[ 
https://issues.apache.org/jira/browse/FLINK-9713?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16614756#comment-16614756
 ] 

ASF GitHub Bot commented on FLINK-9713:
---------------------------------------

twalthr commented on a change in pull request #6299: [FLINK-9713][table][sql] 
Support processing time versioned joins
URL: https://github.com/apache/flink/pull/6299#discussion_r217688365
 
 

 ##########
 File path: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/TemporalJoinHarnessTest.scala
 ##########
 @@ -0,0 +1,458 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.rules.ExpectedException
+import org.junit.{Rule, Test}
+
+class TemporalJoinHarnessTest extends HarnessTestBase {
+  // used for accurate exception information checking.
+  val expectedException = ExpectedException.none()
+
+  @Rule
+  def thrown = expectedException
+
+  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[ValidationException])
+    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] = {
 
 Review comment:
   nit: wrong indention

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


> Support versioned joins in planning phase
> -----------------------------------------
>
>                 Key: FLINK-9713
>                 URL: https://issues.apache.org/jira/browse/FLINK-9713
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Table API & SQL
>    Affects Versions: 1.5.0
>            Reporter: Piotr Nowojski
>            Assignee: Piotr Nowojski
>            Priority: Major
>              Labels: pull-request-available
>
> Queries like:
> {code:java}
> SELECT 
>   o.amount * r.rate 
> FROM 
>   Orders AS o, 
>   LATERAL TABLE (Rates(o.rowtime)) AS r 
> WHERE o.currency = r.currency{code}
> should evaluate to valid plan with versioned joins plan node.



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

Reply via email to