You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by pn...@apache.org on 2018/09/21 11:43:33 UTC

[flink] branch master updated (914dffb -> 00add9c)

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

pnowojski pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git.


    from 914dffb  [FLINK-9913][network] Serialize records only once for multi channel writes in RecordWriter (#6417)
     new f2a67a1  [FLINK-9738][table] Provide a way to define Temporal Table Functions in Table API
     new ed2e360  [hotfix][table] Deduplicate optimize code between stream and batch table environment
     new b1ae9c3  [hotfix][table] Extract computeCost in FlinkLogicalJoin to base class
     new 3cca8b6  [hotfix][table] Deduplicate RelTimeInidicatoConverter logic
     new 81c96ca  [hotfix][table,tests] Reduce mockito usage in TableTestUtil
     new 77c3834  [FLINK-9713][table][sql] Support versioned join in planning phase
     new 6ff0bb6  [hotfix][table] Extract DataStreamJoinToCoProcessTranslator
     new 21082a2  [hotfix][table] Simplify NonWindowJoin class
     new 61c1161  [hotfix][table] Add convienient constructors for CRow
     new dde089b  [hotfix][table,tests] Add convienient verify methods to HarnessTestBase
     new 00add9c  [FLINK-9714][table] Support versioned joins with processing time

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


Summary of changes:
 .../flink/table/api/BatchTableEnvironment.scala    |  49 +-
 .../flink/table/api/StreamTableEnvironment.scala   |  72 +-
 .../apache/flink/table/api/TableEnvironment.scala  |  58 +-
 .../scala/org/apache/flink/table/api/table.scala   |  73 +-
 .../flink/table/calcite/FlinkRelBuilder.scala      |   2 +
 .../table/calcite/RelTimeIndicatorConverter.scala  | 207 +++---
 .../flink/table/expressions/fieldExpression.scala  |  13 +-
 .../table/functions/TemporalTableFunction.scala    |  80 +++
 .../flink/table/plan/logical/operators.scala       |  14 +
 .../logical/rel/LogicalTemporalTableJoin.scala     | 188 +++++
 .../apache/flink/table/plan/nodes/CommonJoin.scala |   9 +
 .../plan/nodes/datastream/DataStreamJoin.scala     | 147 +---
 .../DataStreamJoinToCoProcessTranslator.scala      | 149 ++++
 ...taStreamTemporalJoinToCoProcessTranslator.scala | 237 +++++++
 .../datastream/DataStreamTemporalTableJoin.scala   |  95 +++
 .../plan/nodes/logical/FlinkLogicalJoin.scala      |  26 +-
 .../plan/nodes/logical/FlinkLogicalJoinBase.scala  |  59 ++
 .../logical/FlinkLogicalTableFunctionScan.scala    |  36 +-
 .../logical/FlinkLogicalTemporalTableJoin.scala    |  97 +++
 .../flink/table/plan/rules/FlinkRuleSets.scala     |  10 +
 .../DataStreamTemporalTableJoinRule.scala          |  78 +++
 .../LogicalCorrelateToTemporalTableJoinRule.scala  | 207 ++++++
 .../flink/table/plan/util/RexDefaultVisitor.scala  |  66 ++
 .../table/runtime/join/NonWindowFullJoin.scala     |   3 -
 .../NonWindowFullJoinWithNonEquiPredicates.scala   |   3 -
 .../table/runtime/join/NonWindowInnerJoin.scala    |   3 -
 .../flink/table/runtime/join/NonWindowJoin.scala   |   2 -
 .../runtime/join/NonWindowLeftRightJoin.scala      |   3 -
 ...nWindowLeftRightJoinWithNonEquiPredicates.scala |   2 -
 .../table/runtime/join/NonWindowOuterJoin.scala    |   8 +-
 .../NonWindowOuterJoinWithNonEquiPredicates.scala  |   8 +-
 .../flink/table/runtime/join/TemporalJoin.scala    |  93 +++
 .../apache/flink/table/runtime/types/CRow.scala    |   8 +-
 .../api/batch/sql/TemporalTableJoinTest.scala      | 112 +++
 .../api/batch/table/TemporalTableJoinTest.scala    |  77 +++
 .../api/stream/sql/TemporalTableJoinTest.scala     | 130 ++++
 .../api/stream/table/TemporalTableJoinTest.scala   | 276 ++++++++
 .../TemporalTableJoinValidationTest.scala          | 124 ++++
 .../table/plan/TimeIndicatorConversionTest.scala   | 110 +++
 .../table/runtime/harness/HarnessTestBase.scala    |  16 +
 .../table/runtime/harness/JoinHarnessTest.scala    | 762 +++++++++------------
 .../runtime/harness/NonWindowHarnessTest.scala     |  94 +--
 .../runtime/harness/OverWindowHarnessTest.scala    | 458 ++++++-------
 .../runtime/harness/TemporalJoinHarnessTest.scala  | 452 ++++++++++++
 .../runtime/stream/sql/TemporalJoinITCase.scala    | 135 ++++
 .../runtime/utils/StreamingWithStateTestBase.scala |   7 +-
 .../apache/flink/table/utils/TableTestBase.scala   |  99 ++-
 47 files changed, 3885 insertions(+), 1072 deletions(-)
 create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/TemporalTableFunction.scala
 create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/rel/LogicalTemporalTableJoin.scala
 create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamJoinToCoProcessTranslator.scala
 create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamTemporalJoinToCoProcessTranslator.scala
 create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamTemporalTableJoin.scala
 create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalJoinBase.scala
 create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTemporalTableJoin.scala
 create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamTemporalTableJoinRule.scala
 create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/logical/LogicalCorrelateToTemporalTableJoinRule.scala
 create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexDefaultVisitor.scala
 create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TemporalJoin.scala
 create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/TemporalTableJoinTest.scala
 create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/TemporalTableJoinTest.scala
 create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/TemporalTableJoinTest.scala
 create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/TemporalTableJoinTest.scala
 create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/validation/TemporalTableJoinValidationTest.scala
 create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/TemporalJoinHarnessTest.scala
 create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/TemporalJoinITCase.scala


[flink] 02/11: [hotfix][table] Deduplicate optimize code between stream and batch table environment

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

pnowojski pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit ed2e360a6b2c1cbf8708ea1586e82f8cadbd6f62
Author: Piotr Nowojski <pi...@gmail.com>
AuthorDate: Fri Aug 24 13:17:26 2018 +0200

    [hotfix][table] Deduplicate optimize code between stream and batch table environment
---
 .../flink/table/api/BatchTableEnvironment.scala    | 48 ++-------------
 .../flink/table/api/StreamTableEnvironment.scala   | 71 ++++++----------------
 .../apache/flink/table/api/TableEnvironment.scala  | 50 ++++++++++++++-
 3 files changed, 73 insertions(+), 96 deletions(-)

diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala
index 04a7916..522a03e 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala
@@ -448,48 +448,12 @@ abstract class BatchTableEnvironment(
     * @return The optimized [[RelNode]] tree
     */
   private[flink] def optimize(relNode: RelNode): RelNode = {
-
-    // 0. convert sub-queries before query decorrelation
-    val convSubQueryPlan = runHepPlanner(
-      HepMatchOrder.BOTTOM_UP, FlinkRuleSets.TABLE_SUBQUERY_RULES, relNode, relNode.getTraitSet)
-
-    // 0. convert table references
-    val fullRelNode = runHepPlanner(
-      HepMatchOrder.BOTTOM_UP,
-      FlinkRuleSets.TABLE_REF_RULES,
-      convSubQueryPlan,
-      relNode.getTraitSet)
-
-    // 1. decorrelate
-    val decorPlan = RelDecorrelator.decorrelateQuery(fullRelNode)
-
-    // 2. normalize the logical plan
-    val normRuleSet = getNormRuleSet
-    val normalizedPlan = if (normRuleSet.iterator().hasNext) {
-      runHepPlanner(HepMatchOrder.BOTTOM_UP, normRuleSet, decorPlan, decorPlan.getTraitSet)
-    } else {
-      decorPlan
-    }
-
-    // 3. optimize the logical Flink plan
-    val logicalOptRuleSet = getLogicalOptRuleSet
-    val logicalOutputProps = relNode.getTraitSet.replace(FlinkConventions.LOGICAL).simplify()
-    val logicalPlan = if (logicalOptRuleSet.iterator().hasNext) {
-      runVolcanoPlanner(logicalOptRuleSet, normalizedPlan, logicalOutputProps)
-    } else {
-      normalizedPlan
-    }
-
-    // 4. optimize the physical Flink plan
-    val physicalOptRuleSet = getPhysicalOptRuleSet
-    val physicalOutputProps = relNode.getTraitSet.replace(FlinkConventions.DATASET).simplify()
-    val physicalPlan = if (physicalOptRuleSet.iterator().hasNext) {
-      runVolcanoPlanner(physicalOptRuleSet, logicalPlan, physicalOutputProps)
-    } else {
-      logicalPlan
-    }
-
-    physicalPlan
+    val convSubQueryPlan = optimizeConvertSubQueries(relNode)
+    val fullNode = optimizeConvertTableReferences(convSubQueryPlan)
+    val decorPlan = RelDecorrelator.decorrelateQuery(fullNode)
+    val normalizedPlan = optimizeNormalizeLogicalPlan(decorPlan)
+    val logicalPlan = optimizeLogicalPlan(normalizedPlan)
+    optimizePhysicalPlan(logicalPlan, FlinkConventions.DATASET)
   }
 
   /**
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
index d31ce6c..860f8b2 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
@@ -803,59 +803,28 @@ abstract class StreamTableEnvironment(
     * @return The optimized [[RelNode]] tree
     */
   private[flink] def optimize(relNode: RelNode, updatesAsRetraction: Boolean): RelNode = {
+    val convSubQueryPlan = optimizeConvertSubQueries(relNode)
+    val fullNode = optimizeConvertTableReferences(convSubQueryPlan)
+    val decorPlan = RelDecorrelator.decorrelateQuery(fullNode)
+    val planWithMaterializedTimeAttributes =
+      RelTimeIndicatorConverter.convert(decorPlan, getRelBuilder.getRexBuilder)
+    val normalizedPlan = optimizeNormalizeLogicalPlan(planWithMaterializedTimeAttributes)
+    val logicalPlan = optimizeLogicalPlan(normalizedPlan)
+    val physicalPlan = optimizePhysicalPlan(logicalPlan, FlinkConventions.DATASTREAM)
+    optimizeDecoratePlan(physicalPlan, updatesAsRetraction)
+  }
 
-    // 0. convert sub-queries before query decorrelation
-    val convSubQueryPlan = runHepPlanner(
-      HepMatchOrder.BOTTOM_UP, FlinkRuleSets.TABLE_SUBQUERY_RULES, relNode, relNode.getTraitSet)
-
-    // 0. convert table references
-    val fullRelNode = runHepPlanner(
-      HepMatchOrder.BOTTOM_UP,
-      FlinkRuleSets.TABLE_REF_RULES,
-      convSubQueryPlan,
-      relNode.getTraitSet)
-
-    // 1. decorrelate
-    val decorPlan = RelDecorrelator.decorrelateQuery(fullRelNode)
-
-    // 2. convert time indicators
-    val convPlan = RelTimeIndicatorConverter.convert(decorPlan, getRelBuilder.getRexBuilder)
-
-    // 3. normalize the logical plan
-    val normRuleSet = getNormRuleSet
-    val normalizedPlan = if (normRuleSet.iterator().hasNext) {
-      runHepPlanner(HepMatchOrder.BOTTOM_UP, normRuleSet, convPlan, convPlan.getTraitSet)
-    } else {
-      convPlan
-    }
-
-    // 4. optimize the logical Flink plan
-    val logicalOptRuleSet = getLogicalOptRuleSet
-    val logicalOutputProps = relNode.getTraitSet.replace(FlinkConventions.LOGICAL).simplify()
-    val logicalPlan = if (logicalOptRuleSet.iterator().hasNext) {
-      runVolcanoPlanner(logicalOptRuleSet, normalizedPlan, logicalOutputProps)
-    } else {
-      normalizedPlan
-    }
-
-    // 5. optimize the physical Flink plan
-    val physicalOptRuleSet = getPhysicalOptRuleSet
-    val physicalOutputProps = relNode.getTraitSet.replace(FlinkConventions.DATASTREAM).simplify()
-    val physicalPlan = if (physicalOptRuleSet.iterator().hasNext) {
-      runVolcanoPlanner(physicalOptRuleSet, logicalPlan, physicalOutputProps)
-    } else {
-      logicalPlan
-    }
-
-    // 6. decorate the optimized plan
+  private[flink] def optimizeDecoratePlan(
+      relNode: RelNode,
+      updatesAsRetraction: Boolean): RelNode = {
     val decoRuleSet = getDecoRuleSet
-    val decoratedPlan = if (decoRuleSet.iterator().hasNext) {
+    if (decoRuleSet.iterator().hasNext) {
       val planToDecorate = if (updatesAsRetraction) {
-        physicalPlan.copy(
-          physicalPlan.getTraitSet.plus(new UpdateAsRetractionTrait(true)),
-          physicalPlan.getInputs)
+        relNode.copy(
+          relNode.getTraitSet.plus(new UpdateAsRetractionTrait(true)),
+          relNode.getInputs)
       } else {
-        physicalPlan
+        relNode
       }
       runHepPlanner(
         HepMatchOrder.BOTTOM_UP,
@@ -863,10 +832,8 @@ abstract class StreamTableEnvironment(
         planToDecorate,
         planToDecorate.getTraitSet)
     } else {
-      physicalPlan
+      relNode
     }
-
-    decoratedPlan
   }
 
   /**
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
index 5691ab7..cce270c 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
@@ -26,7 +26,7 @@ import org.apache.calcite.config.Lex
 import org.apache.calcite.jdbc.CalciteSchema
 import org.apache.calcite.plan.RelOptPlanner.CannotPlanException
 import org.apache.calcite.plan.hep.{HepMatchOrder, HepPlanner, HepProgramBuilder}
-import org.apache.calcite.plan.{RelOptPlanner, RelOptUtil, RelTraitSet}
+import org.apache.calcite.plan.{Convention, RelOptPlanner, RelOptUtil, RelTraitSet}
 import org.apache.calcite.rel.RelNode
 import org.apache.calcite.schema.SchemaPlus
 import org.apache.calcite.schema.impl.AbstractTable
@@ -54,6 +54,7 @@ import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils._
 import org.apache.flink.table.functions.{AggregateFunction, ScalarFunction, TableFunction}
 import org.apache.flink.table.plan.cost.DataSetCostFactory
 import org.apache.flink.table.plan.logical.{CatalogNode, LogicalRelNode}
+import org.apache.flink.table.plan.nodes.FlinkConventions
 import org.apache.flink.table.plan.rules.FlinkRuleSets
 import org.apache.flink.table.plan.schema.{RelTable, RowSchema, TableSourceSinkTable}
 import org.apache.flink.table.sinks.TableSink
@@ -233,6 +234,52 @@ abstract class TableEnvironment(val config: TableConfig) {
     */
   protected def getBuiltInPhysicalOptRuleSet: RuleSet
 
+  protected def optimizeConvertSubQueries(relNode: RelNode): RelNode = {
+    runHepPlanner(
+      HepMatchOrder.BOTTOM_UP,
+      FlinkRuleSets.TABLE_SUBQUERY_RULES,
+      relNode,
+      relNode.getTraitSet)
+  }
+
+  protected def optimizeConvertTableReferences(relNode: RelNode): RelNode = {
+    runHepPlanner(
+      HepMatchOrder.BOTTOM_UP,
+      FlinkRuleSets.TABLE_REF_RULES,
+      relNode,
+      relNode.getTraitSet)
+  }
+
+
+  protected def optimizeNormalizeLogicalPlan(relNode: RelNode): RelNode = {
+    val normRuleSet = getNormRuleSet
+    if (normRuleSet.iterator().hasNext) {
+      runHepPlanner(HepMatchOrder.BOTTOM_UP, normRuleSet, relNode, relNode.getTraitSet)
+    } else {
+      relNode
+    }
+  }
+
+  protected def optimizeLogicalPlan(relNode: RelNode): RelNode = {
+    val logicalOptRuleSet = getLogicalOptRuleSet
+    val logicalOutputProps = relNode.getTraitSet.replace(FlinkConventions.LOGICAL).simplify()
+    if (logicalOptRuleSet.iterator().hasNext) {
+      runVolcanoPlanner(logicalOptRuleSet, relNode, logicalOutputProps)
+    } else {
+      relNode
+    }
+  }
+
+  protected def optimizePhysicalPlan(relNode: RelNode, convention: Convention): RelNode = {
+    val physicalOptRuleSet = getPhysicalOptRuleSet
+    val physicalOutputProps = relNode.getTraitSet.replace(convention).simplify()
+    if (physicalOptRuleSet.iterator().hasNext) {
+      runVolcanoPlanner(physicalOptRuleSet, relNode, physicalOutputProps)
+    } else {
+      relNode
+    }
+  }
+
   /**
     * run HEP planner
     */
@@ -1308,5 +1355,4 @@ object TableEnvironment {
       case t: TypeInformation[_] => Array(t.asInstanceOf[TypeInformation[_]])
     }
   }
-
 }


[flink] 01/11: [FLINK-9738][table] Provide a way to define Temporal Table Functions in Table API

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

pnowojski pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit f2a67a1682249d83711030f4e55b824cb18336d7
Author: Piotr Nowojski <pi...@gmail.com>
AuthorDate: Thu Jul 5 19:47:01 2018 +0200

    [FLINK-9738][table] Provide a way to define Temporal Table Functions in Table API
---
 .../scala/org/apache/flink/table/api/table.scala   | 73 ++++++++++++++++++-
 .../flink/table/expressions/fieldExpression.scala  | 13 ++--
 .../table/functions/TemporalTableFunction.scala    | 80 +++++++++++++++++++++
 .../flink/table/plan/logical/operators.scala       | 14 ++++
 .../logical/FlinkLogicalTableFunctionScan.scala    | 36 ++++++++--
 .../api/stream/table/TemporalTableJoinTest.scala   | 84 ++++++++++++++++++++++
 .../TemporalTableJoinValidationTest.scala          | 56 +++++++++++++++
 7 files changed, 344 insertions(+), 12 deletions(-)

diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala
index a44bbaa..6c96834 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala
@@ -21,7 +21,8 @@ import org.apache.calcite.rel.RelNode
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.java.operators.join.JoinType
 import org.apache.flink.table.calcite.{FlinkRelBuilder, FlinkTypeFactory}
-import org.apache.flink.table.expressions.{Alias, Asc, Expression, ExpressionParser, Ordering, UnresolvedAlias, UnresolvedFieldReference, WindowProperty}
+import org.apache.flink.table.expressions.{Alias, Asc, Expression, ExpressionParser, Ordering, ResolvedFieldReference, UnresolvedAlias, UnresolvedFieldReference, WindowProperty}
+import org.apache.flink.table.functions.TemporalTableFunction
 import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils
 import org.apache.flink.table.plan.ProjectionTranslator._
 import org.apache.flink.table.plan.logical.{Minus, _}
@@ -156,6 +157,75 @@ class Table(
   }
 
   /**
+    * Creates [[TemporalTableFunction]] backed up by this table as a history table.
+    * Temporal Tables represent a concept of a table that changes over time and for which
+    * Flink keeps track of those changes. [[TemporalTableFunction]] provides a way how to access
+    * those data.
+    *
+    * For more information please check Flink's documentation on Temporal Tables.
+    *
+    * Currently [[TemporalTableFunction]]s are only supported in streaming.
+    *
+    * @param timeAttribute Must points to a time attribute. Provides a way to compare which records
+    *                      are a newer or older version.
+    * @param primaryKey    Defines the primary key. With primary key it is possible to update
+    *                      a row or to delete it.
+    * @return [[TemporalTableFunction]] which is an instance of
+    *        [[org.apache.flink.table.functions.TableFunction]]. It takes one single argument,
+    *        the `timeAttribute`, for which it returns matching version of the [[Table]], from which
+    *        [[TemporalTableFunction]] was created.
+    */
+  def createTemporalTableFunction(
+      timeAttribute: String,
+      primaryKey: String): TemporalTableFunction = {
+    createTemporalTableFunction(
+      ExpressionParser.parseExpression(timeAttribute),
+      ExpressionParser.parseExpression(primaryKey))
+  }
+
+  /**
+    * Creates [[TemporalTableFunction]] backed up by this table as a history table.
+    * Temporal Tables represent a concept of a table that changes over time and for which
+    * Flink keeps track of those changes. [[TemporalTableFunction]] provides a way how to access
+    * those data.
+    *
+    * For more information please check Flink's documentation on Temporal Tables.
+    *
+    * Currently [[TemporalTableFunction]]s are only supported in streaming.
+    *
+    * @param timeAttribute Must points to a time indicator. Provides a way to compare which records
+    *                      are a newer or older version.
+    * @param primaryKey    Defines the primary key. With primary key it is possible to update
+    *                      a row or to delete it.
+    * @return [[TemporalTableFunction]] which is an instance of
+    *        [[org.apache.flink.table.functions.TableFunction]]. It takes one single argument,
+    *        the `timeAttribute`, for which it returns matching version of the [[Table]], from which
+    *        [[TemporalTableFunction]] was created.
+    */
+  def createTemporalTableFunction(
+      timeAttribute: Expression,
+      primaryKey: Expression): TemporalTableFunction = {
+    val temporalTable = TemporalTable(timeAttribute, primaryKey, logicalPlan)
+      .validate(tableEnv)
+      .asInstanceOf[TemporalTable]
+
+    TemporalTableFunction.create(
+      this,
+      temporalTable.timeAttribute,
+      validatePrimaryKeyExpression(temporalTable.primaryKey))
+  }
+
+  private def validatePrimaryKeyExpression(expression: Expression): String = {
+    expression match {
+      case fieldReference: ResolvedFieldReference =>
+        fieldReference.name
+      case _ => throw new ValidationException(
+        s"Unsupported expression [$expression] as primary key. " +
+          s"Only top-level (not nested) field references are supported.")
+    }
+  }
+
+  /**
     * Renames the fields of the expression result. Use this to disambiguate fields before
     * joining to operations.
     *
@@ -1178,5 +1248,4 @@ class WindowGroupedTable(
     val withResolvedAggFunctionCall = fieldExprs.map(replaceAggFunctionCall(_, table.tableEnv))
     select(withResolvedAggFunctionCall: _*)
   }
-
 }
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/fieldExpression.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/fieldExpression.scala
index f3ef039..3de0175 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/fieldExpression.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/fieldExpression.scala
@@ -174,9 +174,10 @@ case class RowtimeAttribute(expr: Expression) extends TimeAttribute(expr) {
         ValidationSuccess
       case WindowReference(_, _) =>
         ValidationFailure("Reference to a rowtime or proctime window required.")
-      case _ =>
+      case any =>
         ValidationFailure(
-          "The '.rowtime' expression can only be used for table definitions and windows.")
+          s"The '.rowtime' expression can only be used for table definitions and windows, " +
+            s"while [$any] was found.")
     }
   }
 
@@ -189,8 +190,7 @@ case class RowtimeAttribute(expr: Expression) extends TimeAttribute(expr) {
         // batch time window
         Types.SQL_TIMESTAMP
       case _ =>
-        throw TableException("WindowReference of RowtimeAttribute has invalid type. " +
-          "Please report this bug.")
+        throw TableException("RowtimeAttribute has invalid type. Please report this bug.")
     }
   }
 
@@ -208,9 +208,10 @@ case class ProctimeAttribute(expr: Expression) extends TimeAttribute(expr) {
         ValidationSuccess
       case WindowReference(_, _) =>
         ValidationFailure("Reference to a rowtime or proctime window required.")
-      case _ =>
+      case any =>
         ValidationFailure(
-          "The '.proctime' expression can only be used for table definitions and windows.")
+          "The '.proctime' expression can only be used for table definitions and windows, " +
+            s"while [$any] was found.")
     }
   }
 
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/TemporalTableFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/TemporalTableFunction.scala
new file mode 100644
index 0000000..7943144
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/TemporalTableFunction.scala
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.functions
+
+import java.sql.Timestamp
+
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.table.api.Table
+import org.apache.flink.table.expressions.Expression
+import org.apache.flink.types.Row
+
+/**
+  * Class representing temporal table function over some history table.
+  * It takes one single argument, the `timeAttribute`, for which it returns matching version of
+  * the `underlyingHistoryTable`, from which this [[TemporalTableFunction]] was created.
+  *
+  * This function shouldn't be evaluated. Instead calls to it should be rewritten by the optimiser
+  * into other operators (like Temporal Table Join).
+  */
+class TemporalTableFunction private(
+    @transient private val underlyingHistoryTable: Table,
+    private val timeAttribute: Expression,
+    private val primaryKey: String,
+    private val resultType: RowTypeInfo)
+  extends TableFunction[Row] {
+
+  def eval(row: Timestamp): Unit = {
+    throw new IllegalStateException("This should never be called")
+  }
+
+  override def getResultType: RowTypeInfo = {
+    resultType
+  }
+
+  def getTimeAttribute: Expression = {
+    timeAttribute
+  }
+
+  def getPrimaryKey: String = {
+    primaryKey
+  }
+
+  private[flink] def getUnderlyingHistoryTable: Table = {
+    if (underlyingHistoryTable == null) {
+      throw new IllegalStateException("Accessing table field after planing/serialization")
+    }
+    underlyingHistoryTable
+  }
+}
+
+object TemporalTableFunction {
+  def create(
+      table: Table,
+      timeAttribute: Expression,
+      primaryKey: String): TemporalTableFunction = {
+    new TemporalTableFunction(
+      table,
+      timeAttribute,
+      primaryKey,
+      new RowTypeInfo(
+        table.getSchema.getTypes,
+        table.getSchema.getColumnNames))
+  }
+}
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/operators.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/operators.scala
index 7579621..84e3f79 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/operators.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/operators.scala
@@ -686,6 +686,20 @@ case class WindowAggregate(
   }
 }
 
+case class TemporalTable(
+    timeAttribute: Expression,
+    primaryKey: Expression,
+    child: LogicalNode)
+  extends UnaryNode {
+
+  override def output: Seq[Attribute] = child.output
+
+  override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = {
+    throw new UnsupportedOperationException(
+      "This should never be called. This node is supposed to be used only for validation")
+  }
+}
+
 /**
   * LogicalNode for calling a user-defined table functions.
   *
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableFunctionScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableFunctionScan.scala
index 4bf1ca1..0ed2301 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableFunctionScan.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableFunctionScan.scala
@@ -21,14 +21,16 @@ package org.apache.flink.table.plan.nodes.logical
 import java.lang.reflect.Type
 import java.util.{List => JList, Set => JSet}
 
-import org.apache.calcite.plan.{Convention, RelOptCluster, RelTraitSet}
+import org.apache.calcite.plan.{Convention, RelOptCluster, RelOptRuleCall, RelTraitSet}
 import org.apache.calcite.rel.RelNode
 import org.apache.calcite.rel.`type`.RelDataType
 import org.apache.calcite.rel.convert.ConverterRule
-import org.apache.calcite.rel.core.{TableFunctionScan, TableScan}
-import org.apache.calcite.rel.logical.{LogicalTableFunctionScan, LogicalTableScan}
+import org.apache.calcite.rel.core.TableFunctionScan
+import org.apache.calcite.rel.logical.LogicalTableFunctionScan
 import org.apache.calcite.rel.metadata.RelColumnMapping
-import org.apache.calcite.rex.RexNode
+import org.apache.calcite.rex.{RexCall, RexNode, RexVisitorImpl}
+import org.apache.flink.table.functions.TemporalTableFunction
+import org.apache.flink.table.functions.utils.TableSqlFunction
 import org.apache.flink.table.plan.nodes.FlinkConventions
 
 class FlinkLogicalTableFunctionScan(
@@ -74,6 +76,32 @@ class FlinkLogicalTableFunctionScanConverter
     FlinkConventions.LOGICAL,
     "FlinkLogicalTableFunctionScanConverter") {
 
+  /**
+    * This rule do not match to [[TemporalTableFunction]]. We do not support reading from
+    * [[TemporalTableFunction]]s as TableFunctions. We expect them to be rewritten into
+    * [[org.apache.flink.table.plan.nodes.datastream.DataStreamScan]] followed by for
+    * example [[org.apache.flink.table.plan.nodes.datastream.DataStreamTemporalTableJoin]].
+    */
+  override def matches(call: RelOptRuleCall): Boolean = {
+    val logicalTableFunction: LogicalTableFunctionScan = call.rel(0)
+
+    !isTemporalTableFunctionCall(logicalTableFunction)
+  }
+
+  private def isTemporalTableFunctionCall(logicalTableFunction: LogicalTableFunctionScan)
+    : Boolean = {
+
+    if (!logicalTableFunction.getCall.isInstanceOf[RexCall]) {
+      return false
+    }
+    val rexCall = logicalTableFunction.getCall().asInstanceOf[RexCall]
+    if (!rexCall.getOperator.isInstanceOf[TableSqlFunction]) {
+      return false
+    }
+    val tableFunction = rexCall.getOperator.asInstanceOf[TableSqlFunction]
+    tableFunction.getTableFunction.isInstanceOf[TemporalTableFunction]
+  }
+
   def convert(rel: RelNode): RelNode = {
     val scan = rel.asInstanceOf[LogicalTableFunctionScan]
     val traitSet = rel.getTraitSet.replace(FlinkConventions.LOGICAL)
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/TemporalTableJoinTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/TemporalTableJoinTest.scala
new file mode 100644
index 0000000..0942dd3
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/TemporalTableJoinTest.scala
@@ -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.flink.table.api.stream.table
+
+import java.sql.Timestamp
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.{TableSchema, ValidationException}
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.expressions.ResolvedFieldReference
+import org.apache.flink.table.functions.TemporalTableFunction
+import org.apache.flink.table.utils._
+import org.junit.Assert.{assertArrayEquals, assertEquals, assertTrue}
+import org.junit.Test
+
+class TemporalTableJoinTest extends TableTestBase {
+
+  val util: TableTestUtil = streamTestUtil()
+
+  val ratesHistory = util.addTable[(String, Int, Timestamp)](
+    "RatesHistory", 'currency, 'rate, 'rowtime.rowtime)
+
+  val rates = util.addFunction(
+    "Rates",
+    ratesHistory.createTemporalTableFunction('rowtime, 'currency))
+
+  @Test
+  def testTemporalTableFunctionScan(): Unit = {
+    expectedException.expect(classOf[ValidationException])
+    expectedException.expectMessage(
+      "Cannot translate a query with an unbounded table function call")
+
+    val result = rates(java.sql.Timestamp.valueOf("2016-06-27 10:10:42.123"))
+    util.printTable(result)
+  }
+
+  @Test
+  def testProcessingTimeIndicatorVersion(): Unit = {
+    val util: TableTestUtil = streamTestUtil()
+    val ratesHistory = util.addTable[(String, Int)](
+      "RatesHistory", 'currency, 'rate, 'proctime.proctime)
+    val rates = ratesHistory.createTemporalTableFunction('proctime, 'currency)
+    assertRatesFunction(ratesHistory.getSchema, rates, true)
+  }
+
+  @Test
+  def testValidStringFieldReference(): Unit = {
+    val rates = ratesHistory.createTemporalTableFunction("rowtime", "currency")
+    assertRatesFunction(ratesHistory.getSchema, rates)
+  }
+
+  private def assertRatesFunction(
+      expectedSchema: TableSchema,
+      rates: TemporalTableFunction,
+      proctime: Boolean = false): Unit = {
+    assertEquals("currency", rates.getPrimaryKey)
+    assertTrue(rates.getTimeAttribute.isInstanceOf[ResolvedFieldReference])
+    assertEquals(
+      if (proctime) "proctime" else "rowtime",
+      rates.getTimeAttribute.asInstanceOf[ResolvedFieldReference].name)
+    assertArrayEquals(
+      expectedSchema.getColumnNames.asInstanceOf[Array[Object]],
+      rates.getResultType.getFieldNames.asInstanceOf[Array[Object]])
+    assertArrayEquals(
+      expectedSchema.getTypes.asInstanceOf[Array[Object]],
+      rates.getResultType.getFieldTypes.asInstanceOf[Array[Object]])
+  }
+}
+
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/validation/TemporalTableJoinValidationTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/validation/TemporalTableJoinValidationTest.scala
new file mode 100644
index 0000000..71b1585
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/validation/TemporalTableJoinValidationTest.scala
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.stream.table.validation
+
+import java.sql.Timestamp
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.ValidationException
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.utils._
+import org.junit.Test
+
+class TemporalTableJoinValidationTest extends TableTestBase {
+
+  val util: TableTestUtil = streamTestUtil()
+
+  val orders = util.addTable[(Long, String, Timestamp)](
+    "Orders", 'o_amount, 'o_currency, 'o_rowtime.rowtime)
+
+  val ratesHistory = util.addTable[(String, Int, Timestamp)](
+    "RatesHistory", 'currency, 'rate, 'rowtime.rowtime)
+
+  @Test
+  def testInvalidFieldReference(): Unit = {
+    expectedException.expect(classOf[ValidationException])
+    expectedException.expectMessage("Cannot resolve field [foobar]")
+
+    ratesHistory.createTemporalTableFunction('rowtime, 'foobar)
+  }
+
+  @Test
+  def testInvalidStringFieldReference(): Unit = {
+    expectedException.expect(classOf[ValidationException])
+    expectedException.expectMessage("Cannot resolve field [foobar]")
+
+    ratesHistory.createTemporalTableFunction("rowtime", "foobar")
+  }
+}
+
+


[flink] 03/11: [hotfix][table] Extract computeCost in FlinkLogicalJoin to base class

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

pnowojski pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit b1ae9c3228e80c415b30396a7bea01360b54c549
Author: Piotr Nowojski <pi...@gmail.com>
AuthorDate: Thu Jul 5 19:58:05 2018 +0200

    [hotfix][table] Extract computeCost in FlinkLogicalJoin to base class
    
    This commit can be squashed with a following commit after code review
---
 .../plan/nodes/logical/FlinkLogicalJoin.scala      | 26 +++-------
 .../plan/nodes/logical/FlinkLogicalJoinBase.scala  | 59 ++++++++++++++++++++++
 2 files changed, 66 insertions(+), 19 deletions(-)

diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalJoin.scala
index 869ab31..a5ffc90 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalJoin.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalJoin.scala
@@ -24,12 +24,9 @@ import org.apache.calcite.rel.RelNode
 import org.apache.calcite.rel.convert.ConverterRule
 import org.apache.calcite.rel.core._
 import org.apache.calcite.rel.logical.LogicalJoin
-import org.apache.calcite.rel.metadata.RelMetadataQuery
 import org.apache.calcite.rex.RexNode
 import org.apache.flink.table.plan.nodes.FlinkConventions
 
-import scala.collection.JavaConverters._
-
 class FlinkLogicalJoin(
     cluster: RelOptCluster,
     traitSet: RelTraitSet,
@@ -37,8 +34,13 @@ class FlinkLogicalJoin(
     right: RelNode,
     condition: RexNode,
     joinType: JoinRelType)
-  extends Join(cluster, traitSet, left, right, condition, Set.empty[CorrelationId].asJava, joinType)
-  with FlinkLogicalRel {
+  extends FlinkLogicalJoinBase(
+    cluster,
+    traitSet,
+    left,
+    right,
+    condition,
+    joinType) {
 
   override def copy(
       traitSet: RelTraitSet,
@@ -50,20 +52,6 @@ class FlinkLogicalJoin(
 
     new FlinkLogicalJoin(cluster, traitSet, left, right, conditionExpr, joinType)
   }
-
-  override def computeSelfCost (planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = {
-    val leftRowCnt = metadata.getRowCount(getLeft)
-    val leftRowSize = estimateRowSize(getLeft.getRowType)
-
-    val rightRowCnt = metadata.getRowCount(getRight)
-    val rightRowSize = estimateRowSize(getRight.getRowType)
-
-    val ioCost = (leftRowCnt * leftRowSize) + (rightRowCnt * rightRowSize)
-    val cpuCost = leftRowCnt + rightRowCnt
-    val rowCnt = leftRowCnt + rightRowCnt
-
-    planner.getCostFactory.makeCost(rowCnt, cpuCost, ioCost)
-  }
 }
 
 private class FlinkLogicalJoinConverter
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalJoinBase.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalJoinBase.scala
new file mode 100644
index 0000000..7b5c266
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalJoinBase.scala
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.logical
+
+import org.apache.calcite.plan._
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.core._
+import org.apache.calcite.rel.metadata.RelMetadataQuery
+import org.apache.calcite.rex.RexNode
+
+import scala.collection.JavaConverters._
+
+abstract class FlinkLogicalJoinBase(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    left: RelNode,
+    right: RelNode,
+    condition: RexNode,
+    joinType: JoinRelType)
+  extends Join(
+    cluster,
+    traitSet,
+    left,
+    right,
+    condition,
+    Set.empty[CorrelationId].asJava,
+    joinType)
+  with FlinkLogicalRel {
+
+  override def computeSelfCost (planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = {
+    val leftRowCnt = metadata.getRowCount(getLeft)
+    val leftRowSize = estimateRowSize(getLeft.getRowType)
+
+    val rightRowCnt = metadata.getRowCount(getRight)
+    val rightRowSize = estimateRowSize(getRight.getRowType)
+
+    val ioCost = (leftRowCnt * leftRowSize) + (rightRowCnt * rightRowSize)
+    val cpuCost = leftRowCnt + rightRowCnt
+    val rowCnt = leftRowCnt + rightRowCnt
+
+    planner.getCostFactory.makeCost(rowCnt, cpuCost, ioCost)
+  }
+}


[flink] 10/11: [hotfix][table, tests] Add convienient verify methods to HarnessTestBase

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

pnowojski pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit dde089be869c8d2d24f27059530b086326daa5d5
Author: Piotr Nowojski <pi...@gmail.com>
AuthorDate: Fri Jul 20 12:40:36 2018 +0200

    [hotfix][table,tests] Add convienient verify methods to HarnessTestBase
---
 .../table/runtime/harness/HarnessTestBase.scala    |  9 ++++
 .../table/runtime/harness/JoinHarnessTest.scala    | 50 +++++++---------------
 .../runtime/harness/NonWindowHarnessTest.scala     |  4 +-
 .../runtime/harness/OverWindowHarnessTest.scala    | 14 +++---
 4 files changed, 33 insertions(+), 44 deletions(-)

diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/HarnessTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/HarnessTestBase.scala
index 58fe9d3..d494c21 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/HarnessTestBase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/HarnessTestBase.scala
@@ -34,6 +34,7 @@ import org.apache.flink.table.functions.AggregateFunction
 import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils
 import org.apache.flink.table.functions.aggfunctions.{IntSumWithRetractAggFunction, LongMaxWithRetractAggFunction, LongMinWithRetractAggFunction}
 import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils.getAccumulatorTypeOfAggregateFunction
+import org.apache.flink.table.runtime.harness.HarnessTestBase.{RowResultSortComparator, RowResultSortComparatorWithWatermarks}
 import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
 
 class HarnessTestBase {
@@ -318,6 +319,14 @@ class HarnessTestBase {
     new KeyedOneInputStreamOperatorTestHarness[KEY, IN, OUT](operator, keySelector, keyType)
   }
 
+  def verify(expected: JQueue[Object], actual: JQueue[Object]): Unit = {
+    verify(expected, actual, new RowResultSortComparator)
+  }
+
+  def verifyWithWatermarks(expected: JQueue[Object], actual: JQueue[Object]): Unit = {
+    verify(expected, actual, new RowResultSortComparatorWithWatermarks, true)
+  }
+
   def verify(
     expected: JQueue[Object],
     actual: JQueue[Object],
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala
index 86133a0..bd19be8 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala
@@ -231,7 +231,7 @@ class JoinHarnessTest extends HarnessTestBase {
     expectedOutput.add(new StreamRecord(
       CRow(2L: JLong, "2a33", 2L: JLong, "2b33"), 33))
 
-    verify(expectedOutput, result, new RowResultSortComparator())
+    verify(expectedOutput, result)
 
     testHarness.close()
   }
@@ -313,7 +313,7 @@ class JoinHarnessTest extends HarnessTestBase {
     expectedOutput.add(new StreamRecord(
       CRow(1L: JLong, "1a3", 1L: JLong, "1b12"), 12))
 
-    verify(expectedOutput, result, new RowResultSortComparator())
+    verify(expectedOutput, result)
 
     testHarness.close()
   }
@@ -409,11 +409,7 @@ class JoinHarnessTest extends HarnessTestBase {
     expectedOutput.add(new Watermark(41))
 
     val result = testHarness.getOutput
-    verify(
-      expectedOutput,
-      result,
-      new RowResultSortComparatorWithWatermarks(),
-      checkWaterMark = true)
+    verifyWithWatermarks(expectedOutput, result)
     testHarness.close()
   }
 
@@ -491,11 +487,7 @@ class JoinHarnessTest extends HarnessTestBase {
     expectedOutput.add(new Watermark(8))
 
     val result = testHarness.getOutput
-    verify(
-      expectedOutput,
-      result,
-      new RowResultSortComparatorWithWatermarks(),
-      checkWaterMark = true)
+    verifyWithWatermarks(expectedOutput, result)
     testHarness.close()
   }
 
@@ -605,11 +597,7 @@ class JoinHarnessTest extends HarnessTestBase {
 
 
     val result = testHarness.getOutput
-    verify(
-      expectedOutput,
-      result,
-      new RowResultSortComparatorWithWatermarks(),
-      checkWaterMark = true)
+    verifyWithWatermarks(expectedOutput, result)
     testHarness.close()
   }
 
@@ -718,11 +706,7 @@ class JoinHarnessTest extends HarnessTestBase {
     expectedOutput.add(new Watermark(91))
 
     val result = testHarness.getOutput
-    verify(
-      expectedOutput,
-      result,
-      new RowResultSortComparatorWithWatermarks(),
-      checkWaterMark = true)
+    verifyWithWatermarks(expectedOutput, result)
     testHarness.close()
   }
 
@@ -839,11 +823,7 @@ class JoinHarnessTest extends HarnessTestBase {
     expectedOutput.add(new Watermark(91))
 
     val result = testHarness.getOutput
-    verify(
-      expectedOutput,
-      result,
-      new RowResultSortComparatorWithWatermarks(),
-      checkWaterMark = true)
+    verifyWithWatermarks(expectedOutput, result)
     testHarness.close()
   }
 
@@ -944,7 +924,7 @@ class JoinHarnessTest extends HarnessTestBase {
     expectedOutput.add(new StreamRecord(
       CRow(2: JInt, "bbb", 2: JInt, "Hello1")))
 
-    verify(expectedOutput, result, new RowResultSortComparator())
+    verify(expectedOutput, result)
 
     testHarness.close()
   }
@@ -1035,7 +1015,7 @@ class JoinHarnessTest extends HarnessTestBase {
     expectedOutput.add(new StreamRecord(
       CRow(2: JInt, "bbb", 2: JInt, "Hello1")))
 
-    verify(expectedOutput, result, new RowResultSortComparator())
+    verify(expectedOutput, result)
 
     testHarness.close()
   }
@@ -1143,7 +1123,7 @@ class JoinHarnessTest extends HarnessTestBase {
     expectedOutput.add(new StreamRecord(
       CRow(false, 1: JInt, "aaa", null: JInt, null)))
 
-    verify(expectedOutput, result, new RowResultSortComparator())
+    verify(expectedOutput, result)
 
     testHarness.close()
   }
@@ -1273,7 +1253,7 @@ class JoinHarnessTest extends HarnessTestBase {
       CRow(false, 1: JInt, "bbb", 1: JInt, "Hi1")))
     expectedOutput.add(new StreamRecord(
       CRow(1: JInt, "bbb", null: JInt, null)))
-    verify(expectedOutput, result, new RowResultSortComparator())
+    verify(expectedOutput, result)
 
     testHarness.close()
   }
@@ -1381,7 +1361,7 @@ class JoinHarnessTest extends HarnessTestBase {
     expectedOutput.add(new StreamRecord(
       CRow(false, null: JInt, null, 1: JInt, "aaa")))
 
-    verify(expectedOutput, result, new RowResultSortComparator())
+    verify(expectedOutput, result)
 
     testHarness.close()
   }
@@ -1511,7 +1491,7 @@ class JoinHarnessTest extends HarnessTestBase {
       CRow(false, 1: JInt, "Hi1", 1: JInt, "bbb")))
     expectedOutput.add(new StreamRecord(
       CRow(null: JInt, null, 1: JInt, "bbb")))
-    verify(expectedOutput, result, new RowResultSortComparator())
+    verify(expectedOutput, result)
 
     testHarness.close()
   }
@@ -1681,7 +1661,7 @@ class JoinHarnessTest extends HarnessTestBase {
     expectedOutput.add(new StreamRecord(
       CRow(null: JInt, null, 2: JInt, "bbb")))
 
-    verify(expectedOutput, result, new RowResultSortComparator())
+    verify(expectedOutput, result)
     testHarness.close()
   }
 
@@ -1836,7 +1816,7 @@ class JoinHarnessTest extends HarnessTestBase {
     expectedOutput.add(new StreamRecord(
       CRow(null: JInt, null, 2: JInt, "bbb")))
 
-    verify(expectedOutput, result, new RowResultSortComparator())
+    verify(expectedOutput, result)
     testHarness.close()
   }
 }
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala
index d6daa9e..7c4f543 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala
@@ -91,7 +91,7 @@ class NonWindowHarnessTest extends HarnessTestBase {
     expectedOutput.add(new StreamRecord(CRow(9L: JLong, 18: JInt), 1))
     expectedOutput.add(new StreamRecord(CRow(10L: JLong, 3: JInt), 1))
 
-    verify(expectedOutput, result, new RowResultSortComparator())
+    verify(expectedOutput, result)
 
     testHarness.close()
   }
@@ -150,7 +150,7 @@ class NonWindowHarnessTest extends HarnessTestBase {
     expectedOutput.add(new StreamRecord(CRow(false, 10L: JLong, 2: JInt), 10))
     expectedOutput.add(new StreamRecord(CRow(10L: JLong, 5: JInt), 10))
 
-    verify(expectedOutput, result, new RowResultSortComparator())
+    verify(expectedOutput, result)
 
     testHarness.close()
   }
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/OverWindowHarnessTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/OverWindowHarnessTest.scala
index 63d7b5d..95b13a0 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/OverWindowHarnessTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/OverWindowHarnessTest.scala
@@ -130,7 +130,7 @@ class OverWindowHarnessTest extends HarnessTestBase{
     expectedOutput.add(new StreamRecord(
       CRow(2L: JLong, "bbb", 40L: JLong, 40L: JLong, 40L: JLong)))
 
-    verify(expectedOutput, result, new RowResultSortComparator())
+    verify(expectedOutput, result)
 
     testHarness.close()
   }
@@ -261,7 +261,7 @@ class OverWindowHarnessTest extends HarnessTestBase{
     expectedOutput.add(new StreamRecord(
       CRow(0L: JLong, "ccc", 20L: JLong, 10L: JLong, 20L: JLong)))
 
-    verify(expectedOutput, result, new RowResultSortComparator())
+    verify(expectedOutput, result)
 
     testHarness.close()
   }
@@ -351,7 +351,7 @@ class OverWindowHarnessTest extends HarnessTestBase{
     expectedOutput.add(new StreamRecord(
       CRow(0L: JLong, "bbb", 40L: JLong, 40L: JLong, 40L: JLong)))
 
-    verify(expectedOutput, result, new RowResultSortComparator())
+    verify(expectedOutput, result)
     testHarness.close()
   }
 
@@ -504,7 +504,7 @@ class OverWindowHarnessTest extends HarnessTestBase{
     expectedOutput.add(new StreamRecord(
       CRow(20011L: JLong, "ccc", 3L: JLong, 3L: JLong, 3L: JLong)))
 
-    verify(expectedOutput, result, new RowResultSortComparator())
+    verify(expectedOutput, result)
     testHarness.close()
   }
 
@@ -649,7 +649,7 @@ class OverWindowHarnessTest extends HarnessTestBase{
     expectedOutput.add(new StreamRecord(
       CRow(20011L: JLong, "ccc", 3L: JLong, 3L: JLong, 3L: JLong)))
 
-    verify(expectedOutput, result, new RowResultSortComparator())
+    verify(expectedOutput, result)
     testHarness.close()
   }
 
@@ -788,7 +788,7 @@ class OverWindowHarnessTest extends HarnessTestBase{
     expectedOutput.add(new StreamRecord(
       CRow(20002L: JLong, "ccc", 2L: JLong, 1L: JLong, 2L: JLong)))
 
-    verify(expectedOutput, result, new RowResultSortComparator())
+    verify(expectedOutput, result)
     testHarness.close()
   }
 
@@ -923,7 +923,7 @@ class OverWindowHarnessTest extends HarnessTestBase{
     expectedOutput.add(new StreamRecord(
       CRow(20002L: JLong, "ccc", 2L: JLong, 1L: JLong, 2L: JLong)))
 
-    verify(expectedOutput, result, new RowResultSortComparator())
+    verify(expectedOutput, result)
     testHarness.close()
   }
 }


[flink] 06/11: [FLINK-9713][table][sql] Support versioned join in planning phase

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

pnowojski pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 77c38346cb06b6e6c1bb672695c54f4ba253bd3f
Author: Piotr Nowojski <pi...@gmail.com>
AuthorDate: Thu Jul 5 20:02:51 2018 +0200

    [FLINK-9713][table][sql] Support versioned join in planning phase
---
 .../flink/table/api/BatchTableEnvironment.scala    |   3 +-
 .../flink/table/api/StreamTableEnvironment.scala   |   3 +-
 .../apache/flink/table/api/TableEnvironment.scala  |   8 +
 .../flink/table/calcite/FlinkRelBuilder.scala      |   2 +
 .../table/calcite/RelTimeIndicatorConverter.scala  |  53 +++++-
 .../logical/rel/LogicalTemporalTableJoin.scala     | 156 ++++++++++++++++
 .../datastream/DataStreamTemporalTableJoin.scala   |  82 ++++++++
 .../logical/FlinkLogicalTemporalTableJoin.scala    |  97 ++++++++++
 .../flink/table/plan/rules/FlinkRuleSets.scala     |  10 +
 .../DataStreamTemporalTableJoinRule.scala          |  78 ++++++++
 .../LogicalCorrelateToTemporalTableJoinRule.scala  | 207 +++++++++++++++++++++
 .../flink/table/plan/util/RexDefaultVisitor.scala  |  66 +++++++
 .../api/batch/sql/TemporalTableJoinTest.scala      | 112 +++++++++++
 .../api/batch/table/TemporalTableJoinTest.scala    |  77 ++++++++
 .../api/stream/sql/TemporalTableJoinTest.scala     | 130 +++++++++++++
 .../api/stream/table/TemporalTableJoinTest.scala   | 202 +++++++++++++++++++-
 .../TemporalTableJoinValidationTest.scala          |   2 -
 .../table/plan/TimeIndicatorConversionTest.scala   | 110 +++++++++++
 .../apache/flink/table/utils/TableTestBase.scala   |  49 ++++-
 19 files changed, 1428 insertions(+), 19 deletions(-)

diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala
index 522a03e..5a34ee1 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala
@@ -449,7 +449,8 @@ abstract class BatchTableEnvironment(
     */
   private[flink] def optimize(relNode: RelNode): RelNode = {
     val convSubQueryPlan = optimizeConvertSubQueries(relNode)
-    val fullNode = optimizeConvertTableReferences(convSubQueryPlan)
+    val temporalTableJoinPlan = optimizeConvertToTemporalJoin(convSubQueryPlan)
+    val fullNode = optimizeConvertTableReferences(temporalTableJoinPlan)
     val decorPlan = RelDecorrelator.decorrelateQuery(fullNode)
     val normalizedPlan = optimizeNormalizeLogicalPlan(decorPlan)
     val logicalPlan = optimizeLogicalPlan(normalizedPlan)
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
index 860f8b2..5f45cc3 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
@@ -804,7 +804,8 @@ abstract class StreamTableEnvironment(
     */
   private[flink] def optimize(relNode: RelNode, updatesAsRetraction: Boolean): RelNode = {
     val convSubQueryPlan = optimizeConvertSubQueries(relNode)
-    val fullNode = optimizeConvertTableReferences(convSubQueryPlan)
+    val temporalTableJoinPlan = optimizeConvertToTemporalJoin(convSubQueryPlan)
+    val fullNode = optimizeConvertTableReferences(temporalTableJoinPlan)
     val decorPlan = RelDecorrelator.decorrelateQuery(fullNode)
     val planWithMaterializedTimeAttributes =
       RelTimeIndicatorConverter.convert(decorPlan, getRelBuilder.getRexBuilder)
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
index cce270c..d740c3f 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
@@ -242,6 +242,14 @@ abstract class TableEnvironment(val config: TableConfig) {
       relNode.getTraitSet)
   }
 
+  protected def optimizeConvertToTemporalJoin(relNode: RelNode): RelNode = {
+    runHepPlanner(
+      HepMatchOrder.BOTTOM_UP,
+      FlinkRuleSets.TEMPORAL_JOIN_RULES,
+      relNode,
+      relNode.getTraitSet)
+  }
+
   protected def optimizeConvertTableReferences(relNode: RelNode): RelNode = {
     runHepPlanner(
       HepMatchOrder.BOTTOM_UP,
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkRelBuilder.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkRelBuilder.scala
index 1ac9b53..1aecdd8 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkRelBuilder.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkRelBuilder.scala
@@ -46,6 +46,8 @@ class FlinkRelBuilder(
     relOptCluster,
     relOptSchema) {
 
+  def getRelOptSchema: RelOptSchema = relOptSchema
+
   def getPlanner: RelOptPlanner = cluster.getPlanner
 
   def getCluster: RelOptCluster = relOptCluster
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala
index f67b715..34b98a8 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala
@@ -28,7 +28,7 @@ import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo
 import org.apache.flink.table.api.{TableException, ValidationException}
 import org.apache.flink.table.calcite.FlinkTypeFactory.{isRowtimeIndicatorType, _}
 import org.apache.flink.table.functions.sql.ProctimeSqlFunction
-import org.apache.flink.table.plan.logical.rel.LogicalWindowAggregate
+import org.apache.flink.table.plan.logical.rel.{LogicalTemporalTableJoin, LogicalWindowAggregate}
 import org.apache.flink.table.plan.schema.TimeIndicatorRelDataType
 import org.apache.flink.table.validate.BasicOperatorTable
 
@@ -117,11 +117,13 @@ class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttle {
         aggregate.getNamedProperties,
         convAggregate)
 
+    case temporalTableJoin: LogicalTemporalTableJoin =>
+      visit(temporalTableJoin)
+
     case _ =>
       throw new TableException(s"Unsupported logical operator: ${other.getClass.getSimpleName}")
   }
 
-
   override def visit(exchange: LogicalExchange): RelNode =
     throw new TableException("Logical exchange in a stream environment is not supported yet.")
 
@@ -163,9 +165,18 @@ class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttle {
     val right = join.getRight.accept(this)
 
     LogicalJoin.create(left, right, join.getCondition, join.getVariablesSet, join.getJoinType)
-
   }
 
+  def visit(temporalJoin: LogicalTemporalTableJoin): RelNode = {
+    val left = temporalJoin.getLeft.accept(this)
+    val right = temporalJoin.getRight.accept(this)
+
+    val rewrittenTemporalJoin = temporalJoin.copy(temporalJoin.getTraitSet, List(left, right))
+
+    val indicesToMaterialize = gatherIndicesToMaterialize(rewrittenTemporalJoin, left, right)
+
+    materializerUtils.projectAndMaterializeFields(rewrittenTemporalJoin, indicesToMaterialize)
+  }
 
   override def visit(correlate: LogicalCorrelate): RelNode = {
     // visit children and update inputs
@@ -204,13 +215,43 @@ class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttle {
       correlate.getJoinType)
   }
 
+  private def gatherIndicesToMaterialize(
+    temporalJoin: Join,
+    left: RelNode,
+    right: RelNode)
+  : Set[Int] = {
+
+    // Materialize all of the time attributes from the right side of temporal join
+    var indicesToMaterialize =
+      (left.getRowType.getFieldCount until temporalJoin.getRowType.getFieldCount).toSet
+
+    if (!hasRowtimeAttribute(right.getRowType)) {
+      // No rowtime on the right side means that this must be a processing time temporal join
+      // and that we can not provide watermarks even if there is a rowtime time attribute
+      // on the left side (besides processing time attribute used for temporal join).
+      for (fieldIndex <- 0 until left.getRowType.getFieldCount) {
+        val fieldName = left.getRowType.getFieldNames.get(fieldIndex)
+        val fieldType = left.getRowType.getFieldList.get(fieldIndex).getType
+        if (isRowtimeIndicatorType(fieldType)) {
+          indicesToMaterialize += fieldIndex
+        }
+      }
+    }
+
+    indicesToMaterialize
+  }
+
+  private def hasRowtimeAttribute(rowType: RelDataType): Boolean = {
+    rowType.getFieldList.exists(field => isRowtimeIndicatorType(field.getType))
+  }
+
   private def convertAggregate(aggregate: Aggregate): LogicalAggregate = {
     // visit children and update inputs
     val input = aggregate.getInput.accept(this)
 
     // add a project to materialize aggregation arguments/grouping keys
 
-    val indicesToMaterialize = gatherIndicesToMaterialize(aggregate)
+    val indicesToMaterialize = gatherIndicesToMaterialize(aggregate, input)
 
     val needsMaterialization = indicesToMaterialize.exists(idx =>
       isTimeIndicatorType(input.getRowType.getFieldList.get(idx).getType))
@@ -266,13 +307,13 @@ class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttle {
       updatedAggCalls)
   }
 
-  private def gatherIndicesToMaterialize(aggregate: Aggregate): Set[Int] = {
+  private def gatherIndicesToMaterialize(aggregate: Aggregate, input: RelNode): Set[Int] = {
     val indicesToMaterialize = mutable.Set[Int]()
 
     // check arguments of agg calls
     aggregate.getAggCallList.foreach(call => if (call.getArgList.size() == 0) {
       // count(*) has an empty argument list
-      (0 until aggregate.getRowType.getFieldCount).foreach(indicesToMaterialize.add)
+      (0 until input.getRowType.getFieldCount).foreach(indicesToMaterialize.add)
     } else {
       // for other aggregations
       call.getArgList.map(_.asInstanceOf[Int]).foreach(indicesToMaterialize.add)
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/rel/LogicalTemporalTableJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/rel/LogicalTemporalTableJoin.scala
new file mode 100644
index 0000000..3b1d51b
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/rel/LogicalTemporalTableJoin.scala
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.logical.rel
+
+import java.util.Collections
+
+import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.core._
+import org.apache.calcite.rex.{RexBuilder, RexNode}
+import org.apache.calcite.sql.`type`.{OperandTypes, ReturnTypes}
+import org.apache.calcite.sql.{SqlFunction, SqlFunctionCategory, SqlKind}
+import org.apache.flink.util.Preconditions.checkArgument
+
+/**
+  * Represents a join between a table and [[org.apache.flink.table.functions.TemporalTableFunction]]
+  *
+  * @param cluster
+  * @param traitSet
+  * @param left
+  * @param right     table scan (or other more complex table expression) of underlying
+  *                  [[org.apache.flink.table.functions.TemporalTableFunction]]
+  * @param condition must contain [[LogicalTemporalTableJoin#TEMPORAL_JOIN_CONDITION]] with
+  *                  correctly defined references to rightTimeAttribute,
+  *                  rightPrimaryKeyExpression and leftTimeAttribute. We can not implement
+  *                  those references as separate fields, because of problems with Calcite's
+  *                  optimization rules like projections push downs, column
+  *                  pruning/renaming/reordering, etc. Later rightTimeAttribute,
+  *                  rightPrimaryKeyExpression and leftTimeAttribute will be extracted from
+  *                  the condition.
+  */
+class LogicalTemporalTableJoin private(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    left: RelNode,
+    right: RelNode,
+    condition: RexNode)
+  extends Join(
+    cluster,
+    traitSet,
+    left,
+    right,
+    condition,
+    Collections.emptySet().asInstanceOf[java.util.Set[CorrelationId]],
+    JoinRelType.INNER) {
+
+  override def copy(
+       traitSet: RelTraitSet,
+       condition: RexNode,
+       left: RelNode,
+       right: RelNode,
+       joinType: JoinRelType,
+       semiJoinDone: Boolean): LogicalTemporalTableJoin = {
+    checkArgument(joinType == this.getJoinType,
+      "Can not change join type".asInstanceOf[Object])
+    checkArgument(semiJoinDone == this.isSemiJoinDone,
+      "Can not change semiJoinDone".asInstanceOf[Object])
+    new LogicalTemporalTableJoin(
+      cluster,
+      traitSet,
+      left,
+      right,
+      condition)
+  }
+}
+
+object LogicalTemporalTableJoin {
+  /**
+    * See [[LogicalTemporalTableJoin#condition]]
+    */
+  val TEMPORAL_JOIN_CONDITION = new SqlFunction(
+    "__TEMPORAL_JOIN_CONDITION",
+    SqlKind.OTHER_FUNCTION,
+    ReturnTypes.BOOLEAN_NOT_NULL,
+    null,
+    OperandTypes.or(
+      OperandTypes.sequence(
+        "'(LEFT_TIME_ATTRIBUTE, RIGHT_TIME_ATTRIBUTE, PRIMARY_KEY)'",
+        OperandTypes.DATETIME,
+        OperandTypes.DATETIME,
+        OperandTypes.ANY),
+      OperandTypes.sequence(
+        "'(LEFT_TIME_ATTRIBUTE, PRIMARY_KEY)'",
+        OperandTypes.DATETIME,
+        OperandTypes.ANY)),
+    SqlFunctionCategory.SYSTEM)
+
+  /**
+    * See [[LogicalTemporalTableJoin]]
+    */
+  def createRowtime(
+      rexBuilder: RexBuilder,
+      cluster: RelOptCluster,
+      traitSet: RelTraitSet,
+      left: RelNode,
+      right: RelNode,
+      leftTimeAttribute: RexNode,
+      rightTimeAttribute: RexNode,
+      rightPrimaryKeyExpression: RexNode)
+    : LogicalTemporalTableJoin = {
+    new LogicalTemporalTableJoin(
+      cluster,
+      traitSet,
+      left,
+      right,
+      rexBuilder.makeCall(
+        TEMPORAL_JOIN_CONDITION,
+        leftTimeAttribute,
+        rightTimeAttribute,
+        rightPrimaryKeyExpression))
+  }
+
+  /**
+    * See [[LogicalTemporalTableJoin]]
+    *
+    * @param leftTimeAttribute is needed because otherwise,
+    *                          [[LogicalTemporalTableJoin#TEMPORAL_JOIN_CONDITION]] could be pushed
+    *                          down below [[LogicalTemporalTableJoin]], since it wouldn't have any
+    *                          references to the left node.
+    */
+  def createProctime(
+      rexBuilder: RexBuilder,
+      cluster: RelOptCluster,
+      traitSet: RelTraitSet,
+      left: RelNode,
+      right: RelNode,
+      leftTimeAttribute: RexNode,
+      rightPrimaryKeyExpression: RexNode)
+    : LogicalTemporalTableJoin = {
+    new LogicalTemporalTableJoin(
+      cluster,
+      traitSet,
+      left,
+      right,
+      rexBuilder.makeCall(
+        TEMPORAL_JOIN_CONDITION,
+        leftTimeAttribute,
+        rightPrimaryKeyExpression))
+  }
+}
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamTemporalTableJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamTemporalTableJoin.scala
new file mode 100644
index 0000000..60f36d3
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamTemporalTableJoin.scala
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.plan._
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.core.{JoinInfo, JoinRelType}
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.table.api.{StreamQueryConfig, StreamTableEnvironment}
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.util.Preconditions.checkState
+
+/**
+  * RelNode for a stream join with [[org.apache.flink.table.functions.TemporalTableFunction]].
+  */
+class DataStreamTemporalTableJoin(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    leftNode: RelNode,
+    rightNode: RelNode,
+    joinCondition: RexNode,
+    joinInfo: JoinInfo,
+    leftSchema: RowSchema,
+    rightSchema: RowSchema,
+    schema: RowSchema,
+    ruleDescription: String)
+  extends DataStreamJoin(
+    cluster,
+    traitSet,
+    leftNode,
+    rightNode,
+    joinCondition,
+    joinInfo,
+    JoinRelType.INNER,
+    leftSchema,
+    rightSchema,
+    schema,
+    ruleDescription) {
+
+  override def needsUpdatesAsRetraction: Boolean = true
+
+  override def producesRetractions: Boolean = false
+
+  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
+    checkState(inputs.size() == 2)
+    new DataStreamTemporalTableJoin(
+      cluster,
+      traitSet,
+      inputs.get(0),
+      inputs.get(1),
+      joinCondition,
+      joinInfo,
+      leftSchema,
+      rightSchema,
+      schema,
+      ruleDescription)
+  }
+
+  override def translateToPlan(
+      tableEnv: StreamTableEnvironment,
+      queryConfig: StreamQueryConfig): DataStream[CRow] = {
+    throw new NotImplementedError()
+  }
+}
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTemporalTableJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTemporalTableJoin.scala
new file mode 100644
index 0000000..4be2fb9
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTemporalTableJoin.scala
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.logical
+
+import org.apache.calcite.plan._
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.calcite.rel.core._
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.table.plan.logical.rel.LogicalTemporalTableJoin
+import org.apache.flink.table.plan.nodes.FlinkConventions
+import org.apache.flink.util.Preconditions.checkArgument
+
+/**
+  * Represents a join between a table and
+  * [[org.apache.flink.table.functions.TemporalTableFunction]]. For more details please check
+  * [[LogicalTemporalTableJoin]].
+  */
+class FlinkLogicalTemporalTableJoin(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    left: RelNode,
+    right: RelNode,
+    condition: RexNode)
+  extends FlinkLogicalJoinBase(
+    cluster,
+    traitSet,
+    left,
+    right,
+    condition,
+    JoinRelType.INNER) {
+
+  def copy(
+      traitSet: RelTraitSet,
+      condition: RexNode,
+      left: RelNode,
+      right: RelNode,
+      joinType: JoinRelType,
+      semiJoinDone: Boolean): FlinkLogicalTemporalTableJoin = {
+    checkArgument(joinType == this.getJoinType,
+      "Can not change join type".asInstanceOf[Object])
+    checkArgument(semiJoinDone == this.isSemiJoinDone,
+      "Can not change semiJoinDone".asInstanceOf[Object])
+    new FlinkLogicalTemporalTableJoin(
+      cluster,
+      traitSet,
+      left,
+      right,
+      condition)
+  }
+}
+
+class FlinkLogicalTemporalTableJoinConverter
+  extends ConverterRule(
+    classOf[LogicalTemporalTableJoin],
+    Convention.NONE,
+    FlinkConventions.LOGICAL,
+    "FlinkLogicalTemporalTableJoinConverter") {
+
+  override def matches(call: RelOptRuleCall): Boolean = {
+    true
+  }
+
+  override def convert(rel: RelNode): RelNode = {
+    val temporalTableJoin = rel.asInstanceOf[LogicalTemporalTableJoin]
+    val traitSet = rel.getTraitSet.replace(FlinkConventions.LOGICAL)
+    val newLeft = RelOptRule.convert(temporalTableJoin.getLeft, FlinkConventions.LOGICAL)
+    val newRight = RelOptRule.convert(temporalTableJoin.getRight, FlinkConventions.LOGICAL)
+
+    new FlinkLogicalTemporalTableJoin(
+      rel.getCluster,
+      traitSet,
+      newLeft,
+      newRight,
+      temporalTableJoin.getCondition)
+  }
+}
+
+object FlinkLogicalTemporalTableJoin {
+  val CONVERTER = new FlinkLogicalTemporalTableJoinConverter
+}
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala
index 52dab8b..e4cd8d1 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala
@@ -38,6 +38,14 @@ object FlinkRuleSets {
     SubQueryRemoveRule.JOIN)
 
   /**
+    * Handles proper conversion of correlate queries with temporal table functions
+    * into temporal table joins. This can create new table scans in the plan.
+    */
+  val TEMPORAL_JOIN_RULES: RuleSet = RuleSets.ofList(
+    LogicalCorrelateToTemporalTableJoinRule.INSTANCE
+  )
+
+  /**
     * Convert table references before query decorrelation.
     */
   val TABLE_REF_RULES: RuleSet = RuleSets.ofList(
@@ -127,6 +135,7 @@ object FlinkRuleSets {
     FlinkLogicalCorrelate.CONVERTER,
     FlinkLogicalIntersect.CONVERTER,
     FlinkLogicalJoin.CONVERTER,
+    FlinkLogicalTemporalTableJoin.CONVERTER,
     FlinkLogicalMinus.CONVERTER,
     FlinkLogicalSort.CONVERTER,
     FlinkLogicalUnion.CONVERTER,
@@ -211,6 +220,7 @@ object FlinkRuleSets {
     DataStreamCorrelateRule.INSTANCE,
     DataStreamWindowJoinRule.INSTANCE,
     DataStreamJoinRule.INSTANCE,
+    DataStreamTemporalTableJoinRule.INSTANCE,
     StreamTableSourceScanRule.INSTANCE
   )
 
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamTemporalTableJoinRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamTemporalTableJoinRule.scala
new file mode 100644
index 0000000..94ff19c
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamTemporalTableJoinRule.scala
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.rules.datastream
+
+import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelTraitSet}
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.flink.table.api.TableConfig
+import org.apache.flink.table.plan.nodes.FlinkConventions
+import org.apache.flink.table.plan.nodes.datastream.DataStreamTemporalTableJoin
+import org.apache.flink.table.plan.nodes.logical.FlinkLogicalTemporalTableJoin
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.table.runtime.join.WindowJoinUtil
+
+class DataStreamTemporalTableJoinRule
+  extends ConverterRule(
+    classOf[FlinkLogicalTemporalTableJoin],
+    FlinkConventions.LOGICAL,
+    FlinkConventions.DATASTREAM,
+    "DataStreamTemporalTableJoinRule") {
+
+  override def matches(call: RelOptRuleCall): Boolean = {
+    val join: FlinkLogicalTemporalTableJoin = call.rel(0)
+    val joinInfo = join.analyzeCondition
+
+    val (windowBounds, remainingPreds) = WindowJoinUtil.extractWindowBoundsFromPredicate(
+      joinInfo.getRemaining(join.getCluster.getRexBuilder),
+      join.getLeft.getRowType.getFieldCount,
+      join.getRowType,
+      join.getCluster.getRexBuilder,
+      TableConfig.DEFAULT)
+
+    windowBounds.isEmpty && join.getJoinType == JoinRelType.INNER
+  }
+
+  override def convert(rel: RelNode): RelNode = {
+    val temporalJoin = rel.asInstanceOf[FlinkLogicalTemporalTableJoin]
+    val traitSet: RelTraitSet = rel.getTraitSet.replace(FlinkConventions.DATASTREAM)
+    val left: RelNode = RelOptRule.convert(temporalJoin.getInput(0), FlinkConventions.DATASTREAM)
+    val right: RelNode = RelOptRule.convert(temporalJoin.getInput(1), FlinkConventions.DATASTREAM)
+    val joinInfo = temporalJoin.analyzeCondition
+    val leftRowSchema = new RowSchema(left.getRowType)
+    val rightRowSchema = new RowSchema(right.getRowType)
+
+    new DataStreamTemporalTableJoin(
+      rel.getCluster,
+      traitSet,
+      left,
+      right,
+      temporalJoin.getCondition,
+      joinInfo,
+      leftRowSchema,
+      rightRowSchema,
+      new RowSchema(rel.getRowType),
+      description)
+  }
+}
+
+object DataStreamTemporalTableJoinRule {
+  val INSTANCE: RelOptRule = new DataStreamTemporalTableJoinRule
+}
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/logical/LogicalCorrelateToTemporalTableJoinRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/logical/LogicalCorrelateToTemporalTableJoinRule.scala
new file mode 100644
index 0000000..cb666fa
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/logical/LogicalCorrelateToTemporalTableJoinRule.scala
@@ -0,0 +1,207 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.rules.logical
+
+import org.apache.calcite.plan.RelOptRule.{any, none, operand, some}
+import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall}
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.core.TableFunctionScan
+import org.apache.calcite.rel.logical.LogicalCorrelate
+import org.apache.calcite.rex._
+import org.apache.flink.table.api.{Table, Types, ValidationException}
+import org.apache.flink.table.calcite.FlinkTypeFactory.{isProctimeIndicatorType, isTimeIndicatorType}
+import org.apache.flink.table.expressions._
+import org.apache.flink.table.functions.TemporalTableFunction
+import org.apache.flink.table.functions.utils.TableSqlFunction
+import org.apache.flink.table.plan.logical.rel.LogicalTemporalTableJoin
+import org.apache.flink.table.plan.util.RexDefaultVisitor
+import org.apache.flink.util.Preconditions.checkState
+
+class LogicalCorrelateToTemporalTableJoinRule
+  extends RelOptRule(
+    operand(classOf[LogicalCorrelate],
+      some(
+        operand(classOf[RelNode], any()),
+        operand(classOf[TableFunctionScan], none()))),
+    "LogicalCorrelateToTemporalTableJoinRule") {
+
+  def extractNameFromTimeAttribute(timeAttribute: Expression): String = {
+    timeAttribute match {
+      case ResolvedFieldReference(name, _)
+        if timeAttribute.resultType == Types.LONG ||
+          timeAttribute.resultType == Types.SQL_TIMESTAMP ||
+          isTimeIndicatorType(timeAttribute.resultType) =>
+        name
+      case _ => throw new ValidationException(
+        s"Invalid timeAttribute [$timeAttribute] in TemporalTableFunction")
+    }
+  }
+
+  override def onMatch(call: RelOptRuleCall): Unit = {
+    val logicalCorrelate: LogicalCorrelate = call.rel(0)
+    val leftNode: RelNode = call.rel(1)
+    val rightTableFunctionScan: TableFunctionScan = call.rel(2)
+
+    val cluster = logicalCorrelate.getCluster
+
+    new GetTemporalTableFunctionCall(cluster.getRexBuilder, leftNode)
+      .visit(rightTableFunctionScan.getCall) match {
+      case None =>
+        // Do nothing and handle standard TableFunction
+      case Some(TemporalTableFunctionCall(rightTemporalTableFunction, leftTimeAttribute)) =>
+        // If TemporalTableFunction was found, rewrite LogicalCorrelate to TemporalJoin
+        val underlyingHistoryTable: Table = rightTemporalTableFunction.getUnderlyingHistoryTable
+        val relBuilder = this.relBuilderFactory.create(
+          cluster,
+          underlyingHistoryTable.relBuilder.getRelOptSchema)
+        val rexBuilder = cluster.getRexBuilder
+
+        val rightNode: RelNode = underlyingHistoryTable.logicalPlan.toRelNode(relBuilder)
+
+        val rightTimeIndicatorExpression = createRightExpression(
+          rexBuilder,
+          leftNode,
+          rightNode,
+          extractNameFromTimeAttribute(rightTemporalTableFunction.getTimeAttribute))
+
+        val rightPrimaryKeyExpression = createRightExpression(
+          rexBuilder,
+          leftNode,
+          rightNode,
+          rightTemporalTableFunction.getPrimaryKey)
+
+        relBuilder.push(
+          if (isProctimeIndicatorType(rightTemporalTableFunction.getTimeAttribute.resultType)) {
+            LogicalTemporalTableJoin.createProctime(
+              rexBuilder,
+              cluster,
+              logicalCorrelate.getTraitSet,
+              leftNode,
+              rightNode,
+              leftTimeAttribute,
+              rightPrimaryKeyExpression)
+          }
+          else {
+            LogicalTemporalTableJoin.createRowtime(
+              rexBuilder,
+              cluster,
+              logicalCorrelate.getTraitSet,
+              leftNode,
+              rightNode,
+              leftTimeAttribute,
+              rightTimeIndicatorExpression,
+              rightPrimaryKeyExpression)
+          })
+        call.transformTo(relBuilder.build())
+    }
+  }
+
+  private def createRightExpression(
+      rexBuilder: RexBuilder,
+      leftNode: RelNode,
+      rightNode: RelNode,
+      field: String): RexNode = {
+    val rightReferencesOffset = leftNode.getRowType.getFieldCount
+    val rightDataTypeField = rightNode.getRowType.getField(field, false, false)
+    rexBuilder.makeInputRef(
+      rightDataTypeField.getType, rightReferencesOffset + rightDataTypeField.getIndex)
+  }
+}
+
+object LogicalCorrelateToTemporalTableJoinRule {
+  val INSTANCE: RelOptRule = new LogicalCorrelateToTemporalTableJoinRule
+}
+
+/**
+  * Simple pojo class for extracted [[TemporalTableFunction]] with time attribute
+  * extracted from RexNode with [[TemporalTableFunction]] call.
+  */
+case class TemporalTableFunctionCall(
+    var temporalTableFunction: TemporalTableFunction,
+    var timeAttribute: RexNode) {
+}
+
+/**
+  * Find [[TemporalTableFunction]] call and run [[CorrelatedFieldAccessRemoval]] on it's operand.
+  */
+class GetTemporalTableFunctionCall(
+    var rexBuilder: RexBuilder,
+    var leftSide: RelNode)
+  extends RexVisitorImpl[TemporalTableFunctionCall](false) {
+
+  def visit(node: RexNode): Option[TemporalTableFunctionCall] = {
+    val result = node.accept(this)
+    if (result == null) {
+      return None
+    }
+    Some(result)
+  }
+
+  override def visitCall(rexCall: RexCall): TemporalTableFunctionCall = {
+    if (!rexCall.getOperator.isInstanceOf[TableSqlFunction]) {
+      return null
+    }
+    val tableFunction = rexCall.getOperator.asInstanceOf[TableSqlFunction]
+
+    if (!tableFunction.getTableFunction.isInstanceOf[TemporalTableFunction]) {
+      return null
+    }
+    val temporalTableFunction = tableFunction.getTableFunction.asInstanceOf[TemporalTableFunction]
+
+    checkState(
+      rexCall.getOperands.size().equals(1),
+      "TemporalTableFunction call [%s] must have exactly one argument",
+      rexCall)
+    val correlatedFieldAccessRemoval =
+      new CorrelatedFieldAccessRemoval(temporalTableFunction, rexBuilder, leftSide)
+    TemporalTableFunctionCall(
+      temporalTableFunction,
+      rexCall.getOperands.get(0).accept(correlatedFieldAccessRemoval))
+  }
+}
+
+/**
+  * This converts field accesses like `$cor0.o_rowtime` to valid input references
+  * for join condition context without `$cor` reference.
+  */
+class CorrelatedFieldAccessRemoval(
+    var temporalTableFunction: TemporalTableFunction,
+    var rexBuilder: RexBuilder,
+    var leftSide: RelNode) extends RexDefaultVisitor[RexNode] {
+
+  override def visitFieldAccess(fieldAccess: RexFieldAccess): RexNode = {
+    val leftIndex = leftSide.getRowType.getFieldList.indexOf(fieldAccess.getField)
+    if (leftIndex < 0) {
+      throw new IllegalStateException(
+        s"Failed to find reference to field [${fieldAccess.getField}] in node [$leftSide]")
+    }
+    rexBuilder.makeInputRef(leftSide, leftIndex)
+  }
+
+  override def visitInputRef(inputRef: RexInputRef): RexNode = {
+    inputRef
+  }
+
+  override def visitNode(rexNode: RexNode): RexNode = {
+    throw new ValidationException(
+      s"Unsupported argument [$rexNode] " +
+        s"in ${classOf[TemporalTableFunction].getSimpleName} call of " +
+        s"[${temporalTableFunction.getUnderlyingHistoryTable}] table")
+  }
+}
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexDefaultVisitor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexDefaultVisitor.scala
new file mode 100644
index 0000000..7c44616
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexDefaultVisitor.scala
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.util
+
+import org.apache.calcite.rex._
+
+/**
+  * Implementation of [[RexVisitor]] that redirects all calls into generic
+  * [[RexDefaultVisitor#visitNode(org.apache.calcite.rex.RexNode)]] method.
+  */
+abstract class RexDefaultVisitor[R] extends RexVisitor[R] {
+
+  override def visitFieldAccess(fieldAccess: RexFieldAccess): R =
+    visitNode(fieldAccess)
+
+  override def visitCall(call: RexCall): R =
+    visitNode(call)
+
+  override def visitInputRef(inputRef: RexInputRef): R =
+    visitNode(inputRef)
+
+  override def visitOver(over: RexOver): R =
+    visitNode(over)
+
+  override def visitCorrelVariable(correlVariable: RexCorrelVariable): R =
+    visitNode(correlVariable)
+
+  override def visitLocalRef(localRef: RexLocalRef): R =
+    visitNode(localRef)
+
+  override def visitDynamicParam(dynamicParam: RexDynamicParam): R =
+    visitNode(dynamicParam)
+
+  override def visitRangeRef(rangeRef: RexRangeRef): R =
+    visitNode(rangeRef)
+
+  override def visitTableInputRef(tableRef: RexTableInputRef): R =
+    visitNode(tableRef)
+
+  override def visitPatternFieldRef(patternFieldRef: RexPatternFieldRef): R =
+    visitNode(patternFieldRef)
+
+  override def visitSubQuery(subQuery: RexSubQuery): R =
+    visitNode(subQuery)
+
+  override def visitLiteral(literal: RexLiteral): R =
+    visitNode(literal)
+
+  def visitNode(rexNode: RexNode): R
+}
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/TemporalTableJoinTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/TemporalTableJoinTest.scala
new file mode 100644
index 0000000..cca733b
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/TemporalTableJoinTest.scala
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.api.batch.sql
+
+import java.sql.Timestamp
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.{TableException, ValidationException}
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.utils.TableTestUtil._
+import org.apache.flink.table.utils._
+import org.hamcrest.Matchers.startsWith
+import org.junit.Test
+
+class TemporalTableJoinTest extends TableTestBase {
+
+  val util: TableTestUtil = batchTestUtil()
+
+  val orders = util.addTable[(Long, String, Timestamp)](
+    "Orders", 'o_amount, 'o_currency, 'o_rowtime)
+
+  val ratesHistory = util.addTable[(String, Int, Timestamp)](
+    "RatesHistory", 'currency, 'rate, 'rowtime)
+
+  val rates = util.addFunction(
+    "Rates",
+    ratesHistory.createTemporalTableFunction('rowtime, 'currency))
+
+  @Test
+  def testSimpleJoin(): Unit = {
+    expectedException.expect(classOf[TableException])
+    expectedException.expectMessage("Cannot generate a valid execution plan for the given query")
+
+    val sqlQuery = "SELECT " +
+      "o_amount * rate as rate " +
+      "FROM Orders AS o, " +
+      "LATERAL TABLE (Rates(o_rowtime)) AS r " +
+      "WHERE currency = o_currency";
+
+    util.printSql(sqlQuery)
+  }
+
+  /**
+    * Test temporal table joins with more complicated query.
+    * Important thing here is that we have complex OR join condition
+    * and there are some columns that are not being used (are being pruned).
+    */
+  @Test(expected = classOf[TableException])
+  def testComplexJoin(): Unit = {
+    val util = batchTestUtil()
+    util.addTable[(String, Int)]("Table3", 't3_comment, 't3_secondary_key)
+    util.addTable[(Timestamp, String, Long, String, Int)](
+      "Orders", 'o_rowtime, 'o_comment, 'o_amount, 'o_currency, 'o_secondary_key)
+
+    val ratesHistory = util.addTable[(Timestamp, String, String, Int, Int)](
+      "RatesHistory", 'rowtime, 'comment, 'currency, 'rate, 'secondary_key)
+    val rates = ratesHistory.createTemporalTableFunction('rowtime, 'currency)
+    util.addFunction("Rates", rates)
+
+    val sqlQuery =
+      "SELECT * FROM " +
+        "(SELECT " +
+        "o_amount * rate as rate, " +
+        "secondary_key as secondary_key " +
+        "FROM Orders AS o, " +
+        "LATERAL TABLE (Rates(o_rowtime)) AS r " +
+        "WHERE currency = o_currency OR secondary_key = o_secondary_key), " +
+        "Table3 " +
+      "WHERE t3_secondary_key = secondary_key";
+
+    util.printSql(sqlQuery)
+  }
+
+  @Test
+  def testUncorrelatedJoin(): Unit = {
+    expectedException.expect(classOf[TableException])
+    expectedException.expectMessage(startsWith("Cannot generate a valid execution plan"))
+
+    val sqlQuery = "SELECT " +
+      "o_amount * rate as rate " +
+      "FROM Orders AS o, " +
+      "LATERAL TABLE (Rates(TIMESTAMP '2016-06-27 10:10:42.123')) AS r " +
+      "WHERE currency = o_currency";
+
+    util.printSql(sqlQuery)
+  }
+
+  @Test
+  def testTemporalTableFunctionScan(): Unit = {
+    expectedException.expect(classOf[TableException])
+    expectedException.expectMessage(startsWith("Cannot generate a valid execution plan"))
+
+    val sqlQuery = "SELECT * FROM LATERAL TABLE (Rates(TIMESTAMP '2016-06-27 10:10:42.123'))";
+
+    util.printSql(sqlQuery)
+  }
+}
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/TemporalTableJoinTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/TemporalTableJoinTest.scala
new file mode 100644
index 0000000..190eebe
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/TemporalTableJoinTest.scala
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.api.batch.table
+
+import java.sql.Timestamp
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.{TableException, ValidationException}
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.utils._
+import org.hamcrest.Matchers.startsWith
+import org.junit.Test
+
+class TemporalTableJoinTest extends TableTestBase {
+
+  val util: TableTestUtil = batchTestUtil()
+
+  val orders = util.addTable[(Long, String, Timestamp)](
+    "Orders", 'o_amount, 'o_currency, 'o_rowtime)
+
+  val ratesHistory = util.addTable[(String, Int, Timestamp)](
+    "RatesHistory", 'currency, 'rate, 'rowtime)
+
+  val rates = util.addFunction(
+    "Rates",
+    ratesHistory.createTemporalTableFunction('rowtime, 'currency))
+
+  @Test
+  def testSimpleJoin(): Unit = {
+    expectedException.expect(classOf[TableException])
+    expectedException.expectMessage("Cannot generate a valid execution plan for the given query")
+
+    val result = orders
+      .join(rates('o_rowtime), "currency = o_currency")
+      .select("o_amount * rate").as("rate")
+
+    util.printTable(result)
+  }
+
+  @Test
+  def testUncorrelatedJoin(): Unit = {
+    expectedException.expect(classOf[ValidationException])
+    expectedException.expectMessage(startsWith("Unsupported argument"))
+
+    val result = orders
+      .join(rates(java.sql.Timestamp.valueOf("2016-06-27 10:10:42.123")), "o_currency = currency")
+      .select("o_amount * rate")
+
+    util.printTable(result)
+  }
+
+  @Test
+  def testTemporalTableFunctionScan(): Unit = {
+    expectedException.expect(classOf[ValidationException])
+    expectedException.expectMessage(
+      "Cannot translate a query with an unbounded table function call.")
+
+    val result = rates(java.sql.Timestamp.valueOf("2016-06-27 10:10:42.123"))
+
+    util.printTable(result)
+  }
+}
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/TemporalTableJoinTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/TemporalTableJoinTest.scala
new file mode 100644
index 0000000..3c47f56
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/TemporalTableJoinTest.scala
@@ -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.flink.table.api.stream.sql
+
+import java.sql.Timestamp
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.TableException
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.api.stream.table.TemporalTableJoinTest._
+import org.apache.flink.table.utils._
+import org.hamcrest.Matchers.startsWith
+import org.junit.Test
+
+class TemporalTableJoinTest extends TableTestBase {
+
+  val util: TableTestUtil = streamTestUtil()
+
+  val orders = util.addTable[(Long, String, Timestamp)](
+    "Orders", 'o_amount, 'o_currency, 'o_rowtime.rowtime)
+
+  val ratesHistory = util.addTable[(String, Int, Timestamp)](
+    "RatesHistory", 'currency, 'rate, 'rowtime.rowtime)
+
+  val rates = util.addFunction(
+    "Rates",
+    ratesHistory.createTemporalTableFunction('rowtime, 'currency))
+
+  val proctimeOrders = util.addTable[(Long, String)](
+    "ProctimeOrders", 'o_amount, 'o_currency, 'o_proctime.proctime)
+
+  val proctimeRatesHistory = util.addTable[(String, Int)](
+    "ProctimeRatesHistory", 'currency, 'rate, 'proctime.proctime)
+
+  val proctimeRates = util.addFunction(
+    "ProctimeRates",
+    proctimeRatesHistory.createTemporalTableFunction('proctime, 'currency))
+
+  @Test
+  def testSimpleJoin(): Unit = {
+    val sqlQuery = "SELECT " +
+      "o_amount * rate as rate " +
+      "FROM Orders AS o, " +
+      "LATERAL TABLE (Rates(o.o_rowtime)) AS r " +
+      "WHERE currency = o_currency";
+
+    util.verifySql(sqlQuery, getExpectedSimpleJoinPlan())
+  }
+
+  @Test
+  def testSimpleProctimeJoin(): Unit = {
+    val sqlQuery = "SELECT " +
+      "o_amount * rate as rate " +
+      "FROM ProctimeOrders AS o, " +
+      "LATERAL TABLE (ProctimeRates(o.o_proctime)) AS r " +
+      "WHERE currency = o_currency";
+
+    util.verifySql(sqlQuery, getExpectedSimpleProctimeJoinPlan())
+  }
+
+  /**
+    * Test versioned joins with more complicated query.
+    * Important thing here is that we have complex OR join condition
+    * and there are some columns that are not being used (are being pruned).
+    */
+  @Test
+  def testComplexJoin(): Unit = {
+    val util = streamTestUtil()
+    util.addTable[(String, Int)]("Table3", 't3_comment, 't3_secondary_key)
+    util.addTable[(Timestamp, String, Long, String, Int)](
+      "Orders", 'o_rowtime.rowtime, 'o_comment, 'o_amount, 'o_currency, 'o_secondary_key)
+
+    val ratesHistory = util.addTable[(Timestamp, String, String, Int, Int)](
+      "RatesHistory", 'rowtime.rowtime, 'comment, 'currency, 'rate, 'secondary_key)
+    val rates = ratesHistory.createTemporalTableFunction('rowtime, 'currency)
+    util.addFunction("Rates", rates)
+
+    val sqlQuery =
+      "SELECT * FROM " +
+        "(SELECT " +
+        "o_amount * rate as rate, " +
+        "secondary_key as secondary_key " +
+        "FROM Orders AS o, " +
+        "LATERAL TABLE (Rates(o_rowtime)) AS r " +
+        "WHERE currency = o_currency OR secondary_key = o_secondary_key), " +
+        "Table3 " +
+      "WHERE t3_secondary_key = secondary_key";
+
+    util.verifySql(sqlQuery, getExpectedComplexJoinPlan())
+  }
+
+  @Test
+  def testUncorrelatedJoin(): Unit = {
+    expectedException.expect(classOf[TableException])
+    expectedException.expectMessage(startsWith("Cannot generate a valid execution plan"))
+
+    val sqlQuery = "SELECT " +
+      "o_amount * rate as rate " +
+      "FROM Orders AS o, " +
+      "LATERAL TABLE (Rates(TIMESTAMP '2016-06-27 10:10:42.123')) AS r " +
+      "WHERE currency = o_currency";
+
+    util.printSql(sqlQuery)
+  }
+
+  @Test
+  def testTemporalTableFunctionScan(): Unit = {
+    expectedException.expect(classOf[TableException])
+    expectedException.expectMessage(startsWith("Cannot generate a valid execution plan"))
+
+    val sqlQuery = "SELECT * FROM LATERAL TABLE (Rates(TIMESTAMP '2016-06-27 10:10:42.123'))";
+
+    util.printSql(sqlQuery)
+  }
+}
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/TemporalTableJoinTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/TemporalTableJoinTest.scala
index 0942dd3..82fa251 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/TemporalTableJoinTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/TemporalTableJoinTest.scala
@@ -15,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.flink.table.api.stream.table
 
 import java.sql.Timestamp
@@ -22,9 +23,13 @@ import java.sql.Timestamp
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.{TableSchema, ValidationException}
 import org.apache.flink.table.api.scala._
+import org.apache.flink.table.api.stream.table.TemporalTableJoinTest._
 import org.apache.flink.table.expressions.ResolvedFieldReference
 import org.apache.flink.table.functions.TemporalTableFunction
+import org.apache.flink.table.plan.logical.rel.LogicalTemporalTableJoin._
+import org.apache.flink.table.utils.TableTestUtil._
 import org.apache.flink.table.utils._
+import org.hamcrest.Matchers.startsWith
 import org.junit.Assert.{assertArrayEquals, assertEquals, assertTrue}
 import org.junit.Test
 
@@ -32,6 +37,9 @@ class TemporalTableJoinTest extends TableTestBase {
 
   val util: TableTestUtil = streamTestUtil()
 
+  val orders = util.addTable[(Long, String, Timestamp)](
+    "Orders", 'o_amount, 'o_currency, 'o_rowtime.rowtime)
+
   val ratesHistory = util.addTable[(String, Int, Timestamp)](
     "RatesHistory", 'currency, 'rate, 'rowtime.rowtime)
 
@@ -39,6 +47,91 @@ class TemporalTableJoinTest extends TableTestBase {
     "Rates",
     ratesHistory.createTemporalTableFunction('rowtime, 'currency))
 
+  val proctimeOrders = util.addTable[(Long, String)](
+    "ProctimeOrders", 'o_amount, 'o_currency, 'o_proctime.proctime)
+
+  val proctimeRatesHistory = util.addTable[(String, Int)](
+    "ProctimeRatesHistory", 'currency, 'rate, 'proctime.proctime)
+
+  val proctimeRates = proctimeRatesHistory.createTemporalTableFunction('proctime, 'currency)
+
+  @Test
+  def testSimpleJoin(): Unit = {
+    val result = orders
+      .join(rates('o_rowtime), "currency = o_currency")
+      .select("o_amount * rate").as("rate")
+
+    util.verifyTable(result, getExpectedSimpleJoinPlan())
+  }
+
+  @Test
+  def testSimpleProctimeJoin(): Unit = {
+    val result = proctimeOrders
+      .join(proctimeRates('o_proctime), "currency = o_currency")
+      .select("o_amount * rate").as("rate")
+
+    util.verifyTable(result, getExpectedSimpleProctimeJoinPlan())
+  }
+
+  /**
+    * Test versioned joins with more complicated query.
+    * Important thing here is that we have complex OR join condition
+    * and there are some columns that are not being used (are being pruned).
+    */
+  @Test
+  def testComplexJoin(): Unit = {
+    val util = streamTestUtil()
+    val thirdTable = util.addTable[(String, Int)]("ThirdTable", 't3_comment, 't3_secondary_key)
+    val orders = util.addTable[(Timestamp, String, Long, String, Int)](
+      "Orders", 'o_rowtime.rowtime, 'o_comment, 'o_amount, 'o_currency, 'o_secondary_key)
+
+    val ratesHistory = util.addTable[(Timestamp, String, String, Int, Int)](
+      "RatesHistory", 'rowtime.rowtime, 'comment, 'currency, 'rate, 'secondary_key)
+    val rates = ratesHistory.createTemporalTableFunction('rowtime, 'currency)
+    util.addFunction("Rates", rates)
+
+    val result = orders
+      .join(rates('o_rowtime))
+      .filter('currency === 'o_currency || 'secondary_key === 'o_secondary_key)
+      .select('o_amount * 'rate, 'secondary_key).as('rate, 'secondary_key)
+      .join(thirdTable, 't3_secondary_key === 'secondary_key)
+
+    util.verifyTable(result, getExpectedComplexJoinPlan())
+  }
+
+  @Test
+  def testTemporalTableFunctionOnTopOfQuery(): Unit = {
+    val filteredRatesHistory = ratesHistory
+      .filter('rate > 100)
+      .select('currency, 'rate * 2, 'rowtime)
+      .as('currency, 'rate, 'rowtime)
+
+    val filteredRates = util.addFunction(
+      "FilteredRates",
+      filteredRatesHistory.createTemporalTableFunction('rowtime, 'currency))
+
+    val result = orders
+      .join(filteredRates('o_rowtime), "currency = o_currency")
+      .select("o_amount * rate")
+      .as('rate)
+
+    util.verifyTable(result, getExpectedTemporalTableFunctionOnTopOfQueryPlan())
+  }
+
+  @Test
+  def testUncorrelatedJoin(): Unit = {
+    expectedException.expect(classOf[ValidationException])
+    expectedException.expectMessage(startsWith("Unsupported argument"))
+
+    val result = orders
+      .join(rates(
+        java.sql.Timestamp.valueOf("2016-06-27 10:10:42.123")),
+        "o_currency = currency")
+      .select("o_amount * rate")
+
+    util.printTable(result)
+  }
+
   @Test
   def testTemporalTableFunctionScan(): Unit = {
     expectedException.expect(classOf[ValidationException])
@@ -51,11 +144,7 @@ class TemporalTableJoinTest extends TableTestBase {
 
   @Test
   def testProcessingTimeIndicatorVersion(): Unit = {
-    val util: TableTestUtil = streamTestUtil()
-    val ratesHistory = util.addTable[(String, Int)](
-      "RatesHistory", 'currency, 'rate, 'proctime.proctime)
-    val rates = ratesHistory.createTemporalTableFunction('proctime, 'currency)
-    assertRatesFunction(ratesHistory.getSchema, rates, true)
+    assertRatesFunction(proctimeRatesHistory.getSchema, proctimeRates, true)
   }
 
   @Test
@@ -82,3 +171,106 @@ class TemporalTableJoinTest extends TableTestBase {
   }
 }
 
+object TemporalTableJoinTest {
+  def getExpectedSimpleJoinPlan(): String = {
+    unaryNode(
+      "DataStreamCalc",
+      binaryNode(
+        "DataStreamTemporalTableJoin",
+        streamTableNode(0),
+        streamTableNode(1),
+        term("where",
+          "AND(" +
+            s"${TEMPORAL_JOIN_CONDITION.getName}(o_rowtime, rowtime, currency), " +
+            "=(currency, o_currency))"),
+        term("join", "o_amount", "o_currency", "o_rowtime", "currency", "rate", "rowtime"),
+        term("joinType", "InnerJoin")
+      ),
+      term("select", "*(o_amount, rate) AS rate")
+    )
+  }
+
+  def getExpectedSimpleProctimeJoinPlan(): String = {
+    unaryNode(
+      "DataStreamCalc",
+      binaryNode(
+        "DataStreamTemporalTableJoin",
+        streamTableNode(2),
+        unaryNode(
+          "DataStreamCalc",
+          streamTableNode(3),
+          term("select", "currency, rate")),
+        term("where",
+          "AND(" +
+            s"${TEMPORAL_JOIN_CONDITION.getName}(o_proctime, currency), " +
+            "=(currency, o_currency))"),
+        term("join", "o_amount", "o_currency", "o_proctime", "currency", "rate"),
+        term("joinType", "InnerJoin")
+      ),
+      term("select", "*(o_amount, rate) AS rate")
+    )
+  }
+
+  def getExpectedComplexJoinPlan(): String = {
+    binaryNode(
+      "DataStreamJoin",
+      unaryNode(
+        "DataStreamCalc",
+        binaryNode(
+          "DataStreamTemporalTableJoin",
+          unaryNode(
+            "DataStreamCalc",
+            streamTableNode(1),
+            term("select", "o_rowtime, o_amount, o_currency, o_secondary_key")
+          ),
+          unaryNode(
+            "DataStreamCalc",
+            streamTableNode(2),
+            term("select", "rowtime, currency, rate, secondary_key")
+          ),
+          term("where",
+            "AND(" +
+              s"${TEMPORAL_JOIN_CONDITION.getName}(o_rowtime, rowtime, currency), " +
+              "OR(=(currency, o_currency), =(secondary_key, o_secondary_key)))"),
+          term("join",
+            "o_rowtime",
+            "o_amount",
+            "o_currency",
+            "o_secondary_key",
+            "rowtime",
+            "currency",
+            "rate",
+            "secondary_key"),
+          term("joinType", "InnerJoin")
+        ),
+        term("select", "*(o_amount, rate) AS rate", "secondary_key")
+      ),
+      streamTableNode(0),
+      term("where", "=(t3_secondary_key, secondary_key)"),
+      term("join", "rate, secondary_key, t3_comment, t3_secondary_key"),
+      term("joinType", "InnerJoin")
+    )
+  }
+
+  def getExpectedTemporalTableFunctionOnTopOfQueryPlan(): String = {
+    unaryNode(
+      "DataStreamCalc",
+      binaryNode(
+        "DataStreamTemporalTableJoin",
+        streamTableNode(0),
+        unaryNode(
+          "DataStreamCalc",
+          streamTableNode(1),
+          term("select", "currency", "*(rate, 2) AS rate", "rowtime"),
+          term("where", ">(rate, 100)")),
+        term("where",
+          "AND(" +
+            s"${TEMPORAL_JOIN_CONDITION.getName}(o_rowtime, rowtime, currency), " +
+            "=(currency, o_currency))"),
+        term("join", "o_amount", "o_currency", "o_rowtime", "currency", "rate", "rowtime"),
+        term("joinType", "InnerJoin")
+      ),
+      term("select", "*(o_amount, rate) AS rate")
+    )
+  }
+}
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/validation/TemporalTableJoinValidationTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/validation/TemporalTableJoinValidationTest.scala
index 71b1585..ab282ec 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/validation/TemporalTableJoinValidationTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/validation/TemporalTableJoinValidationTest.scala
@@ -52,5 +52,3 @@ class TemporalTableJoinValidationTest extends TableTestBase {
     ratesHistory.createTemporalTableFunction("rowtime", "foobar")
   }
 }
-
-
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/TimeIndicatorConversionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/TimeIndicatorConversionTest.scala
index 6a77f12..1706169 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/TimeIndicatorConversionTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/TimeIndicatorConversionTest.scala
@@ -394,6 +394,116 @@ class TimeIndicatorConversionTest extends TableTestBase {
     util.verifyTable(result, expected)
   }
 
+  @Test
+  def testMaterializeRightSideOfTemporalTableJoin(): Unit = {
+    val util = streamTestUtil()
+
+    val proctimeOrders = util.addTable[(Long, String)](
+      "ProctimeOrders", 'o_amount, 'o_currency, 'o_proctime.proctime)
+
+    val proctimeRatesHistory = util.addTable[(String, Int)](
+      "ProctimeRatesHistory", 'currency, 'rate, 'proctime.proctime)
+
+    val proctimeRates = proctimeRatesHistory.createTemporalTableFunction('proctime, 'currency)
+
+    val result = proctimeOrders
+      .join(proctimeRates('o_proctime), "currency = o_currency")
+      .select("o_amount * rate, currency, proctime").as("converted_amount")
+      .window(Tumble over 1.second on 'proctime as 'w)
+      .groupBy('w, 'currency)
+      .select('converted_amount.sum)
+
+    val expected =
+      unaryAnyNode(
+        unaryAnyNode(
+          unaryNode(
+            "DataStreamCalc",
+            anySubtree(),
+            term(
+              "select",
+              "*(o_amount, rate) AS converted_amount",
+              "currency",
+              "PROCTIME(proctime) AS proctime")
+          )
+        )
+      )
+
+    util.verifyTable(result, expected)
+  }
+
+  @Test
+  def testDoNotMaterializeLeftSideOfTemporalTableJoin(): Unit = {
+    val util = streamTestUtil()
+
+    val proctimeOrders = util.addTable[(Long, String)](
+      "ProctimeOrders", 'o_amount, 'o_currency, 'o_proctime.proctime)
+
+    val proctimeRatesHistory = util.addTable[(String, Int)](
+      "ProctimeRatesHistory", 'currency, 'rate, 'proctime.proctime)
+
+    val proctimeRates = proctimeRatesHistory.createTemporalTableFunction('proctime, 'currency)
+
+    val result = proctimeOrders
+      .join(proctimeRates('o_proctime), "currency = o_currency")
+      .select("o_amount * rate, currency, o_proctime").as("converted_amount")
+      .window(Tumble over 1.second on 'o_proctime as 'w)
+      .groupBy('w, 'currency)
+      .select('converted_amount.sum)
+
+    val expected =
+      unaryAnyNode(
+        unaryAnyNode(
+          unaryNode(
+            "DataStreamCalc",
+            anySubtree(),
+            term(
+              "select",
+              "*(o_amount, rate) AS converted_amount",
+              "currency",
+              "o_proctime")
+          )
+        )
+      )
+
+    util.verifyTable(result, expected)
+  }
+
+  @Test
+  def testMaterializeLeftRowtimeWithProcessingTimeTemporalTableJoin(): Unit = {
+    val util = streamTestUtil()
+
+    val proctimeOrders = util.addTable[(Long, String)](
+      "ProctimeOrders", 'o_amount, 'o_currency, 'o_proctime.proctime, 'o_rowtime.rowtime)
+
+    val proctimeRatesHistory = util.addTable[(String, Int)](
+      "ProctimeRatesHistory", 'currency, 'rate, 'proctime.proctime)
+
+    val proctimeRates = proctimeRatesHistory.createTemporalTableFunction('proctime, 'currency)
+
+    val result = proctimeOrders
+      .join(proctimeRates('o_proctime), "currency = o_currency")
+      .select("o_amount * rate, currency, o_proctime, o_rowtime").as("converted_amount")
+      .window(Tumble over 1.second on 'o_rowtime as 'w)
+      .groupBy('w, 'currency)
+      .select('converted_amount.sum)
+
+    val expected =
+      unaryAnyNode(
+        unaryAnyNode(
+          unaryNode(
+            "DataStreamCalc",
+            anySubtree(),
+            term(
+              "select",
+              "*(o_amount, rate) AS converted_amount",
+              "currency",
+              "CAST(o_rowtime) AS o_rowtime")
+          )
+        )
+      )
+
+    util.verifyTable(result, expected)
+  }
 }
 
 object TimeIndicatorConversionTest {
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala
index b987e34..42b16e9 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala
@@ -33,10 +33,12 @@ import org.apache.flink.table.api.{Table, TableEnvironment, TableSchema}
 import org.apache.flink.table.expressions.Expression
 import org.apache.flink.table.functions.{AggregateFunction, ScalarFunction, TableFunction}
 import org.junit.Assert.assertEquals
-import org.junit.Rule
+import org.junit.{ComparisonFailure, Rule}
 import org.junit.rules.ExpectedException
 import org.mockito.Mockito.{mock, when}
 
+import util.control.Breaks._
+
 /**
   * Test base for testing Table API / SQL plans.
   */
@@ -99,17 +101,48 @@ abstract class TableTestUtil {
     // we remove the charset for testing because it
     // depends on the native machine (Little/Big Endian)
     val actualNoCharset = actual.replace("_UTF-16LE'", "'").replace("_UTF-16BE'", "'")
-    assertEquals(
-      expected.split("\n").map(_.trim).mkString("\n"),
-      actualNoCharset.split("\n").map(_.trim).mkString("\n"))
+
+    val expectedLines = expected.split("\n").map(_.trim)
+    val actualLines = actualNoCharset.split("\n").map(_.trim)
+
+    val expectedMessage = expectedLines.mkString("\n")
+    val actualMessage = actualLines.mkString("\n")
+
+    breakable {
+      for ((expectedLine, actualLine) <- expectedLines.zip(actualLines)) {
+        if (expectedLine == TableTestUtil.ANY_NODE) {
+        }
+        else if (expectedLine == TableTestUtil.ANY_SUBTREE) {
+          break
+        }
+        else if (expectedLine != actualLine) {
+          throw new ComparisonFailure(null, expectedMessage, actualMessage)
+        }
+      }
+    }
   }
+
+  def explain(resultTable: Table): String
 }
 
 object TableTestUtil {
+  val ANY_NODE = "%ANY_NODE%"
+
+  val ANY_SUBTREE = "%ANY_SUBTREE%"
 
   // this methods are currently just for simplifying string construction,
   // we could replace it with logic later
 
+  def unaryAnyNode(input: String): String = {
+    s"""$ANY_NODE
+       |$input
+       |""".stripMargin.stripLineEnd
+  }
+
+  def anySubtree(): String = {
+    ANY_SUBTREE
+  }
+
   def unaryNode(node: String, input: String, term: String*): String = {
     s"""$node(${term.mkString(", ")})
        |$input
@@ -230,6 +263,10 @@ case class BatchTableTestUtil() extends TableTestUtil {
   def printSql(query: String): Unit = {
     printTable(tableEnv.sqlQuery(query))
   }
+
+  def explain(resultTable: Table): String = {
+    tableEnv.explain(resultTable)
+  }
 }
 
 case class StreamTableTestUtil() extends TableTestUtil {
@@ -318,6 +355,10 @@ case class StreamTableTestUtil() extends TableTestUtil {
   def printSql(query: String): Unit = {
     printTable(tableEnv.sqlQuery(query))
   }
+
+  def explain(resultTable: Table): String = {
+    tableEnv.explain(resultTable)
+  }
 }
 
 class EmptySource[T]() extends SourceFunction[T] {


[flink] 07/11: [hotfix][table] Extract DataStreamJoinToCoProcessTranslator

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

pnowojski pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 6ff0bb6c0f23f3810a1265546d217054d5d06417
Author: Piotr Nowojski <pi...@gmail.com>
AuthorDate: Tue Jul 17 18:59:31 2018 +0200

    [hotfix][table] Extract DataStreamJoinToCoProcessTranslator
---
 .../plan/nodes/datastream/DataStreamJoin.scala     | 147 +++++---------------
 .../DataStreamJoinToCoProcessTranslator.scala      | 154 +++++++++++++++++++++
 2 files changed, 190 insertions(+), 111 deletions(-)

diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamJoin.scala
index 1e2311f..d54fd78 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamJoin.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamJoin.scala
@@ -23,19 +23,13 @@ import org.apache.calcite.rel.`type`.RelDataType
 import org.apache.calcite.rel.core.{JoinInfo, JoinRelType}
 import org.apache.calcite.rel.{BiRel, RelNode, RelWriter}
 import org.apache.calcite.rex.RexNode
-import org.apache.flink.api.common.functions.FlatJoinFunction
 import org.apache.flink.streaming.api.datastream.DataStream
 import org.apache.flink.table.api.{StreamQueryConfig, StreamTableEnvironment, TableException}
-import org.apache.flink.table.codegen.FunctionCodeGenerator
 import org.apache.flink.table.plan.nodes.CommonJoin
 import org.apache.flink.table.plan.schema.RowSchema
-import org.apache.flink.table.runtime.CRowKeySelector
-import org.apache.flink.table.runtime.join._
 import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
-import org.apache.flink.types.Row
 
 import scala.collection.JavaConversions._
-import scala.collection.mutable.ArrayBuffer
 
 /**
   * RelNode for a non-windowed stream join.
@@ -103,27 +97,42 @@ class DataStreamJoin(
       tableEnv: StreamTableEnvironment,
       queryConfig: StreamQueryConfig): DataStream[CRow] = {
 
-    val config = tableEnv.getConfig
-    val returnType = schema.typeInfo
-    val keyPairs = joinInfo.pairs().toList
+    validateKeyTypes()
 
-    // get the equality keys
-    val leftKeys = ArrayBuffer.empty[Int]
-    val rightKeys = ArrayBuffer.empty[Int]
+    val leftDataStream =
+      left.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
+    val rightDataStream =
+      right.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
+
+    val connectOperator = leftDataStream.connect(rightDataStream)
+
+    val joinTranslator = createTranslator(tableEnv)
+
+    val joinOpName = joinToString(getRowType, joinCondition, joinType, getExpressionString)
+    val coProcessFunction = joinTranslator.getCoProcessFunction(
+      joinType,
+      schema.fieldNames,
+      ruleDescription,
+      queryConfig)
+    connectOperator
+      .keyBy(
+        joinTranslator.getLeftKeySelector(),
+        joinTranslator.getRightKeySelector())
+      .process(coProcessFunction)
+      .name(joinOpName)
+      .returns(CRowTypeInfo(schema.typeInfo))
+  }
 
+  private def validateKeyTypes(): Unit = {
     // at least one equality expression
     val leftFields = left.getRowType.getFieldList
     val rightFields = right.getRowType.getFieldList
 
-    keyPairs.foreach(pair => {
+    joinInfo.pairs().toList.foreach(pair => {
       val leftKeyType = leftFields.get(pair.source).getType.getSqlTypeName
       val rightKeyType = rightFields.get(pair.target).getType.getSqlTypeName
       // check if keys are compatible
-      if (leftKeyType == rightKeyType) {
-        // add key pair
-        leftKeys.add(pair.source)
-        rightKeys.add(pair.target)
-      } else {
+      if (leftKeyType != rightKeyType) {
         throw TableException(
           "Equality join predicate on incompatible types.\n" +
             s"\tLeft: $left,\n" +
@@ -133,100 +142,16 @@ class DataStreamJoin(
         )
       }
     })
+  }
 
-    val leftDataStream =
-      left.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
-    val rightDataStream =
-      right.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
-
-    val connectOperator = leftDataStream.connect(rightDataStream)
-    // input must not be nullable, because the runtime join function will make sure
-    // the code-generated function won't process null inputs
-    val generator = new FunctionCodeGenerator(
-      config,
-      nullableInput = false,
-      leftSchema.typeInfo,
-      Some(rightSchema.typeInfo))
-    val conversion = generator.generateConverterResultExpression(
+  protected def createTranslator(
+      tableEnv: StreamTableEnvironment): DataStreamJoinToCoProcessTranslator = {
+    new DataStreamJoinToCoProcessTranslator(
+      tableEnv.getConfig,
       schema.typeInfo,
-      schema.fieldNames)
-
-    val body = if (joinInfo.isEqui) {
-      // only equality condition
-      s"""
-         |${conversion.code}
-         |${generator.collectorTerm}.collect(${conversion.resultTerm});
-         |""".stripMargin
-    } else {
-      val nonEquiPredicates = joinInfo.getRemaining(this.cluster.getRexBuilder)
-      val condition = generator.generateExpression(nonEquiPredicates)
-      s"""
-         |${condition.code}
-         |if (${condition.resultTerm}) {
-         |  ${conversion.code}
-         |  ${generator.collectorTerm}.collect(${conversion.resultTerm});
-         |}
-         |""".stripMargin
-    }
-
-    val genFunction = generator.generateFunction(
-      ruleDescription,
-      classOf[FlatJoinFunction[Row, Row, Row]],
-      body,
-      returnType)
-
-    val coMapFun = joinType match {
-      case JoinRelType.INNER =>
-        new NonWindowInnerJoin(
-          leftSchema.typeInfo,
-          rightSchema.typeInfo,
-          CRowTypeInfo(returnType),
-          genFunction.name,
-          genFunction.code,
-          queryConfig)
-      case JoinRelType.LEFT | JoinRelType.RIGHT if joinInfo.isEqui =>
-        new NonWindowLeftRightJoin(
-          leftSchema.typeInfo,
-          rightSchema.typeInfo,
-          CRowTypeInfo(returnType),
-          genFunction.name,
-          genFunction.code,
-          joinType == JoinRelType.LEFT,
-          queryConfig)
-      case JoinRelType.LEFT | JoinRelType.RIGHT =>
-        new NonWindowLeftRightJoinWithNonEquiPredicates(
-          leftSchema.typeInfo,
-          rightSchema.typeInfo,
-          CRowTypeInfo(returnType),
-          genFunction.name,
-          genFunction.code,
-          joinType == JoinRelType.LEFT,
-          queryConfig)
-      case JoinRelType.FULL if joinInfo.isEqui =>
-        new NonWindowFullJoin(
-          leftSchema.typeInfo,
-          rightSchema.typeInfo,
-          CRowTypeInfo(returnType),
-          genFunction.name,
-          genFunction.code,
-          queryConfig)
-      case JoinRelType.FULL =>
-        new NonWindowFullJoinWithNonEquiPredicates(
-          leftSchema.typeInfo,
-          rightSchema.typeInfo,
-          CRowTypeInfo(returnType),
-          genFunction.name,
-          genFunction.code,
-          queryConfig)
-    }
-
-    val joinOpName = joinToString(getRowType, joinCondition, joinType, getExpressionString)
-    connectOperator
-      .keyBy(
-        new CRowKeySelector(leftKeys.toArray, leftSchema.projectedTypeInfo(leftKeys.toArray)),
-        new CRowKeySelector(rightKeys.toArray, rightSchema.projectedTypeInfo(rightKeys.toArray)))
-      .process(coMapFun)
-      .name(joinOpName)
-      .returns(CRowTypeInfo(returnType))
+      leftSchema,
+      rightSchema,
+      joinInfo,
+      cluster.getRexBuilder)
   }
 }
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamJoinToCoProcessTranslator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamJoinToCoProcessTranslator.scala
new file mode 100644
index 0000000..5a8d1a4
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamJoinToCoProcessTranslator.scala
@@ -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.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.rel.core.{JoinInfo, JoinRelType}
+import org.apache.calcite.rex.{RexBuilder, RexNode}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.api.{StreamQueryConfig, TableConfig}
+import org.apache.flink.table.codegen.{FunctionCodeGenerator, GeneratedFunction}
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.table.runtime.CRowKeySelector
+import org.apache.flink.table.runtime.join._
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+
+class DataStreamJoinToCoProcessTranslator(
+    config: TableConfig,
+    returnType: TypeInformation[Row],
+    leftSchema: RowSchema,
+    rightSchema: RowSchema,
+    joinInfo: JoinInfo,
+    rexBuilder: RexBuilder) {
+
+  val nonEquiJoinPredicates: Option[RexNode] = if (joinInfo.isEqui) {
+    None
+  }
+  else {
+    Some(joinInfo.getRemaining(rexBuilder))
+  }
+
+  def getLeftKeySelector(): CRowKeySelector = {
+    new CRowKeySelector(
+      joinInfo.leftKeys.toIntArray,
+      leftSchema.projectedTypeInfo(joinInfo.leftKeys.toIntArray))
+  }
+
+  def getRightKeySelector(): CRowKeySelector = {
+    new CRowKeySelector(
+      joinInfo.rightKeys.toIntArray,
+      rightSchema.projectedTypeInfo(joinInfo.rightKeys.toIntArray))
+  }
+
+  def getCoProcessFunction(
+      joinType: JoinRelType,
+      returnFieldNames: Seq[String],
+      ruleDescription: String,
+      queryConfig: StreamQueryConfig): CoProcessFunction[CRow, CRow, CRow] = {
+    // input must not be nullable, because the runtime join function will make sure
+    // the code-generated function won't process null inputs
+    val generator = new FunctionCodeGenerator(
+      config,
+      nullableInput = false,
+      leftSchema.typeInfo,
+      Some(rightSchema.typeInfo))
+    val conversion = generator.generateConverterResultExpression(
+      returnType,
+      returnFieldNames)
+
+    val body = if (nonEquiJoinPredicates.isEmpty) {
+      // only equality condition
+      s"""
+         |${conversion.code}
+         |${generator.collectorTerm}.collect(${conversion.resultTerm});
+         |""".stripMargin
+    } else {
+      val condition = generator.generateExpression(nonEquiJoinPredicates.get)
+      s"""
+         |${condition.code}
+         |if (${condition.resultTerm}) {
+         |  ${conversion.code}
+         |  ${generator.collectorTerm}.collect(${conversion.resultTerm});
+         |}
+         |""".stripMargin
+    }
+
+    val genFunction = generator.generateFunction(
+      ruleDescription,
+      classOf[FlatJoinFunction[Row, Row, Row]],
+      body,
+      returnType)
+
+    createCoProcessFunction(joinType, queryConfig, genFunction)
+  }
+
+  protected def createCoProcessFunction(
+    joinType: JoinRelType,
+    queryConfig: StreamQueryConfig,
+    genFunction: GeneratedFunction[FlatJoinFunction[Row, Row, Row], Row])
+    : CoProcessFunction[CRow, CRow, CRow] = {
+
+    joinType match {
+      case JoinRelType.INNER =>
+        new NonWindowInnerJoin(
+          leftSchema.typeInfo,
+          rightSchema.typeInfo,
+          CRowTypeInfo(returnType),
+          genFunction.name,
+          genFunction.code,
+          queryConfig)
+      case JoinRelType.LEFT | JoinRelType.RIGHT if joinInfo.isEqui =>
+        new NonWindowLeftRightJoin(
+          leftSchema.typeInfo,
+          rightSchema.typeInfo,
+          CRowTypeInfo(returnType),
+          genFunction.name,
+          genFunction.code,
+          joinType == JoinRelType.LEFT,
+          queryConfig)
+      case JoinRelType.LEFT | JoinRelType.RIGHT =>
+        new NonWindowLeftRightJoinWithNonEquiPredicates(
+          leftSchema.typeInfo,
+          rightSchema.typeInfo,
+          CRowTypeInfo(returnType),
+          genFunction.name,
+          genFunction.code,
+          joinType == JoinRelType.LEFT,
+          queryConfig)
+      case JoinRelType.FULL if joinInfo.isEqui =>
+        new NonWindowFullJoin(
+          leftSchema.typeInfo,
+          rightSchema.typeInfo,
+          CRowTypeInfo(returnType),
+          genFunction.name,
+          genFunction.code,
+          queryConfig)
+      case JoinRelType.FULL =>
+        new NonWindowFullJoinWithNonEquiPredicates(
+          leftSchema.typeInfo,
+          rightSchema.typeInfo,
+          CRowTypeInfo(returnType),
+          genFunction.name,
+          genFunction.code,
+          queryConfig)
+    }
+  }
+}


[flink] 09/11: [hotfix][table] Add convienient constructors for CRow

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

pnowojski pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 61c1161860e6f500a5c11b08bc4f17231a2b79e0
Author: Piotr Nowojski <pi...@gmail.com>
AuthorDate: Fri Jul 20 12:14:44 2018 +0200

    [hotfix][table] Add convienient constructors for CRow
---
 .../apache/flink/table/runtime/types/CRow.scala    |   8 +-
 .../table/runtime/harness/JoinHarnessTest.scala    | 648 ++++++++++-----------
 .../runtime/harness/NonWindowHarnessTest.scala     |  88 +--
 .../runtime/harness/OverWindowHarnessTest.scala    | 444 +++++++-------
 4 files changed, 596 insertions(+), 592 deletions(-)

diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/types/CRow.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/types/CRow.scala
index 25ec8c4..7e6f9b2 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/types/CRow.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/types/CRow.scala
@@ -49,7 +49,11 @@ object CRow {
     new CRow()
   }
 
-  def apply(row: Row, change: Boolean): CRow = {
-    new CRow(row, change)
+  def apply(values: Any*): CRow = {
+    new CRow(Row.of(values.map(_.asInstanceOf[Object]): _*), true)
+  }
+
+  def apply(change: Boolean, values: Any*): CRow = {
+    new CRow(Row.of(values.map(_.asInstanceOf[Object]): _*), change)
   }
 }
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala
index c499a9d..86133a0 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala
@@ -167,28 +167,28 @@ class JoinHarnessTest extends HarnessTestBase {
 
     testHarness.setProcessingTime(1)
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1L: JLong, "1a1"), change = true), 1))
+      CRow(1L: JLong, "1a1"), 1))
     assertEquals(1, testHarness.numProcessingTimeTimers())
     testHarness.setProcessingTime(2)
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(2L: JLong, "2a2"), change = true), 2))
+      CRow(2L: JLong, "2a2"), 2))
 
     // timers for key = 1 and key = 2
     assertEquals(2, testHarness.numProcessingTimeTimers())
 
     testHarness.setProcessingTime(3)
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1L: JLong, "1a3"), change = true), 3))
+      CRow(1L: JLong, "1a3"), 3))
     assertEquals(4, testHarness.numKeyedStateEntries())
 
     // The number of timers won't increase.
     assertEquals(2, testHarness.numProcessingTimeTimers())
 
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1L: JLong, "1b3"), change = true), 3))
+      CRow(1L: JLong, "1b3"), 3))
     testHarness.setProcessingTime(4)
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(2L: JLong, "2b4"), change = true), 4))
+      CRow(2L: JLong, "2b4"), 4))
 
     // The number of states should be doubled.
     assertEquals(8, testHarness.numKeyedStateEntries())
@@ -198,38 +198,38 @@ class JoinHarnessTest extends HarnessTestBase {
     // The left row (key = 1) with timestamp = 1 will be eagerly removed here.
     testHarness.setProcessingTime(13)
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1L: JLong, "1b13"), change = true), 13))
+      CRow(1L: JLong, "1b13"), 13))
 
     // Test for +20 boundary (13 + 20 = 33).
     testHarness.setProcessingTime(33)
     assertEquals(4, testHarness.numKeyedStateEntries())
     assertEquals(2, testHarness.numProcessingTimeTimers())
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1L: JLong, "1a33"), change = true), 33))
+      CRow(1L: JLong, "1a33"), 33))
 
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(2L: JLong, "2a33"), change = true), 33))
+      CRow(2L: JLong, "2a33"), 33))
 
     // The left row (key = 2) with timestamp = 2 will be eagerly removed here.
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(2L: JLong, "2b33"), change = true), 33))
+      CRow(2L: JLong, "2b33"), 33))
 
     val result = testHarness.getOutput
 
     val expectedOutput = new ConcurrentLinkedQueue[Object]()
 
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1L: JLong, "1a1", 1L: JLong, "1b3"), change = true), 3))
+      CRow(1L: JLong, "1a1", 1L: JLong, "1b3"), 3))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1L: JLong, "1a3", 1L: JLong, "1b3"), change = true), 3))
+      CRow(1L: JLong, "1a3", 1L: JLong, "1b3"), 3))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2L: JLong, "2a2", 2L: JLong, "2b4"), change = true), 4))
+      CRow(2L: JLong, "2a2", 2L: JLong, "2b4"), 4))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1L: JLong, "1a3", 1L: JLong, "1b13"), change = true), 13))
+      CRow(1L: JLong, "1a3", 1L: JLong, "1b13"), 13))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1L: JLong, "1a33", 1L: JLong, "1b13"), change = true), 33))
+      CRow(1L: JLong, "1a33", 1L: JLong, "1b13"), 33))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2L: JLong, "2a33", 2L: JLong, "2b33"), change = true), 33))
+      CRow(2L: JLong, "2a33", 2L: JLong, "2b33"), 33))
 
     verify(expectedOutput, result, new RowResultSortComparator())
 
@@ -257,19 +257,19 @@ class JoinHarnessTest extends HarnessTestBase {
 
     testHarness.setProcessingTime(1)
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1L: JLong, "1a1"), change = true), 1))
+      CRow(1L: JLong, "1a1"), 1))
     testHarness.setProcessingTime(2)
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(2L: JLong, "2a2"), change = true), 2))
+      CRow(2L: JLong, "2a2"), 2))
     testHarness.setProcessingTime(3)
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1L: JLong, "1a3"), change = true), 3))
+      CRow(1L: JLong, "1a3"), 3))
     assertEquals(4, testHarness.numKeyedStateEntries())
     assertEquals(2, testHarness.numProcessingTimeTimers())
 
     // All the right rows will not be cached.
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1L: JLong, "1b3"), change = true), 3))
+      CRow(1L: JLong, "1b3"), 3))
     assertEquals(4, testHarness.numKeyedStateEntries())
     assertEquals(2, testHarness.numProcessingTimeTimers())
 
@@ -278,14 +278,14 @@ class JoinHarnessTest extends HarnessTestBase {
     // Meets a.proctime <= b.proctime - 5.
     // This row will only be joined without being cached (7 >= 7 - 5).
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(2L: JLong, "2b7"), change = true), 7))
+      CRow(2L: JLong, "2b7"), 7))
     assertEquals(4, testHarness.numKeyedStateEntries())
     assertEquals(2, testHarness.numProcessingTimeTimers())
 
     testHarness.setProcessingTime(12)
     // The left row (key = 1) with timestamp = 1 will be eagerly removed here.
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1L: JLong, "1b12"), change = true), 12))
+      CRow(1L: JLong, "1b12"), 12))
 
     // We add a delay (relativeWindowSize / 2) for cleaning up state.
     // No timers will be triggered here.
@@ -309,9 +309,9 @@ class JoinHarnessTest extends HarnessTestBase {
     val expectedOutput = new ConcurrentLinkedQueue[Object]()
 
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2L: JLong, "2a2", 2L: JLong, "2b7"), change = true), 7))
+      CRow(2L: JLong, "2a2", 2L: JLong, "2b7"), 7))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1L: JLong, "1a3", 1L: JLong, "1b12"), change = true), 12))
+      CRow(1L: JLong, "1a3", 1L: JLong, "1b12"), 12))
 
     verify(expectedOutput, result, new RowResultSortComparator())
 
@@ -344,24 +344,24 @@ class JoinHarnessTest extends HarnessTestBase {
 
     // Test late data.
     testHarness.processElement1(new StreamRecord[CRow](
-      CRow(Row.of(1L: JLong, "k1"), change = true), 0))
+      CRow(1L: JLong, "k1"), 0))
 
     // Though (1L, "k1") is actually late, it will also be cached.
     assertEquals(1, testHarness.numEventTimeTimers())
 
     testHarness.processElement1(new StreamRecord[CRow](
-      CRow(Row.of(2L: JLong, "k1"), change = true), 0))
+      CRow(2L: JLong, "k1"), 0))
     testHarness.processElement2(new StreamRecord[CRow](
-      CRow(Row.of(2L: JLong, "k1"), change = true), 0))
+      CRow(2L: JLong, "k1"), 0))
 
     assertEquals(2, testHarness.numEventTimeTimers())
     assertEquals(4, testHarness.numKeyedStateEntries())
 
     testHarness.processElement1(new StreamRecord[CRow](
-      CRow(Row.of(5L: JLong, "k1"), change = true), 0))
+      CRow(5L: JLong, "k1"), 0))
 
     testHarness.processElement2(new StreamRecord[CRow](
-      CRow(Row.of(15L: JLong, "k1"), change = true), 0))
+      CRow(15L: JLong, "k1"), 0))
 
     testHarness.processWatermark1(new Watermark(20))
     testHarness.processWatermark2(new Watermark(20))
@@ -369,7 +369,7 @@ class JoinHarnessTest extends HarnessTestBase {
     assertEquals(4, testHarness.numKeyedStateEntries())
 
     testHarness.processElement1(new StreamRecord[CRow](
-      CRow(Row.of(35L: JLong, "k1"), change = true), 0))
+      CRow(35L: JLong, "k1"), 0))
 
     // The right rows with timestamp = 2 and 5 will be removed here.
     // The left rows with timestamp = 2 and 15 will be removed here.
@@ -377,9 +377,9 @@ class JoinHarnessTest extends HarnessTestBase {
     testHarness.processWatermark2(new Watermark(38))
 
     testHarness.processElement1(new StreamRecord[CRow](
-      CRow(Row.of(40L: JLong, "k2"), change = true), 0))
+      CRow(40L: JLong, "k2"), 0))
     testHarness.processElement2(new StreamRecord[CRow](
-      CRow(Row.of(39L: JLong, "k2"), change = true), 0))
+      CRow(39L: JLong, "k2"), 0))
 
     assertEquals(6, testHarness.numKeyedStateEntries())
 
@@ -393,19 +393,19 @@ class JoinHarnessTest extends HarnessTestBase {
     expectedOutput.add(new Watermark(-19))
     // This result is produced by the late row (1, "k1").
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1L: JLong, "k1", 2L: JLong, "k1"), change = true), 0))
+      CRow(1L: JLong, "k1", 2L: JLong, "k1"), 0))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2L: JLong, "k1", 2L: JLong, "k1"), change = true), 0))
+      CRow(2L: JLong, "k1", 2L: JLong, "k1"), 0))
     expectedOutput.add(new StreamRecord(
-        CRow(Row.of(5L: JLong, "k1", 2L: JLong, "k1"), change = true), 0))
+        CRow(5L: JLong, "k1", 2L: JLong, "k1"), 0))
     expectedOutput.add(new StreamRecord(
-        CRow(Row.of(5L: JLong, "k1", 15L: JLong, "k1"), change = true), 0))
+        CRow(5L: JLong, "k1", 15L: JLong, "k1"), 0))
     expectedOutput.add(new Watermark(0))
     expectedOutput.add(new StreamRecord(
-        CRow(Row.of(35L: JLong, "k1", 15L: JLong, "k1"), change = true), 0))
+        CRow(35L: JLong, "k1", 15L: JLong, "k1"), 0))
     expectedOutput.add(new Watermark(18))
     expectedOutput.add(new StreamRecord(
-        CRow(Row.of(40L: JLong, "k2", 39L: JLong, "k2"), change = true), 0))
+        CRow(40L: JLong, "k2", 39L: JLong, "k2"), 0))
     expectedOutput.add(new Watermark(41))
 
     val result = testHarness.getOutput
@@ -443,7 +443,7 @@ class JoinHarnessTest extends HarnessTestBase {
 
     // This row will not be cached.
     testHarness.processElement2(new StreamRecord[CRow](
-      CRow(Row.of(2L: JLong, "k1"), change = true), 0))
+      CRow(2L: JLong, "k1"), 0))
 
     assertEquals(0, testHarness.numKeyedStateEntries())
 
@@ -451,19 +451,19 @@ class JoinHarnessTest extends HarnessTestBase {
     testHarness.processWatermark2(new Watermark(2))
 
     testHarness.processElement1(new StreamRecord[CRow](
-      CRow(Row.of(3L: JLong, "k1"), change = true), 0))
+      CRow(3L: JLong, "k1"), 0))
     testHarness.processElement2(new StreamRecord[CRow](
-      CRow(Row.of(3L: JLong, "k1"), change = true), 0))
+      CRow(3L: JLong, "k1"), 0))
 
     // Test for -10 boundary (13 - 10 = 3).
     // This row from the right stream will be cached.
     // The clean time for the left stream is 13 - 7 + 1 - 1 = 8
     testHarness.processElement2(new StreamRecord[CRow](
-      CRow(Row.of(13L: JLong, "k1"), change = true), 0))
+      CRow(13L: JLong, "k1"), 0))
 
     // Test for -7 boundary (13 - 7 = 6).
     testHarness.processElement1(new StreamRecord[CRow](
-      CRow(Row.of(6L: JLong, "k1"), change = true), 0))
+      CRow(6L: JLong, "k1"), 0))
 
     assertEquals(4, testHarness.numKeyedStateEntries())
 
@@ -484,9 +484,9 @@ class JoinHarnessTest extends HarnessTestBase {
     expectedOutput.add(new Watermark(-9))
     expectedOutput.add(new Watermark(-8))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(3L: JLong, "k1", 13L: JLong, "k1"), change = true), 0))
+      CRow(3L: JLong, "k1", 13L: JLong, "k1"), 0))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(6L: JLong, "k1", 13L: JLong, "k1"), change = true), 0))
+      CRow(6L: JLong, "k1", 13L: JLong, "k1"), 0))
     expectedOutput.add(new Watermark(0))
     expectedOutput.add(new Watermark(8))
 
@@ -521,9 +521,9 @@ class JoinHarnessTest extends HarnessTestBase {
     testHarness.open()
 
     testHarness.processElement1(new StreamRecord[CRow](
-      CRow(Row.of(1L: JLong, "k1"), change = true), 0))
+      CRow(1L: JLong, "k1"), 0))
     testHarness.processElement2(new StreamRecord[CRow](
-      CRow(Row.of(1L: JLong, "k2"), change = true), 0))
+      CRow(1L: JLong, "k2"), 0))
 
     assertEquals(2, testHarness.numEventTimeTimers())
     assertEquals(4, testHarness.numKeyedStateEntries())
@@ -543,9 +543,9 @@ class JoinHarnessTest extends HarnessTestBase {
     assertEquals(0, testHarness.numKeyedStateEntries())
 
     testHarness.processElement1(new StreamRecord[CRow](
-      CRow(Row.of(2L: JLong, "k1"), change = true), 0))
+      CRow(2L: JLong, "k1"), 0))
     testHarness.processElement2(new StreamRecord[CRow](
-      CRow(Row.of(2L: JLong, "k2"), change = true), 0))
+      CRow(2L: JLong, "k2"), 0))
 
     // The late rows with timestamp = 2 will not be cached, but a null padding result for the left
     // row will be emitted.
@@ -554,26 +554,26 @@ class JoinHarnessTest extends HarnessTestBase {
 
     // Make sure the common (inner) join can be performed.
     testHarness.processElement1(new StreamRecord[CRow](
-      CRow(Row.of(19L: JLong, "k1"), change = true), 0))
+      CRow(19L: JLong, "k1"), 0))
     testHarness.processElement1(new StreamRecord[CRow](
-      CRow(Row.of(20L: JLong, "k1"), change = true), 0))
+      CRow(20L: JLong, "k1"), 0))
     testHarness.processElement2(new StreamRecord[CRow](
-      CRow(Row.of(26L: JLong, "k1"), change = true), 0))
+      CRow(26L: JLong, "k1"), 0))
     testHarness.processElement2(new StreamRecord[CRow](
-      CRow(Row.of(25L: JLong, "k1"), change = true), 0))
+      CRow(25L: JLong, "k1"), 0))
     testHarness.processElement1(new StreamRecord[CRow](
-      CRow(Row.of(21L: JLong, "k1"), change = true), 0))
+      CRow(21L: JLong, "k1"), 0))
 
     testHarness.processElement2(new StreamRecord[CRow](
-      CRow(Row.of(39L: JLong, "k2"), change = true), 0))
+      CRow(39L: JLong, "k2"), 0))
     testHarness.processElement2(new StreamRecord[CRow](
-      CRow(Row.of(40L: JLong, "k2"), change = true), 0))
+      CRow(40L: JLong, "k2"), 0))
     testHarness.processElement1(new StreamRecord[CRow](
-      CRow(Row.of(50L: JLong, "k2"), change = true), 0))
+      CRow(50L: JLong, "k2"), 0))
     testHarness.processElement1(new StreamRecord[CRow](
-      CRow(Row.of(49L: JLong, "k2"), change = true), 0))
+      CRow(49L: JLong, "k2"), 0))
     testHarness.processElement2(new StreamRecord[CRow](
-      CRow(Row.of(41L: JLong, "k2"), change = true), 0))
+      CRow(41L: JLong, "k2"), 0))
 
     testHarness.processWatermark1(new Watermark(100))
     testHarness.processWatermark2(new Watermark(100))
@@ -581,26 +581,26 @@ class JoinHarnessTest extends HarnessTestBase {
     val expectedOutput = new ConcurrentLinkedQueue[Object]()
     // The timestamp 14 is set with the triggered timer.
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1L: JLong, "k1", null: JLong, null: String), change = true), 14))
+      CRow(1L: JLong, "k1", null: JLong, null: String), 14))
     expectedOutput.add(new Watermark(5))
     expectedOutput.add(new Watermark(9))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2L: JLong, "k1", null: JLong, null: String), change = true), 0))
+      CRow(2L: JLong, "k1", null: JLong, null: String), 0))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(20L: JLong, "k1", 25L: JLong, "k1": String), change = true), 0))
+      CRow(20L: JLong, "k1", 25L: JLong, "k1": String), 0))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(21L: JLong, "k1", 25L: JLong, "k1": String), change = true), 0))
+      CRow(21L: JLong, "k1", 25L: JLong, "k1": String), 0))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(21L: JLong, "k1", 26L: JLong, "k1": String), change = true), 0))
+      CRow(21L: JLong, "k1", 26L: JLong, "k1": String), 0))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(49L: JLong, "k2", 40L: JLong, "k2": String), change = true), 0))
+      CRow(49L: JLong, "k2", 40L: JLong, "k2": String), 0))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(49L: JLong, "k2", 41L: JLong, "k2": String), change = true), 0))
+      CRow(49L: JLong, "k2", 41L: JLong, "k2": String), 0))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(50L: JLong, "k2", 41L: JLong, "k2": String), change = true), 0))
+      CRow(50L: JLong, "k2", 41L: JLong, "k2": String), 0))
     // The timestamp 32 is set with the triggered timer.
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(19L: JLong, "k1", null: JLong, null: String), change = true), 32))
+      CRow(19L: JLong, "k1", null: JLong, null: String), 32))
     expectedOutput.add(new Watermark(91))
 
 
@@ -635,9 +635,9 @@ class JoinHarnessTest extends HarnessTestBase {
     testHarness.open()
 
     testHarness.processElement1(new StreamRecord[CRow](
-      CRow(Row.of(1L: JLong, "k1"), change = true), 0))
+      CRow(1L: JLong, "k1"), 0))
     testHarness.processElement2(new StreamRecord[CRow](
-      CRow(Row.of(1L: JLong, "k2"), change = true), 0))
+      CRow(1L: JLong, "k2"), 0))
 
     assertEquals(2, testHarness.numEventTimeTimers())
     assertEquals(4, testHarness.numKeyedStateEntries())
@@ -657,9 +657,9 @@ class JoinHarnessTest extends HarnessTestBase {
     assertEquals(0, testHarness.numKeyedStateEntries())
 
     testHarness.processElement1(new StreamRecord[CRow](
-      CRow(Row.of(2L: JLong, "k1"), change = true), 0))
+      CRow(2L: JLong, "k1"), 0))
     testHarness.processElement2(new StreamRecord[CRow](
-      CRow(Row.of(2L: JLong, "k2"), change = true), 0))
+      CRow(2L: JLong, "k2"), 0))
 
     // The late rows with timestamp = 2 will not be cached, but a null padding result for the right
     // row will be emitted.
@@ -668,26 +668,26 @@ class JoinHarnessTest extends HarnessTestBase {
 
     // Make sure the common (inner) join can be performed.
     testHarness.processElement1(new StreamRecord[CRow](
-      CRow(Row.of(19L: JLong, "k1"), change = true), 0))
+      CRow(19L: JLong, "k1"), 0))
     testHarness.processElement1(new StreamRecord[CRow](
-      CRow(Row.of(20L: JLong, "k1"), change = true), 0))
+      CRow(20L: JLong, "k1"), 0))
     testHarness.processElement2(new StreamRecord[CRow](
-      CRow(Row.of(26L: JLong, "k1"), change = true), 0))
+      CRow(26L: JLong, "k1"), 0))
     testHarness.processElement2(new StreamRecord[CRow](
-      CRow(Row.of(25L: JLong, "k1"), change = true), 0))
+      CRow(25L: JLong, "k1"), 0))
     testHarness.processElement1(new StreamRecord[CRow](
-      CRow(Row.of(21L: JLong, "k1"), change = true), 0))
+      CRow(21L: JLong, "k1"), 0))
 
     testHarness.processElement2(new StreamRecord[CRow](
-      CRow(Row.of(39L: JLong, "k2"), change = true), 0))
+      CRow(39L: JLong, "k2"), 0))
     testHarness.processElement2(new StreamRecord[CRow](
-      CRow(Row.of(40L: JLong, "k2"), change = true), 0))
+      CRow(40L: JLong, "k2"), 0))
     testHarness.processElement1(new StreamRecord[CRow](
-      CRow(Row.of(50L: JLong, "k2"), change = true), 0))
+      CRow(50L: JLong, "k2"), 0))
     testHarness.processElement1(new StreamRecord[CRow](
-      CRow(Row.of(49L: JLong, "k2"), change = true), 0))
+      CRow(49L: JLong, "k2"), 0))
     testHarness.processElement2(new StreamRecord[CRow](
-      CRow(Row.of(41L: JLong, "k2"), change = true), 0))
+      CRow(41L: JLong, "k2"), 0))
 
     testHarness.processWatermark1(new Watermark(100))
     testHarness.processWatermark2(new Watermark(100))
@@ -696,25 +696,25 @@ class JoinHarnessTest extends HarnessTestBase {
     expectedOutput.add(new Watermark(5))
     // The timestamp 18 is set with the triggered timer.
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JLong, null: String, 1L: JLong, "k2": String), change = true), 18))
+      CRow(null: JLong, null: String, 1L: JLong, "k2": String), 18))
     expectedOutput.add(new Watermark(9))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JLong, null: String, 2L: JLong, "k2": String), change = true), 0))
+      CRow(null: JLong, null: String, 2L: JLong, "k2": String), 0))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(20L: JLong, "k1", 25L: JLong, "k1": String), change = true), 0))
+      CRow(20L: JLong, "k1", 25L: JLong, "k1": String), 0))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(21L: JLong, "k1", 25L: JLong, "k1": String), change = true), 0))
+      CRow(21L: JLong, "k1", 25L: JLong, "k1": String), 0))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(21L: JLong, "k1", 26L: JLong, "k1": String), change = true), 0))
+      CRow(21L: JLong, "k1", 26L: JLong, "k1": String), 0))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(49L: JLong, "k2", 40L: JLong, "k2": String), change = true), 0))
+      CRow(49L: JLong, "k2", 40L: JLong, "k2": String), 0))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(49L: JLong, "k2", 41L: JLong, "k2": String), change = true), 0))
+      CRow(49L: JLong, "k2", 41L: JLong, "k2": String), 0))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(50L: JLong, "k2", 41L: JLong, "k2": String), change = true), 0))
+      CRow(50L: JLong, "k2", 41L: JLong, "k2": String), 0))
     // The timestamp 56 is set with the triggered timer.
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JLong, null: String, 39L: JLong, "k2": String), change = true), 56))
+      CRow(null: JLong, null: String, 39L: JLong, "k2": String), 56))
     expectedOutput.add(new Watermark(91))
 
     val result = testHarness.getOutput
@@ -748,9 +748,9 @@ class JoinHarnessTest extends HarnessTestBase {
     testHarness.open()
 
     testHarness.processElement1(new StreamRecord[CRow](
-      CRow(Row.of(1L: JLong, "k1"), change = true), 0))
+      CRow(1L: JLong, "k1"), 0))
     testHarness.processElement2(new StreamRecord[CRow](
-      CRow(Row.of(1L: JLong, "k2"), change = true), 0))
+      CRow(1L: JLong, "k2"), 0))
 
     assertEquals(2, testHarness.numEventTimeTimers())
     assertEquals(4, testHarness.numKeyedStateEntries())
@@ -770,9 +770,9 @@ class JoinHarnessTest extends HarnessTestBase {
     assertEquals(0, testHarness.numKeyedStateEntries())
 
     testHarness.processElement1(new StreamRecord[CRow](
-      CRow(Row.of(2L: JLong, "k1"), change = true), 0))
+      CRow(2L: JLong, "k1"), 0))
     testHarness.processElement2(new StreamRecord[CRow](
-      CRow(Row.of(2L: JLong, "k2"), change = true), 0))
+      CRow(2L: JLong, "k2"), 0))
 
     // The late rows with timestamp = 2 will not be cached, but a null padding result for the right
     // row will be emitted.
@@ -781,26 +781,26 @@ class JoinHarnessTest extends HarnessTestBase {
 
     // Make sure the common (inner) join can be performed.
     testHarness.processElement1(new StreamRecord[CRow](
-      CRow(Row.of(19L: JLong, "k1"), change = true), 0))
+      CRow(19L: JLong, "k1"), 0))
     testHarness.processElement1(new StreamRecord[CRow](
-      CRow(Row.of(20L: JLong, "k1"), change = true), 0))
+      CRow(20L: JLong, "k1"), 0))
     testHarness.processElement2(new StreamRecord[CRow](
-      CRow(Row.of(26L: JLong, "k1"), change = true), 0))
+      CRow(26L: JLong, "k1"), 0))
     testHarness.processElement2(new StreamRecord[CRow](
-      CRow(Row.of(25L: JLong, "k1"), change = true), 0))
+      CRow(25L: JLong, "k1"), 0))
     testHarness.processElement1(new StreamRecord[CRow](
-      CRow(Row.of(21L: JLong, "k1"), change = true), 0))
+      CRow(21L: JLong, "k1"), 0))
 
     testHarness.processElement2(new StreamRecord[CRow](
-      CRow(Row.of(39L: JLong, "k2"), change = true), 0))
+      CRow(39L: JLong, "k2"), 0))
     testHarness.processElement2(new StreamRecord[CRow](
-      CRow(Row.of(40L: JLong, "k2"), change = true), 0))
+      CRow(40L: JLong, "k2"), 0))
     testHarness.processElement1(new StreamRecord[CRow](
-      CRow(Row.of(50L: JLong, "k2"), change = true), 0))
+      CRow(50L: JLong, "k2"), 0))
     testHarness.processElement1(new StreamRecord[CRow](
-      CRow(Row.of(49L: JLong, "k2"), change = true), 0))
+      CRow(49L: JLong, "k2"), 0))
     testHarness.processElement2(new StreamRecord[CRow](
-      CRow(Row.of(41L: JLong, "k2"), change = true), 0))
+      CRow(41L: JLong, "k2"), 0))
 
     testHarness.processWatermark1(new Watermark(100))
     testHarness.processWatermark2(new Watermark(100))
@@ -808,34 +808,34 @@ class JoinHarnessTest extends HarnessTestBase {
     val expectedOutput = new ConcurrentLinkedQueue[Object]()
     // The timestamp 14 is set with the triggered timer.
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1L: JLong, "k1", null: JLong, null: String), change = true), 14))
+      CRow(1L: JLong, "k1", null: JLong, null: String), 14))
     expectedOutput.add(new Watermark(5))
     // The timestamp 18 is set with the triggered timer.
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JLong, null: String, 1L: JLong, "k2": String), change = true), 18))
+      CRow(null: JLong, null: String, 1L: JLong, "k2": String), 18))
     expectedOutput.add(new Watermark(9))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2L: JLong, "k1", null: JLong, null: String), change = true), 0))
+      CRow(2L: JLong, "k1", null: JLong, null: String), 0))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JLong, null: String, 2L: JLong, "k2": String), change = true), 0))
+      CRow(null: JLong, null: String, 2L: JLong, "k2": String), 0))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(20L: JLong, "k1", 25L: JLong, "k1": String), change = true), 0))
+      CRow(20L: JLong, "k1", 25L: JLong, "k1": String), 0))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(21L: JLong, "k1", 25L: JLong, "k1": String), change = true), 0))
+      CRow(21L: JLong, "k1", 25L: JLong, "k1": String), 0))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(21L: JLong, "k1", 26L: JLong, "k1": String), change = true), 0))
+      CRow(21L: JLong, "k1", 26L: JLong, "k1": String), 0))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(49L: JLong, "k2", 40L: JLong, "k2": String), change = true), 0))
+      CRow(49L: JLong, "k2", 40L: JLong, "k2": String), 0))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(49L: JLong, "k2", 41L: JLong, "k2": String), change = true), 0))
+      CRow(49L: JLong, "k2", 41L: JLong, "k2": String), 0))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(50L: JLong, "k2", 41L: JLong, "k2": String), change = true), 0))
+      CRow(50L: JLong, "k2", 41L: JLong, "k2": String), 0))
     // The timestamp 32 is set with the triggered timer.
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(19L: JLong, "k1", null: JLong, null: String), change = true), 32))
+      CRow(19L: JLong, "k1", null: JLong, null: String), 32))
     // The timestamp 56 is set with the triggered timer.
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JLong, null: String, 39L: JLong, "k2": String), change = true), 56))
+      CRow(null: JLong, null: String, 39L: JLong, "k2": String), 56))
     expectedOutput.add(new Watermark(91))
 
     val result = testHarness.getOutput
@@ -880,37 +880,37 @@ class JoinHarnessTest extends HarnessTestBase {
     // left stream input
     testHarness.setProcessingTime(1)
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa"), change = true)))
+      CRow(1: JInt, "aaa")))
     assertEquals(1, testHarness.numProcessingTimeTimers())
     assertEquals(2, testHarness.numKeyedStateEntries())
     testHarness.setProcessingTime(2)
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa"), change = true)))
+      CRow(1: JInt, "aaa")))
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(2: JInt, "bbb"), change = true)))
+      CRow(2: JInt, "bbb")))
     assertEquals(2, testHarness.numProcessingTimeTimers())
     assertEquals(4, testHarness.numKeyedStateEntries())
     testHarness.setProcessingTime(3)
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa"), change = true)))
+      CRow(1: JInt, "aaa")))
     assertEquals(4, testHarness.numKeyedStateEntries())
     assertEquals(2, testHarness.numProcessingTimeTimers())
 
     // right stream input and output normally
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1: JInt, "Hi1"), change = true)))
+      CRow(1: JInt, "Hi1")))
     assertEquals(6, testHarness.numKeyedStateEntries())
     assertEquals(3, testHarness.numProcessingTimeTimers())
     testHarness.setProcessingTime(4)
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(2: JInt, "Hello1"), change = true)))
+      CRow(2: JInt, "Hello1")))
     assertEquals(8, testHarness.numKeyedStateEntries())
     assertEquals(4, testHarness.numProcessingTimeTimers())
 
     // expired left stream record with key value of 1
     testHarness.setProcessingTime(5)
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1: JInt, "Hi2"), change = true)))
+      CRow(1: JInt, "Hi2")))
     assertEquals(6, testHarness.numKeyedStateEntries())
     assertEquals(3, testHarness.numProcessingTimeTimers())
 
@@ -936,13 +936,13 @@ class JoinHarnessTest extends HarnessTestBase {
     val expectedOutput = new ConcurrentLinkedQueue[Object]()
 
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa", 1: JInt, "Hi1"), change = true)))
+      CRow(1: JInt, "aaa", 1: JInt, "Hi1")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa", 1: JInt, "Hi1"), change = true)))
+      CRow(1: JInt, "aaa", 1: JInt, "Hi1")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa", 1: JInt, "Hi1"), change = true)))
+      CRow(1: JInt, "aaa", 1: JInt, "Hi1")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2: JInt, "bbb", 2: JInt, "Hello1"), change = true)))
+      CRow(2: JInt, "bbb", 2: JInt, "Hello1")))
 
     verify(expectedOutput, result, new RowResultSortComparator())
 
@@ -974,43 +974,43 @@ class JoinHarnessTest extends HarnessTestBase {
     // left stream input
     testHarness.setProcessingTime(1)
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa"), change = true)))
+      CRow(1: JInt, "aaa")))
     assertEquals(1, testHarness.numProcessingTimeTimers())
     assertEquals(2, testHarness.numKeyedStateEntries())
     testHarness.setProcessingTime(2)
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa"), change = true)))
+      CRow(1: JInt, "aaa")))
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(2: JInt, "bbb"), change = true)))
+      CRow(2: JInt, "bbb")))
     assertEquals(2, testHarness.numProcessingTimeTimers())
     assertEquals(4, testHarness.numKeyedStateEntries())
     testHarness.setProcessingTime(3)
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa"), change = false)))
+      CRow(false, 1: JInt, "aaa")))
     assertEquals(4, testHarness.numKeyedStateEntries())
     assertEquals(2, testHarness.numProcessingTimeTimers())
 
     // right stream input and output normally
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1: JInt, "Hi1"), change = true)))
+      CRow(1: JInt, "Hi1")))
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1: JInt, "Hi1"), change = false)))
+      CRow(false, 1: JInt, "Hi1")))
     assertEquals(5, testHarness.numKeyedStateEntries())
     assertEquals(3, testHarness.numProcessingTimeTimers())
     testHarness.setProcessingTime(4)
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(2: JInt, "Hello1"), change = true)))
+      CRow(2: JInt, "Hello1")))
     assertEquals(7, testHarness.numKeyedStateEntries())
     assertEquals(4, testHarness.numProcessingTimeTimers())
 
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa"), change = false)))
+      CRow(false, 1: JInt, "aaa")))
     // expired left stream record with key value of 1
     testHarness.setProcessingTime(5)
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1: JInt, "Hi2"), change = true)))
+      CRow(1: JInt, "Hi2")))
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1: JInt, "Hi2"), change = false)))
+      CRow(false, 1: JInt, "Hi2")))
     assertEquals(5, testHarness.numKeyedStateEntries())
     assertEquals(3, testHarness.numProcessingTimeTimers())
 
@@ -1029,11 +1029,11 @@ class JoinHarnessTest extends HarnessTestBase {
     val expectedOutput = new ConcurrentLinkedQueue[Object]()
 
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa", 1: JInt, "Hi1"), change = true)))
+      CRow(1: JInt, "aaa", 1: JInt, "Hi1")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa", 1: JInt, "Hi1"), change = false)))
+      CRow(false, 1: JInt, "aaa", 1: JInt, "Hi1")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2: JInt, "bbb", 2: JInt, "Hello1"), change = true)))
+      CRow(2: JInt, "bbb", 2: JInt, "Hello1")))
 
     verify(expectedOutput, result, new RowResultSortComparator())
 
@@ -1066,43 +1066,43 @@ class JoinHarnessTest extends HarnessTestBase {
     // left stream input
     testHarness.setProcessingTime(1)
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa"), change = true)))
+      CRow(1: JInt, "aaa")))
     assertEquals(1, testHarness.numProcessingTimeTimers())
     assertEquals(2, testHarness.numKeyedStateEntries())
     testHarness.setProcessingTime(2)
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa"), change = true)))
+      CRow(1: JInt, "aaa")))
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(2: JInt, "bbb"), change = true)))
+      CRow(2: JInt, "bbb")))
     assertEquals(2, testHarness.numProcessingTimeTimers())
     assertEquals(4, testHarness.numKeyedStateEntries())
     testHarness.setProcessingTime(3)
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa"), change = false)))
+      CRow(false, 1: JInt, "aaa")))
     assertEquals(4, testHarness.numKeyedStateEntries())
     assertEquals(2, testHarness.numProcessingTimeTimers())
 
     // right stream input and output normally
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1: JInt, "Hi1"), change = true)))
+      CRow(1: JInt, "Hi1")))
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1: JInt, "Hi1"), change = false)))
+      CRow(false, 1: JInt, "Hi1")))
     assertEquals(5, testHarness.numKeyedStateEntries())
     assertEquals(3, testHarness.numProcessingTimeTimers())
     testHarness.setProcessingTime(4)
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(2: JInt, "Hello1"), change = true)))
+      CRow(2: JInt, "Hello1")))
     assertEquals(7, testHarness.numKeyedStateEntries())
     assertEquals(4, testHarness.numProcessingTimeTimers())
 
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa"), change = false)))
+      CRow(false, 1: JInt, "aaa")))
     // expired left stream record with key value of 1
     testHarness.setProcessingTime(5)
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1: JInt, "Hi2"), change = true)))
+      CRow(1: JInt, "Hi2")))
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1: JInt, "Hi2"), change = false)))
+      CRow(false, 1: JInt, "Hi2")))
     assertEquals(5, testHarness.numKeyedStateEntries())
     assertEquals(3, testHarness.numProcessingTimeTimers())
 
@@ -1121,27 +1121,27 @@ class JoinHarnessTest extends HarnessTestBase {
     val expectedOutput = new ConcurrentLinkedQueue[Object]()
 
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa", null: JInt, null), change = true)))
+      CRow(1: JInt, "aaa", null: JInt, null)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa", null: JInt, null), change = true)))
+      CRow(1: JInt, "aaa", null: JInt, null)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2: JInt, "bbb", null: JInt, null), change = true)))
+      CRow(2: JInt, "bbb", null: JInt, null)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa", null: JInt, null), change = false)))
+      CRow(false, 1: JInt, "aaa", null: JInt, null)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa", null: JInt, null), change = false)))
+      CRow(false, 1: JInt, "aaa", null: JInt, null)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa", 1: JInt, "Hi1"), change = true)))
+      CRow(1: JInt, "aaa", 1: JInt, "Hi1")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa", 1: JInt, "Hi1"), change = false)))
+      CRow(false, 1: JInt, "aaa", 1: JInt, "Hi1")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa", null: JInt, null), change = true)))
+      CRow(1: JInt, "aaa", null: JInt, null)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2: JInt, "bbb", null: JInt, null), change = false)))
+      CRow(false, 2: JInt, "bbb", null: JInt, null)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2: JInt, "bbb", 2: JInt, "Hello1"), change = true)))
+      CRow(2: JInt, "bbb", 2: JInt, "Hello1")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa", null: JInt, null), change = false)))
+      CRow(false, 1: JInt, "aaa", null: JInt, null)))
 
     verify(expectedOutput, result, new RowResultSortComparator())
 
@@ -1174,19 +1174,19 @@ class JoinHarnessTest extends HarnessTestBase {
     // left stream input
     testHarness.setProcessingTime(1)
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa"), change = true)))
+      CRow(1: JInt, "aaa")))
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa"), change = false)))
+      CRow(false, 1: JInt, "aaa")))
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1: JInt, "bbb"), change = true)))
+      CRow(1: JInt, "bbb")))
     assertEquals(1, testHarness.numProcessingTimeTimers())
     // 1 left timer(5), 1 left key(1), 1 join cnt
     assertEquals(3, testHarness.numKeyedStateEntries())
     testHarness.setProcessingTime(2)
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa"), change = true)))
+      CRow(1: JInt, "aaa")))
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(2: JInt, "bbb"), change = true)))
+      CRow(2: JInt, "bbb")))
     assertEquals(2, testHarness.numProcessingTimeTimers())
     // 2 left timer(5,6), 2 left key(1,2), 2 join cnt
     assertEquals(6, testHarness.numKeyedStateEntries())
@@ -1194,35 +1194,35 @@ class JoinHarnessTest extends HarnessTestBase {
 
     // right stream input and output normally
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1: JInt, "Hi1"), change = true)))
+      CRow(1: JInt, "Hi1")))
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1: JInt, "bbb"), change = false)))
+      CRow(false, 1: JInt, "bbb")))
     // 2 left timer(5,6), 2 left keys(1,2), 2 join cnt, 1 right timer(7), 1 right key(1)
     assertEquals(8, testHarness.numKeyedStateEntries())
     assertEquals(3, testHarness.numProcessingTimeTimers())
     testHarness.setProcessingTime(4)
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(2: JInt, "ccc"), change = true)))
+      CRow(2: JInt, "ccc")))
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(2: JInt, "Hello"), change = true)))
+      CRow(2: JInt, "Hello")))
     // 2 left timer(5,6), 2 left keys(1,2), 2 join cnt, 2 right timer(7,8), 2 right key(1,2)
     assertEquals(10, testHarness.numKeyedStateEntries())
     assertEquals(4, testHarness.numProcessingTimeTimers())
 
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa"), change = false)))
+      CRow(false, 1: JInt, "aaa")))
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1: JInt, "Hi2"), change = true)))
+      CRow(1: JInt, "Hi2")))
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1: JInt, "Hi2"), change = false)))
+      CRow(false, 1: JInt, "Hi2")))
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1: JInt, "Hi1"), change = false)))
+      CRow(false, 1: JInt, "Hi1")))
     // expired left stream record with key value of 1
     testHarness.setProcessingTime(5)
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1: JInt, "Hi3"), change = true)))
+      CRow(1: JInt, "Hi3")))
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1: JInt, "Hi3"), change = false)))
+      CRow(false, 1: JInt, "Hi3")))
     // 1 left timer(6), 1 left keys(2), 1 join cnt, 2 right timer(7,8), 1 right key(2)
     assertEquals(6, testHarness.numKeyedStateEntries())
     assertEquals(3, testHarness.numProcessingTimeTimers())
@@ -1242,37 +1242,37 @@ class JoinHarnessTest extends HarnessTestBase {
     val expectedOutput = new ConcurrentLinkedQueue[Object]()
 
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa", null: JInt, null), change = true)))
+      CRow(1: JInt, "aaa", null: JInt, null)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa", null: JInt, null), change = false)))
+      CRow(false, 1: JInt, "aaa", null: JInt, null)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "bbb", null: JInt, null), change = true)))
+      CRow(1: JInt, "bbb", null: JInt, null)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa", null: JInt, null), change = true)))
+      CRow(1: JInt, "aaa", null: JInt, null)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2: JInt, "bbb", null: JInt, null), change = true)))
+      CRow(2: JInt, "bbb", null: JInt, null)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "bbb", null: JInt, null), change = false)))
+      CRow(false, 1: JInt, "bbb", null: JInt, null)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa", null: JInt, null), change = false)))
+      CRow(false, 1: JInt, "aaa", null: JInt, null)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa", 1: JInt, "Hi1"), change = true)))
+      CRow(1: JInt, "aaa", 1: JInt, "Hi1")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "bbb", 1: JInt, "Hi1"), change = true)))
+      CRow(1: JInt, "bbb", 1: JInt, "Hi1")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2: JInt, "bbb", null: JInt, null), change = false)))
+      CRow(false, 2: JInt, "bbb", null: JInt, null)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2: JInt, "bbb", 2: JInt, "Hello"), change = true)))
+      CRow(2: JInt, "bbb", 2: JInt, "Hello")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa", 1: JInt, "Hi1"), change = false)))
+      CRow(false, 1: JInt, "aaa", 1: JInt, "Hi1")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "bbb", 1: JInt, "Hi2"), change = true)))
+      CRow(1: JInt, "bbb", 1: JInt, "Hi2")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "bbb", 1: JInt, "Hi2"), change = false)))
+      CRow(false, 1: JInt, "bbb", 1: JInt, "Hi2")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "bbb", 1: JInt, "Hi1"), change = false)))
+      CRow(false, 1: JInt, "bbb", 1: JInt, "Hi1")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "bbb", null: JInt, null), change = true)))
+      CRow(1: JInt, "bbb", null: JInt, null)))
     verify(expectedOutput, result, new RowResultSortComparator())
 
     testHarness.close()
@@ -1304,43 +1304,43 @@ class JoinHarnessTest extends HarnessTestBase {
     // right stream input
     testHarness.setProcessingTime(1)
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa"), change = true)))
+      CRow(1: JInt, "aaa")))
     assertEquals(1, testHarness.numProcessingTimeTimers())
     assertEquals(2, testHarness.numKeyedStateEntries())
     testHarness.setProcessingTime(2)
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa"), change = true)))
+      CRow(1: JInt, "aaa")))
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(2: JInt, "bbb"), change = true)))
+      CRow(2: JInt, "bbb")))
     assertEquals(2, testHarness.numProcessingTimeTimers())
     assertEquals(4, testHarness.numKeyedStateEntries())
     testHarness.setProcessingTime(3)
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa"), change = false)))
+      CRow(false, 1: JInt, "aaa")))
     assertEquals(4, testHarness.numKeyedStateEntries())
     assertEquals(2, testHarness.numProcessingTimeTimers())
 
     // left stream input and output normally
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1: JInt, "Hi1"), change = true)))
+      CRow(1: JInt, "Hi1")))
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1: JInt, "Hi1"), change = false)))
+      CRow(false, 1: JInt, "Hi1")))
     assertEquals(5, testHarness.numKeyedStateEntries())
     assertEquals(3, testHarness.numProcessingTimeTimers())
     testHarness.setProcessingTime(4)
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(2: JInt, "Hello1"), change = true)))
+      CRow(2: JInt, "Hello1")))
     assertEquals(7, testHarness.numKeyedStateEntries())
     assertEquals(4, testHarness.numProcessingTimeTimers())
 
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa"), change = false)))
+      CRow(false, 1: JInt, "aaa")))
     // expired right stream record with key value of 1
     testHarness.setProcessingTime(5)
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1: JInt, "Hi2"), change = true)))
+      CRow(1: JInt, "Hi2")))
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1: JInt, "Hi2"), change = false)))
+      CRow(false, 1: JInt, "Hi2")))
     assertEquals(5, testHarness.numKeyedStateEntries())
     assertEquals(3, testHarness.numProcessingTimeTimers())
 
@@ -1359,27 +1359,27 @@ class JoinHarnessTest extends HarnessTestBase {
     val expectedOutput = new ConcurrentLinkedQueue[Object]()
 
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JInt, null, 1: JInt, "aaa"), change = true)))
+      CRow(null: JInt, null, 1: JInt, "aaa")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JInt, null, 1: JInt, "aaa"), change = true)))
+      CRow(null: JInt, null, 1: JInt, "aaa")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JInt, null, 2: JInt, "bbb"), change = true)))
+      CRow(null: JInt, null, 2: JInt, "bbb")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JInt, null, 1: JInt, "aaa"), change = false)))
+      CRow(false, null: JInt, null, 1: JInt, "aaa")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JInt, null, 1: JInt, "aaa"), change = false)))
+      CRow(false, null: JInt, null, 1: JInt, "aaa")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "Hi1", 1: JInt, "aaa"), change = true)))
+      CRow(1: JInt, "Hi1", 1: JInt, "aaa")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "Hi1", 1: JInt, "aaa"), change = false)))
+      CRow(false, 1: JInt, "Hi1", 1: JInt, "aaa")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JInt, null, 1: JInt, "aaa"), change = true)))
+      CRow(null: JInt, null, 1: JInt, "aaa")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JInt, null, 2: JInt, "bbb"), change = false)))
+      CRow(false, null: JInt, null, 2: JInt, "bbb")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2: JInt, "Hello1", 2: JInt, "bbb"), change = true)))
+      CRow(2: JInt, "Hello1", 2: JInt, "bbb")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JInt, null, 1: JInt, "aaa"), change = false)))
+      CRow(false, null: JInt, null, 1: JInt, "aaa")))
 
     verify(expectedOutput, result, new RowResultSortComparator())
 
@@ -1412,19 +1412,19 @@ class JoinHarnessTest extends HarnessTestBase {
     // right stream input
     testHarness.setProcessingTime(1)
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa"), change = true)))
+      CRow(1: JInt, "aaa")))
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa"), change = false)))
+      CRow(false, 1: JInt, "aaa")))
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1: JInt, "bbb"), change = true)))
+      CRow(1: JInt, "bbb")))
     assertEquals(1, testHarness.numProcessingTimeTimers())
     // 1 right timer(5), 1 right key(1), 1 join cnt
     assertEquals(3, testHarness.numKeyedStateEntries())
     testHarness.setProcessingTime(2)
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa"), change = true)))
+      CRow(1: JInt, "aaa")))
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(2: JInt, "bbb"), change = true)))
+      CRow(2: JInt, "bbb")))
     assertEquals(2, testHarness.numProcessingTimeTimers())
     // 2 right timer(5,6), 2 right key(1,2), 2 join cnt
     assertEquals(6, testHarness.numKeyedStateEntries())
@@ -1432,35 +1432,35 @@ class JoinHarnessTest extends HarnessTestBase {
 
     // left stream input and output normally
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1: JInt, "Hi1"), change = true)))
+      CRow(1: JInt, "Hi1")))
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1: JInt, "bbb"), change = false)))
+      CRow(false, 1: JInt, "bbb")))
     // 2 right timer(5,6), 2 right keys(1,2), 2 join cnt, 1 left timer(7), 1 left key(1)
     assertEquals(8, testHarness.numKeyedStateEntries())
     assertEquals(3, testHarness.numProcessingTimeTimers())
     testHarness.setProcessingTime(4)
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(2: JInt, "ccc"), change = true)))
+      CRow(2: JInt, "ccc")))
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(2: JInt, "Hello"), change = true)))
+      CRow(2: JInt, "Hello")))
     // 2 right timer(5,6), 2 right keys(1,2), 2 join cnt, 2 left timer(7,8), 2 left key(1,2)
     assertEquals(10, testHarness.numKeyedStateEntries())
     assertEquals(4, testHarness.numProcessingTimeTimers())
 
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa"), change = false)))
+      CRow(false, 1: JInt, "aaa")))
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1: JInt, "Hi2"), change = true)))
+      CRow(1: JInt, "Hi2")))
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1: JInt, "Hi2"), change = false)))
+      CRow(false, 1: JInt, "Hi2")))
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1: JInt, "Hi1"), change = false)))
+      CRow(false, 1: JInt, "Hi1")))
     // expired right stream record with key value of 1
     testHarness.setProcessingTime(5)
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1: JInt, "Hi3"), change = true)))
+      CRow(1: JInt, "Hi3")))
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1: JInt, "Hi3"), change = false)))
+      CRow(false, 1: JInt, "Hi3")))
     // 1 right timer(6), 1 right keys(2), 1 join cnt, 2 left timer(7,8), 1 left key(2)
     assertEquals(6, testHarness.numKeyedStateEntries())
     assertEquals(3, testHarness.numProcessingTimeTimers())
@@ -1480,37 +1480,37 @@ class JoinHarnessTest extends HarnessTestBase {
     val expectedOutput = new ConcurrentLinkedQueue[Object]()
 
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JInt, null, 1: JInt, "aaa"), change = true)))
+      CRow(null: JInt, null, 1: JInt, "aaa")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JInt, null, 1: JInt, "aaa"), change = false)))
+      CRow(false, null: JInt, null, 1: JInt, "aaa")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JInt, null, 1: JInt, "bbb"), change = true)))
+      CRow(null: JInt, null, 1: JInt, "bbb")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JInt, null, 1: JInt, "aaa"), change = true)))
+      CRow(null: JInt, null, 1: JInt, "aaa")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JInt, null, 2: JInt, "bbb"), change = true)))
+      CRow(null: JInt, null, 2: JInt, "bbb")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JInt, null, 1: JInt, "bbb"), change = false)))
+      CRow(false, null: JInt, null, 1: JInt, "bbb")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JInt, null, 1: JInt, "aaa"), change = false)))
+      CRow(false, null: JInt, null, 1: JInt, "aaa")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "Hi1", 1: JInt, "aaa"), change = true)))
+      CRow(1: JInt, "Hi1", 1: JInt, "aaa")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "Hi1", 1: JInt, "bbb"), change = true)))
+      CRow(1: JInt, "Hi1", 1: JInt, "bbb")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JInt, null, 2: JInt, "bbb"), change = false)))
+      CRow(false, null: JInt, null, 2: JInt, "bbb")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2: JInt, "Hello", 2: JInt, "bbb"), change = true)))
+      CRow(2: JInt, "Hello", 2: JInt, "bbb")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "Hi1", 1: JInt, "aaa"), change = false)))
+      CRow(false, 1: JInt, "Hi1", 1: JInt, "aaa")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "Hi2", 1: JInt, "bbb"), change = true)))
+      CRow(1: JInt, "Hi2", 1: JInt, "bbb")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "Hi2", 1: JInt, "bbb"), change = false)))
+      CRow(false, 1: JInt, "Hi2", 1: JInt, "bbb")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "Hi1", 1: JInt, "bbb"), change = false)))
+      CRow(false, 1: JInt, "Hi1", 1: JInt, "bbb")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JInt, null, 1: JInt, "bbb"), change = true)))
+      CRow(null: JInt, null, 1: JInt, "bbb")))
     verify(expectedOutput, result, new RowResultSortComparator())
 
     testHarness.close()
@@ -1541,18 +1541,18 @@ class JoinHarnessTest extends HarnessTestBase {
     // left stream input
     testHarness.setProcessingTime(1)
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1: JInt, "bbb"), change = true)))
+      CRow(1: JInt, "bbb")))
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1: JInt, "ccc"), change = true)))
+      CRow(1: JInt, "ccc")))
     assertEquals(1, testHarness.numProcessingTimeTimers())
     // 1 left timer(5), 1 left key(1)
     assertEquals(2, testHarness.numKeyedStateEntries())
 
     testHarness.setProcessingTime(2)
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(2: JInt, "bbb"), change = true)))
+      CRow(2: JInt, "bbb")))
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(2: JInt, "ccc"), change = true)))
+      CRow(2: JInt, "ccc")))
     assertEquals(2, testHarness.numProcessingTimeTimers())
     // 1 left timer(5), 1 left key(1)
     // 1 right timer(6), 1 right key(1)
@@ -1560,17 +1560,17 @@ class JoinHarnessTest extends HarnessTestBase {
 
     testHarness.setProcessingTime(3)
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(2: JInt, "aaa"), change = true)))
+      CRow(2: JInt, "aaa")))
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(2: JInt, "ddd"), change = true)))
+      CRow(2: JInt, "ddd")))
     assertEquals(3, testHarness.numProcessingTimeTimers())
     // 2 left timer(5,7), 2 left key(1,2)
     // 1 right timer(6), 1 right key(1)
     assertEquals(6, testHarness.numKeyedStateEntries())
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa"), change = true)))
+      CRow(1: JInt, "aaa")))
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1: JInt, "ddd"), change = true)))
+      CRow(1: JInt, "ddd")))
     assertEquals(4, testHarness.numProcessingTimeTimers())
     // 2 left timer(5,7), 2 left key(1,2)
     // 2 right timer(6,7), 2 right key(1,2)
@@ -1578,13 +1578,13 @@ class JoinHarnessTest extends HarnessTestBase {
 
     testHarness.setProcessingTime(4)
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(2: JInt, "aaa"), change = false)))
+      CRow(false, 2: JInt, "aaa")))
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(2: JInt, "ddd"), change = false)))
+      CRow(false, 2: JInt, "ddd")))
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa"), change = false)))
+      CRow(false, 1: JInt, "aaa")))
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1: JInt, "ddd"), change = false)))
+      CRow(false, 1: JInt, "ddd")))
     assertEquals(4, testHarness.numProcessingTimeTimers())
     // 2 left timer(5,7), 1 left key(1)
     // 2 right timer(6,7), 1 right key(2)
@@ -1608,78 +1608,78 @@ class JoinHarnessTest extends HarnessTestBase {
 
     testHarness.setProcessingTime(8)
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1: JInt, "bbb"), change = true)))
+      CRow(1: JInt, "bbb")))
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(2: JInt, "bbb"), change = true)))
+      CRow(2: JInt, "bbb")))
 
     val result = testHarness.getOutput
     val expectedOutput = new ConcurrentLinkedQueue[Object]()
 
     // processing time 1
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "bbb", null: JInt, null), change = true)))
+      CRow(1: JInt, "bbb", null: JInt, null)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "ccc", null: JInt, null), change = true)))
+      CRow(1: JInt, "ccc", null: JInt, null)))
     // processing time 2
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JInt, null, 2: JInt, "bbb"), change = true)))
+      CRow(null: JInt, null, 2: JInt, "bbb")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JInt, null, 2: JInt, "ccc"), change = true)))
+      CRow(null: JInt, null, 2: JInt, "ccc")))
     // processing time 3
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JInt, null, 2: JInt, "bbb"), change = false)))
+      CRow(false, null: JInt, null, 2: JInt, "bbb")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JInt, null, 2: JInt, "ccc"), change = false)))
+      CRow(false, null: JInt, null, 2: JInt, "ccc")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2: JInt, "aaa", 2: JInt, "bbb"), change = true)))
+      CRow(2: JInt, "aaa", 2: JInt, "bbb")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2: JInt, "aaa", 2: JInt, "ccc"), change = true)))
+      CRow(2: JInt, "aaa", 2: JInt, "ccc")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2: JInt, "ddd", 2: JInt, "bbb"), change = true)))
+      CRow(2: JInt, "ddd", 2: JInt, "bbb")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2: JInt, "ddd", 2: JInt, "ccc"), change = true)))
+      CRow(2: JInt, "ddd", 2: JInt, "ccc")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "bbb", null: JInt, null), change = false)))
+      CRow(false, 1: JInt, "bbb", null: JInt, null)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "ccc", null: JInt, null), change = false)))
+      CRow(false, 1: JInt, "ccc", null: JInt, null)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "bbb", 1: JInt, "aaa"), change = true)))
+      CRow(1: JInt, "bbb", 1: JInt, "aaa")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "ccc", 1: JInt, "aaa"), change = true)))
+      CRow(1: JInt, "ccc", 1: JInt, "aaa")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "bbb", 1: JInt, "ddd"), change = true)))
+      CRow(1: JInt, "bbb", 1: JInt, "ddd")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "ccc", 1: JInt, "ddd"), change = true)))
+      CRow(1: JInt, "ccc", 1: JInt, "ddd")))
     // processing time 4
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2: JInt, "aaa", 2: JInt, "bbb"), change = false)))
+      CRow(false, 2: JInt, "aaa", 2: JInt, "bbb")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2: JInt, "aaa", 2: JInt, "ccc"), change = false)))
+      CRow(false, 2: JInt, "aaa", 2: JInt, "ccc")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2: JInt, "ddd", 2: JInt, "bbb"), change = false)))
+      CRow(false, 2: JInt, "ddd", 2: JInt, "bbb")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2: JInt, "ddd", 2: JInt, "ccc"), change = false)))
+      CRow(false, 2: JInt, "ddd", 2: JInt, "ccc")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JInt, null, 2: JInt, "bbb"), change = true)))
+      CRow(null: JInt, null, 2: JInt, "bbb")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JInt, null, 2: JInt, "ccc"), change = true)))
+      CRow(null: JInt, null, 2: JInt, "ccc")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "bbb", 1: JInt, "aaa"), change = false)))
+      CRow(false, 1: JInt, "bbb", 1: JInt, "aaa")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "ccc", 1: JInt, "aaa"), change = false)))
+      CRow(false, 1: JInt, "ccc", 1: JInt, "aaa")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "bbb", 1: JInt, "ddd"), change = false)))
+      CRow(false, 1: JInt, "bbb", 1: JInt, "ddd")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "ccc", 1: JInt, "ddd"), change = false)))
+      CRow(false, 1: JInt, "ccc", 1: JInt, "ddd")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "bbb", null: JInt, null), change = true)))
+      CRow(1: JInt, "bbb", null: JInt, null)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "ccc", null: JInt, null), change = true)))
+      CRow(1: JInt, "ccc", null: JInt, null)))
     // processing time 8
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "bbb", null: JInt, null), change = true)))
+      CRow(1: JInt, "bbb", null: JInt, null)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JInt, null, 2: JInt, "bbb"), change = true)))
+      CRow(null: JInt, null, 2: JInt, "bbb")))
 
     verify(expectedOutput, result, new RowResultSortComparator())
     testHarness.close()
@@ -1710,18 +1710,18 @@ class JoinHarnessTest extends HarnessTestBase {
     // left stream input
     testHarness.setProcessingTime(1)
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1: JInt, "bbb"), change = true)))
+      CRow(1: JInt, "bbb")))
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1: JInt, "ccc"), change = true)))
+      CRow(1: JInt, "ccc")))
     assertEquals(1, testHarness.numProcessingTimeTimers())
     // 1 left timer(5), 1 left key(1), 1 left joincnt key(1)
     assertEquals(3, testHarness.numKeyedStateEntries())
 
     testHarness.setProcessingTime(2)
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(2: JInt, "bbb"), change = true)))
+      CRow(2: JInt, "bbb")))
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(2: JInt, "ccc"), change = true)))
+      CRow(2: JInt, "ccc")))
     assertEquals(2, testHarness.numProcessingTimeTimers())
     // 1 left timer(5), 1 left key(1), 1 left joincnt key(1)
     // 1 right timer(6), 1 right key(1), 1 right joincnt key(1)
@@ -1729,17 +1729,17 @@ class JoinHarnessTest extends HarnessTestBase {
 
     testHarness.setProcessingTime(3)
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(2: JInt, "aaa"), change = true)))
+      CRow(2: JInt, "aaa")))
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(2: JInt, "ddd"), change = true)))
+      CRow(2: JInt, "ddd")))
     assertEquals(3, testHarness.numProcessingTimeTimers())
     // 2 left timer(5,7), 2 left key(1,2), 2 left joincnt key(1,2)
     // 1 right timer(6), 1 right key(1), 1 right joincnt key(1)
     assertEquals(9, testHarness.numKeyedStateEntries())
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1: JInt, "aaa"), change = true)))
+      CRow(1: JInt, "aaa")))
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1: JInt, "ddd"), change = true)))
+      CRow(1: JInt, "ddd")))
     assertEquals(4, testHarness.numProcessingTimeTimers())
     // 2 left timer(5,7), 2 left key(1,2), 2 left joincnt key(1,2)
     // 2 right timer(6,7), 2 right key(1,2), 2 right joincnt key(1,2)
@@ -1747,9 +1747,9 @@ class JoinHarnessTest extends HarnessTestBase {
 
     testHarness.setProcessingTime(4)
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(2: JInt, "aaa"), change = false)))
+      CRow(false, 2: JInt, "aaa")))
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(1: JInt, "ddd"), change = false)))
+      CRow(false, 1: JInt, "ddd")))
     assertEquals(4, testHarness.numProcessingTimeTimers())
     // 2 left timer(5,7), 2 left key(1,2), 2 left joincnt key(1,2)
     // 2 right timer(6,7), 2 right key(1,2), 2 right joincnt key(1,2)
@@ -1773,68 +1773,68 @@ class JoinHarnessTest extends HarnessTestBase {
 
     testHarness.setProcessingTime(8)
     testHarness.processElement1(new StreamRecord(
-      CRow(Row.of(1: JInt, "bbb"), change = true)))
+      CRow(1: JInt, "bbb")))
     testHarness.processElement2(new StreamRecord(
-      CRow(Row.of(2: JInt, "bbb"), change = true)))
+      CRow(2: JInt, "bbb")))
 
     val result = testHarness.getOutput
     val expectedOutput = new ConcurrentLinkedQueue[Object]()
 
     // processing time 1
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "bbb", null: JInt, null), change = true)))
+      CRow(1: JInt, "bbb", null: JInt, null)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "ccc", null: JInt, null), change = true)))
+      CRow(1: JInt, "ccc", null: JInt, null)))
     // processing time 2
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JInt, null, 2: JInt, "bbb"), change = true)))
+      CRow(null: JInt, null, 2: JInt, "bbb")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JInt, null, 2: JInt, "ccc"), change = true)))
+      CRow(null: JInt, null, 2: JInt, "ccc")))
     // processing time 3
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JInt, null, 2: JInt, "bbb"), change = false)))
+      CRow(false, null: JInt, null, 2: JInt, "bbb")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JInt, null, 2: JInt, "ccc"), change = false)))
+      CRow(false, null: JInt, null, 2: JInt, "ccc")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2: JInt, "aaa", 2: JInt, "bbb"), change = true)))
+      CRow(2: JInt, "aaa", 2: JInt, "bbb")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2: JInt, "aaa", 2: JInt, "ccc"), change = true)))
+      CRow(2: JInt, "aaa", 2: JInt, "ccc")))
     // can not find matched row due to NonEquiJoinPred
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2: JInt, "ddd", null: JInt, null), change = true)))
+      CRow(2: JInt, "ddd", null: JInt, null)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "bbb", null: JInt, null), change = false)))
+      CRow(false, 1: JInt, "bbb", null: JInt, null)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "ccc", null: JInt, null), change = false)))
+      CRow(false, 1: JInt, "ccc", null: JInt, null)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "bbb", 1: JInt, "ddd"), change = true)))
+      CRow(1: JInt, "bbb", 1: JInt, "ddd")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "ccc", 1: JInt, "ddd"), change = true)))
+      CRow(1: JInt, "ccc", 1: JInt, "ddd")))
     // can not find matched row due to NonEquiJoinPred
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JInt, null, 1: JInt, "aaa"), change = true)))
+      CRow(null: JInt, null, 1: JInt, "aaa")))
     // processing time 4
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2: JInt, "aaa", 2: JInt, "bbb"), change = false)))
+      CRow(false, 2: JInt, "aaa", 2: JInt, "bbb")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2: JInt, "aaa", 2: JInt, "ccc"), change = false)))
+      CRow(false, 2: JInt, "aaa", 2: JInt, "ccc")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JInt, null, 2: JInt, "bbb"), change = true)))
+      CRow(null: JInt, null, 2: JInt, "bbb")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JInt, null, 2: JInt, "ccc"), change = true)))
+      CRow(null: JInt, null, 2: JInt, "ccc")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "bbb", 1: JInt, "ddd"), change = false)))
+      CRow(false, 1: JInt, "bbb", 1: JInt, "ddd")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "ccc", 1: JInt, "ddd"), change = false)))
+      CRow(false, 1: JInt, "ccc", 1: JInt, "ddd")))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "bbb", null: JInt, null), change = true)))
+      CRow(1: JInt, "bbb", null: JInt, null)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "ccc", null: JInt, null), change = true)))
+      CRow(1: JInt, "ccc", null: JInt, null)))
     // processing time 8
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1: JInt, "bbb", null: JInt, null), change = true)))
+      CRow(1: JInt, "bbb", null: JInt, null)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(null: JInt, null, 2: JInt, "bbb"), change = true)))
+      CRow(null: JInt, null, 2: JInt, "bbb")))
 
     verify(expectedOutput, result, new RowResultSortComparator())
     testHarness.close()
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala
index 2f4bbfa..d6daa9e 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala
@@ -57,39 +57,39 @@ class NonWindowHarnessTest extends HarnessTestBase {
     // register cleanup timer with 3001
     testHarness.setProcessingTime(1)
 
-    testHarness.processElement(new StreamRecord(CRow(Row.of(1L: JLong, 1: JInt, "aaa"), true), 1))
-    testHarness.processElement(new StreamRecord(CRow(Row.of(2L: JLong, 1: JInt, "bbb"), true), 1))
+    testHarness.processElement(new StreamRecord(CRow(1L: JLong, 1: JInt, "aaa"), 1))
+    testHarness.processElement(new StreamRecord(CRow(2L: JLong, 1: JInt, "bbb"), 1))
     // reuse timer 3001
     testHarness.setProcessingTime(1000)
-    testHarness.processElement(new StreamRecord(CRow(Row.of(3L: JLong, 2: JInt, "aaa"), true), 1))
-    testHarness.processElement(new StreamRecord(CRow(Row.of(4L: JLong, 3: JInt, "aaa"), true), 1))
+    testHarness.processElement(new StreamRecord(CRow(3L: JLong, 2: JInt, "aaa"), 1))
+    testHarness.processElement(new StreamRecord(CRow(4L: JLong, 3: JInt, "aaa"), 1))
 
     // register cleanup timer with 4002
     testHarness.setProcessingTime(1002)
-    testHarness.processElement(new StreamRecord(CRow(Row.of(5L: JLong, 4: JInt, "aaa"), true), 1))
-    testHarness.processElement(new StreamRecord(CRow(Row.of(6L: JLong, 2: JInt, "bbb"), true), 1))
+    testHarness.processElement(new StreamRecord(CRow(5L: JLong, 4: JInt, "aaa"), 1))
+    testHarness.processElement(new StreamRecord(CRow(6L: JLong, 2: JInt, "bbb"), 1))
 
     // trigger cleanup timer and register cleanup timer with 7003
     testHarness.setProcessingTime(4003)
-    testHarness.processElement(new StreamRecord(CRow(Row.of(7L: JLong, 5: JInt, "aaa"), true), 1))
-    testHarness.processElement(new StreamRecord(CRow(Row.of(8L: JLong, 6: JInt, "aaa"), true), 1))
-    testHarness.processElement(new StreamRecord(CRow(Row.of(9L: JLong, 7: JInt, "aaa"), true), 1))
-    testHarness.processElement(new StreamRecord(CRow(Row.of(10L: JLong, 3: JInt, "bbb"), true), 1))
+    testHarness.processElement(new StreamRecord(CRow(7L: JLong, 5: JInt, "aaa"), 1))
+    testHarness.processElement(new StreamRecord(CRow(8L: JLong, 6: JInt, "aaa"), 1))
+    testHarness.processElement(new StreamRecord(CRow(9L: JLong, 7: JInt, "aaa"), 1))
+    testHarness.processElement(new StreamRecord(CRow(10L: JLong, 3: JInt, "bbb"), 1))
 
     val result = testHarness.getOutput
 
     val expectedOutput = new ConcurrentLinkedQueue[Object]()
 
-    expectedOutput.add(new StreamRecord(CRow(Row.of(1L: JLong, 1: JInt), true), 1))
-    expectedOutput.add(new StreamRecord(CRow(Row.of(2L: JLong, 1: JInt), true), 1))
-    expectedOutput.add(new StreamRecord(CRow(Row.of(3L: JLong, 3: JInt), true), 1))
-    expectedOutput.add(new StreamRecord(CRow(Row.of(4L: JLong, 6: JInt), true), 1))
-    expectedOutput.add(new StreamRecord(CRow(Row.of(5L: JLong, 10: JInt), true), 1))
-    expectedOutput.add(new StreamRecord(CRow(Row.of(6L: JLong, 3: JInt), true), 1))
-    expectedOutput.add(new StreamRecord(CRow(Row.of(7L: JLong, 5: JInt), true), 1))
-    expectedOutput.add(new StreamRecord(CRow(Row.of(8L: JLong, 11: JInt), true), 1))
-    expectedOutput.add(new StreamRecord(CRow(Row.of(9L: JLong, 18: JInt), true), 1))
-    expectedOutput.add(new StreamRecord(CRow(Row.of(10L: JLong, 3: JInt), true), 1))
+    expectedOutput.add(new StreamRecord(CRow(1L: JLong, 1: JInt), 1))
+    expectedOutput.add(new StreamRecord(CRow(2L: JLong, 1: JInt), 1))
+    expectedOutput.add(new StreamRecord(CRow(3L: JLong, 3: JInt), 1))
+    expectedOutput.add(new StreamRecord(CRow(4L: JLong, 6: JInt), 1))
+    expectedOutput.add(new StreamRecord(CRow(5L: JLong, 10: JInt), 1))
+    expectedOutput.add(new StreamRecord(CRow(6L: JLong, 3: JInt), 1))
+    expectedOutput.add(new StreamRecord(CRow(7L: JLong, 5: JInt), 1))
+    expectedOutput.add(new StreamRecord(CRow(8L: JLong, 11: JInt), 1))
+    expectedOutput.add(new StreamRecord(CRow(9L: JLong, 18: JInt), 1))
+    expectedOutput.add(new StreamRecord(CRow(10L: JLong, 3: JInt), 1))
 
     verify(expectedOutput, result, new RowResultSortComparator())
 
@@ -117,38 +117,38 @@ class NonWindowHarnessTest extends HarnessTestBase {
     // register cleanup timer with 3001
     testHarness.setProcessingTime(1)
 
-    testHarness.processElement(new StreamRecord(CRow(Row.of(1L: JLong, 1: JInt, "aaa"), true), 1))
-    testHarness.processElement(new StreamRecord(CRow(Row.of(2L: JLong, 1: JInt, "bbb"), true), 2))
-    testHarness.processElement(new StreamRecord(CRow(Row.of(3L: JLong, 2: JInt, "aaa"), true), 3))
-    testHarness.processElement(new StreamRecord(CRow(Row.of(4L: JLong, 3: JInt, "ccc"), true), 4))
+    testHarness.processElement(new StreamRecord(CRow(1L: JLong, 1: JInt, "aaa"), 1))
+    testHarness.processElement(new StreamRecord(CRow(2L: JLong, 1: JInt, "bbb"), 2))
+    testHarness.processElement(new StreamRecord(CRow(3L: JLong, 2: JInt, "aaa"), 3))
+    testHarness.processElement(new StreamRecord(CRow(4L: JLong, 3: JInt, "ccc"), 4))
 
     // trigger cleanup timer and register cleanup timer with 6002
     testHarness.setProcessingTime(3002)
-    testHarness.processElement(new StreamRecord(CRow(Row.of(5L: JLong, 4: JInt, "aaa"), true), 5))
-    testHarness.processElement(new StreamRecord(CRow(Row.of(6L: JLong, 2: JInt, "bbb"), true), 6))
-    testHarness.processElement(new StreamRecord(CRow(Row.of(7L: JLong, 5: JInt, "aaa"), true), 7))
-    testHarness.processElement(new StreamRecord(CRow(Row.of(8L: JLong, 6: JInt, "eee"), true), 8))
-    testHarness.processElement(new StreamRecord(CRow(Row.of(9L: JLong, 7: JInt, "aaa"), true), 9))
-    testHarness.processElement(new StreamRecord(CRow(Row.of(10L: JLong, 3: JInt, "bbb"), true), 10))
+    testHarness.processElement(new StreamRecord(CRow(5L: JLong, 4: JInt, "aaa"), 5))
+    testHarness.processElement(new StreamRecord(CRow(6L: JLong, 2: JInt, "bbb"), 6))
+    testHarness.processElement(new StreamRecord(CRow(7L: JLong, 5: JInt, "aaa"), 7))
+    testHarness.processElement(new StreamRecord(CRow(8L: JLong, 6: JInt, "eee"), 8))
+    testHarness.processElement(new StreamRecord(CRow(9L: JLong, 7: JInt, "aaa"), 9))
+    testHarness.processElement(new StreamRecord(CRow(10L: JLong, 3: JInt, "bbb"), 10))
 
     val result = testHarness.getOutput
 
     val expectedOutput = new ConcurrentLinkedQueue[Object]()
 
-    expectedOutput.add(new StreamRecord(CRow(Row.of(1L: JLong, 1: JInt), true), 1))
-    expectedOutput.add(new StreamRecord(CRow(Row.of(2L: JLong, 1: JInt), true), 2))
-    expectedOutput.add(new StreamRecord(CRow(Row.of(3L: JLong, 1: JInt), false), 3))
-    expectedOutput.add(new StreamRecord(CRow(Row.of(3L: JLong, 3: JInt), true), 3))
-    expectedOutput.add(new StreamRecord(CRow(Row.of(4L: JLong, 3: JInt), true), 4))
-    expectedOutput.add(new StreamRecord(CRow(Row.of(5L: JLong, 4: JInt), true), 5))
-    expectedOutput.add(new StreamRecord(CRow(Row.of(6L: JLong, 2: JInt), true), 6))
-    expectedOutput.add(new StreamRecord(CRow(Row.of(7L: JLong, 4: JInt), false), 7))
-    expectedOutput.add(new StreamRecord(CRow(Row.of(7L: JLong, 9: JInt), true), 7))
-    expectedOutput.add(new StreamRecord(CRow(Row.of(8L: JLong, 6: JInt), true), 8))
-    expectedOutput.add(new StreamRecord(CRow(Row.of(9L: JLong, 9: JInt), false), 9))
-    expectedOutput.add(new StreamRecord(CRow(Row.of(9L: JLong, 16: JInt), true), 9))
-    expectedOutput.add(new StreamRecord(CRow(Row.of(10L: JLong, 2: JInt), false), 10))
-    expectedOutput.add(new StreamRecord(CRow(Row.of(10L: JLong, 5: JInt), true), 10))
+    expectedOutput.add(new StreamRecord(CRow(1L: JLong, 1: JInt), 1))
+    expectedOutput.add(new StreamRecord(CRow(2L: JLong, 1: JInt), 2))
+    expectedOutput.add(new StreamRecord(CRow(false, 3L: JLong, 1: JInt), 3))
+    expectedOutput.add(new StreamRecord(CRow(3L: JLong, 3: JInt), 3))
+    expectedOutput.add(new StreamRecord(CRow(4L: JLong, 3: JInt), 4))
+    expectedOutput.add(new StreamRecord(CRow(5L: JLong, 4: JInt), 5))
+    expectedOutput.add(new StreamRecord(CRow(6L: JLong, 2: JInt), 6))
+    expectedOutput.add(new StreamRecord(CRow(false, 7L: JLong, 4: JInt), 7))
+    expectedOutput.add(new StreamRecord(CRow(7L: JLong, 9: JInt), 7))
+    expectedOutput.add(new StreamRecord(CRow(8L: JLong, 6: JInt), 8))
+    expectedOutput.add(new StreamRecord(CRow(false, 9L: JLong, 9: JInt), 9))
+    expectedOutput.add(new StreamRecord(CRow(9L: JLong, 16: JInt), 9))
+    expectedOutput.add(new StreamRecord(CRow(false, 10L: JLong, 2: JInt), 10))
+    expectedOutput.add(new StreamRecord(CRow(10L: JLong, 5: JInt), 10))
 
     verify(expectedOutput, result, new RowResultSortComparator())
 
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/OverWindowHarnessTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/OverWindowHarnessTest.scala
index 7ad64c6..63d7b5d 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/OverWindowHarnessTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/OverWindowHarnessTest.scala
@@ -60,75 +60,75 @@ class OverWindowHarnessTest extends HarnessTestBase{
     testHarness.setProcessingTime(1)
 
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(1L: JLong, "aaa", 1L: JLong), change = true)))
+      CRow(1L: JLong, "aaa", 1L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(1L: JLong, "bbb", 10L: JLong), change = true)))
+      CRow(1L: JLong, "bbb", 10L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(1L: JLong, "aaa", 2L: JLong), change = true)))
+      CRow(1L: JLong, "aaa", 2L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(1L: JLong, "aaa", 3L: JLong), change = true)))
+      CRow(1L: JLong, "aaa", 3L: JLong)))
 
     // register cleanup timer with 4100
     testHarness.setProcessingTime(1100)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(1L: JLong, "bbb", 20L: JLong), change = true)))
+      CRow(1L: JLong, "bbb", 20L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(1L: JLong, "aaa", 4L: JLong), change = true)))
+      CRow(1L: JLong, "aaa", 4L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(1L: JLong, "aaa", 5L: JLong), change = true)))
+      CRow(1L: JLong, "aaa", 5L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(1L: JLong, "aaa", 6L: JLong), change = true)))
+      CRow(1L: JLong, "aaa", 6L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(1L: JLong, "bbb", 30L: JLong), change = true)))
+      CRow(1L: JLong, "bbb", 30L: JLong)))
 
     // register cleanup timer with 6001
     testHarness.setProcessingTime(3001)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(2L: JLong, "aaa", 7L: JLong), change = true)))
+      CRow(2L: JLong, "aaa", 7L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(2L: JLong, "aaa", 8L: JLong), change = true)))
+      CRow(2L: JLong, "aaa", 8L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(2L: JLong, "aaa", 9L: JLong), change = true)))
+      CRow(2L: JLong, "aaa", 9L: JLong)))
 
     // trigger cleanup timer and register cleanup timer with 9002
     testHarness.setProcessingTime(6002)
     testHarness.processElement(new StreamRecord(
-        CRow(Row.of(2L: JLong, "aaa", 10L: JLong), change = true)))
+        CRow(2L: JLong, "aaa", 10L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(2L: JLong, "bbb", 40L: JLong), change = true)))
+      CRow(2L: JLong, "bbb", 40L: JLong)))
 
     val result = testHarness.getOutput
 
     val expectedOutput = new ConcurrentLinkedQueue[Object]()
 
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1L: JLong, "aaa", 1L: JLong, 1L: JLong, 1L: JLong), change = true)))
+      CRow(1L: JLong, "aaa", 1L: JLong, 1L: JLong, 1L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1L: JLong, "bbb", 10L: JLong, 10L: JLong, 10L: JLong), change = true)))
+      CRow(1L: JLong, "bbb", 10L: JLong, 10L: JLong, 10L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1L: JLong, "aaa", 2L: JLong, 1L: JLong, 2L: JLong), change = true)))
+      CRow(1L: JLong, "aaa", 2L: JLong, 1L: JLong, 2L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1L: JLong, "aaa", 3L: JLong, 2L: JLong, 3L: JLong), change = true)))
+      CRow(1L: JLong, "aaa", 3L: JLong, 2L: JLong, 3L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1L: JLong, "bbb", 20L: JLong, 10L: JLong, 20L: JLong), change = true)))
+      CRow(1L: JLong, "bbb", 20L: JLong, 10L: JLong, 20L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1L: JLong, "aaa", 4L: JLong, 3L: JLong, 4L: JLong), change = true)))
+      CRow(1L: JLong, "aaa", 4L: JLong, 3L: JLong, 4L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1L: JLong, "aaa", 5L: JLong, 4L: JLong, 5L: JLong), change = true)))
+      CRow(1L: JLong, "aaa", 5L: JLong, 4L: JLong, 5L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1L: JLong, "aaa", 6L: JLong, 5L: JLong, 6L: JLong), change = true)))
+      CRow(1L: JLong, "aaa", 6L: JLong, 5L: JLong, 6L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(1L: JLong, "bbb", 30L: JLong, 20L: JLong, 30L: JLong), change = true)))
+      CRow(1L: JLong, "bbb", 30L: JLong, 20L: JLong, 30L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2L: JLong, "aaa", 7L: JLong, 6L: JLong, 7L: JLong), change = true)))
+      CRow(2L: JLong, "aaa", 7L: JLong, 6L: JLong, 7L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2L: JLong, "aaa", 8L: JLong, 7L: JLong, 8L: JLong), change = true)))
+      CRow(2L: JLong, "aaa", 8L: JLong, 7L: JLong, 8L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2L: JLong, "aaa", 9L: JLong, 8L: JLong, 9L: JLong), change = true)))
+      CRow(2L: JLong, "aaa", 9L: JLong, 8L: JLong, 9L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2L: JLong, "aaa", 10L: JLong, 10L: JLong, 10L: JLong), change = true)))
+      CRow(2L: JLong, "aaa", 10L: JLong, 10L: JLong, 10L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2L: JLong, "bbb", 40L: JLong, 40L: JLong, 40L: JLong), change = true)))
+      CRow(2L: JLong, "bbb", 40L: JLong, 40L: JLong, 40L: JLong)))
 
     verify(expectedOutput, result, new RowResultSortComparator())
 
@@ -160,51 +160,51 @@ class OverWindowHarnessTest extends HarnessTestBase{
     // register cleanup timer with 3003
     testHarness.setProcessingTime(3)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 1L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 1L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(0L: JLong, "bbb", 10L: JLong), change = true)))
+      CRow(0L: JLong, "bbb", 10L: JLong)))
 
     testHarness.setProcessingTime(4)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 2L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 2L: JLong)))
 
     // trigger cleanup timer and register cleanup timer with 6003
     testHarness.setProcessingTime(3003)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 3L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 3L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(0L: JLong, "bbb", 20L: JLong), change = true)))
+      CRow(0L: JLong, "bbb", 20L: JLong)))
 
     testHarness.setProcessingTime(5)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 4L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 4L: JLong)))
 
     // register cleanup timer with 9002
     testHarness.setProcessingTime(6002)
 
     testHarness.setProcessingTime(7002)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 5L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 5L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 6L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 6L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(0L: JLong, "bbb", 30L: JLong), change = true)))
+      CRow(0L: JLong, "bbb", 30L: JLong)))
 
     // register cleanup timer with 14002
     testHarness.setProcessingTime(11002)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 7L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 7L: JLong)))
 
     testHarness.setProcessingTime(11004)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 8L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 8L: JLong)))
 
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 9L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 9L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 10L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 10L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(0L: JLong, "bbb", 40L: JLong), change = true)))
+      CRow(0L: JLong, "bbb", 40L: JLong)))
 
     testHarness.setProcessingTime(11006)
 
@@ -213,12 +213,12 @@ class OverWindowHarnessTest extends HarnessTestBase{
 
     // timer registered for 23000
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(0L: JLong, "ccc", 10L: JLong), change = true)))
+      CRow(0L: JLong, "ccc", 10L: JLong)))
 
     // update clean-up timer to 25500. Previous timer should not clean up
     testHarness.setProcessingTime(22500)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(0L: JLong, "ccc", 20L: JLong), change = true)))
+      CRow(0L: JLong, "ccc", 20L: JLong)))
 
     // 23000 clean-up timer should fire but not fail with an NPE
     testHarness.setProcessingTime(23001)
@@ -229,37 +229,37 @@ class OverWindowHarnessTest extends HarnessTestBase{
 
     // all elements at the same proc timestamp have the same value per key
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 1L: JLong, 1L: JLong, 1L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 1L: JLong, 1L: JLong, 1L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(0L: JLong, "bbb", 10L: JLong, 10L: JLong, 10L: JLong), change = true)))
+      CRow(0L: JLong, "bbb", 10L: JLong, 10L: JLong, 10L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 2L: JLong, 1L: JLong, 2L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 2L: JLong, 1L: JLong, 2L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 3L: JLong, 3L: JLong, 4L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 3L: JLong, 3L: JLong, 4L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(0L: JLong, "bbb", 20L: JLong, 20L: JLong, 20L: JLong), change = true)))
+      CRow(0L: JLong, "bbb", 20L: JLong, 20L: JLong, 20L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 4L: JLong, 4L: JLong, 4L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 4L: JLong, 4L: JLong, 4L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 5L: JLong, 5L: JLong, 6L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 5L: JLong, 5L: JLong, 6L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 6L: JLong, 5L: JLong, 6L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 6L: JLong, 5L: JLong, 6L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(0L: JLong, "bbb", 30L: JLong, 30L: JLong, 30L: JLong), change = true)))
+      CRow(0L: JLong, "bbb", 30L: JLong, 30L: JLong, 30L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 7L: JLong, 7L: JLong, 7L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 7L: JLong, 7L: JLong, 7L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 8L: JLong, 7L: JLong, 10L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 8L: JLong, 7L: JLong, 10L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 9L: JLong, 7L: JLong, 10L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 9L: JLong, 7L: JLong, 10L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 10L: JLong, 7L: JLong, 10L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 10L: JLong, 7L: JLong, 10L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(0L: JLong, "bbb", 40L: JLong, 40L: JLong, 40L: JLong), change = true)))
+      CRow(0L: JLong, "bbb", 40L: JLong, 40L: JLong, 40L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(0L: JLong, "ccc", 10L: JLong, 10L: JLong, 10L: JLong), change = true)))
+      CRow(0L: JLong, "ccc", 10L: JLong, 10L: JLong, 10L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(0L: JLong, "ccc", 20L: JLong, 10L: JLong, 20L: JLong), change = true)))
+      CRow(0L: JLong, "ccc", 20L: JLong, 10L: JLong, 20L: JLong)))
 
     verify(expectedOutput, result, new RowResultSortComparator())
 
@@ -287,69 +287,69 @@ class OverWindowHarnessTest extends HarnessTestBase{
     testHarness.setProcessingTime(1003)
 
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 1L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 1L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(0L: JLong, "bbb", 10L: JLong), change = true)))
+      CRow(0L: JLong, "bbb", 10L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 2L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 2L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 3L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 3L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(0L: JLong, "bbb", 20L: JLong), change = true)))
+      CRow(0L: JLong, "bbb", 20L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 4L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 4L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 5L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 5L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 6L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 6L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(0L: JLong, "bbb", 30L: JLong), change = true)))
+      CRow(0L: JLong, "bbb", 30L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 7L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 7L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 8L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 8L: JLong)))
 
     // trigger cleanup timer and register cleanup timer with 8003
     testHarness.setProcessingTime(5003)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 9L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 9L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 10L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 10L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(0L: JLong, "bbb", 40L: JLong), change = true)))
+      CRow(0L: JLong, "bbb", 40L: JLong)))
 
     val result = testHarness.getOutput
 
     val expectedOutput = new ConcurrentLinkedQueue[Object]()
 
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 1L: JLong, 1L: JLong, 1L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 1L: JLong, 1L: JLong, 1L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(0L: JLong, "bbb", 10L: JLong, 10L: JLong, 10L: JLong), change = true)))
+      CRow(0L: JLong, "bbb", 10L: JLong, 10L: JLong, 10L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 2L: JLong, 1L: JLong, 2L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 2L: JLong, 1L: JLong, 2L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 3L: JLong, 1L: JLong, 3L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 3L: JLong, 1L: JLong, 3L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(0L: JLong, "bbb", 20L: JLong, 10L: JLong, 20L: JLong), change = true)))
+      CRow(0L: JLong, "bbb", 20L: JLong, 10L: JLong, 20L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 4L: JLong, 1L: JLong, 4L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 4L: JLong, 1L: JLong, 4L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 5L: JLong, 1L: JLong, 5L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 5L: JLong, 1L: JLong, 5L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 6L: JLong, 1L: JLong, 6L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 6L: JLong, 1L: JLong, 6L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(0L: JLong, "bbb", 30L: JLong, 10L: JLong, 30L: JLong), change = true)))
+      CRow(0L: JLong, "bbb", 30L: JLong, 10L: JLong, 30L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 7L: JLong, 1L: JLong, 7L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 7L: JLong, 1L: JLong, 7L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 8L: JLong, 1L: JLong, 8L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 8L: JLong, 1L: JLong, 8L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 9L: JLong, 9L: JLong, 9L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 9L: JLong, 9L: JLong, 9L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(0L: JLong, "aaa", 10L: JLong, 9L: JLong, 10L: JLong), change = true)))
+      CRow(0L: JLong, "aaa", 10L: JLong, 9L: JLong, 10L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(0L: JLong, "bbb", 40L: JLong, 40L: JLong, 40L: JLong), change = true)))
+      CRow(0L: JLong, "bbb", 40L: JLong, 40L: JLong, 40L: JLong)))
 
     verify(expectedOutput, result, new RowResultSortComparator())
     testHarness.close()
@@ -380,51 +380,51 @@ class OverWindowHarnessTest extends HarnessTestBase{
 
     testHarness.processWatermark(1)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(2L: JLong, "aaa", 1L: JLong), change = true)))
+      CRow(2L: JLong, "aaa", 1L: JLong)))
 
     testHarness.processWatermark(2)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(3L: JLong, "bbb", 10L: JLong), change = true)))
+      CRow(3L: JLong, "bbb", 10L: JLong)))
 
     testHarness.processWatermark(4000)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(4001L: JLong, "aaa", 2L: JLong), change = true)))
+      CRow(4001L: JLong, "aaa", 2L: JLong)))
 
     testHarness.processWatermark(4001)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(4002L: JLong, "aaa", 3L: JLong), change = true)))
+      CRow(4002L: JLong, "aaa", 3L: JLong)))
 
     testHarness.processWatermark(4002)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(4003L: JLong, "aaa", 4L: JLong), change = true)))
+      CRow(4003L: JLong, "aaa", 4L: JLong)))
 
     testHarness.processWatermark(4800)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(4801L: JLong, "bbb", 25L: JLong), change = true)))
+      CRow(4801L: JLong, "bbb", 25L: JLong)))
 
     testHarness.processWatermark(6500)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(6501L: JLong, "aaa", 5L: JLong), change = true)))
+      CRow(6501L: JLong, "aaa", 5L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(6501L: JLong, "aaa", 6L: JLong), change = true)))
+      CRow(6501L: JLong, "aaa", 6L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(6501L: JLong, "bbb", 30L: JLong), change = true)))
+      CRow(6501L: JLong, "bbb", 30L: JLong)))
 
     testHarness.processWatermark(7000)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(7001L: JLong, "aaa", 7L: JLong), change = true)))
+      CRow(7001L: JLong, "aaa", 7L: JLong)))
 
     testHarness.processWatermark(8000)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(8001L: JLong, "aaa", 8L: JLong), change = true)))
+      CRow(8001L: JLong, "aaa", 8L: JLong)))
 
     testHarness.processWatermark(12000)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(12001L: JLong, "aaa", 9L: JLong), change = true)))
+      CRow(12001L: JLong, "aaa", 9L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(12001L: JLong, "aaa", 10L: JLong), change = true)))
+      CRow(12001L: JLong, "aaa", 10L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(12001L: JLong, "bbb", 40L: JLong), change = true)))
+      CRow(12001L: JLong, "bbb", 40L: JLong)))
 
     testHarness.processWatermark(19000)
 
@@ -434,10 +434,10 @@ class OverWindowHarnessTest extends HarnessTestBase{
 
     // check that state is removed after max retention time
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(20001L: JLong, "ccc", 1L: JLong), change = true))) // clean-up 3000
+      CRow(20001L: JLong, "ccc", 1L: JLong))) // clean-up 3000
     testHarness.setProcessingTime(2500)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(20002L: JLong, "ccc", 2L: JLong), change = true))) // clean-up 4500
+      CRow(20002L: JLong, "ccc", 2L: JLong))) // clean-up 4500
     testHarness.processWatermark(20010) // compute output
 
     assert(testHarness.numKeyedStateEntries() > 0) // check that we have state
@@ -449,7 +449,7 @@ class OverWindowHarnessTest extends HarnessTestBase{
 
     // check that state is only removed if all data was processed
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(20011L: JLong, "ccc", 3L: JLong), change = true))) // clean-up 6500
+      CRow(20011L: JLong, "ccc", 3L: JLong))) // clean-up 6500
 
     assert(testHarness.numKeyedStateEntries() > 0) // check that we have state
     testHarness.setProcessingTime(6500) // clean-up attempt but rescheduled to 8500
@@ -469,40 +469,40 @@ class OverWindowHarnessTest extends HarnessTestBase{
 
     // all elements at the same row-time have the same value per key
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2L: JLong, "aaa", 1L: JLong, 1L: JLong, 1L: JLong), change = true)))
+      CRow(2L: JLong, "aaa", 1L: JLong, 1L: JLong, 1L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(3L: JLong, "bbb", 10L: JLong, 10L: JLong, 10L: JLong), change = true)))
+      CRow(3L: JLong, "bbb", 10L: JLong, 10L: JLong, 10L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(4001L: JLong, "aaa", 2L: JLong, 1L: JLong, 2L: JLong), change = true)))
+      CRow(4001L: JLong, "aaa", 2L: JLong, 1L: JLong, 2L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(4002L: JLong, "aaa", 3L: JLong, 1L: JLong, 3L: JLong), change = true)))
+      CRow(4002L: JLong, "aaa", 3L: JLong, 1L: JLong, 3L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(4003L: JLong, "aaa", 4L: JLong, 2L: JLong, 4L: JLong), change = true)))
+      CRow(4003L: JLong, "aaa", 4L: JLong, 2L: JLong, 4L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(4801L: JLong, "bbb", 25L: JLong, 25L: JLong, 25L: JLong), change = true)))
+      CRow(4801L: JLong, "bbb", 25L: JLong, 25L: JLong, 25L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(6501L: JLong, "aaa", 5L: JLong, 2L: JLong, 6L: JLong), change = true)))
+      CRow(6501L: JLong, "aaa", 5L: JLong, 2L: JLong, 6L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(6501L: JLong, "aaa", 6L: JLong, 2L: JLong, 6L: JLong), change = true)))
+      CRow(6501L: JLong, "aaa", 6L: JLong, 2L: JLong, 6L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(7001L: JLong, "aaa", 7L: JLong, 2L: JLong, 7L: JLong), change = true)))
+      CRow(7001L: JLong, "aaa", 7L: JLong, 2L: JLong, 7L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(8001L: JLong, "aaa", 8L: JLong, 2L: JLong, 8L: JLong), change = true)))
+      CRow(8001L: JLong, "aaa", 8L: JLong, 2L: JLong, 8L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(6501L: JLong, "bbb", 30L: JLong, 25L: JLong, 30L: JLong), change = true)))
+      CRow(6501L: JLong, "bbb", 30L: JLong, 25L: JLong, 30L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(12001L: JLong, "aaa", 9L: JLong, 8L: JLong, 10L: JLong), change = true)))
+      CRow(12001L: JLong, "aaa", 9L: JLong, 8L: JLong, 10L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(12001L: JLong, "aaa", 10L: JLong, 8L: JLong, 10L: JLong), change = true)))
+      CRow(12001L: JLong, "aaa", 10L: JLong, 8L: JLong, 10L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(12001L: JLong, "bbb", 40L: JLong, 40L: JLong, 40L: JLong), change = true)))
+      CRow(12001L: JLong, "bbb", 40L: JLong, 40L: JLong, 40L: JLong)))
 
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(20001L: JLong, "ccc", 1L: JLong, 1L: JLong, 1L: JLong), change = true)))
+      CRow(20001L: JLong, "ccc", 1L: JLong, 1L: JLong, 1L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(20002L: JLong, "ccc", 2L: JLong, 1L: JLong, 2L: JLong), change = true)))
+      CRow(20002L: JLong, "ccc", 2L: JLong, 1L: JLong, 2L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(20011L: JLong, "ccc", 3L: JLong, 3L: JLong, 3L: JLong), change = true)))
+      CRow(20011L: JLong, "ccc", 3L: JLong, 3L: JLong, 3L: JLong)))
 
     verify(expectedOutput, result, new RowResultSortComparator())
     testHarness.close()
@@ -530,47 +530,47 @@ class OverWindowHarnessTest extends HarnessTestBase{
 
     testHarness.processWatermark(800)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(801L: JLong, "aaa", 1L: JLong), change = true)))
+      CRow(801L: JLong, "aaa", 1L: JLong)))
 
     testHarness.processWatermark(2500)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(2501L: JLong, "bbb", 10L: JLong), change = true)))
+      CRow(2501L: JLong, "bbb", 10L: JLong)))
 
     testHarness.processWatermark(4000)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(4001L: JLong, "aaa", 2L: JLong), change = true)))
+      CRow(4001L: JLong, "aaa", 2L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(4001L: JLong, "aaa", 3L: JLong), change = true)))
+      CRow(4001L: JLong, "aaa", 3L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(4001L: JLong, "bbb", 20L: JLong), change = true)))
+      CRow(4001L: JLong, "bbb", 20L: JLong)))
 
     testHarness.processWatermark(4800)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(4801L: JLong, "aaa", 4L: JLong), change = true)))
+      CRow(4801L: JLong, "aaa", 4L: JLong)))
 
     testHarness.processWatermark(6500)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(6501L: JLong, "aaa", 5L: JLong), change = true)))
+      CRow(6501L: JLong, "aaa", 5L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(6501L: JLong, "aaa", 6L: JLong), change = true)))
+      CRow(6501L: JLong, "aaa", 6L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(6501L: JLong, "bbb", 30L: JLong), change = true)))
+      CRow(6501L: JLong, "bbb", 30L: JLong)))
 
     testHarness.processWatermark(7000)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(7001L: JLong, "aaa", 7L: JLong), change = true)))
+      CRow(7001L: JLong, "aaa", 7L: JLong)))
 
     testHarness.processWatermark(8000)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(8001L: JLong, "aaa", 8L: JLong), change = true)))
+      CRow(8001L: JLong, "aaa", 8L: JLong)))
 
     testHarness.processWatermark(12000)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(12001L: JLong, "aaa", 9L: JLong), change = true)))
+      CRow(12001L: JLong, "aaa", 9L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(12001L: JLong, "aaa", 10L: JLong), change = true)))
+      CRow(12001L: JLong, "aaa", 10L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(12001L: JLong, "bbb", 40L: JLong), change = true)))
+      CRow(12001L: JLong, "bbb", 40L: JLong)))
 
     testHarness.processWatermark(19000)
 
@@ -580,10 +580,10 @@ class OverWindowHarnessTest extends HarnessTestBase{
 
     // check that state is removed after max retention time
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(20001L: JLong, "ccc", 1L: JLong), change = true))) // clean-up 3000
+      CRow(20001L: JLong, "ccc", 1L: JLong))) // clean-up 3000
     testHarness.setProcessingTime(2500)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(20002L: JLong, "ccc", 2L: JLong), change = true))) // clean-up 4500
+      CRow(20002L: JLong, "ccc", 2L: JLong))) // clean-up 4500
     testHarness.processWatermark(20010) // compute output
 
     assert(testHarness.numKeyedStateEntries() > 0) // check that we have state
@@ -594,7 +594,7 @@ class OverWindowHarnessTest extends HarnessTestBase{
 
     // check that state is only removed if all data was processed
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(20011L: JLong, "ccc", 3L: JLong), change = true))) // clean-up 6500
+      CRow(20011L: JLong, "ccc", 3L: JLong))) // clean-up 6500
 
     assert(testHarness.numKeyedStateEntries() > 0) // check that we have state
     testHarness.setProcessingTime(6500) // clean-up attempt but rescheduled to 8500
@@ -614,40 +614,40 @@ class OverWindowHarnessTest extends HarnessTestBase{
     val expectedOutput = new ConcurrentLinkedQueue[Object]()
 
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(801L: JLong, "aaa", 1L: JLong, 1L: JLong, 1L: JLong), change = true)))
+      CRow(801L: JLong, "aaa", 1L: JLong, 1L: JLong, 1L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2501L: JLong, "bbb", 10L: JLong, 10L: JLong, 10L: JLong), change = true)))
+      CRow(2501L: JLong, "bbb", 10L: JLong, 10L: JLong, 10L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(4001L: JLong, "aaa", 2L: JLong, 1L: JLong, 2L: JLong), change = true)))
+      CRow(4001L: JLong, "aaa", 2L: JLong, 1L: JLong, 2L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(4001L: JLong, "aaa", 3L: JLong, 1L: JLong, 3L: JLong), change = true)))
+      CRow(4001L: JLong, "aaa", 3L: JLong, 1L: JLong, 3L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(4001L: JLong, "bbb", 20L: JLong, 10L: JLong, 20L: JLong), change = true)))
+      CRow(4001L: JLong, "bbb", 20L: JLong, 10L: JLong, 20L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(4801L: JLong, "aaa", 4L: JLong, 2L: JLong, 4L: JLong), change = true)))
+      CRow(4801L: JLong, "aaa", 4L: JLong, 2L: JLong, 4L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(6501L: JLong, "aaa", 5L: JLong, 3L: JLong, 5L: JLong), change = true)))
+      CRow(6501L: JLong, "aaa", 5L: JLong, 3L: JLong, 5L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(6501L: JLong, "aaa", 6L: JLong, 4L: JLong, 6L: JLong), change = true)))
+      CRow(6501L: JLong, "aaa", 6L: JLong, 4L: JLong, 6L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(6501L: JLong, "bbb", 30L: JLong, 10L: JLong, 30L: JLong), change = true)))
+      CRow(6501L: JLong, "bbb", 30L: JLong, 10L: JLong, 30L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(7001L: JLong, "aaa", 7L: JLong, 5L: JLong, 7L: JLong), change = true)))
+      CRow(7001L: JLong, "aaa", 7L: JLong, 5L: JLong, 7L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(8001L: JLong, "aaa", 8L: JLong, 6L: JLong, 8L: JLong), change = true)))
+      CRow(8001L: JLong, "aaa", 8L: JLong, 6L: JLong, 8L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(12001L: JLong, "aaa", 9L: JLong, 7L: JLong, 9L: JLong), change = true)))
+      CRow(12001L: JLong, "aaa", 9L: JLong, 7L: JLong, 9L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(12001L: JLong, "aaa", 10L: JLong, 8L: JLong, 10L: JLong), change = true)))
+      CRow(12001L: JLong, "aaa", 10L: JLong, 8L: JLong, 10L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(12001L: JLong, "bbb", 40L: JLong, 20L: JLong, 40L: JLong), change = true)))
+      CRow(12001L: JLong, "bbb", 40L: JLong, 20L: JLong, 40L: JLong)))
 
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(20001L: JLong, "ccc", 1L: JLong, 1L: JLong, 1L: JLong), change = true)))
+      CRow(20001L: JLong, "ccc", 1L: JLong, 1L: JLong, 1L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(20002L: JLong, "ccc", 2L: JLong, 1L: JLong, 2L: JLong), change = true)))
+      CRow(20002L: JLong, "ccc", 2L: JLong, 1L: JLong, 2L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(20011L: JLong, "ccc", 3L: JLong, 3L: JLong, 3L: JLong), change = true)))
+      CRow(20011L: JLong, "ccc", 3L: JLong, 3L: JLong, 3L: JLong)))
 
     verify(expectedOutput, result, new RowResultSortComparator())
     testHarness.close()
@@ -678,47 +678,47 @@ class OverWindowHarnessTest extends HarnessTestBase{
     testHarness.setProcessingTime(1000)
     testHarness.processWatermark(800)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(801L: JLong, "aaa", 1L: JLong), change = true)))
+      CRow(801L: JLong, "aaa", 1L: JLong)))
 
     testHarness.processWatermark(2500)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(2501L: JLong, "bbb", 10L: JLong), change = true)))
+      CRow(2501L: JLong, "bbb", 10L: JLong)))
 
     testHarness.processWatermark(4000)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(4001L: JLong, "aaa", 2L: JLong), change = true)))
+      CRow(4001L: JLong, "aaa", 2L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(4001L: JLong, "aaa", 3L: JLong), change = true)))
+      CRow(4001L: JLong, "aaa", 3L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(4001L: JLong, "bbb", 20L: JLong), change = true)))
+      CRow(4001L: JLong, "bbb", 20L: JLong)))
 
     testHarness.processWatermark(4800)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(4801L: JLong, "aaa", 4L: JLong), change = true)))
+      CRow(4801L: JLong, "aaa", 4L: JLong)))
 
     testHarness.processWatermark(6500)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(6501L: JLong, "aaa", 5L: JLong), change = true)))
+      CRow(6501L: JLong, "aaa", 5L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(6501L: JLong, "aaa", 6L: JLong), change = true)))
+      CRow(6501L: JLong, "aaa", 6L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(6501L: JLong, "bbb", 30L: JLong), change = true)))
+      CRow(6501L: JLong, "bbb", 30L: JLong)))
 
     testHarness.processWatermark(7000)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(7001L: JLong, "aaa", 7L: JLong), change = true)))
+      CRow(7001L: JLong, "aaa", 7L: JLong)))
 
     testHarness.processWatermark(8000)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(8001L: JLong, "aaa", 8L: JLong), change = true)))
+      CRow(8001L: JLong, "aaa", 8L: JLong)))
 
     testHarness.processWatermark(12000)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(12001L: JLong, "aaa", 9L: JLong), change = true)))
+      CRow(12001L: JLong, "aaa", 9L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(12001L: JLong, "aaa", 10L: JLong), change = true)))
+      CRow(12001L: JLong, "aaa", 10L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(12001L: JLong, "bbb", 40L: JLong), change = true)))
+      CRow(12001L: JLong, "bbb", 40L: JLong)))
 
     testHarness.processWatermark(19000)
 
@@ -731,13 +731,13 @@ class OverWindowHarnessTest extends HarnessTestBase{
 
     testHarness.processWatermark(20000)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(20000L: JLong, "ccc", 1L: JLong), change = true))) // test for late data
+      CRow(20000L: JLong, "ccc", 1L: JLong))) // test for late data
 
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(20001L: JLong, "ccc", 1L: JLong), change = true))) // clean-up 5000
+      CRow(20001L: JLong, "ccc", 1L: JLong))) // clean-up 5000
     testHarness.setProcessingTime(2500)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(20002L: JLong, "ccc", 2L: JLong), change = true))) // clean-up 5000
+      CRow(20002L: JLong, "ccc", 2L: JLong))) // clean-up 5000
 
     assert(testHarness.numKeyedStateEntries() > 0)
     testHarness.setProcessingTime(5000) // does not clean up, because data left. New timer 7000
@@ -755,38 +755,38 @@ class OverWindowHarnessTest extends HarnessTestBase{
 
     // all elements at the same row-time have the same value per key
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(801L: JLong, "aaa", 1L: JLong, 1L: JLong, 1L: JLong), change = true)))
+      CRow(801L: JLong, "aaa", 1L: JLong, 1L: JLong, 1L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2501L: JLong, "bbb", 10L: JLong, 10L: JLong, 10L: JLong), change = true)))
+      CRow(2501L: JLong, "bbb", 10L: JLong, 10L: JLong, 10L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(4001L: JLong, "aaa", 2L: JLong, 1L: JLong, 3L: JLong), change = true)))
+      CRow(4001L: JLong, "aaa", 2L: JLong, 1L: JLong, 3L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(4001L: JLong, "aaa", 3L: JLong, 1L: JLong, 3L: JLong), change = true)))
+      CRow(4001L: JLong, "aaa", 3L: JLong, 1L: JLong, 3L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(4001L: JLong, "bbb", 20L: JLong, 10L: JLong, 20L: JLong), change = true)))
+      CRow(4001L: JLong, "bbb", 20L: JLong, 10L: JLong, 20L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(4801L: JLong, "aaa", 4L: JLong, 1L: JLong, 4L: JLong), change = true)))
+      CRow(4801L: JLong, "aaa", 4L: JLong, 1L: JLong, 4L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(6501L: JLong, "aaa", 5L: JLong, 1L: JLong, 6L: JLong), change = true)))
+      CRow(6501L: JLong, "aaa", 5L: JLong, 1L: JLong, 6L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(6501L: JLong, "aaa", 6L: JLong, 1L: JLong, 6L: JLong), change = true)))
+      CRow(6501L: JLong, "aaa", 6L: JLong, 1L: JLong, 6L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(6501L: JLong, "bbb", 30L: JLong, 10L: JLong, 30L: JLong), change = true)))
+      CRow(6501L: JLong, "bbb", 30L: JLong, 10L: JLong, 30L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(7001L: JLong, "aaa", 7L: JLong, 1L: JLong, 7L: JLong), change = true)))
+      CRow(7001L: JLong, "aaa", 7L: JLong, 1L: JLong, 7L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(8001L: JLong, "aaa", 8L: JLong, 1L: JLong, 8L: JLong), change = true)))
+      CRow(8001L: JLong, "aaa", 8L: JLong, 1L: JLong, 8L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(12001L: JLong, "aaa", 9L: JLong, 1L: JLong, 10L: JLong), change = true)))
+      CRow(12001L: JLong, "aaa", 9L: JLong, 1L: JLong, 10L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(12001L: JLong, "aaa", 10L: JLong, 1L: JLong, 10L: JLong), change = true)))
+      CRow(12001L: JLong, "aaa", 10L: JLong, 1L: JLong, 10L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(12001L: JLong, "bbb", 40L: JLong, 10L: JLong, 40L: JLong), change = true)))
+      CRow(12001L: JLong, "bbb", 40L: JLong, 10L: JLong, 40L: JLong)))
 
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(20001L: JLong, "ccc", 1L: JLong, 1L: JLong, 1L: JLong), change = true)))
+      CRow(20001L: JLong, "ccc", 1L: JLong, 1L: JLong, 1L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(20002L: JLong, "ccc", 2L: JLong, 1L: JLong, 2L: JLong), change = true)))
+      CRow(20002L: JLong, "ccc", 2L: JLong, 1L: JLong, 2L: JLong)))
 
     verify(expectedOutput, result, new RowResultSortComparator())
     testHarness.close()
@@ -814,47 +814,47 @@ class OverWindowHarnessTest extends HarnessTestBase{
     testHarness.setProcessingTime(1000)
     testHarness.processWatermark(800)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(801L: JLong, "aaa", 1L: JLong), change = true)))
+      CRow(801L: JLong, "aaa", 1L: JLong)))
 
     testHarness.processWatermark(2500)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(2501L: JLong, "bbb", 10L: JLong), change = true)))
+      CRow(2501L: JLong, "bbb", 10L: JLong)))
 
     testHarness.processWatermark(4000)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(4001L: JLong, "aaa", 2L: JLong), change = true)))
+      CRow(4001L: JLong, "aaa", 2L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(4001L: JLong, "aaa", 3L: JLong), change = true)))
+      CRow(4001L: JLong, "aaa", 3L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(4001L: JLong, "bbb", 20L: JLong), change = true)))
+      CRow(4001L: JLong, "bbb", 20L: JLong)))
 
     testHarness.processWatermark(4800)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(4801L: JLong, "aaa", 4L: JLong), change = true)))
+      CRow(4801L: JLong, "aaa", 4L: JLong)))
 
     testHarness.processWatermark(6500)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(6501L: JLong, "aaa", 5L: JLong), change = true)))
+      CRow(6501L: JLong, "aaa", 5L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(6501L: JLong, "aaa", 6L: JLong), change = true)))
+      CRow(6501L: JLong, "aaa", 6L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(6501L: JLong, "bbb", 30L: JLong), change = true)))
+      CRow(6501L: JLong, "bbb", 30L: JLong)))
 
     testHarness.processWatermark(7000)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(7001L: JLong, "aaa", 7L: JLong), change = true)))
+      CRow(7001L: JLong, "aaa", 7L: JLong)))
 
     testHarness.processWatermark(8000)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(8001L: JLong, "aaa", 8L: JLong), change = true)))
+      CRow(8001L: JLong, "aaa", 8L: JLong)))
 
     testHarness.processWatermark(12000)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(12001L: JLong, "aaa", 9L: JLong), change = true)))
+      CRow(12001L: JLong, "aaa", 9L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(12001L: JLong, "aaa", 10L: JLong), change = true)))
+      CRow(12001L: JLong, "aaa", 10L: JLong)))
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(12001L: JLong, "bbb", 40L: JLong), change = true)))
+      CRow(12001L: JLong, "bbb", 40L: JLong)))
 
     testHarness.processWatermark(19000)
 
@@ -867,13 +867,13 @@ class OverWindowHarnessTest extends HarnessTestBase{
 
     testHarness.processWatermark(20000)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(20000L: JLong, "ccc", 2L: JLong), change = true))) // test for late data
+      CRow(20000L: JLong, "ccc", 2L: JLong))) // test for late data
 
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(20001L: JLong, "ccc", 1L: JLong), change = true))) // clean-up 5000
+      CRow(20001L: JLong, "ccc", 1L: JLong))) // clean-up 5000
     testHarness.setProcessingTime(2500)
     testHarness.processElement(new StreamRecord(
-      CRow(Row.of(20002L: JLong, "ccc", 2L: JLong), change = true))) // clean-up 5000
+      CRow(20002L: JLong, "ccc", 2L: JLong))) // clean-up 5000
 
     assert(testHarness.numKeyedStateEntries() > 0)
     testHarness.setProcessingTime(5000) // does not clean up, because data left. New timer 7000
@@ -890,38 +890,38 @@ class OverWindowHarnessTest extends HarnessTestBase{
     val expectedOutput = new ConcurrentLinkedQueue[Object]()
 
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(801L: JLong, "aaa", 1L: JLong, 1L: JLong, 1L: JLong), change = true)))
+      CRow(801L: JLong, "aaa", 1L: JLong, 1L: JLong, 1L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(2501L: JLong, "bbb", 10L: JLong, 10L: JLong, 10L: JLong), change = true)))
+      CRow(2501L: JLong, "bbb", 10L: JLong, 10L: JLong, 10L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(4001L: JLong, "aaa", 2L: JLong, 1L: JLong, 2L: JLong), change = true)))
+      CRow(4001L: JLong, "aaa", 2L: JLong, 1L: JLong, 2L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(4001L: JLong, "aaa", 3L: JLong, 1L: JLong, 3L: JLong), change = true)))
+      CRow(4001L: JLong, "aaa", 3L: JLong, 1L: JLong, 3L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(4001L: JLong, "bbb", 20L: JLong, 10L: JLong, 20L: JLong), change = true)))
+      CRow(4001L: JLong, "bbb", 20L: JLong, 10L: JLong, 20L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(4801L: JLong, "aaa", 4L: JLong, 1L: JLong, 4L: JLong), change = true)))
+      CRow(4801L: JLong, "aaa", 4L: JLong, 1L: JLong, 4L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(6501L: JLong, "aaa", 5L: JLong, 1L: JLong, 5L: JLong), change = true)))
+      CRow(6501L: JLong, "aaa", 5L: JLong, 1L: JLong, 5L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(6501L: JLong, "aaa", 6L: JLong, 1L: JLong, 6L: JLong), change = true)))
+      CRow(6501L: JLong, "aaa", 6L: JLong, 1L: JLong, 6L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(6501L: JLong, "bbb", 30L: JLong, 10L: JLong, 30L: JLong), change = true)))
+      CRow(6501L: JLong, "bbb", 30L: JLong, 10L: JLong, 30L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(7001L: JLong, "aaa", 7L: JLong, 1L: JLong, 7L: JLong), change = true)))
+      CRow(7001L: JLong, "aaa", 7L: JLong, 1L: JLong, 7L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(8001L: JLong, "aaa", 8L: JLong, 1L: JLong, 8L: JLong), change = true)))
+      CRow(8001L: JLong, "aaa", 8L: JLong, 1L: JLong, 8L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(12001L: JLong, "aaa", 9L: JLong, 1L: JLong, 9L: JLong), change = true)))
+      CRow(12001L: JLong, "aaa", 9L: JLong, 1L: JLong, 9L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(12001L: JLong, "aaa", 10L: JLong, 1L: JLong, 10L: JLong), change = true)))
+      CRow(12001L: JLong, "aaa", 10L: JLong, 1L: JLong, 10L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(12001L: JLong, "bbb", 40L: JLong, 10L: JLong, 40L: JLong), change = true)))
+      CRow(12001L: JLong, "bbb", 40L: JLong, 10L: JLong, 40L: JLong)))
 
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(20001L: JLong, "ccc", 1L: JLong, 1L: JLong, 1L: JLong), change = true)))
+      CRow(20001L: JLong, "ccc", 1L: JLong, 1L: JLong, 1L: JLong)))
     expectedOutput.add(new StreamRecord(
-      CRow(Row.of(20002L: JLong, "ccc", 2L: JLong, 1L: JLong, 2L: JLong), change = true)))
+      CRow(20002L: JLong, "ccc", 2L: JLong, 1L: JLong, 2L: JLong)))
 
     verify(expectedOutput, result, new RowResultSortComparator())
     testHarness.close()


[flink] 04/11: [hotfix][table] Deduplicate RelTimeInidicatoConverter logic

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

pnowojski pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 3cca8b654b6a245085a0a911a39ff3a8ef2b3ed6
Author: Piotr Nowojski <pi...@gmail.com>
AuthorDate: Thu Sep 20 13:15:43 2018 +0200

    [hotfix][table] Deduplicate RelTimeInidicatoConverter logic
---
 .../table/calcite/RelTimeIndicatorConverter.scala  | 160 +++++++++++----------
 1 file changed, 86 insertions(+), 74 deletions(-)

diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala
index 4f3fbaa..f67b715 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala
@@ -42,10 +42,7 @@ import scala.collection.mutable
   */
 class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttle {
 
-  private val timestamp = rexBuilder
-      .getTypeFactory
-      .asInstanceOf[FlinkTypeFactory]
-      .createTypeFromTypeInfo(SqlTimeTypeInfo.TIMESTAMP, isNullable = false)
+  val materializerUtils = new RexTimeIndicatorMaterializerUtils(rexBuilder)
 
   override def visit(intersect: LogicalIntersect): RelNode =
     throw new TableException("Logical intersect in a stream environment is not supported yet.")
@@ -213,23 +210,9 @@ class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttle {
 
     // add a project to materialize aggregation arguments/grouping keys
 
-    val refIndices = mutable.Set[Int]()
-
-    // check arguments of agg calls
-    aggregate.getAggCallList.foreach(call => if (call.getArgList.size() == 0) {
-        // count(*) has an empty argument list
-        (0 until input.getRowType.getFieldCount).foreach(refIndices.add)
-      } else {
-        // for other aggregations
-        call.getArgList.map(_.asInstanceOf[Int]).foreach(refIndices.add)
-      })
+    val indicesToMaterialize = gatherIndicesToMaterialize(aggregate)
 
-    // check grouping sets
-    aggregate.getGroupSets.foreach(set =>
-      set.asList().map(_.asInstanceOf[Int]).foreach(refIndices.add)
-    )
-
-    val needsMaterialization = refIndices.exists(idx =>
+    val needsMaterialization = indicesToMaterialize.exists(idx =>
       isTimeIndicatorType(input.getRowType.getFieldList.get(idx).getType))
 
     // create project if necessary
@@ -242,17 +225,7 @@ class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttle {
         // merge
         case lp: LogicalProject =>
           val projects = lp.getProjects.zipWithIndex.map { case (expr, idx) =>
-            if (isTimeIndicatorType(expr.getType) && refIndices.contains(idx)) {
-              if (isRowtimeIndicatorType(expr.getType)) {
-                // cast rowtime indicator to regular timestamp
-                rexBuilder.makeAbstractCast(timestamp, expr)
-              } else {
-                // generate proctime access
-                rexBuilder.makeCall(ProctimeSqlFunction, expr)
-              }
-            } else {
-              expr
-            }
+            materializerUtils.materializeIfContains(expr, idx, indicesToMaterialize)
           }
 
           LogicalProject.create(
@@ -262,28 +235,7 @@ class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttle {
 
         // new project
         case _ =>
-          val projects = input.getRowType.getFieldList.map { field =>
-            if (isTimeIndicatorType(field.getType) && refIndices.contains(field.getIndex)) {
-              if (isRowtimeIndicatorType(field.getType)) {
-                // cast rowtime indicator to regular timestamp
-                rexBuilder.makeAbstractCast(
-                  timestamp,
-                  new RexInputRef(field.getIndex, field.getType))
-              } else {
-                // generate proctime access
-                rexBuilder.makeCall(
-                  ProctimeSqlFunction,
-                  new RexInputRef(field.getIndex, field.getType))
-              }
-            } else {
-              new RexInputRef(field.getIndex, field.getType)
-            }
-          }
-
-          LogicalProject.create(
-            input,
-            projects,
-            input.getRowType.getFieldNames)
+          materializerUtils.projectAndMaterializeFields(input, indicesToMaterialize)
       }
     } else {
       // no project necessary
@@ -293,7 +245,7 @@ class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttle {
     // remove time indicator type as agg call return type
     val updatedAggCalls = aggregate.getAggCallList.map { call =>
       val callType = if (isTimeIndicatorType(call.getType)) {
-        timestamp
+        materializerUtils.getTimestamp
       } else {
         call.getType
       }
@@ -314,6 +266,25 @@ class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttle {
       updatedAggCalls)
   }
 
+  private def gatherIndicesToMaterialize(aggregate: Aggregate): Set[Int] = {
+    val indicesToMaterialize = mutable.Set[Int]()
+
+    // check arguments of agg calls
+    aggregate.getAggCallList.foreach(call => if (call.getArgList.size() == 0) {
+      // count(*) has an empty argument list
+      (0 until aggregate.getRowType.getFieldCount).foreach(indicesToMaterialize.add)
+    } else {
+      // for other aggregations
+      call.getArgList.map(_.asInstanceOf[Int]).foreach(indicesToMaterialize.add)
+    })
+
+    // check grouping sets
+    aggregate.getGroupSets.foreach(set =>
+      set.asList().map(_.asInstanceOf[Int]).foreach(indicesToMaterialize.add)
+    )
+
+    indicesToMaterialize.toSet
+  }
 }
 
 object RelTimeIndicatorConverter {
@@ -365,20 +336,21 @@ object RelTimeIndicatorConverter {
   }
 }
 
+/**
+  * Takes `newResolvedInput` types of the [[RexNode]] and if those types have changed rewrites
+  * the [[RexNode]] to make it consistent with new type.
+  */
 class RexTimeIndicatorMaterializer(
   private val rexBuilder: RexBuilder,
-  private val input: Seq[RelDataType])
+  private val newResolvedInput: Seq[RelDataType])
   extends RexShuttle {
 
-  private val timestamp = rexBuilder
-    .getTypeFactory
-    .asInstanceOf[FlinkTypeFactory]
-    .createTypeFromTypeInfo(SqlTimeTypeInfo.TIMESTAMP, isNullable = false)
+  private val materializerUtils = new RexTimeIndicatorMaterializerUtils(rexBuilder)
 
   override def visitInputRef(inputRef: RexInputRef): RexNode = {
     // reference is interesting
     if (isTimeIndicatorType(inputRef.getType)) {
-      val resolvedRefType = input(inputRef.getIndex)
+      val resolvedRefType = newResolvedInput(inputRef.getIndex)
       // input is a valid time indicator
       if (isTimeIndicatorType(resolvedRefType)) {
         inputRef
@@ -405,19 +377,7 @@ class RexTimeIndicatorMaterializer(
         updatedCall.getOperands.toList
 
       case _ =>
-        updatedCall.getOperands.map { o =>
-          if (isTimeIndicatorType(o.getType)) {
-            if (isRowtimeIndicatorType(o.getType)) {
-              // cast rowtime indicator to regular timestamp
-              rexBuilder.makeAbstractCast(timestamp, o)
-            } else {
-              // generate proctime access
-              rexBuilder.makeCall(ProctimeSqlFunction, o)
-            }
-          } else {
-            o
-          }
-        }
+        updatedCall.getOperands.map { materializerUtils.materialize }
     }
 
     // remove time indicator return type
@@ -442,7 +402,7 @@ class RexTimeIndicatorMaterializer(
 
       // materialize function's result and operands
       case _ if isTimeIndicatorType(updatedCall.getType) =>
-        updatedCall.clone(timestamp, materializedOperands)
+        updatedCall.clone(materializerUtils.getTimestamp, materializedOperands)
 
       // materialize function's operands only
       case _ =>
@@ -450,3 +410,55 @@ class RexTimeIndicatorMaterializer(
     }
   }
 }
+
+/**
+  * Helper class for shared logic of materializing time attributes in [[RelNode]] and [[RexNode]].
+  */
+class RexTimeIndicatorMaterializerUtils(rexBuilder: RexBuilder) {
+
+  private val timestamp = rexBuilder
+    .getTypeFactory
+    .asInstanceOf[FlinkTypeFactory]
+    .createTypeFromTypeInfo(SqlTimeTypeInfo.TIMESTAMP, isNullable = false)
+
+  def getTimestamp: RelDataType = {
+    timestamp
+  }
+
+  def projectAndMaterializeFields(input: RelNode, indicesToMaterialize: Set[Int]) : RelNode = {
+    val projects = input.getRowType.getFieldList.map { field =>
+      materializeIfContains(
+        new RexInputRef(field.getIndex, field.getType),
+        field.getIndex,
+        indicesToMaterialize)
+    }
+
+    LogicalProject.create(
+      input,
+      projects,
+      input.getRowType.getFieldNames)
+  }
+
+  def materializeIfContains(expr: RexNode, index: Int, indicesToMaterialize: Set[Int]): RexNode = {
+    if (indicesToMaterialize.contains(index)) {
+      materialize(expr)
+    }
+    else {
+      expr
+    }
+  }
+
+  def materialize(expr: RexNode): RexNode = {
+    if (isTimeIndicatorType(expr.getType)) {
+      if (isRowtimeIndicatorType(expr.getType)) {
+        // cast rowtime indicator to regular timestamp
+        rexBuilder.makeAbstractCast(timestamp, expr)
+      } else {
+        // generate proctime access
+        rexBuilder.makeCall(ProctimeSqlFunction, expr)
+      }
+    } else {
+      expr
+    }
+  }
+}


[flink] 08/11: [hotfix][table] Simplify NonWindowJoin class

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

pnowojski pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 21082a2c349ffc15df064e6250340cdcb748c253
Author: Piotr Nowojski <pi...@gmail.com>
AuthorDate: Thu Jul 19 16:55:47 2018 +0200

    [hotfix][table] Simplify NonWindowJoin class
---
 .../DataStreamJoinToCoProcessTranslator.scala      |  5 --
 .../table/runtime/join/NonWindowFullJoin.scala     |  3 -
 .../NonWindowFullJoinWithNonEquiPredicates.scala   |  3 -
 .../table/runtime/join/NonWindowInnerJoin.scala    |  3 -
 .../flink/table/runtime/join/NonWindowJoin.scala   |  2 -
 .../runtime/join/NonWindowLeftRightJoin.scala      |  3 -
 ...nWindowLeftRightJoinWithNonEquiPredicates.scala |  2 -
 .../table/runtime/join/NonWindowOuterJoin.scala    |  8 +--
 .../NonWindowOuterJoinWithNonEquiPredicates.scala  |  8 +--
 .../table/runtime/harness/JoinHarnessTest.scala    | 64 ----------------------
 10 files changed, 6 insertions(+), 95 deletions(-)

diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamJoinToCoProcessTranslator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamJoinToCoProcessTranslator.scala
index 5a8d1a4..054476a 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamJoinToCoProcessTranslator.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamJoinToCoProcessTranslator.scala
@@ -111,7 +111,6 @@ class DataStreamJoinToCoProcessTranslator(
         new NonWindowInnerJoin(
           leftSchema.typeInfo,
           rightSchema.typeInfo,
-          CRowTypeInfo(returnType),
           genFunction.name,
           genFunction.code,
           queryConfig)
@@ -119,7 +118,6 @@ class DataStreamJoinToCoProcessTranslator(
         new NonWindowLeftRightJoin(
           leftSchema.typeInfo,
           rightSchema.typeInfo,
-          CRowTypeInfo(returnType),
           genFunction.name,
           genFunction.code,
           joinType == JoinRelType.LEFT,
@@ -128,7 +126,6 @@ class DataStreamJoinToCoProcessTranslator(
         new NonWindowLeftRightJoinWithNonEquiPredicates(
           leftSchema.typeInfo,
           rightSchema.typeInfo,
-          CRowTypeInfo(returnType),
           genFunction.name,
           genFunction.code,
           joinType == JoinRelType.LEFT,
@@ -137,7 +134,6 @@ class DataStreamJoinToCoProcessTranslator(
         new NonWindowFullJoin(
           leftSchema.typeInfo,
           rightSchema.typeInfo,
-          CRowTypeInfo(returnType),
           genFunction.name,
           genFunction.code,
           queryConfig)
@@ -145,7 +141,6 @@ class DataStreamJoinToCoProcessTranslator(
         new NonWindowFullJoinWithNonEquiPredicates(
           leftSchema.typeInfo,
           rightSchema.typeInfo,
-          CRowTypeInfo(returnType),
           genFunction.name,
           genFunction.code,
           queryConfig)
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowFullJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowFullJoin.scala
index d2bcb6a..57c60f1 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowFullJoin.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowFullJoin.scala
@@ -33,7 +33,6 @@ import org.apache.flink.util.Collector
   *
   * @param leftType        the input type of left stream
   * @param rightType       the input type of right stream
-  * @param resultType      the output type of join
   * @param genJoinFuncName the function code without any non-equi condition
   * @param genJoinFuncCode the function name without any non-equi condition
   * @param queryConfig     the configuration for the query to generate
@@ -41,14 +40,12 @@ import org.apache.flink.util.Collector
 class NonWindowFullJoin(
     leftType: TypeInformation[Row],
     rightType: TypeInformation[Row],
-    resultType: TypeInformation[CRow],
     genJoinFuncName: String,
     genJoinFuncCode: String,
     queryConfig: StreamQueryConfig)
   extends NonWindowOuterJoin(
     leftType,
     rightType,
-    resultType,
     genJoinFuncName,
     genJoinFuncCode,
     false,
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowFullJoinWithNonEquiPredicates.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowFullJoinWithNonEquiPredicates.scala
index b442a88..9c27eb4 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowFullJoinWithNonEquiPredicates.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowFullJoinWithNonEquiPredicates.scala
@@ -35,7 +35,6 @@ import org.apache.flink.util.Collector
   *
   * @param leftType        the input type of left stream
   * @param rightType       the input type of right stream
-  * @param resultType      the output type of join
   * @param genJoinFuncName the function code of other non-equi condition
   * @param genJoinFuncCode the function name of other non-equi condition
   * @param queryConfig     the configuration for the query to generate
@@ -43,14 +42,12 @@ import org.apache.flink.util.Collector
 class NonWindowFullJoinWithNonEquiPredicates(
     leftType: TypeInformation[Row],
     rightType: TypeInformation[Row],
-    resultType: TypeInformation[CRow],
     genJoinFuncName: String,
     genJoinFuncCode: String,
     queryConfig: StreamQueryConfig)
   extends NonWindowOuterJoinWithNonEquiPredicates(
     leftType,
     rightType,
-    resultType,
     genJoinFuncName,
     genJoinFuncCode,
     false,
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowInnerJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowInnerJoin.scala
index e511ed1..2e5832c 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowInnerJoin.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowInnerJoin.scala
@@ -32,7 +32,6 @@ import org.apache.flink.util.Collector
   *
   * @param leftType          the input type of left stream
   * @param rightType         the input type of right stream
-  * @param resultType        the output type of join
   * @param genJoinFuncName   the function code of other non-equi condition
   * @param genJoinFuncCode   the function name of other non-equi condition
   * @param queryConfig       the configuration for the query to generate
@@ -40,14 +39,12 @@ import org.apache.flink.util.Collector
 class NonWindowInnerJoin(
     leftType: TypeInformation[Row],
     rightType: TypeInformation[Row],
-    resultType: TypeInformation[CRow],
     genJoinFuncName: String,
     genJoinFuncCode: String,
     queryConfig: StreamQueryConfig)
   extends NonWindowJoin(
     leftType,
     rightType,
-    resultType,
     genJoinFuncName,
     genJoinFuncCode,
     queryConfig) {
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowJoin.scala
index 51db755..0fe2e39 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowJoin.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowJoin.scala
@@ -37,7 +37,6 @@ import org.apache.flink.util.Collector
   *
   * @param leftType          the input type of left stream
   * @param rightType         the input type of right stream
-  * @param resultType        the output type of join
   * @param genJoinFuncName   the function code of other non-equi condition
   * @param genJoinFuncCode   the function name of other non-equi condition
   * @param queryConfig       the configuration for the query to generate
@@ -45,7 +44,6 @@ import org.apache.flink.util.Collector
 abstract class NonWindowJoin(
     leftType: TypeInformation[Row],
     rightType: TypeInformation[Row],
-    resultType: TypeInformation[CRow],
     genJoinFuncName: String,
     genJoinFuncCode: String,
     queryConfig: StreamQueryConfig)
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowLeftRightJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowLeftRightJoin.scala
index a595712..b4f14e4 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowLeftRightJoin.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowLeftRightJoin.scala
@@ -33,7 +33,6 @@ import org.apache.flink.util.Collector
   *
   * @param leftType        the input type of left stream
   * @param rightType       the input type of right stream
-  * @param resultType      the output type of join
   * @param genJoinFuncName the function code without any non-equi condition
   * @param genJoinFuncCode the function name without any non-equi condition
   * @param isLeftJoin      the type of join, whether it is the type of left join
@@ -42,7 +41,6 @@ import org.apache.flink.util.Collector
 class NonWindowLeftRightJoin(
     leftType: TypeInformation[Row],
     rightType: TypeInformation[Row],
-    resultType: TypeInformation[CRow],
     genJoinFuncName: String,
     genJoinFuncCode: String,
     isLeftJoin: Boolean,
@@ -50,7 +48,6 @@ class NonWindowLeftRightJoin(
   extends NonWindowOuterJoin(
     leftType,
     rightType,
-    resultType,
     genJoinFuncName,
     genJoinFuncCode,
     isLeftJoin,
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowLeftRightJoinWithNonEquiPredicates.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowLeftRightJoinWithNonEquiPredicates.scala
index f3a499a..33517cc 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowLeftRightJoinWithNonEquiPredicates.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowLeftRightJoinWithNonEquiPredicates.scala
@@ -44,7 +44,6 @@ import org.apache.flink.util.Collector
 class NonWindowLeftRightJoinWithNonEquiPredicates(
     leftType: TypeInformation[Row],
     rightType: TypeInformation[Row],
-    resultType: TypeInformation[CRow],
     genJoinFuncName: String,
     genJoinFuncCode: String,
     isLeftJoin: Boolean,
@@ -52,7 +51,6 @@ class NonWindowLeftRightJoinWithNonEquiPredicates(
   extends NonWindowOuterJoinWithNonEquiPredicates(
     leftType,
     rightType,
-    resultType,
     genJoinFuncName,
     genJoinFuncCode,
     isLeftJoin,
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowOuterJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowOuterJoin.scala
index 8877b89..0018a16 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowOuterJoin.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowOuterJoin.scala
@@ -31,7 +31,6 @@ import org.apache.flink.util.Collector
   *
   * @param leftType        the input type of left stream
   * @param rightType       the input type of right stream
-  * @param resultType      the output type of join
   * @param genJoinFuncName the function code of other non-equi condition
   * @param genJoinFuncCode the function name of other non-equi condition
   * @param isLeftJoin      the type of join, whether it is the type of left join
@@ -40,7 +39,6 @@ import org.apache.flink.util.Collector
 abstract class NonWindowOuterJoin(
     leftType: TypeInformation[Row],
     rightType: TypeInformation[Row],
-    resultType: TypeInformation[CRow],
     genJoinFuncName: String,
     genJoinFuncCode: String,
     isLeftJoin: Boolean,
@@ -48,7 +46,6 @@ abstract class NonWindowOuterJoin(
   extends NonWindowJoin(
     leftType,
     rightType,
-    resultType,
     genJoinFuncName,
     genJoinFuncCode,
     queryConfig) {
@@ -60,8 +57,9 @@ abstract class NonWindowOuterJoin(
 
   override def open(parameters: Configuration): Unit = {
     super.open(parameters)
-    leftResultRow = new Row(resultType.getArity)
-    rightResultRow = new Row(resultType.getArity)
+    val arity = leftType.getArity + rightType.getArity
+    leftResultRow = new Row(arity)
+    rightResultRow = new Row(arity)
     LOG.debug(s"Instantiating NonWindowOuterJoin")
   }
 
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowOuterJoinWithNonEquiPredicates.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowOuterJoinWithNonEquiPredicates.scala
index 6812a06..8fe2f4f 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowOuterJoinWithNonEquiPredicates.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowOuterJoinWithNonEquiPredicates.scala
@@ -32,7 +32,6 @@ import org.apache.flink.types.Row
   *
   * @param leftType        the input type of left stream
   * @param rightType       the input type of right stream
-  * @param resultType      the output type of join
   * @param genJoinFuncName the function code of other non-equi condition
   * @param genJoinFuncCode the function name of other non-equi condition
   * @param isLeftJoin      the type of join, whether it is the type of left join
@@ -41,7 +40,6 @@ import org.apache.flink.types.Row
   abstract class NonWindowOuterJoinWithNonEquiPredicates(
     leftType: TypeInformation[Row],
     rightType: TypeInformation[Row],
-    resultType: TypeInformation[CRow],
     genJoinFuncName: String,
     genJoinFuncCode: String,
     isLeftJoin: Boolean,
@@ -49,7 +47,6 @@ import org.apache.flink.types.Row
   extends NonWindowOuterJoin(
     leftType,
     rightType,
-    resultType,
     genJoinFuncName,
     genJoinFuncCode,
     isLeftJoin,
@@ -64,8 +61,9 @@ import org.apache.flink.types.Row
   override def open(parameters: Configuration): Unit = {
     super.open(parameters)
 
-    leftResultRow = new Row(resultType.getArity)
-    rightResultRow = new Row(resultType.getArity)
+    val arity = leftType.getArity + rightType.getArity
+    leftResultRow = new Row(arity)
+    rightResultRow = new Row(arity)
 
     joinCntState = new Array[MapState[Row, Long]](2)
     val leftJoinCntStateDescriptor = new MapStateDescriptor[Row, Long](
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala
index 132ac4e..c499a9d 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala
@@ -861,7 +861,6 @@ class JoinHarnessTest extends HarnessTestBase {
     val joinProcessFunc = new NonWindowInnerJoin(
       rowType,
       rowType,
-      joinReturnType,
       "TestJoinFunction",
       funcCode,
       queryConfig)
@@ -953,18 +952,9 @@ class JoinHarnessTest extends HarnessTestBase {
   @Test
   def testNonWindowInnerJoinWithRetract() {
 
-    val joinReturnType = CRowTypeInfo(new RowTypeInfo(
-      Array[TypeInformation[_]](
-        INT_TYPE_INFO,
-        STRING_TYPE_INFO,
-        INT_TYPE_INFO,
-        STRING_TYPE_INFO),
-      Array("a", "b", "c", "d")))
-
     val joinProcessFunc = new NonWindowInnerJoin(
       rowType,
       rowType,
-      joinReturnType,
       "TestJoinFunction",
       funcCode,
       queryConfig)
@@ -1053,18 +1043,9 @@ class JoinHarnessTest extends HarnessTestBase {
   @Test
   def testNonWindowLeftJoinWithoutNonEqualPred() {
 
-    val joinReturnType = CRowTypeInfo(new RowTypeInfo(
-      Array[TypeInformation[_]](
-        INT_TYPE_INFO,
-        STRING_TYPE_INFO,
-        INT_TYPE_INFO,
-        STRING_TYPE_INFO),
-      Array("a", "b", "c", "d")))
-
     val joinProcessFunc = new NonWindowLeftRightJoin(
       rowType,
       rowType,
-      joinReturnType,
       "TestJoinFunction",
       funcCode,
       true,
@@ -1170,18 +1151,9 @@ class JoinHarnessTest extends HarnessTestBase {
   @Test
   def testNonWindowLeftJoinWithNonEqualPred() {
 
-    val joinReturnType = CRowTypeInfo(new RowTypeInfo(
-      Array[TypeInformation[_]](
-        INT_TYPE_INFO,
-        STRING_TYPE_INFO,
-        INT_TYPE_INFO,
-        STRING_TYPE_INFO),
-      Array("a", "b", "c", "d")))
-
     val joinProcessFunc = new NonWindowLeftRightJoinWithNonEquiPredicates(
       rowType,
       rowType,
-      joinReturnType,
       "TestJoinFunction",
       funcCodeWithNonEqualPred,
       true,
@@ -1309,18 +1281,9 @@ class JoinHarnessTest extends HarnessTestBase {
   @Test
   def testNonWindowRightJoinWithoutNonEqualPred() {
 
-    val joinReturnType = CRowTypeInfo(new RowTypeInfo(
-      Array[TypeInformation[_]](
-        INT_TYPE_INFO,
-        STRING_TYPE_INFO,
-        INT_TYPE_INFO,
-        STRING_TYPE_INFO),
-      Array("a", "b", "c", "d")))
-
     val joinProcessFunc = new NonWindowLeftRightJoin(
       rowType,
       rowType,
-      joinReturnType,
       "TestJoinFunction",
       funcCode,
       false,
@@ -1426,18 +1389,9 @@ class JoinHarnessTest extends HarnessTestBase {
   @Test
   def testNonWindowRightJoinWithNonEqualPred() {
 
-    val joinReturnType = CRowTypeInfo(new RowTypeInfo(
-      Array[TypeInformation[_]](
-        INT_TYPE_INFO,
-        STRING_TYPE_INFO,
-        INT_TYPE_INFO,
-        STRING_TYPE_INFO),
-      Array("a", "b", "c", "d")))
-
     val joinProcessFunc = new NonWindowLeftRightJoinWithNonEquiPredicates(
       rowType,
       rowType,
-      joinReturnType,
       "TestJoinFunction",
       funcCodeWithNonEqualPred2,
       false,
@@ -1565,18 +1519,9 @@ class JoinHarnessTest extends HarnessTestBase {
   @Test
   def testNonWindowFullJoinWithoutNonEqualPred() {
 
-    val joinReturnType = CRowTypeInfo(new RowTypeInfo(
-      Array[TypeInformation[_]](
-        INT_TYPE_INFO,
-        STRING_TYPE_INFO,
-        INT_TYPE_INFO,
-        STRING_TYPE_INFO),
-      Array("a", "b", "c", "d")))
-
     val joinProcessFunc = new NonWindowFullJoin(
       rowType,
       rowType,
-      joinReturnType,
       "TestJoinFunction",
       funcCode,
       queryConfig)
@@ -1743,18 +1688,9 @@ class JoinHarnessTest extends HarnessTestBase {
   @Test
   def testNonWindowFullJoinWithNonEqualPred() {
 
-    val joinReturnType = CRowTypeInfo(new RowTypeInfo(
-      Array[TypeInformation[_]](
-        INT_TYPE_INFO,
-        STRING_TYPE_INFO,
-        INT_TYPE_INFO,
-        STRING_TYPE_INFO),
-      Array("a", "b", "c", "d")))
-
     val joinProcessFunc = new NonWindowFullJoinWithNonEquiPredicates(
       rowType,
       rowType,
-      joinReturnType,
       "TestJoinFunction",
       funcCodeWithNonEqualPred2,
       queryConfig)


[flink] 05/11: [hotfix][table, tests] Reduce mockito usage in TableTestUtil

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

pnowojski pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 81c96cac656d1dfc61d06aa85e18d4edf4eab59e
Author: Piotr Nowojski <pi...@gmail.com>
AuthorDate: Mon Aug 27 16:49:55 2018 +0200

    [hotfix][table,tests] Reduce mockito usage in TableTestUtil
---
 .../apache/flink/table/utils/TableTestBase.scala   | 50 +++++++++++-----------
 1 file changed, 25 insertions(+), 25 deletions(-)

diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala
index 5e1aabe..b987e34 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala
@@ -21,12 +21,13 @@ package org.apache.flink.table.utils
 import org.apache.calcite.plan.RelOptUtil
 import org.apache.calcite.rel.RelNode
 import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.{DataSet => JDataSet, ExecutionEnvironment => JExecutionEnvironment}
+import org.apache.flink.api.java.{LocalEnvironment, DataSet => JDataSet, ExecutionEnvironment => JExecutionEnvironment}
 import org.apache.flink.api.scala.{DataSet, ExecutionEnvironment}
 import org.apache.flink.streaming.api.TimeCharacteristic
 import org.apache.flink.streaming.api.datastream.{DataStream => JDataStream}
-import org.apache.flink.streaming.api.environment.{StreamExecutionEnvironment => JStreamExecutionEnvironment}
-import org.apache.flink.streaming.api.scala.{DataStream, StreamExecutionEnvironment}
+import org.apache.flink.streaming.api.environment.{LocalStreamEnvironment, StreamExecutionEnvironment => JStreamExecutionEnvironment}
+import org.apache.flink.streaming.api.functions.source.SourceFunction
+import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.api.{Table, TableEnvironment, TableSchema}
 import org.apache.flink.table.expressions.Expression
@@ -152,9 +153,9 @@ object TableTestUtil {
 }
 
 case class BatchTableTestUtil() extends TableTestUtil {
-  val javaEnv = mock(classOf[JExecutionEnvironment])
+  val javaEnv = new LocalEnvironment()
   val javaTableEnv = TableEnvironment.getTableEnvironment(javaEnv)
-  val env = mock(classOf[ExecutionEnvironment])
+  val env = new ExecutionEnvironment(javaEnv)
   val tableEnv = TableEnvironment.getTableEnvironment(env)
 
   def addTable[T: TypeInformation](
@@ -232,11 +233,11 @@ case class BatchTableTestUtil() extends TableTestUtil {
 }
 
 case class StreamTableTestUtil() extends TableTestUtil {
-  val javaEnv = mock(classOf[JStreamExecutionEnvironment])
-  when(javaEnv.getStreamTimeCharacteristic).thenReturn(TimeCharacteristic.EventTime)
+  val javaEnv = new LocalStreamEnvironment()
+  javaEnv.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+
   val javaTableEnv = TableEnvironment.getTableEnvironment(javaEnv)
-  val env = mock(classOf[StreamExecutionEnvironment])
-  when(env.getWrappedStreamExecutionEnvironment).thenReturn(javaEnv)
+  val env = new StreamExecutionEnvironment(javaEnv)
   val tableEnv = TableEnvironment.getTableEnvironment(env)
 
   def addTable[T: TypeInformation](
@@ -244,25 +245,16 @@ case class StreamTableTestUtil() extends TableTestUtil {
       fields: Expression*)
     : Table = {
 
-    val ds = mock(classOf[DataStream[T]])
-    val jDs = mock(classOf[JDataStream[T]])
-    when(ds.javaStream).thenReturn(jDs)
-    val typeInfo: TypeInformation[T] = implicitly[TypeInformation[T]]
-    when(jDs.getType).thenReturn(typeInfo)
-
-    val t = ds.toTable(tableEnv, fields: _*)
-    tableEnv.registerTable(name, t)
-    t
+    val table = env.fromElements().toTable(tableEnv, fields: _*)
+    tableEnv.registerTable(name, table)
+    table
   }
 
   def addJavaTable[T](typeInfo: TypeInformation[T], name: String, fields: String): Table = {
-
-    val jDs = mock(classOf[JDataStream[T]])
-    when(jDs.getType).thenReturn(typeInfo)
-
-    val t = javaTableEnv.fromDataStream(jDs, fields)
-    javaTableEnv.registerTable(name, t)
-    t
+    val stream = javaEnv.addSource(new EmptySource[T], typeInfo)
+    val table = javaTableEnv.fromDataStream(stream, fields)
+    javaTableEnv.registerTable(name, table)
+    table
   }
 
   def addFunction[T: TypeInformation](
@@ -327,3 +319,11 @@ case class StreamTableTestUtil() extends TableTestUtil {
     printTable(tableEnv.sqlQuery(query))
   }
 }
+
+class EmptySource[T]() extends SourceFunction[T] {
+  override def run(ctx: SourceFunction.SourceContext[T]): Unit = {
+  }
+
+  override def cancel(): Unit = {
+  }
+}


[flink] 11/11: [FLINK-9714][table] Support versioned joins with processing time

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

pnowojski pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 00add9cdc02ccfacb38a566099d877e4669c6f65
Author: Piotr Nowojski <pi...@gmail.com>
AuthorDate: Thu Sep 20 14:15:48 2018 +0200

    [FLINK-9714][table] Support versioned joins with processing time
---
 .../logical/rel/LogicalTemporalTableJoin.scala     |  42 +-
 .../apache/flink/table/plan/nodes/CommonJoin.scala |   9 +
 ...taStreamTemporalJoinToCoProcessTranslator.scala | 237 +++++++++++
 .../datastream/DataStreamTemporalTableJoin.scala   |  29 +-
 .../flink/table/runtime/join/TemporalJoin.scala    |  93 +++++
 .../TemporalTableJoinValidationTest.scala          |  72 +++-
 .../table/runtime/harness/HarnessTestBase.scala    |   7 +
 .../runtime/harness/TemporalJoinHarnessTest.scala  | 452 +++++++++++++++++++++
 .../runtime/stream/sql/TemporalJoinITCase.scala    | 135 ++++++
 .../runtime/utils/StreamingWithStateTestBase.scala |   7 +-
 10 files changed, 1068 insertions(+), 15 deletions(-)

diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/rel/LogicalTemporalTableJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/rel/LogicalTemporalTableJoin.scala
index 3b1d51b..7fdc5e9 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/rel/LogicalTemporalTableJoin.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/rel/LogicalTemporalTableJoin.scala
@@ -23,7 +23,7 @@ import java.util.Collections
 import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
 import org.apache.calcite.rel.RelNode
 import org.apache.calcite.rel.core._
-import org.apache.calcite.rex.{RexBuilder, RexNode}
+import org.apache.calcite.rex.{RexBuilder, RexCall, RexNode}
 import org.apache.calcite.sql.`type`.{OperandTypes, ReturnTypes}
 import org.apache.calcite.sql.{SqlFunction, SqlFunctionCategory, SqlKind}
 import org.apache.flink.util.Preconditions.checkArgument
@@ -101,6 +101,38 @@ object LogicalTemporalTableJoin {
         OperandTypes.ANY)),
     SqlFunctionCategory.SYSTEM)
 
+  def isRowtimeCall(call: RexCall): Boolean = {
+    checkArgument(call.getOperator == TEMPORAL_JOIN_CONDITION)
+    call.getOperands.size() == 3
+  }
+
+  def isProctimeCall(call: RexCall): Boolean = {
+    checkArgument(call.getOperator == TEMPORAL_JOIN_CONDITION)
+    call.getOperands.size() == 2
+  }
+
+  def makeRowTimeTemporalJoinConditionCall(
+      rexBuilder: RexBuilder,
+      leftTimeAttribute: RexNode,
+      rightTimeAttribute: RexNode,
+      rightPrimaryKeyExpression: RexNode): RexNode = {
+    rexBuilder.makeCall(
+      TEMPORAL_JOIN_CONDITION,
+      leftTimeAttribute,
+      rightTimeAttribute,
+      rightPrimaryKeyExpression)
+  }
+
+  def makeProcTimeTemporalJoinConditionCall(
+      rexBuilder: RexBuilder,
+      leftTimeAttribute: RexNode,
+      rightPrimaryKeyExpression: RexNode): RexNode = {
+    rexBuilder.makeCall(
+      TEMPORAL_JOIN_CONDITION,
+      leftTimeAttribute,
+      rightPrimaryKeyExpression)
+  }
+
   /**
     * See [[LogicalTemporalTableJoin]]
     */
@@ -119,8 +151,8 @@ object LogicalTemporalTableJoin {
       traitSet,
       left,
       right,
-      rexBuilder.makeCall(
-        TEMPORAL_JOIN_CONDITION,
+      makeRowTimeTemporalJoinConditionCall(
+        rexBuilder,
         leftTimeAttribute,
         rightTimeAttribute,
         rightPrimaryKeyExpression))
@@ -148,8 +180,8 @@ object LogicalTemporalTableJoin {
       traitSet,
       left,
       right,
-      rexBuilder.makeCall(
-        TEMPORAL_JOIN_CONDITION,
+      makeProcTimeTemporalJoinConditionCall(
+        rexBuilder,
         leftTimeAttribute,
         rightPrimaryKeyExpression))
   }
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonJoin.scala
index 7d0ca35..3d98a4d 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonJoin.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonJoin.scala
@@ -48,6 +48,15 @@ trait CommonJoin {
     }
   }
 
+  private[flink] def temporalJoinToString(
+      inputType: RelDataType,
+      joinCondition: RexNode,
+      joinType: JoinRelType,
+      expression: (RexNode, List[String], Option[List[RexNode]]) => String): String = {
+
+    "Temporal" + joinToString(inputType, joinCondition, joinType, expression)
+  }
+
   private[flink] def joinToString(
       inputType: RelDataType,
       joinCondition: RexNode,
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamTemporalJoinToCoProcessTranslator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamTemporalJoinToCoProcessTranslator.scala
new file mode 100644
index 0000000..467e84f
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamTemporalJoinToCoProcessTranslator.scala
@@ -0,0 +1,237 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.rel.core.{JoinInfo, JoinRelType}
+import org.apache.calcite.rex._
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.api.{StreamQueryConfig, TableConfig, TableException, ValidationException}
+import org.apache.flink.table.calcite.FlinkTypeFactory._
+import org.apache.flink.table.codegen.GeneratedFunction
+import org.apache.flink.table.plan.logical.rel.LogicalTemporalTableJoin
+import org.apache.flink.table.plan.logical.rel.LogicalTemporalTableJoin._
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.table.plan.util.RexDefaultVisitor
+import org.apache.flink.table.runtime.join.TemporalJoin
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Preconditions.checkState
+
+class DataStreamTemporalJoinToCoProcessTranslator private (
+    textualRepresentation: String,
+    config: TableConfig,
+    returnType: TypeInformation[Row],
+    leftSchema: RowSchema,
+    rightSchema: RowSchema,
+    joinInfo: JoinInfo,
+    rexBuilder: RexBuilder,
+    leftTimeAttribute: RexNode,
+    rightTimeAttribute: Option[RexNode],
+    rightPrimaryKeyExpression: RexNode,
+    remainingNonEquiJoinPredicates: RexNode)
+  extends DataStreamJoinToCoProcessTranslator(
+    config,
+    returnType,
+    leftSchema,
+    rightSchema,
+    joinInfo,
+    rexBuilder) {
+
+  override val nonEquiJoinPredicates: Option[RexNode] = Some(remainingNonEquiJoinPredicates)
+
+  override protected def createCoProcessFunction(
+      joinType: JoinRelType,
+      queryConfig: StreamQueryConfig,
+      joinFunction: GeneratedFunction[FlatJoinFunction[Row, Row, Row], Row])
+    : CoProcessFunction[CRow, CRow, CRow] = {
+
+    if (rightTimeAttribute.isDefined) {
+      throw new ValidationException(
+        s"Currently only proctime temporal joins are supported in [$textualRepresentation]")
+    }
+
+    joinType match {
+      case JoinRelType.INNER =>
+        new TemporalJoin(
+          leftSchema.typeInfo,
+          rightSchema.typeInfo,
+          joinFunction.name,
+          joinFunction.code,
+          queryConfig)
+      case _ =>
+       throw new ValidationException(
+         s"Only ${JoinRelType.INNER} temporal join is supported in [$textualRepresentation]")
+    }
+  }
+}
+
+object DataStreamTemporalJoinToCoProcessTranslator {
+  def create(
+    textualRepresentation: String,
+    config: TableConfig,
+    returnType: TypeInformation[Row],
+    leftSchema: RowSchema,
+    rightSchema: RowSchema,
+    joinInfo: JoinInfo,
+    rexBuilder: RexBuilder): DataStreamTemporalJoinToCoProcessTranslator = {
+
+    checkState(
+      !joinInfo.isEqui,
+      "Missing %s in join condition",
+      TEMPORAL_JOIN_CONDITION)
+
+    val nonEquiJoinRex: RexNode = joinInfo.getRemaining(rexBuilder)
+    val temporalJoinConditionExtractor = new TemporalJoinConditionExtractor(
+      textualRepresentation,
+      leftSchema.typeInfo.getTotalFields,
+      joinInfo,
+      rexBuilder)
+
+    val remainingNonEquiJoinPredicates = temporalJoinConditionExtractor.apply(nonEquiJoinRex)
+
+    checkState(
+      temporalJoinConditionExtractor.leftTimeAttribute.isDefined &&
+        temporalJoinConditionExtractor.rightPrimaryKeyExpression.isDefined,
+      "Missing %s in join condition",
+      TEMPORAL_JOIN_CONDITION)
+
+    new DataStreamTemporalJoinToCoProcessTranslator(
+      textualRepresentation,
+      config,
+      returnType,
+      leftSchema,
+      rightSchema,
+      joinInfo,
+      rexBuilder,
+      temporalJoinConditionExtractor.leftTimeAttribute.get,
+      temporalJoinConditionExtractor.rightTimeAttribute,
+      temporalJoinConditionExtractor.rightPrimaryKeyExpression.get,
+      remainingNonEquiJoinPredicates)
+  }
+
+  private class TemporalJoinConditionExtractor(
+      textualRepresentation: String,
+      rightKeysStartingOffset: Int,
+      joinInfo: JoinInfo,
+      rexBuilder: RexBuilder)
+
+    extends RexShuttle {
+
+    var leftTimeAttribute: Option[RexNode] = None
+
+    var rightTimeAttribute: Option[RexNode] = None
+
+    var rightPrimaryKeyExpression: Option[RexNode] = None
+
+    override def visitCall(call: RexCall): RexNode = {
+      if (call.getOperator != TEMPORAL_JOIN_CONDITION) {
+        return super.visitCall(call)
+      }
+
+      checkState(
+        leftTimeAttribute.isEmpty
+        && rightPrimaryKeyExpression.isEmpty
+        && rightTimeAttribute.isEmpty,
+        "Multiple %s functions in [%s]",
+        TEMPORAL_JOIN_CONDITION,
+        textualRepresentation)
+
+      if (LogicalTemporalTableJoin.isRowtimeCall(call)) {
+        leftTimeAttribute = Some(call.getOperands.get(0))
+        rightTimeAttribute = Some(call.getOperands.get(1))
+
+        rightPrimaryKeyExpression = Some(validateRightPrimaryKey(call.getOperands.get(2)))
+
+        if (!isRowtimeIndicatorType(rightTimeAttribute.get.getType)) {
+          throw new ValidationException(
+            s"Non rowtime timeAttribute [${rightTimeAttribute.get.getType}] " +
+              s"used to create TemporalTableFunction")
+        }
+        if (!isRowtimeIndicatorType(leftTimeAttribute.get.getType)) {
+          throw new ValidationException(
+            s"Non rowtime timeAttribute [${leftTimeAttribute.get.getType}] " +
+              s"passed as the argument to TemporalTableFunction")
+        }
+
+        throw new TableException("Event time temporal joins are not yet supported.")
+      }
+      else if (LogicalTemporalTableJoin.isProctimeCall(call)) {
+        leftTimeAttribute = Some(call.getOperands.get(0))
+        rightPrimaryKeyExpression = Some(validateRightPrimaryKey(call.getOperands.get(1)))
+
+        if (!isProctimeIndicatorType(leftTimeAttribute.get.getType)) {
+          throw new ValidationException(
+            s"Non processing timeAttribute [${leftTimeAttribute.get.getType}] " +
+              s"passed as the argument to TemporalTableFunction")
+        }
+      }
+      else {
+        throw new IllegalStateException(
+          s"Unsupported invocation $call in [$textualRepresentation]")
+      }
+      rexBuilder.makeLiteral(true)
+    }
+
+    private def validateRightPrimaryKey(rightPrimaryKey: RexNode): RexNode = {
+      if (joinInfo.rightKeys.size() != 1) {
+        throw new ValidationException(
+          s"Only single column join key is supported. " +
+            s"Found ${joinInfo.rightKeys} in [$textualRepresentation]")
+      }
+      val rightKey = joinInfo.rightKeys.get(0) + rightKeysStartingOffset
+
+      val primaryKeyVisitor = new PrimaryKeyVisitor(textualRepresentation)
+      rightPrimaryKey.accept(primaryKeyVisitor)
+
+      primaryKeyVisitor.inputReference match {
+        case None =>
+          throw new IllegalStateException(
+            s"Failed to find primary key reference in [$textualRepresentation]")
+        case Some(primaryKeyInputReference) if primaryKeyInputReference != rightKey =>
+          throw new ValidationException(
+            s"Join key [$rightKey] must be the same as " +
+              s"temporal table's primary key [$primaryKeyInputReference] " +
+              s"in [$textualRepresentation]")
+        case _ =>
+          rightPrimaryKey
+      }
+    }
+  }
+
+  /**
+    * Extracts input references from primary key expression.
+    */
+  private class PrimaryKeyVisitor(textualRepresentation: String)
+    extends RexDefaultVisitor[RexNode] {
+
+    var inputReference: Option[Int] = None
+
+    override def visitInputRef(inputRef: RexInputRef): RexNode = {
+      inputReference = Some(inputRef.getIndex)
+      inputRef
+    }
+
+    override def visitNode(rexNode: RexNode): RexNode = {
+      throw new ValidationException(
+        s"Unsupported right primary key expression [$rexNode] in [$textualRepresentation]")
+    }
+  }
+}
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamTemporalTableJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamTemporalTableJoin.scala
index 60f36d3..d0c2e96 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamTemporalTableJoin.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamTemporalTableJoin.scala
@@ -22,12 +22,20 @@ import org.apache.calcite.plan._
 import org.apache.calcite.rel.RelNode
 import org.apache.calcite.rel.core.{JoinInfo, JoinRelType}
 import org.apache.calcite.rex.RexNode
+import org.apache.flink.api.common.functions.FlatJoinFunction
 import org.apache.flink.streaming.api.datastream.DataStream
-import org.apache.flink.table.api.{StreamQueryConfig, StreamTableEnvironment}
+import org.apache.flink.table.api.{StreamQueryConfig, StreamTableEnvironment, TableException}
+import org.apache.flink.table.codegen.FunctionCodeGenerator
 import org.apache.flink.table.plan.schema.RowSchema
-import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.runtime.CRowKeySelector
+import org.apache.flink.table.runtime.join._
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+import org.apache.flink.types.Row
 import org.apache.flink.util.Preconditions.checkState
 
+import scala.collection.JavaConversions._
+import scala.collection.mutable.ArrayBuffer
+
 /**
   * RelNode for a stream join with [[org.apache.flink.table.functions.TemporalTableFunction]].
   */
@@ -74,9 +82,14 @@ class DataStreamTemporalTableJoin(
       ruleDescription)
   }
 
-  override def translateToPlan(
-      tableEnv: StreamTableEnvironment,
-      queryConfig: StreamQueryConfig): DataStream[CRow] = {
-    throw new NotImplementedError()
-  }
-}
+  override protected def createTranslator(
+      tableEnv: StreamTableEnvironment): DataStreamJoinToCoProcessTranslator = {
+    DataStreamTemporalJoinToCoProcessTranslator.create(
+      this.toString,
+      tableEnv.getConfig,
+      schema.typeInfo,
+      leftSchema,
+      rightSchema,
+      joinInfo,
+      cluster.getRexBuilder)
+  }}
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TemporalJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TemporalJoin.scala
new file mode 100644
index 0000000..5087515
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TemporalJoin.scala
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.join
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state.{ValueState, ValueStateDescriptor}
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.api.StreamQueryConfig
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.typeutils.TypeCheckUtils._
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+class TemporalJoin(
+    leftType: TypeInformation[Row],
+    rightType: TypeInformation[Row],
+    genJoinFuncName: String,
+    genJoinFuncCode: String,
+    queryConfig: StreamQueryConfig)
+  extends CoProcessFunction[CRow, CRow, CRow]
+  with Compiler[FlatJoinFunction[Row, Row, Row]]
+  with Logging {
+
+  validateEqualsHashCode("join", leftType)
+  validateEqualsHashCode("join", rightType)
+
+  protected var rightState: ValueState[Row] = _
+  protected var cRowWrapper: CRowWrappingCollector = _
+
+  protected var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  override def open(parameters: Configuration): Unit = {
+    val clazz = compile(
+      getRuntimeContext.getUserCodeClassLoader,
+      genJoinFuncName,
+      genJoinFuncCode)
+
+    joinFunction = clazz.newInstance()
+
+    val rightStateDescriptor = new ValueStateDescriptor[Row]("right", rightType)
+    rightState = getRuntimeContext.getState(rightStateDescriptor)
+
+    cRowWrapper = new CRowWrappingCollector()
+  }
+
+  override def processElement1(
+      value: CRow,
+      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
+      out: Collector[CRow]): Unit = {
+
+    if (rightState.value() == null) {
+      return
+    }
+
+    cRowWrapper.out = out
+    cRowWrapper.setChange(value.change)
+
+    val rightSideRow = rightState.value()
+    joinFunction.join(value.row, rightSideRow, cRowWrapper)
+  }
+
+  override def processElement2(
+      value: CRow,
+      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
+      out: Collector[CRow]): Unit = {
+
+    if (value.change) {
+      rightState.update(value.row)
+    } else {
+      rightState.clear()
+    }
+  }
+}
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/validation/TemporalTableJoinValidationTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/validation/TemporalTableJoinValidationTest.scala
index ab282ec..58ad493 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/validation/TemporalTableJoinValidationTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/validation/TemporalTableJoinValidationTest.scala
@@ -21,7 +21,7 @@ package org.apache.flink.table.api.stream.table.validation
 import java.sql.Timestamp
 
 import org.apache.flink.api.scala._
-import org.apache.flink.table.api.ValidationException
+import org.apache.flink.table.api.{TableException, ValidationException}
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.utils._
 import org.junit.Test
@@ -33,9 +33,18 @@ class TemporalTableJoinValidationTest extends TableTestBase {
   val orders = util.addTable[(Long, String, Timestamp)](
     "Orders", 'o_amount, 'o_currency, 'o_rowtime.rowtime)
 
+  val ordersProctime = util.addTable[(Long, String)](
+    "OrdersProctime", 'o_amount, 'o_currency, 'o_rowtime.proctime)
+
+  val ordersWithoutTimeAttribute = util.addTable[(Long, String, Timestamp)](
+    "OrdersWithoutTimeAttribute", 'o_amount, 'o_currency, 'o_rowtime)
+
   val ratesHistory = util.addTable[(String, Int, Timestamp)](
     "RatesHistory", 'currency, 'rate, 'rowtime.rowtime)
 
+  val ratesHistoryWithoutTimeAttribute = util.addTable[(String, Int, Timestamp)](
+    "ratesHistoryWithoutTimeAttribute", 'currency, 'rate, 'rowtime)
+
   @Test
   def testInvalidFieldReference(): Unit = {
     expectedException.expect(classOf[ValidationException])
@@ -51,4 +60,65 @@ class TemporalTableJoinValidationTest extends TableTestBase {
 
     ratesHistory.createTemporalTableFunction("rowtime", "foobar")
   }
+
+  @Test
+  def testNonTimeIndicatorOnRightSide(): Unit = {
+    expectedException.expect(classOf[ValidationException])
+    expectedException.expectMessage(
+      "Non rowtime timeAttribute [TIMESTAMP(3)] used to create TemporalTableFunction")
+
+    val rates = ratesHistoryWithoutTimeAttribute.createTemporalTableFunction('rowtime, 'currency)
+
+    val result = orders
+      .join(rates('o_rowtime), "currency = o_currency")
+      .select("o_amount * rate").as("rate")
+
+    util.explain(result)
+  }
+
+  @Test
+  def testNonTimeIndicatorOnLeftSide(): Unit = {
+    expectedException.expect(classOf[ValidationException])
+    expectedException.expectMessage(
+      "Non rowtime timeAttribute [TIMESTAMP(3)] passed as the argument to TemporalTableFunction")
+
+    val rates = ratesHistory.createTemporalTableFunction('rowtime, 'currency)
+
+    val result = ordersWithoutTimeAttribute
+      .join(rates('o_rowtime), "currency = o_currency")
+      .select("o_amount * rate").as("rate")
+
+    util.explain(result)
+  }
+
+  @Test
+  def testMixedTimeIndicators(): Unit = {
+    expectedException.expect(classOf[ValidationException])
+    expectedException.expectMessage(
+      "Non rowtime timeAttribute [TIME ATTRIBUTE(PROCTIME)] passed as the argument " +
+        "to TemporalTableFunction")
+
+    val rates = ratesHistory.createTemporalTableFunction('rowtime, 'currency)
+
+    val result = ordersProctime
+      .join(rates('o_rowtime), "currency = o_currency")
+      .select("o_amount * rate").as("rate")
+
+    util.explain(result)
+  }
+
+  @Test
+  def testEventTimeIndicators(): Unit = {
+    expectedException.expect(classOf[TableException])
+    expectedException.expectMessage(
+      "Event time temporal joins are not yet supported.")
+
+    val rates = ratesHistory.createTemporalTableFunction('rowtime, 'currency)
+
+    val result = orders
+      .join(rates('o_rowtime), "currency = o_currency")
+      .select("o_amount * rate").as("rate")
+
+    util.explain(result)
+  }
 }
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/HarnessTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/HarnessTestBase.scala
index d494c21..28b7d14 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/HarnessTestBase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/HarnessTestBase.scala
@@ -36,8 +36,15 @@ import org.apache.flink.table.functions.aggfunctions.{IntSumWithRetractAggFuncti
 import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils.getAccumulatorTypeOfAggregateFunction
 import org.apache.flink.table.runtime.harness.HarnessTestBase.{RowResultSortComparator, RowResultSortComparatorWithWatermarks}
 import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+import org.junit.Rule
+import org.junit.rules.ExpectedException
 
 class HarnessTestBase {
+  // used for accurate exception information checking.
+  val expectedException = ExpectedException.none()
+
+  @Rule
+  def thrown = expectedException
 
   val longMinWithRetractAggFunction: String =
     UserDefinedFunctionUtils.serialize(new LongMinWithRetractAggFunction)
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/TemporalJoinHarnessTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/TemporalJoinHarnessTest.scala
new file mode 100644
index 0000000..43f3548
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/TemporalJoinHarnessTest.scala
@@ -0,0 +1,452 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.harness
+
+import java.util.concurrent.ConcurrentLinkedQueue
+
+import org.apache.calcite.rel.core.{JoinInfo, JoinRelType}
+import org.apache.calcite.rex.{RexBuilder, RexNode}
+import org.apache.calcite.sql.fun.SqlStdOperatorTable
+import org.apache.calcite.util.ImmutableIntList
+import org.apache.flink.api.common.time.Time
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
+import org.apache.flink.api.java.functions.KeySelector
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.streaming.api.operators.co.KeyedCoProcessOperator
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord
+import org.apache.flink.streaming.util.KeyedTwoInputStreamOperatorTestHarness
+import org.apache.flink.table.api.{TableConfig, Types, ValidationException}
+import org.apache.flink.table.calcite.{FlinkTypeFactory, FlinkTypeSystem}
+import org.apache.flink.table.plan.logical.rel.LogicalTemporalTableJoin
+import org.apache.flink.table.plan.logical.rel.LogicalTemporalTableJoin.TEMPORAL_JOIN_CONDITION
+import org.apache.flink.table.plan.nodes.datastream.DataStreamTemporalJoinToCoProcessTranslator
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.table.runtime.CRowKeySelector
+import org.apache.flink.table.runtime.harness.HarnessTestBase.{RowResultSortComparator, TestStreamQueryConfig}
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo
+import org.hamcrest.Matchers.startsWith
+import org.junit.Test
+
+class TemporalJoinHarnessTest extends HarnessTestBase {
+
+  private val typeFactory = new FlinkTypeFactory(new FlinkTypeSystem)
+
+  private val tableConfig = new TableConfig
+
+  private val queryConfig =
+    new TestStreamQueryConfig(Time.milliseconds(2), Time.milliseconds(4))
+
+  private val ORDERS_KEY = "o_currency"
+
+  private val ORDERS_PROCTIME = "o_proctime"
+
+  private val RATES_KEY = "r_currency"
+
+  private val ordersRowtimeType = new RowTypeInfo(
+    Array[TypeInformation[_]](
+      Types.LONG,
+      Types.STRING,
+      TimeIndicatorTypeInfo.ROWTIME_INDICATOR),
+    Array("o_amount", ORDERS_KEY, "o_rowtime"))
+
+  private val ordersProctimeType = new RowTypeInfo(
+    Array[TypeInformation[_]](
+      Types.LONG,
+      Types.STRING,
+      TimeIndicatorTypeInfo.PROCTIME_INDICATOR),
+    Array("o_amount", ORDERS_KEY, ORDERS_PROCTIME))
+
+  private val ratesRowtimeType = new RowTypeInfo(
+    Array[TypeInformation[_]](
+      Types.STRING,
+      Types.LONG,
+      TimeIndicatorTypeInfo.ROWTIME_INDICATOR),
+    Array(RATES_KEY, "r_rate", "r_rowtime"))
+
+  private val ratesProctimeType = new RowTypeInfo(
+    Array[TypeInformation[_]](
+      Types.STRING,
+      Types.LONG,
+      TimeIndicatorTypeInfo.PROCTIME_INDICATOR),
+    Array(RATES_KEY, "r_rate", "r_proctime"))
+
+  private val joinRowtimeType = new RowTypeInfo(
+    ordersRowtimeType.getFieldTypes ++ ratesRowtimeType.getFieldTypes,
+    ordersRowtimeType.getFieldNames ++ ratesRowtimeType.getFieldNames)
+
+  private val rexBuilder = new RexBuilder(typeFactory)
+
+  @Test
+  def testProctime() {
+    val testHarness = createTestHarness(new OrdersRatesProctimeTemporalJoinInfo)
+
+    testHarness.open()
+    val expectedOutput = new ConcurrentLinkedQueue[Object]()
+
+    // process without conversion rates
+    testHarness.processElement1(new StreamRecord(CRow(2L, "Euro", null)))
+
+    // initiate conversion rates
+    testHarness.processElement2(new StreamRecord(CRow("US Dollar", 102L, null)))
+    testHarness.processElement2(new StreamRecord(CRow("Euro", 114L, null)))
+    testHarness.processElement2(new StreamRecord(CRow("Yen", 1L, null)))
+
+    // process with conversion rates
+    testHarness.processElement1(new StreamRecord(CRow(2L, "Euro", null)))
+    testHarness.processElement1(new StreamRecord(CRow(1L, "US Dollar", null)))
+    testHarness.processElement1(new StreamRecord(CRow(50L, "Yen", null)))
+
+    expectedOutput.add(new StreamRecord(CRow(2L, "Euro", null, "Euro", 114L, null)))
+    expectedOutput.add(new StreamRecord(CRow(1L, "US Dollar", null, "US Dollar", 102L, null)))
+    expectedOutput.add(new StreamRecord(CRow(50L, "Yen", null, "Yen", 1L, null)))
+
+    // update Euro
+    testHarness.processElement2(new StreamRecord(CRow("Euro", 116L, null)))
+
+    // process Euro
+    testHarness.processElement1(new StreamRecord(CRow(3L, "Euro", null)))
+
+    expectedOutput.add(new StreamRecord(CRow(3L, "Euro", null, "Euro", 116L, null)))
+
+    // again update Euro
+    testHarness.processElement2(new StreamRecord(CRow("Euro", 119L, null)))
+
+    // process US Dollar
+    testHarness.processElement1(new StreamRecord(CRow(5L, "US Dollar", null)))
+
+    expectedOutput.add(new StreamRecord(CRow(5L, "US Dollar", null, "US Dollar", 102L, null)))
+
+    verify(expectedOutput, testHarness.getOutput, new RowResultSortComparator())
+
+    testHarness.close()
+  }
+
+  @Test
+  def testNonEquiProctime() {
+    val testHarness = createTestHarness(
+      new ProctimeTemporalJoinInfo(
+        new RowTypeInfo(
+          ordersProctimeType.getFieldTypes :+ Types.INT,
+          ordersProctimeType.getFieldNames :+ "foo"),
+        new RowTypeInfo(
+          ratesProctimeType.getFieldTypes :+ Types.INT,
+          ratesProctimeType.getFieldNames :+ "bar"),
+        ORDERS_KEY,
+        RATES_KEY,
+        ORDERS_PROCTIME) {
+        /**
+          * @return [[LogicalTemporalTableJoin.TEMPORAL_JOIN_CONDITION]](...) AND
+          *        leftInputRef(3) > rightInputRef(3)
+          */
+        override def getRemaining(rexBuilder: RexBuilder): RexNode = {
+          rexBuilder.makeCall(
+            SqlStdOperatorTable.AND,
+            super.getRemaining(rexBuilder),
+            rexBuilder.makeCall(
+              SqlStdOperatorTable.GREATER_THAN,
+              makeLeftInputRef("foo"),
+              makeRightInputRef("bar")))
+        }
+      })
+
+    testHarness.open()
+    val expectedOutput = new ConcurrentLinkedQueue[Object]()
+
+    // initiate conversion rates
+    testHarness.processElement2(new StreamRecord(CRow("Euro", 114L, null, 42)))
+    testHarness.processElement2(new StreamRecord(CRow("Yen", 1L, null, 42)))
+
+    // process with conversion rates
+    testHarness.processElement1(new StreamRecord(CRow(2L, "Euro", null, 0)))
+    testHarness.processElement1(new StreamRecord(CRow(50L, "Yen", null, 44)))
+
+    expectedOutput.add(new StreamRecord(CRow(50L, "Yen", null, 44, "Yen", 1L, null, 42)))
+
+    // update Euro
+    testHarness.processElement2(new StreamRecord(CRow("Euro", 116L, null, 44)))
+
+    // process Euro
+    testHarness.processElement1(new StreamRecord(CRow(3L, "Euro", null, 42)))
+    testHarness.processElement1(new StreamRecord(CRow(4L, "Euro", null, 44)))
+    testHarness.processElement1(new StreamRecord(CRow(5L, "Euro", null, 1337)))
+
+    expectedOutput.add(new StreamRecord(CRow(5L, "Euro", null, 1337, "Euro", 116L, null, 44)))
+
+    // process US Dollar
+    testHarness.processElement1(new StreamRecord(CRow(5L, "US Dollar", null, 1337)))
+
+    verify(expectedOutput, testHarness.getOutput, new RowResultSortComparator())
+
+    testHarness.close()
+  }
+
+  @Test
+  def testMissingTemporalJoinCondition() {
+    expectedException.expect(classOf[IllegalStateException])
+    expectedException.expectMessage(startsWith(s"Missing ${TEMPORAL_JOIN_CONDITION.getName}"))
+
+    translateJoin(new TemporalJoinInfo(
+      ordersProctimeType,
+      ratesProctimeType,
+      ORDERS_KEY,
+      RATES_KEY) {
+
+      override def isEqui: Boolean = true
+
+      override def getRemaining(rexBuilder: RexBuilder): RexNode = rexBuilder.makeLiteral(true)
+    })
+  }
+
+  @Test
+  def testNonEquiMissingTemporalJoinCondition() {
+    expectedException.expect(classOf[IllegalStateException])
+    expectedException.expectMessage(startsWith(s"Missing ${TEMPORAL_JOIN_CONDITION.getName}"))
+
+    translateJoin(new TemporalJoinInfo(
+      ordersProctimeType,
+      ratesProctimeType,
+      ORDERS_KEY,
+      RATES_KEY) {
+
+      override def isEqui: Boolean = true
+
+      override def getRemaining(rexBuilder: RexBuilder): RexNode = {
+        rexBuilder.makeCall(
+          SqlStdOperatorTable.GREATER_THAN,
+          rexBuilder.makeCall(
+            SqlStdOperatorTable.CONCAT,
+            rexBuilder.makeLiteral("A"),
+            makeLeftInputRef(ORDERS_KEY)),
+          makeRightInputRef(RATES_KEY))
+      }
+    })
+  }
+
+  @Test
+  def testTwoTemporalJoinConditions() {
+    expectedException.expect(classOf[IllegalStateException])
+    expectedException.expectMessage(startsWith(s"Multiple $TEMPORAL_JOIN_CONDITION functions"))
+
+    translateJoin(
+      new OrdersRatesProctimeTemporalJoinInfo() {
+        override def getRemaining(rexBuilder: RexBuilder): RexNode = {
+          rexBuilder.makeCall(
+            SqlStdOperatorTable.OR,
+            super.getRemaining(rexBuilder),
+            super.getRemaining(rexBuilder))
+        }
+      })
+  }
+
+  @Test
+  def testIncorrectTemporalJoinCondition() {
+    expectedException.expect(classOf[IllegalStateException])
+    expectedException.expectMessage(startsWith(s"Unsupported invocation"))
+
+    translateJoin(
+      new OrdersRatesProctimeTemporalJoinInfo() {
+        override def getRemaining(rexBuilder: RexBuilder): RexNode = {
+          rexBuilder.makeCall(
+            TEMPORAL_JOIN_CONDITION,
+            makeLeftInputRef(leftKey),
+            makeLeftInputRef(leftKey),
+            makeLeftInputRef(leftKey),
+            makeRightInputRef(rightKey))
+        }
+      })
+  }
+
+  @Test
+  def testUnsupportedPrimaryKeyInTemporalJoinCondition() {
+    expectedException.expect(classOf[ValidationException])
+    expectedException.expectMessage(startsWith("Unsupported right primary key expression"))
+
+    translateJoin(
+      new OrdersRatesProctimeTemporalJoinInfo() {
+        override def getRemaining(rexBuilder: RexBuilder): RexNode = {
+          LogicalTemporalTableJoin.makeProcTimeTemporalJoinConditionCall(
+            rexBuilder,
+            makeLeftInputRef(leftTimeAttribute),
+            rexBuilder.makeCall(
+              SqlStdOperatorTable.CONCAT,
+              rexBuilder.makeLiteral("A"),
+              makeRightInputRef(RATES_KEY)))
+        }
+      })
+  }
+
+  @Test
+  def testMultipleJoinKeys() {
+    expectedException.expect(classOf[ValidationException])
+    expectedException.expectMessage(startsWith(s"Only single column join key"))
+
+    translateJoin(
+      new TemporalJoinInfo(
+        ordersProctimeType,
+        ratesProctimeType,
+        ImmutableIntList.of(0, 1),
+        ImmutableIntList.of(1, 0)) {
+
+        override def getRemaining(rexBuilder: RexBuilder): RexNode = {
+          LogicalTemporalTableJoin.makeProcTimeTemporalJoinConditionCall(
+            rexBuilder,
+            makeLeftInputRef(ORDERS_PROCTIME),
+            makeRightInputRef(RATES_KEY))
+        }
+      })
+  }
+
+  @Test
+  def testNonInnerJoin() {
+    expectedException.expect(classOf[ValidationException])
+    expectedException.expectMessage(startsWith(s"Only ${JoinRelType.INNER} temporal join"))
+
+    translateJoin(new OrdersRatesProctimeTemporalJoinInfo, JoinRelType.FULL)
+  }
+
+  def createTestHarness(temporalJoinInfo: TemporalJoinInfo)
+    : KeyedTwoInputStreamOperatorTestHarness[String, CRow, CRow, CRow] = {
+
+    val (leftKeySelector, rightKeySelector, joinCoProcessFunction) =
+      translateJoin(temporalJoinInfo)
+
+    val operator: KeyedCoProcessOperator[String, CRow, CRow, CRow] =
+      new KeyedCoProcessOperator[String, CRow, CRow, CRow](joinCoProcessFunction)
+
+    new KeyedTwoInputStreamOperatorTestHarness[String, CRow, CRow, CRow](
+      operator,
+      leftKeySelector.asInstanceOf[KeySelector[CRow, String]],
+      rightKeySelector.asInstanceOf[KeySelector[CRow, String]],
+      BasicTypeInfo.STRING_TYPE_INFO,
+      1,
+      1,
+      0)
+  }
+
+  def translateJoin(joinInfo: TemporalJoinInfo, joinRelType: JoinRelType = JoinRelType.INNER)
+    : (CRowKeySelector, CRowKeySelector, CoProcessFunction[CRow, CRow, CRow]) = {
+
+    val leftType = joinInfo.leftRowType
+    val rightType = joinInfo.rightRowType
+    val joinType = new RowTypeInfo(
+      leftType.getFieldTypes ++ rightType.getFieldTypes,
+      leftType.getFieldNames ++ rightType.getFieldNames)
+
+    val joinTranslator = DataStreamTemporalJoinToCoProcessTranslator.create(
+      "TemporalJoin",
+      tableConfig,
+      joinType,
+      new RowSchema(typeFactory.createTypeFromTypeInfo(leftType, false)),
+      new RowSchema(typeFactory.createTypeFromTypeInfo(rightType, false)),
+      joinInfo,
+      rexBuilder)
+
+    val joinCoProcessFunction = joinTranslator.getCoProcessFunction(
+      joinRelType,
+      joinType.getFieldNames,
+      "TemporalJoin",
+      queryConfig)
+
+    (joinTranslator.getLeftKeySelector(),
+      joinTranslator.getRightKeySelector(),
+      joinCoProcessFunction)
+  }
+
+  abstract class TemporalJoinInfo(
+      val leftRowType: RowTypeInfo,
+      val rightRowType: RowTypeInfo,
+      leftKeys: ImmutableIntList,
+      rightKeys: ImmutableIntList)
+    extends JoinInfo(leftKeys, rightKeys) {
+
+    def this(
+      leftRowType: RowTypeInfo,
+      rightRowType: RowTypeInfo,
+      leftKey: String,
+      rightKey: String) =
+      this(
+        leftRowType,
+        rightRowType,
+        ImmutableIntList.of(leftRowType.getFieldIndex(leftKey)),
+        ImmutableIntList.of(rightRowType.getFieldIndex(rightKey)))
+
+    override def isEqui: Boolean = false
+
+    def makeLeftInputRef(leftField: String): RexNode = {
+      rexBuilder.makeInputRef(
+        typeFactory.createTypeFromTypeInfo(leftRowType.getTypeAt(leftField), false),
+        leftRowType.getFieldIndex(leftField))
+    }
+
+    def makeRightInputRef(rightField: String): RexNode = {
+      rexBuilder.makeInputRef(
+        typeFactory.createTypeFromTypeInfo(rightRowType.getTypeAt(rightField), false),
+        rightRowType.getFieldIndex(rightField) + leftRowType.getFieldTypes.length)
+    }
+  }
+
+  class OrdersRatesProctimeTemporalJoinInfo()
+    extends ProctimeTemporalJoinInfo(
+      ordersProctimeType,
+      ratesProctimeType,
+      ORDERS_KEY,
+      RATES_KEY,
+      ORDERS_PROCTIME)
+
+  class ProctimeTemporalJoinInfo(
+      leftRowType: RowTypeInfo,
+      rightRowType: RowTypeInfo,
+      val leftKey: String,
+      val rightKey: String,
+      val leftTimeAttribute: String)
+    extends TemporalJoinInfo(leftRowType, rightRowType, leftKey, rightKey) {
+
+    override def getRemaining(rexBuilder: RexBuilder): RexNode = {
+      LogicalTemporalTableJoin.makeProcTimeTemporalJoinConditionCall(
+        rexBuilder,
+        makeLeftInputRef(leftTimeAttribute),
+        makeRightInputRef(rightKey))
+    }
+  }
+
+  class MissingTemporalJoinConditionJoinInfo(
+      leftRowType: RowTypeInfo,
+      rightRowType: RowTypeInfo,
+      leftKey: String,
+      rightKey: String,
+      isEquiJoin: Boolean)
+    extends TemporalJoinInfo(leftRowType, rightRowType, leftKey, rightKey) {
+
+    override def isEqui: Boolean = isEquiJoin
+
+    override def getRemaining(rexBuilder: RexBuilder): RexNode = if (isEquiJoin) {
+      rexBuilder.makeLiteral(true)
+    }
+    else {
+      rexBuilder.makeCall(
+        SqlStdOperatorTable.GREATER_THAN,
+        rexBuilder.makeCall(
+          SqlStdOperatorTable.CONCAT,
+          rexBuilder.makeLiteral("A"),
+          makeLeftInputRef(leftKey)),
+        makeRightInputRef(rightKey))
+    }
+  }
+}
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/TemporalJoinITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/TemporalJoinITCase.scala
new file mode 100644
index 0000000..b0304a3
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/TemporalJoinITCase.scala
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.runtime.stream.sql
+
+import java.sql.Timestamp
+
+import org.apache.flink.api.scala._
+import org.apache.flink.streaming.api.TimeCharacteristic
+import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.api.{TableEnvironment, TableException}
+import org.apache.flink.table.runtime.utils.{StreamITCase, StreamingWithStateTestBase}
+import org.apache.flink.types.Row
+import org.junit._
+
+import scala.collection.mutable
+
+class TemporalJoinITCase extends StreamingWithStateTestBase {
+
+  /**
+    * Because of nature of the processing time, we can not (or at least it is not that easy)
+    * validate the result here. Instead of that, here we are just testing whether there are no
+    * exceptions in a full blown ITCase. Actual correctness is tested in unit tests.
+    */
+  @Test
+  def testProcessTimeInnerJoin(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    env.setStateBackend(getStateBackend)
+    StreamITCase.clear
+    env.setParallelism(1)
+    env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime)
+
+    val sqlQuery =
+      """
+        |SELECT
+        |  o.amount * r.rate AS amount
+        |FROM
+        |  Orders AS o,
+        |  LATERAL TABLE (Rates(o.proctime)) AS r
+        |WHERE r.currency = o.currency
+        |""".stripMargin
+
+    val ordersData = new mutable.MutableList[(Long, String)]
+    ordersData.+=((2L, "Euro"))
+    ordersData.+=((1L, "US Dollar"))
+    ordersData.+=((50L, "Yen"))
+    ordersData.+=((3L, "Euro"))
+    ordersData.+=((5L, "US Dollar"))
+
+    val ratesHistoryData = new mutable.MutableList[(String, Long)]
+    ratesHistoryData.+=(("US Dollar", 102L))
+    ratesHistoryData.+=(("Euro", 114L))
+    ratesHistoryData.+=(("Yen", 1L))
+    ratesHistoryData.+=(("Euro", 116L))
+    ratesHistoryData.+=(("Euro", 119L))
+
+    val orders = env
+      .fromCollection(ordersData)
+      .toTable(tEnv, 'amount, 'currency, 'proctime.proctime)
+    val ratesHistory = env
+      .fromCollection(ratesHistoryData)
+      .toTable(tEnv, 'currency, 'rate, 'proctime.proctime)
+
+    tEnv.registerTable("Orders", orders)
+    tEnv.registerTable("RatesHistory", ratesHistory)
+    tEnv.registerFunction(
+      "Rates",
+      ratesHistory.createTemporalTableFunction('proctime, 'currency))
+
+    val result = tEnv.sqlQuery(sqlQuery).toAppendStream[Row]
+    result.addSink(new StreamITCase.StringSink[Row])
+    env.execute()
+  }
+
+  @Test
+  def testEventTimeInnerJoin(): Unit = {
+    expectedException.expect(classOf[TableException])
+    expectedException.expectMessage("Event time temporal joins are not yet supported")
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    env.setStateBackend(getStateBackend)
+    StreamITCase.clear
+    env.setParallelism(1)
+    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+
+    val sqlQuery =
+      """
+        |SELECT
+        |  o.amount * r.rate AS amount
+        |FROM
+        |  Orders AS o,
+        |  LATERAL TABLE (Rates(o.rowtime)) AS r
+        |WHERE r.currency = o.currency
+        |""".stripMargin
+
+    val ordersData = new mutable.MutableList[(Long, String, Timestamp)]
+
+    val ratesHistoryData = new mutable.MutableList[(String, Long, Timestamp)]
+
+    val orders = env
+      .fromCollection(ordersData)
+      .toTable(tEnv, 'amount, 'currency, 'rowtime.rowtime)
+    val ratesHistory = env
+      .fromCollection(ratesHistoryData)
+      .toTable(tEnv, 'currency, 'rate, 'rowtime.rowtime)
+
+    tEnv.registerTable("Orders", orders)
+    tEnv.registerTable("RatesHistory", ratesHistory)
+    tEnv.registerFunction(
+      "Rates",
+      ratesHistory.createTemporalTableFunction('rowtime, 'currency))
+
+    val result = tEnv.sqlQuery(sqlQuery).toAppendStream[Row]
+    result.addSink(new StreamITCase.StringSink[Row])
+    env.execute()
+  }
+}
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/utils/StreamingWithStateTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/utils/StreamingWithStateTestBase.scala
index b3eeb59..b8c414f 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/utils/StreamingWithStateTestBase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/utils/StreamingWithStateTestBase.scala
@@ -21,10 +21,15 @@ import org.apache.flink.contrib.streaming.state.RocksDBStateBackend
 import org.apache.flink.runtime.state.StateBackend
 import org.apache.flink.test.util.AbstractTestBase
 import org.junit.Rule
-import org.junit.rules.TemporaryFolder
+import org.junit.rules.{ExpectedException, TemporaryFolder}
 
 class StreamingWithStateTestBase extends AbstractTestBase {
 
+  val expectedException = ExpectedException.none()
+
+  @Rule
+  def thrown = expectedException
+
   val _tempFolder = new TemporaryFolder
 
   @Rule