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

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_r123238701
  
    --- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala
 ---
    @@ -0,0 +1,200 @@
    +/*
    + * 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 java.lang.{Integer => JInt}
    +
    +import org.apache.flink.api.common.functions.FlatJoinFunction
    +import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
    +import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
    +import org.apache.flink.api.java.typeutils.RowTypeInfo
    +import org.apache.flink.streaming.api.operators.co.KeyedCoProcessOperator
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord
    +import 
org.apache.flink.streaming.util.{KeyedTwoInputStreamOperatorTestHarness, 
TwoInputStreamOperatorTestHarness}
    +import org.apache.flink.table.codegen.GeneratedFunction
    +import 
org.apache.flink.table.runtime.harness.HarnessTestBase.{RowResultSortComparator,
 TupleRowKeySelector}
    +import org.apache.flink.table.runtime.join.ProcTimeInnerJoin
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.junit.Test
    +
    +
    +class JoinHarnessTest extends HarnessTestBase{
    +
    +  private val rT = new RowTypeInfo(Array[TypeInformation[_]](
    +    INT_TYPE_INFO,
    +    STRING_TYPE_INFO),
    +    Array("a", "b"))
    +
    +
    +  val funcCode: String =
    +    """
    +      |public class TestJoinFunction
    +      |          extends 
org.apache.flink.api.common.functions.RichFlatJoinFunction {
    +      |  transient org.apache.flink.types.Row out =
    +      |            new org.apache.flink.types.Row(4);
    +      |  public TestJoinFunction() throws Exception {}
    +      |
    +      |  @Override
    +      |  public void open(org.apache.flink.configuration.Configuration 
parameters)
    +      |  throws Exception {}
    +      |
    +      |  @Override
    +      |  public void join(Object _in1, Object _in2, 
org.apache.flink.util.Collector c)
    +      |   throws Exception {
    +      |   org.apache.flink.types.Row in1 = (org.apache.flink.types.Row) 
_in1;
    +      |   org.apache.flink.types.Row in2 = (org.apache.flink.types.Row) 
_in2;
    +      |
    +      |   out.setField(0, in1.getField(0));
    +      |   out.setField(1, in1.getField(1));
    +      |   out.setField(2, in2.getField(0));
    +      |   out.setField(3, in2.getField(1));
    +      |
    +      |   c.collect(out);
    +      |
    +      |  }
    +      |
    +      |  @Override
    +      |  public void close() throws Exception {}
    +      |}
    +    """.stripMargin
    +
    +  @Test
    +  def testProcTimeJoin() {
    --- End diff --
    
    Please add comments for the scenarios that this test covers.


> 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