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

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

Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3715#discussion_r123189670
  
    --- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/JoinITCase.scala
 ---
    @@ -0,0 +1,204 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF 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.scala.stream.sql
    +
    +import org.apache.flink.api.scala._
    +import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
    +import org.apache.flink.table.api.scala._
    +import org.apache.flink.table.api.scala.stream.utils.{StreamITCase, 
StreamingWithStateTestBase}
    +import org.apache.flink.table.api.{TableEnvironment, TableException}
    +import org.apache.flink.types.Row
    +import org.junit.Assert._
    +import org.junit._
    +
    +import scala.collection.mutable
    +
    +class JoinITCase extends StreamingWithStateTestBase {
    +
    +  val data = List(
    +    (1L, 1, "Hello"),
    +    (2L, 2, "Hello"),
    +    (3L, 3, "Hello"),
    +    (4L, 4, "Hello"),
    +    (5L, 5, "Hello"),
    +    (6L, 6, "Hello"),
    +    (7L, 7, "Hello World"),
    +    (8L, 8, "Hello World"),
    +    (20L, 20, "Hello World"))
    +
    +  /**
    +    * both stream should have boundary
    +    */
    +  @Test(expected = classOf[TableException])
    +  def testJoinException0(): Unit = {
    +    val env = StreamExecutionEnvironment.getExecutionEnvironment
    +    env.setStateBackend(getStateBackend)
    +    val tEnv = TableEnvironment.getTableEnvironment(env)
    +    StreamITCase.testResults = mutable.MutableList()
    +
    +    val t1 = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c, 
'proctime.proctime)
    +    val t2 = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c, 
'proctime.proctime)
    +
    +    tEnv.registerTable("T1", t1)
    +    tEnv.registerTable("T2", t2)
    +
    +    val sqlQuery = "SELECT t2.a, t2.c, t1.c from T1 as t1 join T2 as t2 on 
t1.a = t2.a"
    +
    +    val result = tEnv.sql(sqlQuery).toDataStream[Row]
    +    result.addSink(new StreamITCase.StringSink)
    +    env.execute()
    +  }
    +
    +  /**
    +    * both stream should have boundary
    +    */
    +  @Test(expected = classOf[TableException])
    +  def testJoinException1(): Unit = {
    +    val env = StreamExecutionEnvironment.getExecutionEnvironment
    +    env.setStateBackend(getStateBackend)
    +    val tEnv = TableEnvironment.getTableEnvironment(env)
    +    StreamITCase.testResults = mutable.MutableList()
    +
    +    val t1 = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c, 
'proctime.proctime)
    +    val t2 = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c, 
'proctime.proctime)
    +
    +    tEnv.registerTable("T1", t1)
    +    tEnv.registerTable("T2", t2)
    +
    +    val sqlQuery = "SELECT t2.a, t2.c, t1.c from T1 as t1 join T2 as t2 on 
t1.a = t2.a " +
    +      "and t1.proctime > t2.proctime - interval '5' second"
    +
    +    val result = tEnv.sql(sqlQuery).toDataStream[Row]
    +    result.addSink(new StreamITCase.StringSink)
    +    env.execute()
    +  }
    +
    +  /**
    +    * both stream should use same time indicator
    +    */
    +  @Test(expected = classOf[TableException])
    +  def testJoinException2(): Unit = {
    +    val env = StreamExecutionEnvironment.getExecutionEnvironment
    +    env.setStateBackend(getStateBackend)
    +    val tEnv = TableEnvironment.getTableEnvironment(env)
    +    StreamITCase.testResults = mutable.MutableList()
    +
    +    val t1 = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c, 
'proctime.proctime)
    +    val t2 = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c, 
'rowtime.rowtime)
    +
    +    tEnv.registerTable("T1", t1)
    +    tEnv.registerTable("T2", t2)
    +
    +    val sqlQuery = "SELECT t2.a, t2.c, t1.c from T1 as t1 join T2 as t2 on 
t1.a = t2.a " +
    +      "and t1.proctime > t2.rowtime - interval '5' second "
    +
    +    val result = tEnv.sql(sqlQuery).toDataStream[Row]
    +    result.addSink(new StreamITCase.StringSink)
    +    env.execute()
    +  }
    +
    +
    +  /** test process time inner join **/
    +  @Test
    +  def testProcessTimeInnerJoin(): Unit = {
    --- End diff --
    
    We cannot test this operator with a regular ITCase. Since the join result 
is based on processing time, we cannot guarantee the result. Very wide bound 
have the problem that they don't test important edge cases (the join result 
becomes a Cartesian product). Also the test might fail due to GC stalls, on 
slower machines, etc. The only way to test this operator is the test harness.


> Support proctime inner equi-join between two streams in the SQL API
> -------------------------------------------------------------------
>
>                 Key: FLINK-6232
>                 URL: https://issues.apache.org/jira/browse/FLINK-6232
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Table API & SQL
>            Reporter: hongyuhong
>            Assignee: hongyuhong
>
> The goal of this issue is to add support for inner equi-join on proc time 
> streams to the SQL interface.
> Queries similar to the following should be supported:
> {code}
> SELECT o.proctime, o.productId, o.orderId, s.proctime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.proctime BETWEEN s.proctime AND s.proctime + INTERVAL '1' HOUR;
> {code}
> The following restrictions should initially apply:
> * The join hint only support inner join
> * The ON clause should include equi-join condition
> * The time-condition {{o.proctime BETWEEN s.proctime AND s.proctime + 
> INTERVAL '1' HOUR}} only can use proctime that is a system attribute, the 
> time condition only support bounded time range like {{o.proctime BETWEEN 
> s.proctime - INTERVAL '1' HOUR AND s.proctime + INTERVAL '1' HOUR}}, not 
> support unbounded like {{o.proctime > s.protime}},  and  should include both 
> two stream's proctime attribute, {{o.proctime between proctime() and 
> proctime() + 1}} should also not be supported.
> This issue includes:
> * Design of the DataStream operator to deal with stream join
> * Translation from Calcite's RelNode representation (LogicalJoin). 



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to