You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by hequn8128 <gi...@git.apache.org> on 2017/08/03 15:04:00 UTC

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

GitHub user hequn8128 opened a pull request:

    https://github.com/apache/flink/pull/4471

    [FLINK-6094] [table] Implement stream-stream proctime non-window inner join

    
    ## What is the purpose of the change
    
    Implement stream-stream proctime non-window inner join for table-api/sql
    
    
    ## Brief change log
    
      - Implement stream-stream inner join. `DataStreamJoinRule`, `DataStreamJoin`...
      - Implement retraction for stream-stream inner join.
      - Support state retrain time configuration for stream-stream inner join.
      - Add key extractor for `DataStreamJoin` in `UniqueKeyExtractor`.
      - Modify UniqueKeyExtractor to support cases like `select(pk as pk1, pk as pk2)`, in thoes cases, pk1 or pk2 can either be the unique key.
    
    
    ## Verifying this change
    
    This change added tests and can be verified as follows:
    
      - Add JoinValidation Tests 
      - Add JoinHarnessTest to test data expiration 
      - Add JoinITCase for SQL and table-api 
    
    ## Does this pull request potentially affect one of the following parts:
    
      - Dependencies (does it add or upgrade a dependency): ( no)
      - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (no)
      - The serializers: (no)
      - The runtime per-record code paths (performance sensitive): (yes)
      - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: (no)
    
    ## Documentation
    
      - Does this pull request introduce a new feature? (yes)
      - If yes, how is the feature documented? (docs, I have checked the sql and table-api docs and find 
     the inner join has already been added)
    


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/hequn8128/flink 6094_pr

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/4471.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #4471
    
----
commit 459c3b5b7801a15eac907ab5c11f8ca01acdc1c6
Author: 军长 <he...@alibaba-inc.com>
Date:   2017-07-30T10:45:45Z

    [FLINK-6094] [table] Implement stream-stream proctime non-window inner join

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132456337
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/UpdatingPlanChecker.scala ---
    @@ -90,40 +96,86 @@ object UpdatingPlanChecker {
                   // resolve names of input fields
                   .map(io => (inNames.get(io._1), io._2))
     
    -            // filter by input keys
    -            val outKeys = inOutNames.filter(io => keys.get.contains(io._1)).map(_._2)
    -            // check if all keys have been preserved
    -            if (outKeys.nonEmpty && outKeys.length == keys.get.length) {
    +            // filter by input keyAncestors
    +            val outKeyAncesters = inOutNames
    +              .filter(io => keyAncestors.get.map(e => e._1).contains(io._1))
    +              .map(io => (io._2, keyAncestors.get.find(ka => ka._1 == io._1).get._2))
    +
    +            // check if all keyAncestors have been preserved
    +            if (outKeyAncesters.nonEmpty &&
    +              outKeyAncesters.map(ka => ka._2).distinct.length ==
    +                keyAncestors.get.map(ka => ka._2).distinct.length) {
                   // all key have been preserved (but possibly renamed)
    -              keys = Some(outKeys.toArray)
    +              Some(outKeyAncesters.toList)
                 } else {
                   // some (or all) keys have been removed. Keys are no longer unique and removed
    -              keys = None
    +              None
                 }
    +          } else {
    +            None
               }
    +
             case _: DataStreamOverAggregate =>
    -          super.visit(node, ordinal, parent)
    -        // keys are always forwarded by Over aggregate
    +          // keyAncestors are always forwarded by Over aggregate
    +          visit(node.getInput(0))
             case a: DataStreamGroupAggregate =>
    -          // get grouping keys
    +          // get grouping keyAncestors
               val groupKeys = a.getRowType.getFieldNames.asScala.take(a.getGroupings.length)
    -          keys = Some(groupKeys.toArray)
    +          Some(groupKeys.map(e => (e, e)).toList)
             case w: DataStreamGroupWindowAggregate =>
    -          // get grouping keys
    +          // get grouping keyAncestors
               val groupKeys =
                 w.getRowType.getFieldNames.asScala.take(w.getGroupings.length).toArray
               // get window start and end time
               val windowStartEnd = w.getWindowProperties.map(_.name)
               // we have only a unique key if at least one window property is selected
               if (windowStartEnd.nonEmpty) {
    -            keys = Some(groupKeys ++ windowStartEnd)
    +            Some((groupKeys ++ windowStartEnd).map(e => (e, e)).toList)
    +          } else {
    +            None
    +          }
    +
    +        case j: DataStreamJoin =>
    +          val leftKeyAncestors = visit(j.getLeft)
    +          val rightKeyAncestors = visit(j.getRight)
    +          if (!leftKeyAncestors.isDefined || !rightKeyAncestors.isDefined) {
    +            None
    +          } else {
    +            // both left and right contain keys
    +            val leftJoinKeys =
    +              j.getLeft.getRowType.getFieldNames.asScala.zipWithIndex
    +              .filter(e => j.getJoinInfo.leftKeys.contains(e._2))
    +              .map(e => e._1)
    +            val rightJoinKeys =
    +              j.getRight.getRowType.getFieldNames.asScala.zipWithIndex
    +                .filter(e => j.getJoinInfo.rightKeys.contains(e._2))
    +                .map(e => e._1)
    +
    +            val leftKeys = leftKeyAncestors.get.map(e => e._1)
    +            val rightKeys = rightKeyAncestors.get.map(e => e._1)
    +
    +            //1. join key = left key = right key
    +            if (leftJoinKeys == leftKeys && rightJoinKeys == rightKeys) {
    +              Some(leftKeyAncestors.get ::: (rightKeyAncestors.get.map(e => (e._1)) zip
    +                leftKeyAncestors.get.map(e => (e._2))))
    +            }
    +            //2. join key = left key
    +            else if (leftJoinKeys == leftKeys && rightJoinKeys != rightKeys) {
    +              rightKeyAncestors
    --- End diff --
    
    all fields which are equal to a right key field become a key as well (with the same ancestor as the right key field).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r152818061
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/JoinITCase.scala ---
    @@ -0,0 +1,262 @@
    +/*
    + * 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.table
    +
    +import org.apache.flink.api.scala._
    +import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
    +import org.apache.flink.table.api.{StreamQueryConfig, TableEnvironment}
    +import org.apache.flink.table.api.scala._
    +import org.apache.flink.table.runtime.utils.{StreamITCase, StreamingWithStateTestBase}
    +import org.junit.Assert._
    +import org.junit.Test
    +import org.apache.flink.api.common.time.Time
    +import org.apache.flink.streaming.api.TimeCharacteristic
    +import org.apache.flink.table.functions.aggfunctions.CountAggFunction
    +import org.apache.flink.table.runtime.stream.table.GroupWindowITCase.TimestampAndWatermarkWithOffset
    +import org.apache.flink.table.runtime.utils.JavaUserDefinedAggFunctions.{CountDistinct, WeightedAvg}
    +import org.apache.flink.types.Row
    +
    +import scala.collection.mutable
    +
    +class JoinITCase extends StreamingWithStateTestBase {
    +
    +  private val queryConfig = new StreamQueryConfig()
    +  queryConfig.withIdleStateRetentionTime(Time.hours(1), Time.hours(2))
    --- End diff --
    
    Do we need to set the retention time if it is not relevant anyway?


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by hequn8128 <gi...@git.apache.org>.
Github user hequn8128 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r158488989
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/JoinITCase.scala ---
    @@ -0,0 +1,262 @@
    +/*
    + * 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.table
    +
    +import org.apache.flink.api.scala._
    +import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
    +import org.apache.flink.table.api.{StreamQueryConfig, TableEnvironment}
    +import org.apache.flink.table.api.scala._
    +import org.apache.flink.table.runtime.utils.{StreamITCase, StreamingWithStateTestBase}
    +import org.junit.Assert._
    +import org.junit.Test
    +import org.apache.flink.api.common.time.Time
    +import org.apache.flink.streaming.api.TimeCharacteristic
    +import org.apache.flink.table.functions.aggfunctions.CountAggFunction
    +import org.apache.flink.table.runtime.stream.table.GroupWindowITCase.TimestampAndWatermarkWithOffset
    +import org.apache.flink.table.runtime.utils.JavaUserDefinedAggFunctions.{CountDistinct, WeightedAvg}
    +import org.apache.flink.types.Row
    +
    +import scala.collection.mutable
    +
    +class JoinITCase extends StreamingWithStateTestBase {
    +
    +  private val queryConfig = new StreamQueryConfig()
    +  queryConfig.withIdleStateRetentionTime(Time.hours(1), Time.hours(2))
    --- End diff --
    
    Hi, maybe we can keep it. At least we can cover some logics in if (`if (stateCleaningEnabled && timerState.value() == 0)`).


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r152817360
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/JoinITCase.scala ---
    @@ -414,6 +416,56 @@ class JoinITCase extends StreamingWithStateTestBase {
         StreamITCase.compareWithList(expected)
       }
     
    +  /** test process time non-window inner join **/
    +  @Test
    +  def testProcessTimeNonWindowInnerJoin(): Unit = {
    --- End diff --
    
    `Proctime` or `ProcessingTime`


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r152742134
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamJoin.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.nodes.datastream
    +
    +import org.apache.calcite.plan._
    +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.join.DataStreamInnerJoin
    +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.
    +  */
    +class DataStreamJoin(
    +    cluster: RelOptCluster,
    +    traitSet: RelTraitSet,
    +    leftNode: RelNode,
    +    rightNode: RelNode,
    +    joinCondition: RexNode,
    +    joinInfo: JoinInfo,
    +    joinType: JoinRelType,
    +    leftSchema: RowSchema,
    +    rightSchema: RowSchema,
    +    schema: RowSchema,
    +    ruleDescription: String)
    +  extends BiRel(cluster, traitSet, leftNode, rightNode)
    +          with CommonJoin
    +          with DataStreamRel {
    +
    +  override def deriveRowType(): RelDataType = schema.relDataType
    +
    +  override def needsUpdatesAsRetraction: Boolean = true
    +
    +  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
    +    new DataStreamJoin(
    +      cluster,
    +      traitSet,
    +      inputs.get(0),
    +      inputs.get(1),
    +      joinCondition,
    +      joinInfo,
    +      joinType,
    +      leftSchema,
    +      rightSchema,
    +      schema,
    +      ruleDescription)
    +  }
    +
    +  def getJoinInfo: JoinInfo = joinInfo
    +
    +  def getJoinType: JoinRelType = joinType
    +
    +  override def toString: String = {
    +    joinToString(
    +      schema.relDataType,
    +      joinCondition,
    +      joinType,
    +      getExpressionString)
    +  }
    +
    +  override def explainTerms(pw: RelWriter): RelWriter = {
    +    joinExplainTerms(
    +      super.explainTerms(pw),
    +      schema.relDataType,
    +      joinCondition,
    +      joinType,
    +      getExpressionString)
    +  }
    +
    +  override def translateToPlan(
    +      tableEnv: StreamTableEnvironment,
    +      queryConfig: StreamQueryConfig): DataStream[CRow] = {
    +
    +    val config = tableEnv.getConfig
    +    val returnType = schema.typeInfo
    +    val keyPairs = joinInfo.pairs().toList
    +
    +    // get the equality keys
    +    val leftKeys = ArrayBuffer.empty[Int]
    +    val rightKeys = ArrayBuffer.empty[Int]
    +    if (keyPairs.isEmpty) {
    +      // if no equality keys => not supported
    +      throw TableException(
    +        "Joins should have at least one equality condition.\n" +
    +          s"\tLeft: ${left.toString},\n" +
    --- End diff --
    
    `toString` is not necessary here, it is called implicitly. 


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r148948471
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/DataStreamInnerJoin.scala ---
    @@ -0,0 +1,287 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +
    +package org.apache.flink.table.runtime.join
    +
    +import org.apache.flink.api.common.functions.FlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.java.typeutils.TupleTypeInfo
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.runtime.CRowWrappingMultiOuputCollector
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
    +import org.slf4j.LoggerFactory
    +import org.apache.flink.table.codegen.Compiler
    +
    +
    +/**
    +  * Connect data for left stream and right stream. Only use for innerJoin.
    +  *
    +  * @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
    +  */
    +class DataStreamInnerJoin(
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    resultType: TypeInformation[CRow],
    +    genJoinFuncName: String,
    +    genJoinFuncCode: String,
    +    queryConfig: StreamQueryConfig)
    +  extends CoProcessFunction[CRow, CRow, CRow]
    +          with Compiler[FlatJoinFunction[Row, Row, Row]] {
    +
    +  // state to hold left stream element
    +  private var leftState: MapState[Row, JTuple2[Int, Long]] = _
    +  // state to hold right stream element
    +  private var rightState: MapState[Row, JTuple2[Int, Long]] = _
    +  private var cRowWrapper: CRowWrappingMultiOuputCollector = _
    +
    +  private val minRetentionTime: Long = queryConfig.getMinIdleStateRetentionTime
    +  private val maxRetentionTime: Long = queryConfig.getMaxIdleStateRetentionTime
    +  private val stateCleaningEnabled: Boolean = minRetentionTime > 1
    +
    +  // state to record last timer of left stream, 0 means no timer
    +  private var leftTimer: ValueState[Long] = _
    +  // state to record last timer of right stream, 0 means no timer
    +  private var rightTimer: ValueState[Long] = _
    +
    +  // other condition function
    +  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
    +
    +  val LOG = LoggerFactory.getLogger(this.getClass)
    +
    +  override def open(parameters: Configuration): Unit = {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +                s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    joinFunction = clazz.newInstance()
    +
    +    // initialize left and right state, the first element of tuple2 indicates how many rows of
    +    // this row, while the second element represents the expired time of this row.
    +    val tupleTypeInfo = new TupleTypeInfo[JTuple2[Int, Long]](Types.INT, Types.LONG)
    +    val leftStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "left", leftType, tupleTypeInfo)
    +    val rightStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "right", rightType, tupleTypeInfo)
    +    leftState = getRuntimeContext.getMapState(leftStateDescriptor)
    +    rightState = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // initialize timer state
    +    val valueStateDescriptor1 = new ValueStateDescriptor[Long]("timervaluestate1", classOf[Long])
    +    leftTimer = getRuntimeContext.getState(valueStateDescriptor1)
    +    val valueStateDescriptor2 = new ValueStateDescriptor[Long]("timervaluestate2", classOf[Long])
    +    rightTimer = getRuntimeContext.getState(valueStateDescriptor2)
    +
    +    cRowWrapper = new CRowWrappingMultiOuputCollector()
    +  }
    +
    +  /**
    +    * Process left stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx    The ctx to register timer or get current time
    +    * @param out    The collector for returning result values.
    +    *
    +    */
    +  override def processElement1(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, leftTimer, leftState, rightState, true)
    +  }
    +
    +  /**
    +    * Process right stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx    The ctx to register timer or get current time
    +    * @param out    The collector for returning result values.
    +    *
    +    */
    +  override def processElement2(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, rightTimer, rightState, leftState, false)
    +  }
    +
    +
    +  /**
    +    * Called when a processing timer trigger.
    +    * Expire left/right records which are expired in left and right state.
    +    *
    +    * @param timestamp The timestamp of the firing timer.
    +    * @param ctx       The ctx to register timer or get current time
    +    * @param out       The collector for returning result values.
    +    */
    +  override def onTimer(
    +      timestamp: Long,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +
    +    if (stateCleaningEnabled && leftTimer.value == timestamp) {
    --- End diff --
    
    Then the `onTimer()` method will be invoked only if `stateCleaningEnabled = true`. The condition can be removed here.


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132235747
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/UpdatingPlanChecker.scala ---
    @@ -90,40 +96,86 @@ object UpdatingPlanChecker {
                   // resolve names of input fields
                   .map(io => (inNames.get(io._1), io._2))
     
    -            // filter by input keys
    -            val outKeys = inOutNames.filter(io => keys.get.contains(io._1)).map(_._2)
    -            // check if all keys have been preserved
    -            if (outKeys.nonEmpty && outKeys.length == keys.get.length) {
    +            // filter by input keyAncestors
    +            val outKeyAncesters = inOutNames
    +              .filter(io => keyAncestors.get.map(e => e._1).contains(io._1))
    +              .map(io => (io._2, keyAncestors.get.find(ka => ka._1 == io._1).get._2))
    +
    +            // check if all keyAncestors have been preserved
    +            if (outKeyAncesters.nonEmpty &&
    --- End diff --
    
    I think we can remove the `outKeyAncesters.nonEmpty` condition. 
    `keyAncestors` is not empty, so `outKeyAncestors` cannot be empty if we pass the length check.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by hequn8128 <gi...@git.apache.org>.
Github user hequn8128 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r143320024
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamJoinRule.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.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.flink.table.api.TableConfig
    +import org.apache.flink.table.calcite.FlinkTypeFactory
    +import org.apache.flink.table.plan.nodes.FlinkConventions
    +import org.apache.flink.table.plan.nodes.datastream.DataStreamJoin
    +import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
    +import org.apache.flink.table.plan.schema.RowSchema
    +import org.apache.flink.table.runtime.join.WindowJoinUtil
    +import scala.collection.JavaConverters._
    +
    +class DataStreamJoinRule
    +  extends ConverterRule(
    +    classOf[FlinkLogicalJoin],
    +    FlinkConventions.LOGICAL,
    +    FlinkConventions.DATASTREAM,
    +    "DataStreamJoinRule") {
    +
    +  override def matches(call: RelOptRuleCall): Boolean = {
    +    val join: FlinkLogicalJoin = call.rel(0).asInstanceOf[FlinkLogicalJoin]
    +    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)
    +
    +    // remaining predicate must not access time attributes
    +    val remainingPredsAccessTime = remainingPreds.isDefined &&
    +      WindowJoinUtil.accessesTimeAttribute(remainingPreds.get, join.getRowType)
    +
    +    // Check that no event-time attributes are in the input.
    +    val rowTimeAttrInOutput = join.getRowType.getFieldList.asScala
    +      .exists(f => FlinkTypeFactory.isRowtimeIndicatorType(f.getType))
    +
    +    if (!windowBounds.isDefined && !remainingPredsAccessTime && !rowTimeAttrInOutput) {
    --- End diff --
    
    Hi @fhueske , thanks a lot for this review. Maybe we need several iterations before everything is ok. I will update the PR and address your comments(except for this one). We can have more discussions then. I will update the PR in one or two days. 
    Thanks, Hequn


---

[GitHub] flink issue #4471: [FLINK-6094] [table] Implement stream-stream proctime non...

Posted by hequn8128 <gi...@git.apache.org>.
Github user hequn8128 commented on the issue:

    https://github.com/apache/flink/pull/4471
  
    hi @twalthr ~ Thanks for your review. I will give another pr to fix a small bug for FLINK-6094. 
    
    PS: Outer joins are coming soon. Also, some PRs to optimize the stream-stream (inner/outer) join has been planed.  : )


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r142779879
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamJoinRule.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.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.flink.table.api.TableConfig
    +import org.apache.flink.table.calcite.FlinkTypeFactory
    +import org.apache.flink.table.plan.nodes.FlinkConventions
    +import org.apache.flink.table.plan.nodes.datastream.DataStreamJoin
    +import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
    +import org.apache.flink.table.plan.schema.RowSchema
    +import org.apache.flink.table.runtime.join.WindowJoinUtil
    +import scala.collection.JavaConverters._
    +
    +class DataStreamJoinRule
    +  extends ConverterRule(
    +    classOf[FlinkLogicalJoin],
    +    FlinkConventions.LOGICAL,
    +    FlinkConventions.DATASTREAM,
    +    "DataStreamJoinRule") {
    +
    +  override def matches(call: RelOptRuleCall): Boolean = {
    +    val join: FlinkLogicalJoin = call.rel(0).asInstanceOf[FlinkLogicalJoin]
    +    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)
    +
    +    // remaining predicate must not access time attributes
    +    val remainingPredsAccessTime = remainingPreds.isDefined &&
    +      WindowJoinUtil.accessesTimeAttribute(remainingPreds.get, join.getRowType)
    +
    +    // Check that no event-time attributes are in the input.
    +    val rowTimeAttrInOutput = join.getRowType.getFieldList.asScala
    +      .exists(f => FlinkTypeFactory.isRowtimeIndicatorType(f.getType))
    +
    +    if (!windowBounds.isDefined && !remainingPredsAccessTime && !rowTimeAttrInOutput) {
    --- End diff --
    
    @shaoxuan-wang, I thought about this issue again and think you are right. It would be quite difficult for users to get the queries right and also difficult to properly document the restrictions.
    
    IMO, it would be good to evolve the relational APIs such that most operators can be executed on time indicator attributes (event or proc time) or not. In case of time indicator attributes, we can generate more efficient plans with built-in state clean-up. A generic stream-stream join such as the one proposed in the PR would be a first step in this direction.
    
    As you said before, a major challenge with this approach would be to help users configuring state cleanup timers. I would propose to extend the EXPLAIN information with state size estimates. This would help users users to correctly set the query configuration.
    
    I will go over my comments for this PR again and adapt them where necessary.
    Thanks, Fabian


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r152744007
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamJoin.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.nodes.datastream
    +
    +import org.apache.calcite.plan._
    +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.join.DataStreamInnerJoin
    +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.
    +  */
    +class DataStreamJoin(
    +    cluster: RelOptCluster,
    +    traitSet: RelTraitSet,
    +    leftNode: RelNode,
    +    rightNode: RelNode,
    +    joinCondition: RexNode,
    +    joinInfo: JoinInfo,
    +    joinType: JoinRelType,
    +    leftSchema: RowSchema,
    +    rightSchema: RowSchema,
    +    schema: RowSchema,
    +    ruleDescription: String)
    +  extends BiRel(cluster, traitSet, leftNode, rightNode)
    +          with CommonJoin
    +          with DataStreamRel {
    +
    +  override def deriveRowType(): RelDataType = schema.relDataType
    +
    +  override def needsUpdatesAsRetraction: Boolean = true
    +
    +  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
    +    new DataStreamJoin(
    +      cluster,
    +      traitSet,
    +      inputs.get(0),
    +      inputs.get(1),
    +      joinCondition,
    +      joinInfo,
    +      joinType,
    +      leftSchema,
    +      rightSchema,
    +      schema,
    +      ruleDescription)
    +  }
    +
    +  def getJoinInfo: JoinInfo = joinInfo
    +
    +  def getJoinType: JoinRelType = joinType
    +
    +  override def toString: String = {
    +    joinToString(
    +      schema.relDataType,
    +      joinCondition,
    +      joinType,
    +      getExpressionString)
    +  }
    +
    +  override def explainTerms(pw: RelWriter): RelWriter = {
    +    joinExplainTerms(
    +      super.explainTerms(pw),
    +      schema.relDataType,
    +      joinCondition,
    +      joinType,
    +      getExpressionString)
    +  }
    +
    +  override def translateToPlan(
    +      tableEnv: StreamTableEnvironment,
    +      queryConfig: StreamQueryConfig): DataStream[CRow] = {
    +
    +    val config = tableEnv.getConfig
    +    val returnType = schema.typeInfo
    +    val keyPairs = joinInfo.pairs().toList
    +
    +    // get the equality keys
    +    val leftKeys = ArrayBuffer.empty[Int]
    +    val rightKeys = ArrayBuffer.empty[Int]
    +    if (keyPairs.isEmpty) {
    +      // if no equality keys => not supported
    +      throw TableException(
    +        "Joins should have at least one equality condition.\n" +
    +          s"\tLeft: ${left.toString},\n" +
    +          s"\tRight: ${right.toString},\n" +
    +          s"\tCondition: (${joinConditionToString(schema.relDataType,
    +             joinCondition, getExpressionString)})"
    +      )
    +    }
    +    else {
    +      // at least one equality expression
    +      val leftFields = left.getRowType.getFieldList
    +      val rightFields = right.getRowType.getFieldList
    +
    +      keyPairs.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 {
    +          throw TableException(
    +            "Equality join predicate on incompatible types.\n" +
    +              s"\tLeft: ${left.toString},\n" +
    +              s"\tRight: ${right.toString},\n" +
    +              s"\tCondition: (${joinConditionToString(schema.relDataType,
    +                joinCondition, getExpressionString)})"
    +          )
    +        }
    +      })
    +    }
    +
    +    val leftDataStream =
    +      left.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
    +    val rightDataStream =
    +      right.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
    +
    +    val (connectOperator, nullCheck) = joinType match {
    +      case JoinRelType.INNER => (leftDataStream.connect(rightDataStream), false)
    +      case _ => throw new UnsupportedOperationException(s"An Unsupported JoinType [ $joinType ]")
    +    }
    +
    +    val generator = new FunctionCodeGenerator(
    +      config,
    +      nullCheck,
    +      leftSchema.typeInfo,
    +      Some(rightSchema.typeInfo))
    +    val conversion = generator.generateConverterResultExpression(
    +      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)
    +
    --- End diff --
    
    Remove double empty line (also in line 164).


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r152812041
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/validation/JoinValidationTest.scala ---
    @@ -92,4 +92,85 @@ class JoinValidationTest extends TableTestBase {
     
         util.verifyTable(resultTable, "")
       }
    +
    +
    +  private val util = streamTestUtil()
    +  private val ds1 = util.addTable[(Int, Long, String)]("Table3",'a, 'b, 'c)
    +  private val ds2 = util.addTable[(Int, Long, Int, String, Long)]("Table5", 'd, 'e, 'f, 'g, 'h)
    +
    +  @Test(expected = classOf[ValidationException])
    +  def testJoinNonExistingKey(): Unit = {
    +    ds1.join(ds2)
    +      // must fail. Field 'foo does not exist
    +      .where('foo === 'e)
    +      .select('c, 'g)
    +  }
    +
    +  @Test(expected = classOf[ValidationException])
    +  def testJoinWithNonMatchingKeyTypes(): Unit = {
    +    ds1.join(ds2)
    +      // must fail. Field 'a is Int, and 'g is String
    +      .where('a === 'g)
    +      .select('c, 'g)
    +  }
    +
    +
    --- End diff --
    
    Remove empty line.


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r152819401
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/JoinITCase.scala ---
    @@ -0,0 +1,262 @@
    +/*
    + * 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.table
    +
    +import org.apache.flink.api.scala._
    +import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
    +import org.apache.flink.table.api.{StreamQueryConfig, TableEnvironment}
    +import org.apache.flink.table.api.scala._
    +import org.apache.flink.table.runtime.utils.{StreamITCase, StreamingWithStateTestBase}
    +import org.junit.Assert._
    +import org.junit.Test
    +import org.apache.flink.api.common.time.Time
    +import org.apache.flink.streaming.api.TimeCharacteristic
    +import org.apache.flink.table.functions.aggfunctions.CountAggFunction
    +import org.apache.flink.table.runtime.stream.table.GroupWindowITCase.TimestampAndWatermarkWithOffset
    +import org.apache.flink.table.runtime.utils.JavaUserDefinedAggFunctions.{CountDistinct, WeightedAvg}
    +import org.apache.flink.types.Row
    +
    +import scala.collection.mutable
    +
    +class JoinITCase extends StreamingWithStateTestBase {
    +
    +  private val queryConfig = new StreamQueryConfig()
    +  queryConfig.withIdleStateRetentionTime(Time.hours(1), Time.hours(2))
    +
    +  @Test
    +  def testOutputWithPk(): Unit = {
    +    // data input
    +
    +    val data1 = List(
    +      (0, 0),
    +      (1, 0),
    +      (1, 1),
    +      (2, 2),
    +      (3, 3),
    +      (4, 4),
    +      (5, 4),
    +      (5, 5)
    +    )
    +
    +    val data2 = List(
    +      (1, 1),
    +      (2, 0),
    +      (2, 1),
    +      (2, 2),
    +      (3, 3),
    +      (4, 4),
    +      (5, 4),
    +      (5, 5),
    +      (6, 6)
    +    )
    +
    +    val env = StreamExecutionEnvironment.getExecutionEnvironment
    +    val tEnv = TableEnvironment.getTableEnvironment(env)
    +    StreamITCase.clear
    +    env.setStateBackend(getStateBackend)
    +    env.setParallelism(1)
    +
    +    val leftTable = env.fromCollection(data1).toTable(tEnv, 'a, 'b)
    +    val rightTable = env.fromCollection(data2).toTable(tEnv, 'bb, 'c)
    +
    +    val leftTableWithPk = leftTable
    +      .groupBy('a)
    +      .select('a, 'b.max as 'b)
    +
    +    val rightTableWithPk = rightTable
    +        .groupBy('bb)
    +        .select('bb, 'c.max as 'c)
    +
    +    leftTableWithPk
    +      .join(rightTableWithPk, 'b === 'bb)
    +      .select('a, 'b, 'c)
    +      .writeToSink(new TestUpsertSink(Array("a,b"), false), queryConfig)
    +
    +    env.execute()
    +    val results = RowCollector.getAndClearValues
    +    val retracted = RowCollector.upsertResults(results, Array(0)).sorted
    +
    +    val expected = Seq("1,1,1", "2,2,2", "3,3,3", "4,4,4", "5,5,5")
    +    assertEquals(expected, retracted)
    +
    +  }
    +
    +
    +  @Test
    +  def testOutputWithoutPk(): Unit = {
    +    // data input
    +
    +    val data1 = List(
    +      (0, 0),
    +      (1, 0),
    +      (1, 1),
    +      (2, 2),
    +      (3, 3),
    +      (4, 4),
    +      (5, 4),
    +      (5, 5)
    +    )
    +
    +    val data2 = List(
    +      (1, 1, 1),
    +      (1, 1, 1),
    +      (1, 1, 1),
    +      (1, 1, 1),
    +      (2, 2, 2),
    +      (3, 3, 3),
    +      (4, 4, 4),
    +      (5, 5, 5),
    +      (5, 5, 5),
    +      (6, 6, 6)
    +    )
    +
    +    val env = StreamExecutionEnvironment.getExecutionEnvironment
    +    val tEnv = TableEnvironment.getTableEnvironment(env)
    +    StreamITCase.clear
    +    env.setStateBackend(getStateBackend)
    +    env.setParallelism(1)
    +
    +    val leftTable = env.fromCollection(data1).toTable(tEnv, 'a, 'b)
    +    val rightTable = env.fromCollection(data2).toTable(tEnv, 'bb, 'c, 'd)
    +
    +    val leftTableWithPk = leftTable
    +      .groupBy('a)
    +      .select('a, 'b.max as 'b)
    +
    +    leftTableWithPk
    +      .join(rightTable, 'a === 'bb && ('a < 4 || 'a > 4))
    +      .select('a, 'b, 'c, 'd)
    +      .writeToSink(new TestRetractSink, queryConfig)
    +
    +    env.execute()
    +    val results = RowCollector.getAndClearValues
    +
    +    val retracted = RowCollector.retractResults(results).sorted
    +
    +    val expected = Seq("1,1,1,1", "1,1,1,1", "1,1,1,1", "1,1,1,1", "2,2,2,2", "3,3,3,3",
    +                       "5,5,5,5", "5,5,5,5")
    +    assertEquals(expected, retracted)
    +  }
    +
    +
    +  @Test
    +  def testJoinWithTimeAttributeOutput() {
    +
    +    val data1 = List(
    +      (1L, 1, "LEFT:Hi"),
    +      (2L, 2, "LEFT:Hello"),
    +      (4L, 2, "LEFT:Hello"),
    +      (8L, 3, "LEFT:Hello world"),
    +      (16L, 3, "LEFT:Hello world"))
    +
    +    val data2 = List(
    +      (1L, 1, "RIGHT:Hi"),
    +      (2L, 2, "RIGHT:Hello"),
    +      (4L, 2, "RIGHT:Hello"),
    +      (8L, 3, "RIGHT:Hello world"),
    +      (16L, 3, "RIGHT:Hello world"))
    +
    +    val env = StreamExecutionEnvironment.getExecutionEnvironment
    +    env.setParallelism(1)
    +    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
    +    val tEnv = TableEnvironment.getTableEnvironment(env)
    +    StreamITCase.testResults = mutable.MutableList()
    +
    +    val stream1 = env
    +      .fromCollection(data1)
    +      .assignTimestampsAndWatermarks(new TimestampAndWatermarkWithOffset[(Long, Int, String)](0L))
    +    val stream2 = env
    +      .fromCollection(data2)
    +      .assignTimestampsAndWatermarks(new TimestampAndWatermarkWithOffset[(Long, Int, String)](0L))
    +
    +    val table1 = stream1.toTable(tEnv, 'long_l, 'int_l, 'string_l, 'rowtime_l.rowtime)
    --- End diff --
    
    Test proctime as well?


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r146105266
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/DataStreamInnerJoin.scala ---
    @@ -0,0 +1,287 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +
    +package org.apache.flink.table.runtime.join
    +
    +import org.apache.flink.api.common.functions.FlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.java.typeutils.TupleTypeInfo
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.runtime.CRowWrappingMultiOuputCollector
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
    +import org.slf4j.LoggerFactory
    +import org.apache.flink.table.codegen.Compiler
    +
    +
    +/**
    +  * Connect data for left stream and right stream. Only use for innerJoin.
    +  *
    +  * @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
    +  */
    +class DataStreamInnerJoin(
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    resultType: TypeInformation[CRow],
    +    genJoinFuncName: String,
    +    genJoinFuncCode: String,
    +    queryConfig: StreamQueryConfig)
    +  extends CoProcessFunction[CRow, CRow, CRow]
    +          with Compiler[FlatJoinFunction[Row, Row, Row]] {
    +
    +  // state to hold left stream element
    +  private var leftState: MapState[Row, JTuple2[Int, Long]] = _
    +  // state to hold right stream element
    +  private var rightState: MapState[Row, JTuple2[Int, Long]] = _
    +  private var cRowWrapper: CRowWrappingMultiOuputCollector = _
    +
    +  private val minRetentionTime: Long = queryConfig.getMinIdleStateRetentionTime
    +  private val maxRetentionTime: Long = queryConfig.getMaxIdleStateRetentionTime
    +  private val stateCleaningEnabled: Boolean = minRetentionTime > 1
    +
    +  // state to record last timer of left stream, 0 means no timer
    +  private var leftTimer: ValueState[Long] = _
    +  // state to record last timer of right stream, 0 means no timer
    +  private var rightTimer: ValueState[Long] = _
    +
    +  // other condition function
    +  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
    +
    +  val LOG = LoggerFactory.getLogger(this.getClass)
    +
    +  override def open(parameters: Configuration): Unit = {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +                s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    joinFunction = clazz.newInstance()
    +
    +    // initialize left and right state, the first element of tuple2 indicates how many rows of
    +    // this row, while the second element represents the expired time of this row.
    +    val tupleTypeInfo = new TupleTypeInfo[JTuple2[Int, Long]](Types.INT, Types.LONG)
    +    val leftStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "left", leftType, tupleTypeInfo)
    +    val rightStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "right", rightType, tupleTypeInfo)
    +    leftState = getRuntimeContext.getMapState(leftStateDescriptor)
    +    rightState = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // initialize timer state
    +    val valueStateDescriptor1 = new ValueStateDescriptor[Long]("timervaluestate1", classOf[Long])
    +    leftTimer = getRuntimeContext.getState(valueStateDescriptor1)
    +    val valueStateDescriptor2 = new ValueStateDescriptor[Long]("timervaluestate2", classOf[Long])
    +    rightTimer = getRuntimeContext.getState(valueStateDescriptor2)
    +
    +    cRowWrapper = new CRowWrappingMultiOuputCollector()
    +  }
    +
    +  /**
    +    * Process left stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx    The ctx to register timer or get current time
    +    * @param out    The collector for returning result values.
    +    *
    +    */
    +  override def processElement1(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, leftTimer, leftState, rightState, true)
    +  }
    +
    +  /**
    +    * Process right stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx    The ctx to register timer or get current time
    +    * @param out    The collector for returning result values.
    +    *
    +    */
    +  override def processElement2(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, rightTimer, rightState, leftState, false)
    +  }
    +
    +
    +  /**
    +    * Called when a processing timer trigger.
    +    * Expire left/right records which are expired in left and right state.
    +    *
    +    * @param timestamp The timestamp of the firing timer.
    +    * @param ctx       The ctx to register timer or get current time
    +    * @param out       The collector for returning result values.
    +    */
    +  override def onTimer(
    +      timestamp: Long,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +
    +    if (stateCleaningEnabled && leftTimer.value == timestamp) {
    --- End diff --
    
    It seems if `stateCleaningEnabled = false`, there will be no timer registered, right?


---

[GitHub] flink issue #4471: [FLINK-6094] [table] Implement stream-stream proctime non...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on the issue:

    https://github.com/apache/flink/pull/4471
  
    Thanks for the update @hequn8128. I will go through the code a last time and merge this :)


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132306893
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/UpdatingPlanCheckerTest.scala ---
    @@ -0,0 +1,203 @@
    +/*
    + * 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
    +
    +import org.apache.flink.table.api.Table
    +import org.apache.flink.table.plan.util.UpdatingPlanChecker
    +import org.apache.flink.table.utils.StreamTableTestUtil
    +import org.junit.Assert._
    +import org.apache.flink.table.api.scala._
    +import org.apache.flink.api.scala._
    +import org.junit.Test
    +
    +
    +class UpdatingPlanCheckerTest {
    +
    +  @Test
    +  def testSelect(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(String, Int)]('word, 'number)
    +    val resultTable = table.select('word, 'number)
    +
    +    util.verifyTableUniqueKey(resultTable, Nil)
    +  }
    +
    +  @Test
    +  def testGroupByWithoutKey(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(String, Int)]('word, 'number)
    +
    +    val resultTable = table
    +      .groupBy('word)
    +      .select('number.count)
    +
    +    util.verifyTableUniqueKey(resultTable, Nil)
    +  }
    +
    +  @Test
    +  def testGroupBy(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(String, Int)]('word, 'number)
    +
    +    val resultTable = table
    +      .groupBy('word)
    +      .select('word, 'number.count)
    +
    +    util.verifyTableUniqueKey(resultTable, Seq("word"))
    +  }
    +
    +  @Test
    +  def testGroupByWithDumplicateKey(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(String, Int)]('word, 'number)
    +
    +    val resultTable = table
    +      .groupBy('word)
    +      .select('word as 'word1, 'word as 'word2, 'number.count)
    +
    +    util.verifyTableUniqueKey(resultTable, Seq("word1", "word2"))
    +  }
    +
    --- End diff --
    
    Please add some tests for window aggregations.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132242877
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/UpdatingPlanChecker.scala ---
    @@ -90,40 +96,86 @@ object UpdatingPlanChecker {
                   // resolve names of input fields
                   .map(io => (inNames.get(io._1), io._2))
     
    -            // filter by input keys
    -            val outKeys = inOutNames.filter(io => keys.get.contains(io._1)).map(_._2)
    -            // check if all keys have been preserved
    -            if (outKeys.nonEmpty && outKeys.length == keys.get.length) {
    +            // filter by input keyAncestors
    +            val outKeyAncesters = inOutNames
    +              .filter(io => keyAncestors.get.map(e => e._1).contains(io._1))
    +              .map(io => (io._2, keyAncestors.get.find(ka => ka._1 == io._1).get._2))
    +
    +            // check if all keyAncestors have been preserved
    +            if (outKeyAncesters.nonEmpty &&
    +              outKeyAncesters.map(ka => ka._2).distinct.length ==
    +                keyAncestors.get.map(ka => ka._2).distinct.length) {
                   // all key have been preserved (but possibly renamed)
    -              keys = Some(outKeys.toArray)
    +              Some(outKeyAncesters.toList)
                 } else {
                   // some (or all) keys have been removed. Keys are no longer unique and removed
    -              keys = None
    +              None
                 }
    +          } else {
    +            None
               }
    +
             case _: DataStreamOverAggregate =>
    -          super.visit(node, ordinal, parent)
    -        // keys are always forwarded by Over aggregate
    +          // keyAncestors are always forwarded by Over aggregate
    +          visit(node.getInput(0))
             case a: DataStreamGroupAggregate =>
    -          // get grouping keys
    +          // get grouping keyAncestors
               val groupKeys = a.getRowType.getFieldNames.asScala.take(a.getGroupings.length)
    -          keys = Some(groupKeys.toArray)
    +          Some(groupKeys.map(e => (e, e)).toList)
             case w: DataStreamGroupWindowAggregate =>
    -          // get grouping keys
    +          // get grouping keyAncestors
               val groupKeys =
                 w.getRowType.getFieldNames.asScala.take(w.getGroupings.length).toArray
               // get window start and end time
               val windowStartEnd = w.getWindowProperties.map(_.name)
               // we have only a unique key if at least one window property is selected
               if (windowStartEnd.nonEmpty) {
    -            keys = Some(groupKeys ++ windowStartEnd)
    +            Some((groupKeys ++ windowStartEnd).map(e => (e, e)).toList)
    +          } else {
    +            None
    +          }
    +
    +        case j: DataStreamJoin =>
    +          val leftKeyAncestors = visit(j.getLeft)
    +          val rightKeyAncestors = visit(j.getRight)
    +          if (!leftKeyAncestors.isDefined || !rightKeyAncestors.isDefined) {
    +            None
    +          } else {
    +            // both left and right contain keys
    +            val leftJoinKeys =
    --- End diff --
    
    It would be helpful to add the type of `leftJoinKeys` -> `val leftJoinKeys: Seq[String] = `


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by hequn8128 <gi...@git.apache.org>.
Github user hequn8128 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r158489047
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/UpdatingPlanChecker.scala ---
    @@ -116,14 +135,100 @@ object UpdatingPlanChecker {
               val windowStartEnd = w.getWindowProperties.map(_.name)
    --- End diff --
    
    Window can only output start or end field, right ?


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r152753324
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamJoinRule.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.plan.rules.datastream
    +
    +import org.apache.calcite.plan.{RelOptRule, 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.rex.{RexCall, RexInputRef, RexNode}
    +import org.apache.flink.table.api.TableConfig
    +import org.apache.flink.table.plan.nodes.FlinkConventions
    +import org.apache.flink.table.plan.nodes.datastream.DataStreamJoin
    +import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
    +import org.apache.flink.table.plan.schema.{RowSchema, TimeIndicatorRelDataType}
    +import org.apache.flink.table.runtime.join.WindowJoinUtil
    +
    +import scala.collection.JavaConverters._
    +
    +class DataStreamJoinRule
    +  extends ConverterRule(
    +    classOf[FlinkLogicalJoin],
    +    FlinkConventions.LOGICAL,
    +    FlinkConventions.DATASTREAM,
    +    "DataStreamJoinRule") {
    +
    +  /**
    +    * Checks if an expression accesses a time attribute.
    +    *
    +    * @param expr The expression to check.
    +    * @param inputType The input type of the expression.
    +    * @return True, if the expression accesses a time attribute. False otherwise.
    +    */
    +  def accessesTimeAttribute(expr: RexNode, inputType: RelDataType): Boolean = {
    +    expr match {
    +      case i: RexInputRef =>
    +        val accessedType = inputType.getFieldList.get(i.getIndex).getType
    +        accessedType match {
    +          case _: TimeIndicatorRelDataType => true
    +          case _ => false
    +        }
    +      case c: RexCall =>
    +        c.operands.asScala.exists(accessesTimeAttribute(_, inputType))
    +      case _ => false
    +    }
    +  }
    +
    +  override def matches(call: RelOptRuleCall): Boolean = {
    +    val join: FlinkLogicalJoin = call.rel(0).asInstanceOf[FlinkLogicalJoin]
    +    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)
    +
    +    // remaining predicate must not access time attributes
    +    val remainingPredsAccessTime = remainingPreds.isDefined &&
    +      accessesTimeAttribute(remainingPreds.get, join.getRowType)
    --- End diff --
    
    I also think that we don't need special treatment here. The rowtime is a regular field and proctime should be evaluated by the code generator, no?


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r152753104
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamJoinRule.scala ---
    @@ -0,0 +1,108 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.table.plan.rules.datastream
    +
    +import org.apache.calcite.plan.{RelOptRule, 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.rex.{RexCall, RexInputRef, RexNode}
    +import org.apache.flink.table.api.TableConfig
    +import org.apache.flink.table.plan.nodes.FlinkConventions
    +import org.apache.flink.table.plan.nodes.datastream.DataStreamJoin
    +import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
    +import org.apache.flink.table.plan.schema.{RowSchema, TimeIndicatorRelDataType}
    +import org.apache.flink.table.runtime.join.WindowJoinUtil
    +
    +import scala.collection.JavaConverters._
    +
    +class DataStreamJoinRule
    +  extends ConverterRule(
    +    classOf[FlinkLogicalJoin],
    +    FlinkConventions.LOGICAL,
    +    FlinkConventions.DATASTREAM,
    +    "DataStreamJoinRule") {
    +
    +  /**
    +    * Checks if an expression accesses a time attribute.
    +    *
    +    * @param expr The expression to check.
    +    * @param inputType The input type of the expression.
    +    * @return True, if the expression accesses a time attribute. False otherwise.
    +    */
    +  def accessesTimeAttribute(expr: RexNode, inputType: RelDataType): Boolean = {
    +    expr match {
    +      case i: RexInputRef =>
    +        val accessedType = inputType.getFieldList.get(i.getIndex).getType
    +        accessedType match {
    +          case _: TimeIndicatorRelDataType => true
    +          case _ => false
    +        }
    +      case c: RexCall =>
    +        c.operands.asScala.exists(accessesTimeAttribute(_, inputType))
    +      case _ => false
    +    }
    +  }
    +
    +  override def matches(call: RelOptRuleCall): Boolean = {
    +    val join: FlinkLogicalJoin = call.rel(0).asInstanceOf[FlinkLogicalJoin]
    +    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)
    +
    +    // remaining predicate must not access time attributes
    +    val remainingPredsAccessTime = remainingPreds.isDefined &&
    +      accessesTimeAttribute(remainingPreds.get, join.getRowType)
    +
    +    !windowBounds.isDefined && !remainingPredsAccessTime
    --- End diff --
    
    Replace with `windowBounds.isEmpty`


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132300708
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeNonWindowInnerJoin.scala ---
    @@ -0,0 +1,275 @@
    +/*
    + * 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.RichFlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.java.typeutils.{ResultTypeQueryable, TupleTypeInfo}
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.runtime.CRowWrappingMultiOuputCollector
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
    +
    +/**
    +  * Connect data for left stream and right stream. Only use for innerJoin.
    +  *
    +  * @param joiner           join function
    +  * @param leftType         the input type of left stream
    +  * @param rightType        the input type of right stream
    +  * @param resultType       the output type of join
    +  * @param queryConfig
    +  */
    +class ProcTimeNonWindowInnerJoin(
    +    joiner: RichFlatJoinFunction[Row, Row, Row],
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    resultType: TypeInformation[CRow],
    +    queryConfig: StreamQueryConfig) extends
    +  CoProcessFunction[CRow, CRow, CRow] with ResultTypeQueryable[CRow] {
    +
    +
    +  // state to hold left stream element
    +  private var leftState: MapState[Row, JTuple2[Int, Long]] = null
    +  // state to hold right stream element
    +  private var rightState: MapState[Row, JTuple2[Int, Long]] = null
    +  private var cRowWrapper: CRowWrappingMultiOuputCollector = null
    +
    +  private val minRetentionTime: Long = queryConfig.getMinIdleStateRetentionTime
    +  private val maxRetentionTime: Long = queryConfig.getMaxIdleStateRetentionTime
    +  private val stateCleaningEnabled: Boolean = minRetentionTime > 1
    +
    +  // state to record last timer of left stream, 0 means no timer
    +  private var timerState1: ValueState[Long] = _
    +  // state to record last timer of right stream, 0 means no timer
    +  private var timerState2: ValueState[Long] = _
    +
    +
    +  override def open(parameters: Configuration): Unit = {
    +    // initialize left and right state
    +    val tupleTypeInfo = new TupleTypeInfo[JTuple2[Int, Long]](Types.INT, Types.LONG)
    +    val leftStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "left", leftType, tupleTypeInfo)
    +    val rightStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "right", rightType, tupleTypeInfo)
    +    leftState = getRuntimeContext.getMapState(leftStateDescriptor)
    +    rightState = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // initialize timer state
    +    val valueStateDescriptor1 = new ValueStateDescriptor[Long]("timervaluestate1", classOf[Long])
    +    timerState1 = getRuntimeContext.getState(valueStateDescriptor1)
    +    val valueStateDescriptor2 = new ValueStateDescriptor[Long]("timervaluestate2", classOf[Long])
    +    timerState2 = getRuntimeContext.getState(valueStateDescriptor2)
    +
    +    cRowWrapper = new CRowWrappingMultiOuputCollector()
    +    joiner.setRuntimeContext(getRuntimeContext)
    +    joiner.open(parameters)
    +  }
    +
    +  /**
    +    * Process left stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx   The ctx to register timer or get current time
    +    * @param out   The collector for returning result values.
    +    *
    +    */
    +  override def processElement1(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, timerState1, leftState, rightState, true)
    +  }
    +
    +  /**
    +    * Process right stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx   The ctx to register timer or get current time
    +    * @param out   The collector for returning result values.
    +    *
    +    */
    +  override def processElement2(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, timerState2, rightState, leftState, false)
    +  }
    +
    +
    +  /**
    +    * Called when a processing timer trigger.
    +    * Expire left/right records which are expired in left and right state.
    +    *
    +    * @param timestamp The timestamp of the firing timer.
    +    * @param ctx       The ctx to register timer or get current time
    +    * @param out       The collector for returning result values.
    +    */
    +  override def onTimer(
    +      timestamp: Long,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +
    +    if (stateCleaningEnabled && timerState1.value == timestamp) {
    +      expireOutTimeRow(
    +        timestamp,
    +        leftState,
    +        timerState1,
    +        ctx
    +      )
    +    }
    +
    +    if (stateCleaningEnabled && timerState2.value == timestamp) {
    +      expireOutTimeRow(
    +        timestamp,
    +        rightState,
    +        timerState2,
    +        ctx
    +      )
    +    }
    +  }
    +
    +
    +  def getNewExpiredTime(
    +      curProcessTime: Long,
    +      oldExpiredTime: Long): Long = {
    +
    +    var newExpiredTime = oldExpiredTime
    +    if (stateCleaningEnabled) {
    +      if (-1 == oldExpiredTime || (curProcessTime + minRetentionTime) > oldExpiredTime) {
    +        newExpiredTime = curProcessTime + maxRetentionTime
    +      }
    +    }
    +    newExpiredTime
    +  }
    +
    +  /**
    +    * Puts or Retract an element from the input stream into state and search the other state to
    +    * output records meet the condition. Records will be expired in state if state retention time
    +    * has been specified.
    +    */
    +  def processElement(
    +      value: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow],
    +      timerState: ValueState[Long],
    +      currentSideState: MapState[Row, JTuple2[Int, Long]],
    +      otherSideState: MapState[Row, JTuple2[Int, Long]],
    +      isLeft: Boolean): Unit = {
    +
    +    cRowWrapper.out = out
    +    cRowWrapper.setChange(value.change)
    +
    +    val curProcessTime = ctx.timerService.currentProcessingTime
    +    var oldCnt = 0
    +    var oldExpiredTime: Long = -1
    +
    +    val currentRowCntAndExpiredTime = currentSideState.get(value.row)
    +    if (currentRowCntAndExpiredTime != null) {
    +      oldCnt = currentRowCntAndExpiredTime.f0
    +      oldExpiredTime = currentRowCntAndExpiredTime.f1
    +    }
    +
    +    val newExpiredTime = getNewExpiredTime(curProcessTime, oldExpiredTime)
    +    if (stateCleaningEnabled && timerState.value() == 0) {
    +      timerState.update(newExpiredTime)
    +      ctx.timerService().registerProcessingTimeTimer(newExpiredTime)
    +    }
    +
    +    // update current side stream state
    +    if (!value.asInstanceOf[CRow].change) {
    +      oldCnt = oldCnt - 1
    --- End diff --
    
    -> `val newCnt = oldCnt - 1`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #4471: [FLINK-6094] [table] Implement stream-stream proctime non...

Posted by hequn8128 <gi...@git.apache.org>.
Github user hequn8128 commented on the issue:

    https://github.com/apache/flink/pull/4471
  
    hi @twalthr , Sorry for the late reply. The notification of github has been ignored mistakenly. I will give an update ASAP. Thanks very much. 


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132456285
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/UpdatingPlanChecker.scala ---
    @@ -90,40 +96,86 @@ object UpdatingPlanChecker {
                   // resolve names of input fields
                   .map(io => (inNames.get(io._1), io._2))
     
    -            // filter by input keys
    -            val outKeys = inOutNames.filter(io => keys.get.contains(io._1)).map(_._2)
    -            // check if all keys have been preserved
    -            if (outKeys.nonEmpty && outKeys.length == keys.get.length) {
    +            // filter by input keyAncestors
    +            val outKeyAncesters = inOutNames
    +              .filter(io => keyAncestors.get.map(e => e._1).contains(io._1))
    +              .map(io => (io._2, keyAncestors.get.find(ka => ka._1 == io._1).get._2))
    +
    +            // check if all keyAncestors have been preserved
    +            if (outKeyAncesters.nonEmpty &&
    +              outKeyAncesters.map(ka => ka._2).distinct.length ==
    +                keyAncestors.get.map(ka => ka._2).distinct.length) {
                   // all key have been preserved (but possibly renamed)
    -              keys = Some(outKeys.toArray)
    +              Some(outKeyAncesters.toList)
                 } else {
                   // some (or all) keys have been removed. Keys are no longer unique and removed
    -              keys = None
    +              None
                 }
    +          } else {
    +            None
               }
    +
             case _: DataStreamOverAggregate =>
    -          super.visit(node, ordinal, parent)
    -        // keys are always forwarded by Over aggregate
    +          // keyAncestors are always forwarded by Over aggregate
    +          visit(node.getInput(0))
             case a: DataStreamGroupAggregate =>
    -          // get grouping keys
    +          // get grouping keyAncestors
               val groupKeys = a.getRowType.getFieldNames.asScala.take(a.getGroupings.length)
    -          keys = Some(groupKeys.toArray)
    +          Some(groupKeys.map(e => (e, e)).toList)
             case w: DataStreamGroupWindowAggregate =>
    -          // get grouping keys
    +          // get grouping keyAncestors
               val groupKeys =
                 w.getRowType.getFieldNames.asScala.take(w.getGroupings.length).toArray
               // get window start and end time
               val windowStartEnd = w.getWindowProperties.map(_.name)
               // we have only a unique key if at least one window property is selected
               if (windowStartEnd.nonEmpty) {
    -            keys = Some(groupKeys ++ windowStartEnd)
    +            Some((groupKeys ++ windowStartEnd).map(e => (e, e)).toList)
    +          } else {
    +            None
    +          }
    +
    +        case j: DataStreamJoin =>
    +          val leftKeyAncestors = visit(j.getLeft)
    +          val rightKeyAncestors = visit(j.getRight)
    +          if (!leftKeyAncestors.isDefined || !rightKeyAncestors.isDefined) {
    +            None
    +          } else {
    +            // both left and right contain keys
    +            val leftJoinKeys =
    +              j.getLeft.getRowType.getFieldNames.asScala.zipWithIndex
    +              .filter(e => j.getJoinInfo.leftKeys.contains(e._2))
    +              .map(e => e._1)
    +            val rightJoinKeys =
    +              j.getRight.getRowType.getFieldNames.asScala.zipWithIndex
    +                .filter(e => j.getJoinInfo.rightKeys.contains(e._2))
    +                .map(e => e._1)
    +
    +            val leftKeys = leftKeyAncestors.get.map(e => e._1)
    +            val rightKeys = rightKeyAncestors.get.map(e => e._1)
    +
    +            //1. join key = left key = right key
    +            if (leftJoinKeys == leftKeys && rightJoinKeys == rightKeys) {
    +              Some(leftKeyAncestors.get ::: (rightKeyAncestors.get.map(e => (e._1)) zip
    +                leftKeyAncestors.get.map(e => (e._2))))
    +            }
    +            //2. join key = left key
    +            else if (leftJoinKeys == leftKeys && rightJoinKeys != rightKeys) {
    +              rightKeyAncestors
    +            }
    +            //3. join key = right key
    +            else if (leftJoinKeys != leftKeys && rightJoinKeys == rightKeys) {
    +              leftKeyAncestors
    --- End diff --
    
    all fields which are equal to a left key field become a key as well (with the same ancestor as the left key field).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r152819187
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/JoinITCase.scala ---
    @@ -0,0 +1,262 @@
    +/*
    + * 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.table
    +
    +import org.apache.flink.api.scala._
    +import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
    +import org.apache.flink.table.api.{StreamQueryConfig, TableEnvironment}
    +import org.apache.flink.table.api.scala._
    +import org.apache.flink.table.runtime.utils.{StreamITCase, StreamingWithStateTestBase}
    +import org.junit.Assert._
    +import org.junit.Test
    +import org.apache.flink.api.common.time.Time
    +import org.apache.flink.streaming.api.TimeCharacteristic
    +import org.apache.flink.table.functions.aggfunctions.CountAggFunction
    +import org.apache.flink.table.runtime.stream.table.GroupWindowITCase.TimestampAndWatermarkWithOffset
    +import org.apache.flink.table.runtime.utils.JavaUserDefinedAggFunctions.{CountDistinct, WeightedAvg}
    +import org.apache.flink.types.Row
    +
    +import scala.collection.mutable
    +
    +class JoinITCase extends StreamingWithStateTestBase {
    +
    +  private val queryConfig = new StreamQueryConfig()
    +  queryConfig.withIdleStateRetentionTime(Time.hours(1), Time.hours(2))
    +
    +  @Test
    +  def testOutputWithPk(): Unit = {
    +    // data input
    +
    +    val data1 = List(
    +      (0, 0),
    +      (1, 0),
    +      (1, 1),
    +      (2, 2),
    +      (3, 3),
    +      (4, 4),
    +      (5, 4),
    +      (5, 5)
    +    )
    +
    +    val data2 = List(
    +      (1, 1),
    +      (2, 0),
    +      (2, 1),
    +      (2, 2),
    +      (3, 3),
    +      (4, 4),
    +      (5, 4),
    +      (5, 5),
    +      (6, 6)
    +    )
    +
    +    val env = StreamExecutionEnvironment.getExecutionEnvironment
    +    val tEnv = TableEnvironment.getTableEnvironment(env)
    +    StreamITCase.clear
    +    env.setStateBackend(getStateBackend)
    +    env.setParallelism(1)
    +
    +    val leftTable = env.fromCollection(data1).toTable(tEnv, 'a, 'b)
    +    val rightTable = env.fromCollection(data2).toTable(tEnv, 'bb, 'c)
    +
    +    val leftTableWithPk = leftTable
    +      .groupBy('a)
    +      .select('a, 'b.max as 'b)
    +
    +    val rightTableWithPk = rightTable
    +        .groupBy('bb)
    +        .select('bb, 'c.max as 'c)
    +
    +    leftTableWithPk
    +      .join(rightTableWithPk, 'b === 'bb)
    +      .select('a, 'b, 'c)
    +      .writeToSink(new TestUpsertSink(Array("a,b"), false), queryConfig)
    +
    +    env.execute()
    +    val results = RowCollector.getAndClearValues
    +    val retracted = RowCollector.upsertResults(results, Array(0)).sorted
    +
    +    val expected = Seq("1,1,1", "2,2,2", "3,3,3", "4,4,4", "5,5,5")
    +    assertEquals(expected, retracted)
    +
    +  }
    +
    +
    +  @Test
    +  def testOutputWithoutPk(): Unit = {
    +    // data input
    +
    +    val data1 = List(
    +      (0, 0),
    +      (1, 0),
    +      (1, 1),
    +      (2, 2),
    +      (3, 3),
    +      (4, 4),
    +      (5, 4),
    +      (5, 5)
    +    )
    +
    +    val data2 = List(
    +      (1, 1, 1),
    +      (1, 1, 1),
    +      (1, 1, 1),
    +      (1, 1, 1),
    +      (2, 2, 2),
    +      (3, 3, 3),
    +      (4, 4, 4),
    +      (5, 5, 5),
    +      (5, 5, 5),
    +      (6, 6, 6)
    +    )
    +
    +    val env = StreamExecutionEnvironment.getExecutionEnvironment
    +    val tEnv = TableEnvironment.getTableEnvironment(env)
    +    StreamITCase.clear
    +    env.setStateBackend(getStateBackend)
    +    env.setParallelism(1)
    +
    +    val leftTable = env.fromCollection(data1).toTable(tEnv, 'a, 'b)
    +    val rightTable = env.fromCollection(data2).toTable(tEnv, 'bb, 'c, 'd)
    +
    +    val leftTableWithPk = leftTable
    +      .groupBy('a)
    +      .select('a, 'b.max as 'b)
    +
    +    leftTableWithPk
    +      .join(rightTable, 'a === 'bb && ('a < 4 || 'a > 4))
    +      .select('a, 'b, 'c, 'd)
    +      .writeToSink(new TestRetractSink, queryConfig)
    +
    +    env.execute()
    +    val results = RowCollector.getAndClearValues
    +
    +    val retracted = RowCollector.retractResults(results).sorted
    +
    +    val expected = Seq("1,1,1,1", "1,1,1,1", "1,1,1,1", "1,1,1,1", "2,2,2,2", "3,3,3,3",
    +                       "5,5,5,5", "5,5,5,5")
    +    assertEquals(expected, retracted)
    +  }
    +
    +
    +  @Test
    +  def testJoinWithTimeAttributeOutput() {
    +
    +    val data1 = List(
    +      (1L, 1, "LEFT:Hi"),
    +      (2L, 2, "LEFT:Hello"),
    +      (4L, 2, "LEFT:Hello"),
    +      (8L, 3, "LEFT:Hello world"),
    +      (16L, 3, "LEFT:Hello world"))
    +
    +    val data2 = List(
    +      (1L, 1, "RIGHT:Hi"),
    +      (2L, 2, "RIGHT:Hello"),
    +      (4L, 2, "RIGHT:Hello"),
    +      (8L, 3, "RIGHT:Hello world"),
    +      (16L, 3, "RIGHT:Hello world"))
    +
    +    val env = StreamExecutionEnvironment.getExecutionEnvironment
    +    env.setParallelism(1)
    --- End diff --
    
    Enable default parallelism?


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r146104452
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamJoin.scala ---
    @@ -0,0 +1,211 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.table.plan.nodes.datastream
    +
    +import org.apache.calcite.plan._
    +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.join.DataStreamInnerJoin
    +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.
    +  */
    +class DataStreamJoin(
    +    cluster: RelOptCluster,
    +    traitSet: RelTraitSet,
    +    leftNode: RelNode,
    +    rightNode: RelNode,
    +    joinCondition: RexNode,
    +    joinInfo: JoinInfo,
    +    joinType: JoinRelType,
    +    leftSchema: RowSchema,
    +    rightSchema: RowSchema,
    +    schema: RowSchema,
    +    ruleDescription: String)
    +  extends BiRel(cluster, traitSet, leftNode, rightNode)
    +          with CommonJoin
    +          with DataStreamRel {
    +
    +  override def deriveRowType(): RelDataType = schema.relDataType
    +
    +  override def needsUpdatesAsRetraction: Boolean = true
    +
    +  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
    +    new DataStreamJoin(
    +      cluster,
    +      traitSet,
    +      inputs.get(0),
    +      inputs.get(1),
    +      joinCondition,
    +      joinInfo,
    +      joinType,
    +      leftSchema,
    +      rightSchema,
    +      schema,
    +      ruleDescription)
    +  }
    +
    +  def getJoinInfo: JoinInfo = joinInfo
    +
    +  def getJoinType: JoinRelType = joinType
    +
    +  override def toString: String = {
    +    joinToString(
    +      schema.relDataType,
    +      joinCondition,
    +      joinType,
    +      getExpressionString)
    +  }
    +
    +  override def explainTerms(pw: RelWriter): RelWriter = {
    +    joinExplainTerms(
    +      super.explainTerms(pw),
    +      schema.relDataType,
    +      joinCondition,
    +      joinType,
    +      getExpressionString)
    +  }
    +
    +  override def translateToPlan(
    +      tableEnv: StreamTableEnvironment,
    +      queryConfig: StreamQueryConfig): DataStream[CRow] = {
    +
    +    val config = tableEnv.getConfig
    +    val returnType = schema.typeInfo
    +    val keyPairs = joinInfo.pairs().toList
    +
    +    // get the equality keys
    +    val leftKeys = ArrayBuffer.empty[Int]
    +    val rightKeys = ArrayBuffer.empty[Int]
    +    if (keyPairs.isEmpty) {
    +      // if no equality keys => not supported
    +      throw TableException(
    +        "Joins should have at least one equality condition.\n" +
    +          s"\tLeft: ${left.toString},\n" +
    +          s"\tRight: ${right.toString},\n" +
    +          s"\tCondition: (${joinConditionToString(schema.relDataType,
    +             joinCondition, getExpressionString)})"
    +      )
    +    }
    +    else {
    +      // at least one equality expression
    +      val leftFields = left.getRowType.getFieldList
    +      val rightFields = right.getRowType.getFieldList
    +
    +      keyPairs.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 {
    +          throw TableException(
    +            "Equality join predicate on incompatible types.\n" +
    +              s"\tLeft: ${left.toString},\n" +
    +              s"\tRight: ${right.toString},\n" +
    +              s"\tCondition: (${joinConditionToString(schema.relDataType,
    +                joinCondition, getExpressionString)})"
    +          )
    +        }
    +      })
    +    }
    +
    +    val leftDataStream =
    +      left.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
    +    val rightDataStream =
    +      right.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
    +
    +    val (connectOperator, nullCheck) = joinType match {
    +      case JoinRelType.INNER => (leftDataStream.connect(rightDataStream), false)
    +      case _ => throw new UnsupportedOperationException(s"An Unsupported JoinType [ $joinType ]")
    +    }
    +
    +    if (nullCheck && !config.getNullCheck) {
    +      throw TableException("Null check in TableConfig must be enabled for outer joins.")
    +    }
    +
    +
    +    val generator = new FunctionCodeGenerator(
    +      config,
    +      nullCheck,
    +      leftSchema.typeInfo,
    +      Some(rightSchema.typeInfo))
    +    val conversion = generator.generateConverterResultExpression(
    +      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 joinOpName = joinToString(getRowType, joinCondition, joinType, getExpressionString)
    +
    +    val coMapFun =
    +      new DataStreamInnerJoin(
    +        leftSchema.typeInfo,
    +        rightSchema.typeInfo,
    +        CRowTypeInfo(returnType),
    +        genFunction.name,
    +        genFunction.code,
    +        queryConfig)
    +
    +    connectOperator
    +      .keyBy(leftKeys.toArray, rightKeys.toArray)
    +      .process(coMapFun).name(joinOpName)
    --- End diff --
    
    One line for each method invoking.


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r152775960
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/DataStreamInnerJoin.scala ---
    @@ -0,0 +1,285 @@
    +/*
    + * 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._
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.java.typeutils.TupleTypeInfo
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.runtime.CRowWrappingMultiOuputCollector
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
    +import org.slf4j.LoggerFactory
    +import org.apache.flink.table.codegen.Compiler
    +
    +
    +/**
    +  * Connect data for left stream and right stream. Only use for innerJoin.
    +  *
    +  * @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
    +  */
    +class DataStreamInnerJoin(
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    resultType: TypeInformation[CRow],
    +    genJoinFuncName: String,
    +    genJoinFuncCode: String,
    +    queryConfig: StreamQueryConfig)
    +  extends CoProcessFunction[CRow, CRow, CRow]
    +          with Compiler[FlatJoinFunction[Row, Row, Row]] {
    +
    +  // state to hold left stream element
    +  private var leftState: MapState[Row, JTuple2[Int, Long]] = _
    +  // state to hold right stream element
    +  private var rightState: MapState[Row, JTuple2[Int, Long]] = _
    +  private var cRowWrapper: CRowWrappingMultiOuputCollector = _
    +
    +  private val minRetentionTime: Long = queryConfig.getMinIdleStateRetentionTime
    +  private val maxRetentionTime: Long = queryConfig.getMaxIdleStateRetentionTime
    +  private val stateCleaningEnabled: Boolean = minRetentionTime > 1
    +
    +  // state to record last timer of left stream, 0 means no timer
    +  private var leftTimer: ValueState[Long] = _
    +  // state to record last timer of right stream, 0 means no timer
    +  private var rightTimer: ValueState[Long] = _
    +
    +  // other condition function
    +  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
    +
    +  val LOG = LoggerFactory.getLogger(this.getClass)
    +
    +  override def open(parameters: Configuration): Unit = {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +                s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    joinFunction = clazz.newInstance()
    +
    +    // initialize left and right state, the first element of tuple2 indicates how many rows of
    +    // this row, while the second element represents the expired time of this row.
    +    val tupleTypeInfo = new TupleTypeInfo[JTuple2[Int, Long]](Types.INT, Types.LONG)
    +    val leftStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "left", leftType, tupleTypeInfo)
    +    val rightStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "right", rightType, tupleTypeInfo)
    +    leftState = getRuntimeContext.getMapState(leftStateDescriptor)
    +    rightState = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // initialize timer state
    +    val valueStateDescriptor1 = new ValueStateDescriptor[Long]("timervaluestate1", classOf[Long])
    +    leftTimer = getRuntimeContext.getState(valueStateDescriptor1)
    +    val valueStateDescriptor2 = new ValueStateDescriptor[Long]("timervaluestate2", classOf[Long])
    +    rightTimer = getRuntimeContext.getState(valueStateDescriptor2)
    +
    +    cRowWrapper = new CRowWrappingMultiOuputCollector()
    +  }
    +
    +  /**
    +    * Process left stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx    The ctx to register timer or get current time
    +    * @param out    The collector for returning result values.
    +    *
    +    */
    +  override def processElement1(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, leftTimer, leftState, rightState, true)
    +  }
    +
    +  /**
    +    * Process right stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx    The ctx to register timer or get current time
    +    * @param out    The collector for returning result values.
    +    *
    +    */
    +  override def processElement2(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, rightTimer, rightState, leftState, false)
    +  }
    +
    +
    +  /**
    +    * Called when a processing timer trigger.
    +    * Expire left/right records which are expired in left and right state.
    +    *
    +    * @param timestamp The timestamp of the firing timer.
    +    * @param ctx       The ctx to register timer or get current time
    +    * @param out       The collector for returning result values.
    +    */
    +  override def onTimer(
    +      timestamp: Long,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +
    +    if (stateCleaningEnabled && leftTimer.value == timestamp) {
    +      expireOutTimeRow(
    +        timestamp,
    +        leftState,
    +        leftTimer,
    +        ctx
    +      )
    +    }
    +
    +    if (stateCleaningEnabled && rightTimer.value == timestamp) {
    +      expireOutTimeRow(
    +        timestamp,
    +        rightState,
    +        rightTimer,
    +        ctx
    +      )
    +    }
    +  }
    +
    +
    +  def getNewExpiredTime(
    +      curProcessTime: Long,
    +      oldExpiredTime: Long): Long = {
    +
    +    if (stateCleaningEnabled && curProcessTime + minRetentionTime > oldExpiredTime) {
    +      curProcessTime + maxRetentionTime
    +    } else {
    +      oldExpiredTime
    +    }
    +  }
    +
    +  /**
    +    * Puts or Retract an element from the input stream into state and search the other state to
    +    * output records meet the condition. Records will be expired in state if state retention time
    +    * has been specified.
    +    */
    +  def processElement(
    +      value: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow],
    +      timerState: ValueState[Long],
    +      currentSideState: MapState[Row, JTuple2[Int, Long]],
    +      otherSideState: MapState[Row, JTuple2[Int, Long]],
    +      isLeft: Boolean): Unit = {
    +
    +    cRowWrapper.setCollector(out)
    +    cRowWrapper.setChange(value.change)
    +
    +    val curProcessTime = ctx.timerService.currentProcessingTime
    +    val oldCntAndExpiredTime = currentSideState.get(value.row)
    +    val cntAndExpiredTime = if (null == oldCntAndExpiredTime) {
    +      JTuple2.of(0, -1L)
    +    } else {
    +      oldCntAndExpiredTime
    +    }
    +
    +    cntAndExpiredTime.f1 = getNewExpiredTime(curProcessTime, cntAndExpiredTime.f1)
    +    if (stateCleaningEnabled && timerState.value() == 0) {
    +      timerState.update(cntAndExpiredTime.f1)
    +      ctx.timerService().registerProcessingTimeTimer(cntAndExpiredTime.f1)
    +    }
    +
    +    // update current side stream state
    +    if (!value.change) {
    +      cntAndExpiredTime.f0 = cntAndExpiredTime.f0 - 1
    +      if (cntAndExpiredTime.f0 <= 0) {
    +        currentSideState.remove(value.row)
    +      } else {
    +        currentSideState.put(value.row, cntAndExpiredTime)
    +      }
    +    } else {
    +      cntAndExpiredTime.f0 = cntAndExpiredTime.f0 + 1
    +      currentSideState.put(value.row, cntAndExpiredTime)
    +    }
    +
    +    val otherSideRowsIterator = otherSideState.keys().iterator()
    +    // join other side data
    +    while (otherSideRowsIterator.hasNext) {
    +      val otherSideRow = otherSideRowsIterator.next()
    +      val cntAndExpiredTime = otherSideState.get(otherSideRow)
    --- End diff --
    
    Wouldn't it make sense to iterate over the entries instead of keys if we retrieve every value anyway?


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132422891
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala ---
    @@ -232,4 +238,111 @@ class JoinHarnessTest extends HarnessTestBase{
         testHarness.close()
       }
     
    +  @Test
    +  def testProcTimeNonWindowInnerJoin() {
    --- End diff --
    
    Add a test that includes retraction messages


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r146105057
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowWrappingMultiOuputCollector.scala ---
    @@ -0,0 +1,48 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.table.runtime
    +
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +
    +/**
    +  * The collector is used to wrap a [[Row]] to a [[CRow]]. Otherwise, set times to output repeatedly
    --- End diff --
    
    The collector to warp a [[Row]] into a [[CRow]] and collect it multiple times.


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r152777181
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/DataStreamInnerJoin.scala ---
    @@ -0,0 +1,285 @@
    +/*
    + * 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._
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.java.typeutils.TupleTypeInfo
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.runtime.CRowWrappingMultiOuputCollector
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
    +import org.slf4j.LoggerFactory
    +import org.apache.flink.table.codegen.Compiler
    +
    +
    +/**
    +  * Connect data for left stream and right stream. Only use for innerJoin.
    +  *
    +  * @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
    +  */
    +class DataStreamInnerJoin(
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    resultType: TypeInformation[CRow],
    +    genJoinFuncName: String,
    +    genJoinFuncCode: String,
    +    queryConfig: StreamQueryConfig)
    +  extends CoProcessFunction[CRow, CRow, CRow]
    +          with Compiler[FlatJoinFunction[Row, Row, Row]] {
    +
    +  // state to hold left stream element
    +  private var leftState: MapState[Row, JTuple2[Int, Long]] = _
    +  // state to hold right stream element
    +  private var rightState: MapState[Row, JTuple2[Int, Long]] = _
    +  private var cRowWrapper: CRowWrappingMultiOuputCollector = _
    +
    +  private val minRetentionTime: Long = queryConfig.getMinIdleStateRetentionTime
    +  private val maxRetentionTime: Long = queryConfig.getMaxIdleStateRetentionTime
    +  private val stateCleaningEnabled: Boolean = minRetentionTime > 1
    +
    +  // state to record last timer of left stream, 0 means no timer
    +  private var leftTimer: ValueState[Long] = _
    +  // state to record last timer of right stream, 0 means no timer
    +  private var rightTimer: ValueState[Long] = _
    +
    +  // other condition function
    +  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
    +
    +  val LOG = LoggerFactory.getLogger(this.getClass)
    +
    +  override def open(parameters: Configuration): Unit = {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +                s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    joinFunction = clazz.newInstance()
    +
    +    // initialize left and right state, the first element of tuple2 indicates how many rows of
    +    // this row, while the second element represents the expired time of this row.
    +    val tupleTypeInfo = new TupleTypeInfo[JTuple2[Int, Long]](Types.INT, Types.LONG)
    +    val leftStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "left", leftType, tupleTypeInfo)
    +    val rightStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "right", rightType, tupleTypeInfo)
    +    leftState = getRuntimeContext.getMapState(leftStateDescriptor)
    +    rightState = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // initialize timer state
    +    val valueStateDescriptor1 = new ValueStateDescriptor[Long]("timervaluestate1", classOf[Long])
    +    leftTimer = getRuntimeContext.getState(valueStateDescriptor1)
    +    val valueStateDescriptor2 = new ValueStateDescriptor[Long]("timervaluestate2", classOf[Long])
    +    rightTimer = getRuntimeContext.getState(valueStateDescriptor2)
    +
    +    cRowWrapper = new CRowWrappingMultiOuputCollector()
    +  }
    +
    +  /**
    +    * Process left stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx    The ctx to register timer or get current time
    +    * @param out    The collector for returning result values.
    +    *
    +    */
    +  override def processElement1(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, leftTimer, leftState, rightState, true)
    +  }
    +
    +  /**
    +    * Process right stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx    The ctx to register timer or get current time
    +    * @param out    The collector for returning result values.
    +    *
    +    */
    +  override def processElement2(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, rightTimer, rightState, leftState, false)
    +  }
    +
    +
    +  /**
    +    * Called when a processing timer trigger.
    +    * Expire left/right records which are expired in left and right state.
    +    *
    +    * @param timestamp The timestamp of the firing timer.
    +    * @param ctx       The ctx to register timer or get current time
    +    * @param out       The collector for returning result values.
    +    */
    +  override def onTimer(
    +      timestamp: Long,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +
    +    if (stateCleaningEnabled && leftTimer.value == timestamp) {
    +      expireOutTimeRow(
    +        timestamp,
    +        leftState,
    +        leftTimer,
    +        ctx
    +      )
    +    }
    +
    +    if (stateCleaningEnabled && rightTimer.value == timestamp) {
    +      expireOutTimeRow(
    +        timestamp,
    +        rightState,
    +        rightTimer,
    +        ctx
    +      )
    +    }
    +  }
    +
    +
    +  def getNewExpiredTime(
    +      curProcessTime: Long,
    +      oldExpiredTime: Long): Long = {
    +
    +    if (stateCleaningEnabled && curProcessTime + minRetentionTime > oldExpiredTime) {
    +      curProcessTime + maxRetentionTime
    +    } else {
    +      oldExpiredTime
    +    }
    +  }
    +
    +  /**
    +    * Puts or Retract an element from the input stream into state and search the other state to
    +    * output records meet the condition. Records will be expired in state if state retention time
    +    * has been specified.
    +    */
    +  def processElement(
    +      value: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow],
    +      timerState: ValueState[Long],
    +      currentSideState: MapState[Row, JTuple2[Int, Long]],
    +      otherSideState: MapState[Row, JTuple2[Int, Long]],
    +      isLeft: Boolean): Unit = {
    +
    +    cRowWrapper.setCollector(out)
    +    cRowWrapper.setChange(value.change)
    +
    +    val curProcessTime = ctx.timerService.currentProcessingTime
    +    val oldCntAndExpiredTime = currentSideState.get(value.row)
    +    val cntAndExpiredTime = if (null == oldCntAndExpiredTime) {
    +      JTuple2.of(0, -1L)
    +    } else {
    +      oldCntAndExpiredTime
    +    }
    +
    +    cntAndExpiredTime.f1 = getNewExpiredTime(curProcessTime, cntAndExpiredTime.f1)
    +    if (stateCleaningEnabled && timerState.value() == 0) {
    +      timerState.update(cntAndExpiredTime.f1)
    +      ctx.timerService().registerProcessingTimeTimer(cntAndExpiredTime.f1)
    +    }
    +
    +    // update current side stream state
    +    if (!value.change) {
    +      cntAndExpiredTime.f0 = cntAndExpiredTime.f0 - 1
    +      if (cntAndExpiredTime.f0 <= 0) {
    +        currentSideState.remove(value.row)
    +      } else {
    +        currentSideState.put(value.row, cntAndExpiredTime)
    +      }
    +    } else {
    +      cntAndExpiredTime.f0 = cntAndExpiredTime.f0 + 1
    +      currentSideState.put(value.row, cntAndExpiredTime)
    +    }
    +
    +    val otherSideRowsIterator = otherSideState.keys().iterator()
    +    // join other side data
    +    while (otherSideRowsIterator.hasNext) {
    +      val otherSideRow = otherSideRowsIterator.next()
    +      val cntAndExpiredTime = otherSideState.get(otherSideRow)
    +      // join
    +      cRowWrapper.setTimes(cntAndExpiredTime.f0)
    +      if (isLeft) {
    +        joinFunction.join(value.row, otherSideRow, cRowWrapper)
    --- End diff --
    
    Some micro-optimization: store `value.row` in a variable at the beginning of the method to reduce field accesses.


---

[GitHub] flink issue #4471: [FLINK-6094] [table] Implement stream-stream proctime non...

Posted by hequn8128 <gi...@git.apache.org>.
Github user hequn8128 commented on the issue:

    https://github.com/apache/flink/pull/4471
  
    hi @twalthr  The pr has been submitted. https://issues.apache.org/jira/browse/FLINK-8400
    Thanks, Hequn.


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by hequn8128 <gi...@git.apache.org>.
Github user hequn8128 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r158489025
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/DataStreamInnerJoin.scala ---
    @@ -0,0 +1,285 @@
    +/*
    + * 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._
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.java.typeutils.TupleTypeInfo
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.runtime.CRowWrappingMultiOuputCollector
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
    +import org.slf4j.LoggerFactory
    +import org.apache.flink.table.codegen.Compiler
    +
    +
    +/**
    +  * Connect data for left stream and right stream. Only use for innerJoin.
    +  *
    +  * @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
    +  */
    +class DataStreamInnerJoin(
    --- End diff --
    
    Maybe `NonWindowInnerJoin` is better. It is consistent with `non-window aggregate`. What do you think ?


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by hequn8128 <gi...@git.apache.org>.
Github user hequn8128 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r148947767
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/DataStreamInnerJoin.scala ---
    @@ -0,0 +1,287 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +
    +package org.apache.flink.table.runtime.join
    +
    +import org.apache.flink.api.common.functions.FlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.java.typeutils.TupleTypeInfo
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.runtime.CRowWrappingMultiOuputCollector
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
    +import org.slf4j.LoggerFactory
    +import org.apache.flink.table.codegen.Compiler
    +
    +
    +/**
    +  * Connect data for left stream and right stream. Only use for innerJoin.
    +  *
    +  * @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
    +  */
    +class DataStreamInnerJoin(
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    resultType: TypeInformation[CRow],
    +    genJoinFuncName: String,
    +    genJoinFuncCode: String,
    +    queryConfig: StreamQueryConfig)
    +  extends CoProcessFunction[CRow, CRow, CRow]
    +          with Compiler[FlatJoinFunction[Row, Row, Row]] {
    +
    +  // state to hold left stream element
    +  private var leftState: MapState[Row, JTuple2[Int, Long]] = _
    +  // state to hold right stream element
    +  private var rightState: MapState[Row, JTuple2[Int, Long]] = _
    +  private var cRowWrapper: CRowWrappingMultiOuputCollector = _
    +
    +  private val minRetentionTime: Long = queryConfig.getMinIdleStateRetentionTime
    +  private val maxRetentionTime: Long = queryConfig.getMaxIdleStateRetentionTime
    +  private val stateCleaningEnabled: Boolean = minRetentionTime > 1
    +
    +  // state to record last timer of left stream, 0 means no timer
    +  private var leftTimer: ValueState[Long] = _
    +  // state to record last timer of right stream, 0 means no timer
    +  private var rightTimer: ValueState[Long] = _
    +
    +  // other condition function
    +  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
    +
    +  val LOG = LoggerFactory.getLogger(this.getClass)
    +
    +  override def open(parameters: Configuration): Unit = {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +                s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    joinFunction = clazz.newInstance()
    +
    +    // initialize left and right state, the first element of tuple2 indicates how many rows of
    +    // this row, while the second element represents the expired time of this row.
    +    val tupleTypeInfo = new TupleTypeInfo[JTuple2[Int, Long]](Types.INT, Types.LONG)
    +    val leftStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "left", leftType, tupleTypeInfo)
    +    val rightStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "right", rightType, tupleTypeInfo)
    +    leftState = getRuntimeContext.getMapState(leftStateDescriptor)
    +    rightState = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // initialize timer state
    +    val valueStateDescriptor1 = new ValueStateDescriptor[Long]("timervaluestate1", classOf[Long])
    +    leftTimer = getRuntimeContext.getState(valueStateDescriptor1)
    +    val valueStateDescriptor2 = new ValueStateDescriptor[Long]("timervaluestate2", classOf[Long])
    +    rightTimer = getRuntimeContext.getState(valueStateDescriptor2)
    +
    +    cRowWrapper = new CRowWrappingMultiOuputCollector()
    +  }
    +
    +  /**
    +    * Process left stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx    The ctx to register timer or get current time
    +    * @param out    The collector for returning result values.
    +    *
    +    */
    +  override def processElement1(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, leftTimer, leftState, rightState, true)
    +  }
    +
    +  /**
    +    * Process right stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx    The ctx to register timer or get current time
    +    * @param out    The collector for returning result values.
    +    *
    +    */
    +  override def processElement2(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, rightTimer, rightState, leftState, false)
    +  }
    +
    +
    +  /**
    +    * Called when a processing timer trigger.
    +    * Expire left/right records which are expired in left and right state.
    +    *
    +    * @param timestamp The timestamp of the firing timer.
    +    * @param ctx       The ctx to register timer or get current time
    +    * @param out       The collector for returning result values.
    +    */
    +  override def onTimer(
    +      timestamp: Long,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +
    +    if (stateCleaningEnabled && leftTimer.value == timestamp) {
    --- End diff --
    
    yes~


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132213679
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamJoin.scala ---
    @@ -0,0 +1,212 @@
    +/*
    + * 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.`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.FlatJoinRunner
    +import org.apache.flink.table.runtime.join.ProcTimeNonWindowInnerJoin
    +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.
    +  */
    +class DataStreamJoin(
    +    cluster: RelOptCluster,
    +    traitSet: RelTraitSet,
    +    leftNode: RelNode,
    +    rightNode: RelNode,
    +    joinCondition: RexNode,
    +    joinInfo: JoinInfo,
    +    joinType: JoinRelType,
    +    leftSchema: RowSchema,
    +    rightSchema: RowSchema,
    +    schema: RowSchema,
    +    ruleDescription: String)
    +  extends BiRel(cluster, traitSet, leftNode, rightNode)
    +          with CommonJoin
    +          with DataStreamRel {
    +
    +  override def deriveRowType(): RelDataType = schema.logicalType
    +
    +  override def needsUpdatesAsRetraction: Boolean = true
    +
    +  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
    +    new DataStreamJoin(
    +      cluster,
    +      traitSet,
    +      inputs.get(0),
    +      inputs.get(1),
    +      joinCondition,
    +      joinInfo,
    +      joinType,
    +      leftSchema,
    +      rightSchema,
    +      schema,
    +      ruleDescription)
    +  }
    +
    +  def getJoinInfo: JoinInfo = joinInfo
    +
    +  override def toString: String = {
    +    joinToString(
    +      schema.logicalType,
    +      joinCondition,
    +      joinType,
    +      getExpressionString)
    +  }
    +
    +  override def explainTerms(pw: RelWriter): RelWriter = {
    +    joinExplainTerms(
    +      super.explainTerms(pw),
    +      schema.logicalType,
    +      joinCondition,
    +      joinType,
    +      getExpressionString)
    +  }
    +
    +  override def translateToPlan(
    +      tableEnv: StreamTableEnvironment,
    +      queryConfig: StreamQueryConfig): DataStream[CRow] = {
    +
    +    val config = tableEnv.getConfig
    +    val returnType = schema.physicalTypeInfo
    +    val keyPairs = joinInfo.pairs().toList
    +
    +    // get the equality keys
    +    val leftKeys = ArrayBuffer.empty[Int]
    +    val rightKeys = ArrayBuffer.empty[Int]
    +    if (keyPairs.isEmpty) {
    +      // if no equality keys => not supported
    +      throw TableException(
    +        "Joins should have at least one equality condition.\n" +
    +          s"\tLeft: ${left.toString},\n" +
    +          s"\tRight: ${right.toString},\n" +
    +          s"\tCondition: (${joinConditionToString(schema.logicalType,
    +             joinCondition, getExpressionString)})"
    +      )
    +    }
    +    else {
    +      // at least one equality expression
    +      val leftFields = left.getRowType.getFieldList
    +      val rightFields = right.getRowType.getFieldList
    +
    +      keyPairs.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 {
    +          throw TableException(
    +            "Equality join predicate on incompatible types.\n" +
    +              s"\tLeft: ${left.toString},\n" +
    +              s"\tRight: ${right.toString},\n" +
    +              s"\tCondition: (${joinConditionToString(schema.logicalType,
    +                joinCondition, getExpressionString)})"
    +          )
    +        }
    +      })
    +    }
    +
    +    val leftDataStream =
    +      left.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
    +    val rightDataStream =
    +      right.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
    +
    +    val (connectOperator, nullCheck) = joinType match {
    +      case JoinRelType.INNER => (leftDataStream.connect(rightDataStream), false)
    +      case _ => throw new UnsupportedOperationException(s"An Unsupported JoinType [ $joinType ]")
    +    }
    +
    +    if (nullCheck && !config.getNullCheck) {
    +      throw TableException("Null check in TableConfig must be enabled for outer joins.")
    +    }
    +
    +
    +    val generator = new FunctionCodeGenerator(
    +      config,
    +      nullCheck,
    +      leftSchema.physicalTypeInfo,
    +      Some(rightSchema.physicalTypeInfo))
    +    val conversion = generator.generateConverterResultExpression(
    +      schema.physicalTypeInfo,
    +      schema.physicalType.getFieldNames)
    +
    +
    +    var body = ""
    +
    +    if (joinInfo.isEqui) {
    +      // only equality condition
    +      body = s"""
    +                |${conversion.code}
    +                |${generator.collectorTerm}.collect(${conversion.resultTerm});
    +                |""".stripMargin
    +    } else {
    +      val condition = generator.generateExpression(joinCondition)
    +      body = 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 joinFun = new FlatJoinRunner[Row, Row, Row](
    --- End diff --
    
    Do we need to wrap the generated function in a `FlatJoinRunner`? 
    Can't we use the code-gen'd function directly in the `ProcTimeNonWindowInnerJoin`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132284969
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeNonWindowInnerJoin.scala ---
    @@ -0,0 +1,275 @@
    +/*
    + * 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.RichFlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.java.typeutils.{ResultTypeQueryable, TupleTypeInfo}
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.runtime.CRowWrappingMultiOuputCollector
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
    +
    +/**
    +  * Connect data for left stream and right stream. Only use for innerJoin.
    +  *
    +  * @param joiner           join function
    +  * @param leftType         the input type of left stream
    +  * @param rightType        the input type of right stream
    +  * @param resultType       the output type of join
    +  * @param queryConfig
    +  */
    +class ProcTimeNonWindowInnerJoin(
    +    joiner: RichFlatJoinFunction[Row, Row, Row],
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    resultType: TypeInformation[CRow],
    +    queryConfig: StreamQueryConfig) extends
    +  CoProcessFunction[CRow, CRow, CRow] with ResultTypeQueryable[CRow] {
    +
    +
    +  // state to hold left stream element
    +  private var leftState: MapState[Row, JTuple2[Int, Long]] = null
    +  // state to hold right stream element
    +  private var rightState: MapState[Row, JTuple2[Int, Long]] = null
    +  private var cRowWrapper: CRowWrappingMultiOuputCollector = null
    +
    +  private val minRetentionTime: Long = queryConfig.getMinIdleStateRetentionTime
    +  private val maxRetentionTime: Long = queryConfig.getMaxIdleStateRetentionTime
    +  private val stateCleaningEnabled: Boolean = minRetentionTime > 1
    +
    +  // state to record last timer of left stream, 0 means no timer
    +  private var timerState1: ValueState[Long] = _
    +  // state to record last timer of right stream, 0 means no timer
    +  private var timerState2: ValueState[Long] = _
    +
    +
    +  override def open(parameters: Configuration): Unit = {
    +    // initialize left and right state
    +    val tupleTypeInfo = new TupleTypeInfo[JTuple2[Int, Long]](Types.INT, Types.LONG)
    --- End diff --
    
    Add a comment about the structure of the state. What do we need the `Tuple2<Integer, Long>` for?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/flink/pull/4471


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132212670
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamJoin.scala ---
    @@ -0,0 +1,212 @@
    +/*
    + * 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.`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.FlatJoinRunner
    +import org.apache.flink.table.runtime.join.ProcTimeNonWindowInnerJoin
    +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.
    +  */
    +class DataStreamJoin(
    +    cluster: RelOptCluster,
    +    traitSet: RelTraitSet,
    +    leftNode: RelNode,
    +    rightNode: RelNode,
    +    joinCondition: RexNode,
    +    joinInfo: JoinInfo,
    +    joinType: JoinRelType,
    +    leftSchema: RowSchema,
    +    rightSchema: RowSchema,
    +    schema: RowSchema,
    +    ruleDescription: String)
    +  extends BiRel(cluster, traitSet, leftNode, rightNode)
    +          with CommonJoin
    +          with DataStreamRel {
    +
    +  override def deriveRowType(): RelDataType = schema.logicalType
    +
    +  override def needsUpdatesAsRetraction: Boolean = true
    +
    +  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
    +    new DataStreamJoin(
    +      cluster,
    +      traitSet,
    +      inputs.get(0),
    +      inputs.get(1),
    +      joinCondition,
    +      joinInfo,
    +      joinType,
    +      leftSchema,
    +      rightSchema,
    +      schema,
    +      ruleDescription)
    +  }
    +
    +  def getJoinInfo: JoinInfo = joinInfo
    +
    +  override def toString: String = {
    +    joinToString(
    +      schema.logicalType,
    +      joinCondition,
    +      joinType,
    +      getExpressionString)
    +  }
    +
    +  override def explainTerms(pw: RelWriter): RelWriter = {
    +    joinExplainTerms(
    +      super.explainTerms(pw),
    +      schema.logicalType,
    +      joinCondition,
    +      joinType,
    +      getExpressionString)
    +  }
    +
    +  override def translateToPlan(
    +      tableEnv: StreamTableEnvironment,
    +      queryConfig: StreamQueryConfig): DataStream[CRow] = {
    +
    +    val config = tableEnv.getConfig
    +    val returnType = schema.physicalTypeInfo
    +    val keyPairs = joinInfo.pairs().toList
    +
    +    // get the equality keys
    +    val leftKeys = ArrayBuffer.empty[Int]
    +    val rightKeys = ArrayBuffer.empty[Int]
    +    if (keyPairs.isEmpty) {
    +      // if no equality keys => not supported
    +      throw TableException(
    +        "Joins should have at least one equality condition.\n" +
    +          s"\tLeft: ${left.toString},\n" +
    +          s"\tRight: ${right.toString},\n" +
    +          s"\tCondition: (${joinConditionToString(schema.logicalType,
    +             joinCondition, getExpressionString)})"
    +      )
    +    }
    +    else {
    +      // at least one equality expression
    +      val leftFields = left.getRowType.getFieldList
    +      val rightFields = right.getRowType.getFieldList
    +
    +      keyPairs.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 {
    +          throw TableException(
    +            "Equality join predicate on incompatible types.\n" +
    +              s"\tLeft: ${left.toString},\n" +
    +              s"\tRight: ${right.toString},\n" +
    +              s"\tCondition: (${joinConditionToString(schema.logicalType,
    +                joinCondition, getExpressionString)})"
    +          )
    +        }
    +      })
    +    }
    +
    +    val leftDataStream =
    +      left.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
    +    val rightDataStream =
    +      right.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
    +
    +    val (connectOperator, nullCheck) = joinType match {
    +      case JoinRelType.INNER => (leftDataStream.connect(rightDataStream), false)
    +      case _ => throw new UnsupportedOperationException(s"An Unsupported JoinType [ $joinType ]")
    +    }
    +
    +    if (nullCheck && !config.getNullCheck) {
    +      throw TableException("Null check in TableConfig must be enabled for outer joins.")
    +    }
    +
    +
    +    val generator = new FunctionCodeGenerator(
    +      config,
    +      nullCheck,
    +      leftSchema.physicalTypeInfo,
    +      Some(rightSchema.physicalTypeInfo))
    +    val conversion = generator.generateConverterResultExpression(
    +      schema.physicalTypeInfo,
    +      schema.physicalType.getFieldNames)
    +
    +
    +    var body = ""
    +
    +    if (joinInfo.isEqui) {
    +      // only equality condition
    +      body = s"""
    +                |${conversion.code}
    +                |${generator.collectorTerm}.collect(${conversion.resultTerm});
    +                |""".stripMargin
    +    } else {
    +      val condition = generator.generateExpression(joinCondition)
    --- End diff --
    
    `joinCondition` still includes the equi-join predicates. These should be removed before the condition is code-gen'd:
    
    ```
    val nonEquiPredicates = joinInfo.getRemaining(this.cluster.getRexBuilder)
    val condition = generator.generateExpression(nonEquiPredicates)
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by hequn8128 <gi...@git.apache.org>.
Github user hequn8128 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r158489057
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamJoin.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.nodes.datastream
    +
    +import org.apache.calcite.plan._
    +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.join.DataStreamInnerJoin
    +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.
    +  */
    +class DataStreamJoin(
    +    cluster: RelOptCluster,
    +    traitSet: RelTraitSet,
    +    leftNode: RelNode,
    +    rightNode: RelNode,
    +    joinCondition: RexNode,
    +    joinInfo: JoinInfo,
    +    joinType: JoinRelType,
    +    leftSchema: RowSchema,
    +    rightSchema: RowSchema,
    +    schema: RowSchema,
    +    ruleDescription: String)
    +  extends BiRel(cluster, traitSet, leftNode, rightNode)
    +          with CommonJoin
    +          with DataStreamRel {
    +
    +  override def deriveRowType(): RelDataType = schema.relDataType
    +
    +  override def needsUpdatesAsRetraction: Boolean = true
    +
    +  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
    +    new DataStreamJoin(
    +      cluster,
    +      traitSet,
    +      inputs.get(0),
    +      inputs.get(1),
    +      joinCondition,
    +      joinInfo,
    +      joinType,
    +      leftSchema,
    +      rightSchema,
    +      schema,
    +      ruleDescription)
    +  }
    +
    +  def getJoinInfo: JoinInfo = joinInfo
    +
    +  def getJoinType: JoinRelType = joinType
    +
    +  override def toString: String = {
    +    joinToString(
    +      schema.relDataType,
    +      joinCondition,
    +      joinType,
    +      getExpressionString)
    +  }
    +
    +  override def explainTerms(pw: RelWriter): RelWriter = {
    +    joinExplainTerms(
    +      super.explainTerms(pw),
    +      schema.relDataType,
    +      joinCondition,
    +      joinType,
    +      getExpressionString)
    +  }
    +
    +  override def translateToPlan(
    +      tableEnv: StreamTableEnvironment,
    +      queryConfig: StreamQueryConfig): DataStream[CRow] = {
    +
    +    val config = tableEnv.getConfig
    +    val returnType = schema.typeInfo
    +    val keyPairs = joinInfo.pairs().toList
    +
    +    // get the equality keys
    +    val leftKeys = ArrayBuffer.empty[Int]
    +    val rightKeys = ArrayBuffer.empty[Int]
    +    if (keyPairs.isEmpty) {
    --- End diff --
    
    Hi, I have removed this check, because equal join check has been done in `FlinkLogicalJoinConverter`. Also i have added an equal join test case in `org.apache.flink.table.api.stream.table.validation.JoinValidationTest`  in case of later changes in `FlinkLogicalJoinConverter`. What do you think?


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132208317
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamJoin.scala ---
    @@ -0,0 +1,212 @@
    +/*
    + * 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.`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.FlatJoinRunner
    +import org.apache.flink.table.runtime.join.ProcTimeNonWindowInnerJoin
    +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.
    +  */
    +class DataStreamJoin(
    +    cluster: RelOptCluster,
    +    traitSet: RelTraitSet,
    +    leftNode: RelNode,
    +    rightNode: RelNode,
    +    joinCondition: RexNode,
    +    joinInfo: JoinInfo,
    +    joinType: JoinRelType,
    +    leftSchema: RowSchema,
    +    rightSchema: RowSchema,
    +    schema: RowSchema,
    +    ruleDescription: String)
    +  extends BiRel(cluster, traitSet, leftNode, rightNode)
    +          with CommonJoin
    +          with DataStreamRel {
    +
    +  override def deriveRowType(): RelDataType = schema.logicalType
    +
    +  override def needsUpdatesAsRetraction: Boolean = true
    +
    +  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
    +    new DataStreamJoin(
    +      cluster,
    +      traitSet,
    +      inputs.get(0),
    +      inputs.get(1),
    +      joinCondition,
    +      joinInfo,
    +      joinType,
    +      leftSchema,
    +      rightSchema,
    +      schema,
    +      ruleDescription)
    +  }
    +
    +  def getJoinInfo: JoinInfo = joinInfo
    +
    +  override def toString: String = {
    +    joinToString(
    +      schema.logicalType,
    +      joinCondition,
    +      joinType,
    +      getExpressionString)
    +  }
    +
    +  override def explainTerms(pw: RelWriter): RelWriter = {
    +    joinExplainTerms(
    +      super.explainTerms(pw),
    +      schema.logicalType,
    +      joinCondition,
    +      joinType,
    +      getExpressionString)
    +  }
    +
    +  override def translateToPlan(
    +      tableEnv: StreamTableEnvironment,
    +      queryConfig: StreamQueryConfig): DataStream[CRow] = {
    +
    +    val config = tableEnv.getConfig
    +    val returnType = schema.physicalTypeInfo
    +    val keyPairs = joinInfo.pairs().toList
    +
    +    // get the equality keys
    +    val leftKeys = ArrayBuffer.empty[Int]
    +    val rightKeys = ArrayBuffer.empty[Int]
    +    if (keyPairs.isEmpty) {
    +      // if no equality keys => not supported
    +      throw TableException(
    +        "Joins should have at least one equality condition.\n" +
    +          s"\tLeft: ${left.toString},\n" +
    +          s"\tRight: ${right.toString},\n" +
    +          s"\tCondition: (${joinConditionToString(schema.logicalType,
    +             joinCondition, getExpressionString)})"
    +      )
    +    }
    +    else {
    +      // at least one equality expression
    +      val leftFields = left.getRowType.getFieldList
    +      val rightFields = right.getRowType.getFieldList
    +
    +      keyPairs.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 {
    +          throw TableException(
    +            "Equality join predicate on incompatible types.\n" +
    +              s"\tLeft: ${left.toString},\n" +
    +              s"\tRight: ${right.toString},\n" +
    +              s"\tCondition: (${joinConditionToString(schema.logicalType,
    +                joinCondition, getExpressionString)})"
    +          )
    +        }
    +      })
    +    }
    +
    +    val leftDataStream =
    +      left.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
    +    val rightDataStream =
    +      right.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
    +
    +    val (connectOperator, nullCheck) = joinType match {
    +      case JoinRelType.INNER => (leftDataStream.connect(rightDataStream), false)
    +      case _ => throw new UnsupportedOperationException(s"An Unsupported JoinType [ $joinType ]")
    +    }
    +
    +    if (nullCheck && !config.getNullCheck) {
    +      throw TableException("Null check in TableConfig must be enabled for outer joins.")
    +    }
    +
    +
    +    val generator = new FunctionCodeGenerator(
    +      config,
    +      nullCheck,
    +      leftSchema.physicalTypeInfo,
    +      Some(rightSchema.physicalTypeInfo))
    +    val conversion = generator.generateConverterResultExpression(
    +      schema.physicalTypeInfo,
    +      schema.physicalType.getFieldNames)
    +
    +
    +    var body = ""
    +
    +    if (joinInfo.isEqui) {
    +      // only equality condition
    +      body = s"""
    +                |${conversion.code}
    +                |${generator.collectorTerm}.collect(${conversion.resultTerm});
    +                |""".stripMargin
    +    } else {
    +      val condition = generator.generateExpression(joinCondition)
    +      body = 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 joinFun = new FlatJoinRunner[Row, Row, Row](
    +      genFunction.name,
    +      genFunction.code,
    +      genFunction.returnType)
    +    val joinOpName = joinToString(getRowType, joinCondition, joinType, getExpressionString)
    +
    +    val coMapFun =
    +      new ProcTimeNonWindowInnerJoin(
    +        joinFun,
    +        leftSchema.physicalTypeInfo,
    +        rightSchema.physicalTypeInfo,
    +        CRowTypeInfo(returnType),
    +        queryConfig)
    +
    +    connectOperator
    +      .keyBy(leftKeys.toArray, rightKeys.toArray)
    +      .process(coMapFun).name(joinOpName).asInstanceOf[DataStream[CRow]]
    --- End diff --
    
    add `.returns(CRowTypeInfo(returnType))`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132228222
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/UpdatingPlanChecker.scala ---
    @@ -90,40 +96,86 @@ object UpdatingPlanChecker {
                   // resolve names of input fields
                   .map(io => (inNames.get(io._1), io._2))
     
    -            // filter by input keys
    -            val outKeys = inOutNames.filter(io => keys.get.contains(io._1)).map(_._2)
    -            // check if all keys have been preserved
    -            if (outKeys.nonEmpty && outKeys.length == keys.get.length) {
    +            // filter by input keyAncestors
    +            val outKeyAncesters = inOutNames
    +              .filter(io => keyAncestors.get.map(e => e._1).contains(io._1))
    +              .map(io => (io._2, keyAncestors.get.find(ka => ka._1 == io._1).get._2))
    +
    +            // check if all keyAncestors have been preserved
    +            if (outKeyAncesters.nonEmpty &&
    +              outKeyAncesters.map(ka => ka._2).distinct.length ==
    +                keyAncestors.get.map(ka => ka._2).distinct.length) {
                   // all key have been preserved (but possibly renamed)
    -              keys = Some(outKeys.toArray)
    +              Some(outKeyAncesters.toList)
                 } else {
                   // some (or all) keys have been removed. Keys are no longer unique and removed
    -              keys = None
    +              None
                 }
    +          } else {
    +            None
               }
    +
             case _: DataStreamOverAggregate =>
    -          super.visit(node, ordinal, parent)
    -        // keys are always forwarded by Over aggregate
    +          // keyAncestors are always forwarded by Over aggregate
    +          visit(node.getInput(0))
             case a: DataStreamGroupAggregate =>
    -          // get grouping keys
    +          // get grouping keyAncestors
               val groupKeys = a.getRowType.getFieldNames.asScala.take(a.getGroupings.length)
    -          keys = Some(groupKeys.toArray)
    +          Some(groupKeys.map(e => (e, e)).toList)
             case w: DataStreamGroupWindowAggregate =>
    -          // get grouping keys
    +          // get grouping keyAncestors
               val groupKeys =
                 w.getRowType.getFieldNames.asScala.take(w.getGroupings.length).toArray
               // get window start and end time
               val windowStartEnd = w.getWindowProperties.map(_.name)
               // we have only a unique key if at least one window property is selected
               if (windowStartEnd.nonEmpty) {
    -            keys = Some(groupKeys ++ windowStartEnd)
    +            Some((groupKeys ++ windowStartEnd).map(e => (e, e)).toList)
    +          } else {
    +            None
    +          }
    +
    +        case j: DataStreamJoin =>
    +          val leftKeyAncestors = visit(j.getLeft)
    +          val rightKeyAncestors = visit(j.getRight)
    +          if (!leftKeyAncestors.isDefined || !rightKeyAncestors.isDefined) {
    --- End diff --
    
    `!x.isDefined` -> `x.isEmpty`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132243950
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/UpdatingPlanChecker.scala ---
    @@ -90,40 +96,86 @@ object UpdatingPlanChecker {
                   // resolve names of input fields
                   .map(io => (inNames.get(io._1), io._2))
     
    -            // filter by input keys
    -            val outKeys = inOutNames.filter(io => keys.get.contains(io._1)).map(_._2)
    -            // check if all keys have been preserved
    -            if (outKeys.nonEmpty && outKeys.length == keys.get.length) {
    +            // filter by input keyAncestors
    +            val outKeyAncesters = inOutNames
    +              .filter(io => keyAncestors.get.map(e => e._1).contains(io._1))
    +              .map(io => (io._2, keyAncestors.get.find(ka => ka._1 == io._1).get._2))
    +
    +            // check if all keyAncestors have been preserved
    +            if (outKeyAncesters.nonEmpty &&
    +              outKeyAncesters.map(ka => ka._2).distinct.length ==
    +                keyAncestors.get.map(ka => ka._2).distinct.length) {
                   // all key have been preserved (but possibly renamed)
    -              keys = Some(outKeys.toArray)
    +              Some(outKeyAncesters.toList)
                 } else {
                   // some (or all) keys have been removed. Keys are no longer unique and removed
    -              keys = None
    +              None
                 }
    +          } else {
    +            None
               }
    +
             case _: DataStreamOverAggregate =>
    -          super.visit(node, ordinal, parent)
    -        // keys are always forwarded by Over aggregate
    +          // keyAncestors are always forwarded by Over aggregate
    +          visit(node.getInput(0))
             case a: DataStreamGroupAggregate =>
    -          // get grouping keys
    +          // get grouping keyAncestors
               val groupKeys = a.getRowType.getFieldNames.asScala.take(a.getGroupings.length)
    -          keys = Some(groupKeys.toArray)
    +          Some(groupKeys.map(e => (e, e)).toList)
             case w: DataStreamGroupWindowAggregate =>
    -          // get grouping keys
    +          // get grouping keyAncestors
               val groupKeys =
                 w.getRowType.getFieldNames.asScala.take(w.getGroupings.length).toArray
               // get window start and end time
               val windowStartEnd = w.getWindowProperties.map(_.name)
               // we have only a unique key if at least one window property is selected
               if (windowStartEnd.nonEmpty) {
    -            keys = Some(groupKeys ++ windowStartEnd)
    +            Some((groupKeys ++ windowStartEnd).map(e => (e, e)).toList)
    +          } else {
    +            None
    +          }
    +
    +        case j: DataStreamJoin =>
    +          val leftKeyAncestors = visit(j.getLeft)
    +          val rightKeyAncestors = visit(j.getRight)
    +          if (!leftKeyAncestors.isDefined || !rightKeyAncestors.isDefined) {
    +            None
    +          } else {
    +            // both left and right contain keys
    +            val leftJoinKeys =
    +              j.getLeft.getRowType.getFieldNames.asScala.zipWithIndex
    +              .filter(e => j.getJoinInfo.leftKeys.contains(e._2))
    +              .map(e => e._1)
    +            val rightJoinKeys =
    +              j.getRight.getRowType.getFieldNames.asScala.zipWithIndex
    +                .filter(e => j.getJoinInfo.rightKeys.contains(e._2))
    +                .map(e => e._1)
    +
    +            val leftKeys = leftKeyAncestors.get.map(e => e._1)
    +            val rightKeys = rightKeyAncestors.get.map(e => e._1)
    +
    +            //1. join key = left key = right key
    +            if (leftJoinKeys == leftKeys && rightJoinKeys == rightKeys) {
    --- End diff --
    
    I think this condition is too strict. We have to check for containment not equality. 
    Equi join predicates on additional attributes are fine and also the assignment of keys to join predicates is not relevant (`l_key1 = r_key1 AND l_key2 = r_key2` is equivalent to `l_key1 = r_key2 AND l_key2 = r_key1`). 
    
    Moreover, we have to handle fields with common ancestors when checking for containment in the join attributes. If we have an input table that has two fields which resolve to the same key field, only on of the fields needs to be included in the join predicates.
    
    So we could do something like:
    
        // create maps to look up the ancestor of each key field
        val leftKeyAncestorMap: Map[String, String] = leftKeyAncestors.get.toMap
        val rightKeyAncestorMap: Map[String, String] = rightKeyAncestors.get.toMap
    
        // resolve fields to their ancestor if known
        val resolvedLeftJoinKeys = leftJoinKeys
          .map(k => leftKeyAncestorMap.getOrElse(k, k))
          .distinct
        val resolvedRightJoinKeys = rightJoinKeys
          .map(k => rightKeyAncestorMap.getOrElse(k, k))
          .distinct
    
        // resolve keys to their ancestor
        val resolvedLeftKeys = leftKeyAncestors.get.map(_._2).distinct
        val resolvedRightKeys = rightKeyAncestors.get.map(_._2).distinct
    
        // check that if all unique keys are included in the join fields 
        val joinsOnLeftKey = resolvedLeftKeys.forall(resolvedLeftJoinKeys.contains)
        val joinsOnRightKey = resolvedRightKeys.forall(resolvedRightJoinKeys.contains)
    
        if (joinsOnLeftKey && joinsOnRightKey) {
          // forward both keys
        } else if (joinsOnLeftKey) {
          // forward right keys
        } else if (joinsOnRightKey) {
          // forward left keys
        } else {
          // do not forward any keys
        }
    
    When forwarding keys, we have to make sure that all fields that are equal resolve to the same ancestor field. This means if we have a condition such as `WHERE l_key1 == r_key1 && l._key1 = r_nonKey1`, all three fields become keys and have the same ancestor. I propose the lexicographical smallest field name. So we would have in this case `[(l_key1, l_key1), (r_key1, l_key1), (r_nonKey1, l_key1)]`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132305360
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/UpdatingPlanCheckerTest.scala ---
    @@ -0,0 +1,203 @@
    +/*
    + * 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
    +
    +import org.apache.flink.table.api.Table
    +import org.apache.flink.table.plan.util.UpdatingPlanChecker
    +import org.apache.flink.table.utils.StreamTableTestUtil
    +import org.junit.Assert._
    +import org.apache.flink.table.api.scala._
    +import org.apache.flink.api.scala._
    +import org.junit.Test
    +
    +
    +class UpdatingPlanCheckerTest {
    +
    +  @Test
    +  def testSelect(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(String, Int)]('word, 'number)
    +    val resultTable = table.select('word, 'number)
    +
    +    util.verifyTableUniqueKey(resultTable, Nil)
    +  }
    +
    +  @Test
    +  def testGroupByWithoutKey(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(String, Int)]('word, 'number)
    +
    +    val resultTable = table
    +      .groupBy('word)
    +      .select('number.count)
    +
    +    util.verifyTableUniqueKey(resultTable, Nil)
    +  }
    +
    +  @Test
    +  def testGroupBy(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(String, Int)]('word, 'number)
    +
    +    val resultTable = table
    +      .groupBy('word)
    +      .select('word, 'number.count)
    +
    +    util.verifyTableUniqueKey(resultTable, Seq("word"))
    +  }
    +
    +  @Test
    +  def testGroupByWithDumplicateKey(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(String, Int)]('word, 'number)
    +
    +    val resultTable = table
    +      .groupBy('word)
    +      .select('word as 'word1, 'word as 'word2, 'number.count)
    +
    +    util.verifyTableUniqueKey(resultTable, Seq("word1", "word2"))
    +  }
    +
    +  //1. join key = left key = right key
    +  @Test
    +  def testJoinKeysEqualsleftAndRightKeys(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(Int, Int)]('pk, 'a)
    +
    +    val leftTableWithPk = table
    +      .groupBy('pk)
    +      .select('pk as 'leftpk, 'a.max as 'lefta)
    +
    +    val rightTableWithPk = table
    +      .groupBy('pk)
    +      .select('pk as 'rightpk, 'a.max as 'righta)
    +
    +    val resultTable = leftTableWithPk
    +      .join(rightTableWithPk)
    +      .where('leftpk === 'rightpk)
    +      .select('leftpk, 'lefta, 'righta)
    +
    +    util.verifyTableUniqueKey(resultTable, Seq("leftpk"))
    +  }
    +
    +  //2. join key = left key
    +  @Test
    +  def testJoinKeysEqualsLeftKeys(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(Int, Int)]('pk, 'a)
    +
    +    val leftTableWithPk = table
    +      .groupBy('pk)
    +      .select('pk as 'leftpk, 'a.max as 'lefta)
    +
    +    val rightTableWithPk = table
    +      .groupBy('pk)
    +      .select('pk as 'rightpk, 'a.max as 'righta)
    +
    +    val resultTable = leftTableWithPk
    +      .join(rightTableWithPk)
    +      .where('leftpk === 'righta)
    +      .select('rightpk, 'lefta, 'righta)
    +
    +    util.verifyTableUniqueKey(resultTable, Seq("rightpk"))
    +  }
    +
    +  //3. join key = right key
    +  @Test
    +  def testJoinKeysEqualsRightKeys(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(Int, Int)]('pk, 'a)
    +
    +    val leftTableWithPk = table
    +      .groupBy('pk)
    +      .select('pk as 'leftpk, 'a.max as 'lefta)
    +
    +    val rightTableWithPk = table
    +      .groupBy('pk)
    +      .select('pk as 'rightpk, 'a.max as 'righta)
    +
    +    val resultTable = leftTableWithPk
    +      .join(rightTableWithPk)
    +      .where('lefta === 'rightpk)
    +      .select('leftpk, 'lefta, 'righta)
    +
    +    util.verifyTableUniqueKey(resultTable, Seq("leftpk"))
    +  }
    +
    +  //4. join key not left or right key
    +  @Test
    +  def testJoinKeysWithoutLeftRightKeys(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(Int, Int)]('pk, 'a)
    +
    +    val leftTableWithPk = table
    +      .groupBy('pk)
    +      .select('pk as 'leftpk, 'a.max as 'lefta)
    +
    +    val rightTableWithPk = table
    +      .groupBy('pk)
    +      .select('pk as 'rightpk, 'a.max as 'righta)
    +
    +    val resultTable = leftTableWithPk
    +      .join(rightTableWithPk)
    +      .where('lefta === 'righta)
    +      .select('leftpk, 'rightpk, 'lefta, 'righta)
    +
    +    util.verifyTableUniqueKey(resultTable, Seq("leftpk", "rightpk"))
    +  }
    +
    +  @Test
    +  def testNonKeysJoin(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(Int, Int)]('a, 'b)
    +
    +    val leftTable = table
    +      .select('a as 'a, 'b as 'b)
    +
    +    val rightTable = table
    +      .select('a as 'aa, 'b as 'bb)
    +
    +    val resultTable = leftTable
    +      .join(rightTable)
    +      .where('a === 'aa)
    +      .select('a, 'aa, 'b, 'bb)
    +
    +    util.verifyTableUniqueKey(resultTable, Nil)
    +  }
    +}
    +
    +
    +class UpdatePlanChecnkerUtil extends StreamTableTestUtil {
    --- End diff --
    
    typo in name -> `UpdatePlanCheckerUtil`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132446011
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSinkITCase.scala ---
    @@ -506,4 +467,43 @@ object RowCollector {
         sink.clear()
         out
       }
    +
    +  /** Converts a list of retraction messages into a list of final results. */
    +  def restractResults(results: List[JTuple2[JBool, Row]]): List[String] = {
    --- End diff --
    
    please fix the typo in the method name -> `retractResults`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #4471: [FLINK-6094] [table] Implement stream-stream proctime non...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the issue:

    https://github.com/apache/flink/pull/4471
  
    Thanks for the update @hequn8128! 
    I'll be quite busy with testing and bug fixing for the upcoming 1.4 release in the next 2-3 weeks. I'll make another pass on this PR once the 1.4 work becomes less.
    
    Best, Fabian


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by hequn8128 <gi...@git.apache.org>.
Github user hequn8128 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r158489035
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/UpdatingPlanChecker.scala ---
    @@ -116,14 +135,100 @@ object UpdatingPlanChecker {
               val windowStartEnd = w.getWindowProperties.map(_.name)
               // we have only a unique key if at least one window property is selected
               if (windowStartEnd.nonEmpty) {
    -            keys = Some(groupKeys ++ windowStartEnd)
    +            val smallestAttribute = windowStartEnd.sorted.head
    --- End diff --
    
    Thanks, `windowStartEnd.min` is better, i want to get the lexicographic smallest attribute from `windowStartEnd` .  I have sorted it before get the first string.


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132305574
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/UpdatingPlanCheckerTest.scala ---
    @@ -0,0 +1,203 @@
    +/*
    + * 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
    +
    +import org.apache.flink.table.api.Table
    +import org.apache.flink.table.plan.util.UpdatingPlanChecker
    +import org.apache.flink.table.utils.StreamTableTestUtil
    +import org.junit.Assert._
    +import org.apache.flink.table.api.scala._
    +import org.apache.flink.api.scala._
    +import org.junit.Test
    +
    +
    +class UpdatingPlanCheckerTest {
    +
    +  @Test
    +  def testSelect(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(String, Int)]('word, 'number)
    +    val resultTable = table.select('word, 'number)
    +
    +    util.verifyTableUniqueKey(resultTable, Nil)
    +  }
    +
    +  @Test
    +  def testGroupByWithoutKey(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(String, Int)]('word, 'number)
    +
    +    val resultTable = table
    +      .groupBy('word)
    +      .select('number.count)
    +
    +    util.verifyTableUniqueKey(resultTable, Nil)
    +  }
    +
    +  @Test
    +  def testGroupBy(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(String, Int)]('word, 'number)
    +
    +    val resultTable = table
    +      .groupBy('word)
    +      .select('word, 'number.count)
    +
    +    util.verifyTableUniqueKey(resultTable, Seq("word"))
    +  }
    +
    +  @Test
    +  def testGroupByWithDumplicateKey(): Unit = {
    --- End diff --
    
    Typo -> `Dumplicate` -> `Duplicate`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r152742893
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamJoin.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.nodes.datastream
    +
    +import org.apache.calcite.plan._
    +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.join.DataStreamInnerJoin
    +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.
    +  */
    +class DataStreamJoin(
    +    cluster: RelOptCluster,
    +    traitSet: RelTraitSet,
    +    leftNode: RelNode,
    +    rightNode: RelNode,
    +    joinCondition: RexNode,
    +    joinInfo: JoinInfo,
    +    joinType: JoinRelType,
    +    leftSchema: RowSchema,
    +    rightSchema: RowSchema,
    +    schema: RowSchema,
    +    ruleDescription: String)
    +  extends BiRel(cluster, traitSet, leftNode, rightNode)
    +          with CommonJoin
    +          with DataStreamRel {
    +
    +  override def deriveRowType(): RelDataType = schema.relDataType
    +
    +  override def needsUpdatesAsRetraction: Boolean = true
    +
    +  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
    +    new DataStreamJoin(
    +      cluster,
    +      traitSet,
    +      inputs.get(0),
    +      inputs.get(1),
    +      joinCondition,
    +      joinInfo,
    +      joinType,
    +      leftSchema,
    +      rightSchema,
    +      schema,
    +      ruleDescription)
    +  }
    +
    +  def getJoinInfo: JoinInfo = joinInfo
    +
    +  def getJoinType: JoinRelType = joinType
    +
    +  override def toString: String = {
    +    joinToString(
    +      schema.relDataType,
    +      joinCondition,
    +      joinType,
    +      getExpressionString)
    +  }
    +
    +  override def explainTerms(pw: RelWriter): RelWriter = {
    +    joinExplainTerms(
    +      super.explainTerms(pw),
    +      schema.relDataType,
    +      joinCondition,
    +      joinType,
    +      getExpressionString)
    +  }
    +
    +  override def translateToPlan(
    +      tableEnv: StreamTableEnvironment,
    +      queryConfig: StreamQueryConfig): DataStream[CRow] = {
    +
    +    val config = tableEnv.getConfig
    +    val returnType = schema.typeInfo
    +    val keyPairs = joinInfo.pairs().toList
    +
    +    // get the equality keys
    +    val leftKeys = ArrayBuffer.empty[Int]
    +    val rightKeys = ArrayBuffer.empty[Int]
    +    if (keyPairs.isEmpty) {
    +      // if no equality keys => not supported
    +      throw TableException(
    +        "Joins should have at least one equality condition.\n" +
    +          s"\tLeft: ${left.toString},\n" +
    +          s"\tRight: ${right.toString},\n" +
    +          s"\tCondition: (${joinConditionToString(schema.relDataType,
    +             joinCondition, getExpressionString)})"
    +      )
    +    }
    +    else {
    +      // at least one equality expression
    +      val leftFields = left.getRowType.getFieldList
    +      val rightFields = right.getRowType.getFieldList
    +
    +      keyPairs.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 {
    +          throw TableException(
    +            "Equality join predicate on incompatible types.\n" +
    +              s"\tLeft: ${left.toString},\n" +
    +              s"\tRight: ${right.toString},\n" +
    +              s"\tCondition: (${joinConditionToString(schema.relDataType,
    +                joinCondition, getExpressionString)})"
    +          )
    +        }
    +      })
    +    }
    +
    +    val leftDataStream =
    +      left.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
    +    val rightDataStream =
    +      right.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
    +
    +    val (connectOperator, nullCheck) = joinType match {
    +      case JoinRelType.INNER => (leftDataStream.connect(rightDataStream), false)
    +      case _ => throw new UnsupportedOperationException(s"An Unsupported JoinType [ $joinType ]")
    --- End diff --
    
    throw a table exception here like "Unsupported join type xxx. Currently only inner joins with at least one equality predicate are supported."


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132285502
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeNonWindowInnerJoin.scala ---
    @@ -0,0 +1,275 @@
    +/*
    + * 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.RichFlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.java.typeutils.{ResultTypeQueryable, TupleTypeInfo}
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.runtime.CRowWrappingMultiOuputCollector
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
    +
    +/**
    +  * Connect data for left stream and right stream. Only use for innerJoin.
    +  *
    +  * @param joiner           join function
    +  * @param leftType         the input type of left stream
    +  * @param rightType        the input type of right stream
    +  * @param resultType       the output type of join
    +  * @param queryConfig
    +  */
    +class ProcTimeNonWindowInnerJoin(
    +    joiner: RichFlatJoinFunction[Row, Row, Row],
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    resultType: TypeInformation[CRow],
    +    queryConfig: StreamQueryConfig) extends
    +  CoProcessFunction[CRow, CRow, CRow] with ResultTypeQueryable[CRow] {
    +
    +
    +  // state to hold left stream element
    +  private var leftState: MapState[Row, JTuple2[Int, Long]] = null
    +  // state to hold right stream element
    +  private var rightState: MapState[Row, JTuple2[Int, Long]] = null
    +  private var cRowWrapper: CRowWrappingMultiOuputCollector = null
    +
    +  private val minRetentionTime: Long = queryConfig.getMinIdleStateRetentionTime
    +  private val maxRetentionTime: Long = queryConfig.getMaxIdleStateRetentionTime
    +  private val stateCleaningEnabled: Boolean = minRetentionTime > 1
    +
    +  // state to record last timer of left stream, 0 means no timer
    +  private var timerState1: ValueState[Long] = _
    --- End diff --
    
    `leftTimer` instead of `timerState1`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r152818820
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/JoinITCase.scala ---
    @@ -0,0 +1,262 @@
    +/*
    + * 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.table
    +
    +import org.apache.flink.api.scala._
    +import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
    +import org.apache.flink.table.api.{StreamQueryConfig, TableEnvironment}
    +import org.apache.flink.table.api.scala._
    +import org.apache.flink.table.runtime.utils.{StreamITCase, StreamingWithStateTestBase}
    +import org.junit.Assert._
    +import org.junit.Test
    +import org.apache.flink.api.common.time.Time
    +import org.apache.flink.streaming.api.TimeCharacteristic
    +import org.apache.flink.table.functions.aggfunctions.CountAggFunction
    +import org.apache.flink.table.runtime.stream.table.GroupWindowITCase.TimestampAndWatermarkWithOffset
    +import org.apache.flink.table.runtime.utils.JavaUserDefinedAggFunctions.{CountDistinct, WeightedAvg}
    +import org.apache.flink.types.Row
    +
    +import scala.collection.mutable
    +
    +class JoinITCase extends StreamingWithStateTestBase {
    +
    +  private val queryConfig = new StreamQueryConfig()
    +  queryConfig.withIdleStateRetentionTime(Time.hours(1), Time.hours(2))
    +
    +  @Test
    +  def testOutputWithPk(): Unit = {
    +    // data input
    +
    +    val data1 = List(
    +      (0, 0),
    --- End diff --
    
    Can you also test some null keys and fields in both inputs?


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132238828
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/UpdatingPlanChecker.scala ---
    @@ -90,40 +96,86 @@ object UpdatingPlanChecker {
                   // resolve names of input fields
                   .map(io => (inNames.get(io._1), io._2))
     
    -            // filter by input keys
    -            val outKeys = inOutNames.filter(io => keys.get.contains(io._1)).map(_._2)
    -            // check if all keys have been preserved
    -            if (outKeys.nonEmpty && outKeys.length == keys.get.length) {
    +            // filter by input keyAncestors
    +            val outKeyAncesters = inOutNames
    +              .filter(io => keyAncestors.get.map(e => e._1).contains(io._1))
    +              .map(io => (io._2, keyAncestors.get.find(ka => ka._1 == io._1).get._2))
    +
    +            // check if all keyAncestors have been preserved
    +            if (outKeyAncesters.nonEmpty &&
    +              outKeyAncesters.map(ka => ka._2).distinct.length ==
    +                keyAncestors.get.map(ka => ka._2).distinct.length) {
                   // all key have been preserved (but possibly renamed)
    -              keys = Some(outKeys.toArray)
    +              Some(outKeyAncesters.toList)
                 } else {
                   // some (or all) keys have been removed. Keys are no longer unique and removed
    -              keys = None
    +              None
                 }
    +          } else {
    +            None
               }
    +
             case _: DataStreamOverAggregate =>
    -          super.visit(node, ordinal, parent)
    -        // keys are always forwarded by Over aggregate
    +          // keyAncestors are always forwarded by Over aggregate
    +          visit(node.getInput(0))
             case a: DataStreamGroupAggregate =>
    -          // get grouping keys
    +          // get grouping keyAncestors
               val groupKeys = a.getRowType.getFieldNames.asScala.take(a.getGroupings.length)
    -          keys = Some(groupKeys.toArray)
    +          Some(groupKeys.map(e => (e, e)).toList)
             case w: DataStreamGroupWindowAggregate =>
    -          // get grouping keys
    +          // get grouping keyAncestors
               val groupKeys =
                 w.getRowType.getFieldNames.asScala.take(w.getGroupings.length).toArray
               // get window start and end time
               val windowStartEnd = w.getWindowProperties.map(_.name)
               // we have only a unique key if at least one window property is selected
               if (windowStartEnd.nonEmpty) {
    -            keys = Some(groupKeys ++ windowStartEnd)
    +            Some((groupKeys ++ windowStartEnd).map(e => (e, e)).toList)
    --- End diff --
    
    I think that window properties should be treated as keys derived from the window. If we have start and end one of both could be removed and we would still have a unique key.
    So I would add them as follows:
    
    ```
    Some((groupKeys.map(e => (e, e)) ++ windowStartEnd.map((_, w.getWindowAlias))).toList)
    ```
    
    (you have to add `def getWindowAlias: String = window.aliasAttribute.toString` to `DataStreamGroupWindowAggregate`).
    
    With this we treat the window (by its alias) as the ancestor of the key.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #4471: [FLINK-6094] [table] Implement stream-stream proctime non...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on the issue:

    https://github.com/apache/flink/pull/4471
  
    @hequn8128 that sounds great :) Looking forward to review the PRs. Can you open an issue for the bug so that we are all informed about it?


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132234963
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/UpdatingPlanChecker.scala ---
    @@ -90,40 +96,86 @@ object UpdatingPlanChecker {
                   // resolve names of input fields
                   .map(io => (inNames.get(io._1), io._2))
     
    -            // filter by input keys
    -            val outKeys = inOutNames.filter(io => keys.get.contains(io._1)).map(_._2)
    -            // check if all keys have been preserved
    -            if (outKeys.nonEmpty && outKeys.length == keys.get.length) {
    +            // filter by input keyAncestors
    +            val outKeyAncesters = inOutNames
    +              .filter(io => keyAncestors.get.map(e => e._1).contains(io._1))
    +              .map(io => (io._2, keyAncestors.get.find(ka => ka._1 == io._1).get._2))
    +
    +            // check if all keyAncestors have been preserved
    +            if (outKeyAncesters.nonEmpty &&
    +              outKeyAncesters.map(ka => ka._2).distinct.length ==
    --- End diff --
    
    `map(ka => ka._2)` -> `map(_._2)`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132302566
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeNonWindowInnerJoin.scala ---
    @@ -0,0 +1,275 @@
    +/*
    + * 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.RichFlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.java.typeutils.{ResultTypeQueryable, TupleTypeInfo}
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.runtime.CRowWrappingMultiOuputCollector
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
    +
    +/**
    +  * Connect data for left stream and right stream. Only use for innerJoin.
    +  *
    +  * @param joiner           join function
    +  * @param leftType         the input type of left stream
    +  * @param rightType        the input type of right stream
    +  * @param resultType       the output type of join
    +  * @param queryConfig
    +  */
    +class ProcTimeNonWindowInnerJoin(
    +    joiner: RichFlatJoinFunction[Row, Row, Row],
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    resultType: TypeInformation[CRow],
    +    queryConfig: StreamQueryConfig) extends
    +  CoProcessFunction[CRow, CRow, CRow] with ResultTypeQueryable[CRow] {
    +
    +
    +  // state to hold left stream element
    +  private var leftState: MapState[Row, JTuple2[Int, Long]] = null
    --- End diff --
    
    don't initialize with `null`. Use `_` instead


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132302597
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeNonWindowInnerJoin.scala ---
    @@ -0,0 +1,275 @@
    +/*
    + * 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.RichFlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.java.typeutils.{ResultTypeQueryable, TupleTypeInfo}
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.runtime.CRowWrappingMultiOuputCollector
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
    +
    +/**
    +  * Connect data for left stream and right stream. Only use for innerJoin.
    +  *
    +  * @param joiner           join function
    +  * @param leftType         the input type of left stream
    +  * @param rightType        the input type of right stream
    +  * @param resultType       the output type of join
    +  * @param queryConfig
    --- End diff --
    
    Please add parameter description


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132229209
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/UpdatingPlanChecker.scala ---
    @@ -90,40 +96,86 @@ object UpdatingPlanChecker {
                   // resolve names of input fields
                   .map(io => (inNames.get(io._1), io._2))
     
    -            // filter by input keys
    -            val outKeys = inOutNames.filter(io => keys.get.contains(io._1)).map(_._2)
    -            // check if all keys have been preserved
    -            if (outKeys.nonEmpty && outKeys.length == keys.get.length) {
    +            // filter by input keyAncestors
    +            val outKeyAncesters = inOutNames
    +              .filter(io => keyAncestors.get.map(e => e._1).contains(io._1))
    +              .map(io => (io._2, keyAncestors.get.find(ka => ka._1 == io._1).get._2))
    +
    +            // check if all keyAncestors have been preserved
    +            if (outKeyAncesters.nonEmpty &&
    +              outKeyAncesters.map(ka => ka._2).distinct.length ==
    +                keyAncestors.get.map(ka => ka._2).distinct.length) {
                   // all key have been preserved (but possibly renamed)
    -              keys = Some(outKeys.toArray)
    +              Some(outKeyAncesters.toList)
                 } else {
                   // some (or all) keys have been removed. Keys are no longer unique and removed
    -              keys = None
    +              None
                 }
    +          } else {
    +            None
               }
    +
             case _: DataStreamOverAggregate =>
    -          super.visit(node, ordinal, parent)
    -        // keys are always forwarded by Over aggregate
    +          // keyAncestors are always forwarded by Over aggregate
    +          visit(node.getInput(0))
             case a: DataStreamGroupAggregate =>
    -          // get grouping keys
    +          // get grouping keyAncestors
               val groupKeys = a.getRowType.getFieldNames.asScala.take(a.getGroupings.length)
    -          keys = Some(groupKeys.toArray)
    +          Some(groupKeys.map(e => (e, e)).toList)
             case w: DataStreamGroupWindowAggregate =>
    -          // get grouping keys
    +          // get grouping keyAncestors
               val groupKeys =
                 w.getRowType.getFieldNames.asScala.take(w.getGroupings.length).toArray
               // get window start and end time
               val windowStartEnd = w.getWindowProperties.map(_.name)
               // we have only a unique key if at least one window property is selected
               if (windowStartEnd.nonEmpty) {
    -            keys = Some(groupKeys ++ windowStartEnd)
    +            Some((groupKeys ++ windowStartEnd).map(e => (e, e)).toList)
    +          } else {
    +            None
    +          }
    +
    +        case j: DataStreamJoin =>
    +          val leftKeyAncestors = visit(j.getLeft)
    +          val rightKeyAncestors = visit(j.getRight)
    +          if (!leftKeyAncestors.isDefined || !rightKeyAncestors.isDefined) {
    +            None
    +          } else {
    +            // both left and right contain keys
    +            val leftJoinKeys =
    +              j.getLeft.getRowType.getFieldNames.asScala.zipWithIndex
    +              .filter(e => j.getJoinInfo.leftKeys.contains(e._2))
    +              .map(e => e._1)
    +            val rightJoinKeys =
    +              j.getRight.getRowType.getFieldNames.asScala.zipWithIndex
    +                .filter(e => j.getJoinInfo.rightKeys.contains(e._2))
    +                .map(e => e._1)
    +
    +            val leftKeys = leftKeyAncestors.get.map(e => e._1)
    +            val rightKeys = rightKeyAncestors.get.map(e => e._1)
    +
    +            //1. join key = left key = right key
    +            if (leftJoinKeys == leftKeys && rightJoinKeys == rightKeys) {
    +              Some(leftKeyAncestors.get ::: (rightKeyAncestors.get.map(e => (e._1)) zip
    --- End diff --
    
    `map(e => (e._1))` -> `map(_._1)`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132458139
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/UpdatingPlanChecker.scala ---
    @@ -90,40 +96,86 @@ object UpdatingPlanChecker {
                   // resolve names of input fields
                   .map(io => (inNames.get(io._1), io._2))
     
    -            // filter by input keys
    -            val outKeys = inOutNames.filter(io => keys.get.contains(io._1)).map(_._2)
    -            // check if all keys have been preserved
    -            if (outKeys.nonEmpty && outKeys.length == keys.get.length) {
    +            // filter by input keyAncestors
    +            val outKeyAncesters = inOutNames
    +              .filter(io => keyAncestors.get.map(e => e._1).contains(io._1))
    +              .map(io => (io._2, keyAncestors.get.find(ka => ka._1 == io._1).get._2))
    +
    +            // check if all keyAncestors have been preserved
    +            if (outKeyAncesters.nonEmpty &&
    +              outKeyAncesters.map(ka => ka._2).distinct.length ==
    +                keyAncestors.get.map(ka => ka._2).distinct.length) {
                   // all key have been preserved (but possibly renamed)
    -              keys = Some(outKeys.toArray)
    +              Some(outKeyAncesters.toList)
                 } else {
                   // some (or all) keys have been removed. Keys are no longer unique and removed
    -              keys = None
    +              None
                 }
    +          } else {
    +            None
               }
    +
             case _: DataStreamOverAggregate =>
    -          super.visit(node, ordinal, parent)
    -        // keys are always forwarded by Over aggregate
    +          // keyAncestors are always forwarded by Over aggregate
    +          visit(node.getInput(0))
             case a: DataStreamGroupAggregate =>
    -          // get grouping keys
    +          // get grouping keyAncestors
               val groupKeys = a.getRowType.getFieldNames.asScala.take(a.getGroupings.length)
    -          keys = Some(groupKeys.toArray)
    +          Some(groupKeys.map(e => (e, e)).toList)
             case w: DataStreamGroupWindowAggregate =>
    -          // get grouping keys
    +          // get grouping keyAncestors
               val groupKeys =
                 w.getRowType.getFieldNames.asScala.take(w.getGroupings.length).toArray
               // get window start and end time
               val windowStartEnd = w.getWindowProperties.map(_.name)
               // we have only a unique key if at least one window property is selected
               if (windowStartEnd.nonEmpty) {
    -            keys = Some(groupKeys ++ windowStartEnd)
    +            Some((groupKeys ++ windowStartEnd).map(e => (e, e)).toList)
    +          } else {
    +            None
    +          }
    +
    +        case j: DataStreamJoin =>
    +          val leftKeyAncestors = visit(j.getLeft)
    +          val rightKeyAncestors = visit(j.getRight)
    +          if (!leftKeyAncestors.isDefined || !rightKeyAncestors.isDefined) {
    +            None
    +          } else {
    +            // both left and right contain keys
    +            val leftJoinKeys =
    +              j.getLeft.getRowType.getFieldNames.asScala.zipWithIndex
    +              .filter(e => j.getJoinInfo.leftKeys.contains(e._2))
    +              .map(e => e._1)
    +            val rightJoinKeys =
    +              j.getRight.getRowType.getFieldNames.asScala.zipWithIndex
    +                .filter(e => j.getJoinInfo.rightKeys.contains(e._2))
    +                .map(e => e._1)
    +
    +            val leftKeys = leftKeyAncestors.get.map(e => e._1)
    +            val rightKeys = rightKeyAncestors.get.map(e => e._1)
    +
    +            //1. join key = left key = right key
    +            if (leftJoinKeys == leftKeys && rightJoinKeys == rightKeys) {
    +              Some(leftKeyAncestors.get ::: (rightKeyAncestors.get.map(e => (e._1)) zip
    +                leftKeyAncestors.get.map(e => (e._2))))
    +            }
    +            //2. join key = left key
    +            else if (leftJoinKeys == leftKeys && rightJoinKeys != rightKeys) {
    +              rightKeyAncestors
    +            }
    +            //3. join key = right key
    +            else if (leftJoinKeys != leftKeys && rightJoinKeys == rightKeys) {
    +              leftKeyAncestors
    +            }
    +            //4. join key not left or right key
    +            else {
    +              Some(leftKeyAncestors.get ++ rightKeyAncestors.get)
    --- End diff --
    
    In this case no keys are preserved. We have to return `None`. If no key is completely included in the equi join predicates, we have an n-m join and each row might join multiple times, so none of the attributes is guaranteed to be unique anymore.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r152833454
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/UpdatingPlanChecker.scala ---
    @@ -116,14 +135,100 @@ object UpdatingPlanChecker {
               val windowStartEnd = w.getWindowProperties.map(_.name)
               // we have only a unique key if at least one window property is selected
               if (windowStartEnd.nonEmpty) {
    -            keys = Some(groupKeys ++ windowStartEnd)
    +            val smallestAttribute = windowStartEnd.sorted.head
    +            Some((groupKeys.map(e => (e, e)) ++ windowStartEnd.map((_, smallestAttribute))).toList)
    +          } else {
    +            None
    +          }
    +
    +        case j: DataStreamJoin =>
    +          val joinType = j.getJoinType
    +          joinType match {
    +            case JoinRelType.INNER => {
    +              // get key(s) for inner join
    +              val lInputKeys = visit(j.getLeft)
    +              val rInputKeys = visit(j.getRight)
    +              if (lInputKeys.isEmpty || rInputKeys.isEmpty) {
    +                None
    +              } else {
    +                // Output of inner join must have keys if left and right both contain key(s).
    +                // Key groups from both side will be merged by join equi-predicates
    +                val lFieldNames: Seq[String] = j.getLeft.getRowType.getFieldNames
    +                val rFieldNames: Seq[String] = j.getRight.getRowType.getFieldNames
    +                val lJoinKeys: Seq[String] = j.getJoinInfo.leftKeys.map(lFieldNames.get(_))
    +                val rJoinKeys: Seq[String] = j.getJoinInfo.rightKeys.map(rFieldNames.get(_))
    +
    +                getOutputKeysForInnerJoin(
    +                  lFieldNames ++ rFieldNames,
    +                  lInputKeys.get ++ rInputKeys.get,
    +                  lJoinKeys.zip(rJoinKeys).toList
    +                )
    +              }
    +            }
    +            case _ => throw new UnsupportedOperationException(
    +              s"An Unsupported JoinType [ $joinType ]")
               }
             case _: DataStreamRel =>
    -          // anything else does not forward keys or might duplicate key, so we can stop
    -          keys = None
    +          // anything else does not forward keys, so we can stop
    +          None
           }
         }
     
    -  }
     
    +    def getOutputKeysForInnerJoin(
    +        inNames: Seq[String],
    +        inKeys: List[(String, String)],
    +        joinKeys: List[(String, String)])
    +    : Option[List[(String, String)]] = {
    +
    +      val nameToGroups = mutable.HashMap.empty[String,String]
    +
    +      // merge two groups
    +      def merge(nameA: String, nameB: String): Unit = {
    +        val ga: String = findGroup(nameA);
    --- End diff --
    
    Remove semicolons.


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r152807064
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/DataStreamInnerJoin.scala ---
    @@ -0,0 +1,285 @@
    +/*
    + * 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._
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.java.typeutils.TupleTypeInfo
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.runtime.CRowWrappingMultiOuputCollector
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
    +import org.slf4j.LoggerFactory
    +import org.apache.flink.table.codegen.Compiler
    +
    +
    +/**
    +  * Connect data for left stream and right stream. Only use for innerJoin.
    +  *
    +  * @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
    +  */
    +class DataStreamInnerJoin(
    --- End diff --
    
    Call this `UnboundedInnerJoin`?


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132300021
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeNonWindowInnerJoin.scala ---
    @@ -0,0 +1,275 @@
    +/*
    + * 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.RichFlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.java.typeutils.{ResultTypeQueryable, TupleTypeInfo}
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.runtime.CRowWrappingMultiOuputCollector
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
    +
    +/**
    +  * Connect data for left stream and right stream. Only use for innerJoin.
    +  *
    +  * @param joiner           join function
    +  * @param leftType         the input type of left stream
    +  * @param rightType        the input type of right stream
    +  * @param resultType       the output type of join
    +  * @param queryConfig
    +  */
    +class ProcTimeNonWindowInnerJoin(
    +    joiner: RichFlatJoinFunction[Row, Row, Row],
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    resultType: TypeInformation[CRow],
    +    queryConfig: StreamQueryConfig) extends
    +  CoProcessFunction[CRow, CRow, CRow] with ResultTypeQueryable[CRow] {
    +
    +
    +  // state to hold left stream element
    +  private var leftState: MapState[Row, JTuple2[Int, Long]] = null
    +  // state to hold right stream element
    +  private var rightState: MapState[Row, JTuple2[Int, Long]] = null
    +  private var cRowWrapper: CRowWrappingMultiOuputCollector = null
    +
    +  private val minRetentionTime: Long = queryConfig.getMinIdleStateRetentionTime
    +  private val maxRetentionTime: Long = queryConfig.getMaxIdleStateRetentionTime
    +  private val stateCleaningEnabled: Boolean = minRetentionTime > 1
    +
    +  // state to record last timer of left stream, 0 means no timer
    +  private var timerState1: ValueState[Long] = _
    +  // state to record last timer of right stream, 0 means no timer
    +  private var timerState2: ValueState[Long] = _
    +
    +
    +  override def open(parameters: Configuration): Unit = {
    +    // initialize left and right state
    +    val tupleTypeInfo = new TupleTypeInfo[JTuple2[Int, Long]](Types.INT, Types.LONG)
    +    val leftStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "left", leftType, tupleTypeInfo)
    +    val rightStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "right", rightType, tupleTypeInfo)
    +    leftState = getRuntimeContext.getMapState(leftStateDescriptor)
    +    rightState = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // initialize timer state
    +    val valueStateDescriptor1 = new ValueStateDescriptor[Long]("timervaluestate1", classOf[Long])
    +    timerState1 = getRuntimeContext.getState(valueStateDescriptor1)
    +    val valueStateDescriptor2 = new ValueStateDescriptor[Long]("timervaluestate2", classOf[Long])
    +    timerState2 = getRuntimeContext.getState(valueStateDescriptor2)
    +
    +    cRowWrapper = new CRowWrappingMultiOuputCollector()
    +    joiner.setRuntimeContext(getRuntimeContext)
    +    joiner.open(parameters)
    +  }
    +
    +  /**
    +    * Process left stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx   The ctx to register timer or get current time
    +    * @param out   The collector for returning result values.
    +    *
    +    */
    +  override def processElement1(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, timerState1, leftState, rightState, true)
    +  }
    +
    +  /**
    +    * Process right stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx   The ctx to register timer or get current time
    +    * @param out   The collector for returning result values.
    +    *
    +    */
    +  override def processElement2(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, timerState2, rightState, leftState, false)
    +  }
    +
    +
    +  /**
    +    * Called when a processing timer trigger.
    +    * Expire left/right records which are expired in left and right state.
    +    *
    +    * @param timestamp The timestamp of the firing timer.
    +    * @param ctx       The ctx to register timer or get current time
    +    * @param out       The collector for returning result values.
    +    */
    +  override def onTimer(
    +      timestamp: Long,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +
    +    if (stateCleaningEnabled && timerState1.value == timestamp) {
    +      expireOutTimeRow(
    +        timestamp,
    +        leftState,
    +        timerState1,
    +        ctx
    +      )
    +    }
    +
    +    if (stateCleaningEnabled && timerState2.value == timestamp) {
    +      expireOutTimeRow(
    +        timestamp,
    +        rightState,
    +        timerState2,
    +        ctx
    +      )
    +    }
    +  }
    +
    +
    +  def getNewExpiredTime(
    +      curProcessTime: Long,
    +      oldExpiredTime: Long): Long = {
    +
    +    var newExpiredTime = oldExpiredTime
    +    if (stateCleaningEnabled) {
    +      if (-1 == oldExpiredTime || (curProcessTime + minRetentionTime) > oldExpiredTime) {
    +        newExpiredTime = curProcessTime + maxRetentionTime
    +      }
    +    }
    +    newExpiredTime
    +  }
    +
    +  /**
    +    * Puts or Retract an element from the input stream into state and search the other state to
    +    * output records meet the condition. Records will be expired in state if state retention time
    +    * has been specified.
    +    */
    +  def processElement(
    +      value: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow],
    +      timerState: ValueState[Long],
    +      currentSideState: MapState[Row, JTuple2[Int, Long]],
    +      otherSideState: MapState[Row, JTuple2[Int, Long]],
    +      isLeft: Boolean): Unit = {
    +
    +    cRowWrapper.out = out
    +    cRowWrapper.setChange(value.change)
    +
    +    val curProcessTime = ctx.timerService.currentProcessingTime
    +    var oldCnt = 0
    +    var oldExpiredTime: Long = -1
    +
    +    val currentRowCntAndExpiredTime = currentSideState.get(value.row)
    +    if (currentRowCntAndExpiredTime != null) {
    +      oldCnt = currentRowCntAndExpiredTime.f0
    +      oldExpiredTime = currentRowCntAndExpiredTime.f1
    +    }
    +
    +    val newExpiredTime = getNewExpiredTime(curProcessTime, oldExpiredTime)
    +    if (stateCleaningEnabled && timerState.value() == 0) {
    +      timerState.update(newExpiredTime)
    +      ctx.timerService().registerProcessingTimeTimer(newExpiredTime)
    +    }
    +
    +    // update current side stream state
    +    if (!value.asInstanceOf[CRow].change) {
    +      oldCnt = oldCnt - 1
    +      if (oldCnt <= 0) {
    +        currentSideState.remove(value.row)
    +      } else {
    +        currentSideState.put(value.row, JTuple2.of(oldCnt, newExpiredTime))
    +      }
    +    } else {
    +      oldCnt = oldCnt + 1
    +      currentSideState.put(value.row, JTuple2.of(oldCnt, newExpiredTime))
    --- End diff --
    
    We can reuse `currentRowCntAndExpiredTime` instead of creating a new `Tuple2`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132412037
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/UpdatingPlanCheckerTest.scala ---
    @@ -0,0 +1,203 @@
    +/*
    + * 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
    +
    +import org.apache.flink.table.api.Table
    +import org.apache.flink.table.plan.util.UpdatingPlanChecker
    +import org.apache.flink.table.utils.StreamTableTestUtil
    +import org.junit.Assert._
    +import org.apache.flink.table.api.scala._
    +import org.apache.flink.api.scala._
    +import org.junit.Test
    +
    +
    +class UpdatingPlanCheckerTest {
    +
    +  @Test
    +  def testSelect(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(String, Int)]('word, 'number)
    +    val resultTable = table.select('word, 'number)
    +
    +    util.verifyTableUniqueKey(resultTable, Nil)
    +  }
    +
    +  @Test
    +  def testGroupByWithoutKey(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(String, Int)]('word, 'number)
    +
    +    val resultTable = table
    +      .groupBy('word)
    +      .select('number.count)
    +
    +    util.verifyTableUniqueKey(resultTable, Nil)
    +  }
    +
    +  @Test
    +  def testGroupBy(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(String, Int)]('word, 'number)
    +
    +    val resultTable = table
    +      .groupBy('word)
    +      .select('word, 'number.count)
    +
    +    util.verifyTableUniqueKey(resultTable, Seq("word"))
    +  }
    +
    +  @Test
    +  def testGroupByWithDumplicateKey(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(String, Int)]('word, 'number)
    +
    +    val resultTable = table
    +      .groupBy('word)
    +      .select('word as 'word1, 'word as 'word2, 'number.count)
    +
    +    util.verifyTableUniqueKey(resultTable, Seq("word1", "word2"))
    +  }
    +
    +  //1. join key = left key = right key
    +  @Test
    +  def testJoinKeysEqualsleftAndRightKeys(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(Int, Int)]('pk, 'a)
    +
    +    val leftTableWithPk = table
    +      .groupBy('pk)
    +      .select('pk as 'leftpk, 'a.max as 'lefta)
    +
    +    val rightTableWithPk = table
    +      .groupBy('pk)
    +      .select('pk as 'rightpk, 'a.max as 'righta)
    +
    +    val resultTable = leftTableWithPk
    +      .join(rightTableWithPk)
    +      .where('leftpk === 'rightpk)
    +      .select('leftpk, 'lefta, 'righta)
    +
    +    util.verifyTableUniqueKey(resultTable, Seq("leftpk"))
    +  }
    +
    +  //2. join key = left key
    +  @Test
    +  def testJoinKeysEqualsLeftKeys(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(Int, Int)]('pk, 'a)
    +
    +    val leftTableWithPk = table
    +      .groupBy('pk)
    +      .select('pk as 'leftpk, 'a.max as 'lefta)
    +
    +    val rightTableWithPk = table
    +      .groupBy('pk)
    +      .select('pk as 'rightpk, 'a.max as 'righta)
    +
    +    val resultTable = leftTableWithPk
    +      .join(rightTableWithPk)
    +      .where('leftpk === 'righta)
    +      .select('rightpk, 'lefta, 'righta)
    +
    +    util.verifyTableUniqueKey(resultTable, Seq("rightpk"))
    +  }
    +
    +  //3. join key = right key
    +  @Test
    +  def testJoinKeysEqualsRightKeys(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(Int, Int)]('pk, 'a)
    +
    +    val leftTableWithPk = table
    +      .groupBy('pk)
    +      .select('pk as 'leftpk, 'a.max as 'lefta)
    +
    +    val rightTableWithPk = table
    +      .groupBy('pk)
    +      .select('pk as 'rightpk, 'a.max as 'righta)
    +
    +    val resultTable = leftTableWithPk
    +      .join(rightTableWithPk)
    +      .where('lefta === 'rightpk)
    +      .select('leftpk, 'lefta, 'righta)
    +
    +    util.verifyTableUniqueKey(resultTable, Seq("leftpk"))
    +  }
    +
    +  //4. join key not left or right key
    +  @Test
    +  def testJoinKeysWithoutLeftRightKeys(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(Int, Int)]('pk, 'a)
    +
    +    val leftTableWithPk = table
    +      .groupBy('pk)
    +      .select('pk as 'leftpk, 'a.max as 'lefta)
    +
    +    val rightTableWithPk = table
    +      .groupBy('pk)
    +      .select('pk as 'rightpk, 'a.max as 'righta)
    +
    +    val resultTable = leftTableWithPk
    +      .join(rightTableWithPk)
    +      .where('lefta === 'righta)
    +      .select('leftpk, 'rightpk, 'lefta, 'righta)
    +
    +    util.verifyTableUniqueKey(resultTable, Seq("leftpk", "rightpk"))
    +  }
    +
    +  @Test
    +  def testNonKeysJoin(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(Int, Int)]('a, 'b)
    +
    +    val leftTable = table
    +      .select('a as 'a, 'b as 'b)
    +
    +    val rightTable = table
    +      .select('a as 'aa, 'b as 'bb)
    +
    +    val resultTable = leftTable
    +      .join(rightTable)
    +      .where('a === 'aa)
    +      .select('a, 'aa, 'b, 'bb)
    +
    +    util.verifyTableUniqueKey(resultTable, Nil)
    +  }
    +}
    +
    +
    +class UpdatePlanChecnkerUtil extends StreamTableTestUtil {
    +
    +  def verifySqlUniqueKey(query: String, expected: Seq[String]): Unit = {
    +    verifyTableUniqueKey(tableEnv.sql(query), expected)
    +  }
    +
    +  def verifyTableUniqueKey(resultTable: Table, expected: Seq[String]): Unit = {
    +    val relNode = resultTable.getRelNode
    +    val optimized = tableEnv.optimize(relNode, updatesAsRetraction = false)
    +    val actual = UpdatingPlanChecker.getUniqueKeyFields(optimized)
    +
    +    if (actual.isDefined) {
    --- End diff --
    
    This check evaluates always to `true` if `actual.isEmpty` regardless of what is expected.
    
    Should be
    
    ```
    (expected, actual) match {
          case (Nil, None) => assert(true)
          case (Nil, Some(ak)) => fail(s"No keys expected, but found: [${ak.mkString(", ")}]")
          case (ek, None) => fail(s"No keys found, but expected: [${ek.mkString(", ")}]")
          case (ek, Some(ak)) => assertEquals(ek.sorted, ak.toList.sorted)
        }
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r160121422
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/UpdatingPlanChecker.scala ---
    @@ -116,14 +135,100 @@ object UpdatingPlanChecker {
               val windowStartEnd = w.getWindowProperties.map(_.name)
    --- End diff --
    
    No, it can also emit the rowtime and proctime.


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132230160
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/UpdatingPlanChecker.scala ---
    @@ -56,17 +59,20 @@ object UpdatingPlanChecker {
       }
     
       /** Identifies unique key fields in the output of a RelNode. */
    -  private class UniqueKeyExtractor extends RelVisitor {
    -
    -    var keys: Option[Array[String]] = None
    +  private class UniqueKeyExtractor {
     
    -    override def visit(node: RelNode, ordinal: Int, parent: RelNode): Unit = {
    +    // visit() function will return a tuple, the first element of tuple is the key, the second is
    +    // the key's corresponding ancestor. Ancestors are used to identify same keys, for example:
    +    // select('pk as pk1, 'pk as pk2), both pk1 and pk2 have the same ancestor, i.e., pk.
    +    // A node having keys means: 1.it generates keys by itself 2.all ancestors from it's upstream
    --- End diff --
    
    `A node can have keys if it generates the keys by itself or it forwards keys from its input(s).`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132303241
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/validation/JoinValidationTest.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.stream.table.validation
    +
    +import org.apache.flink.api.scala._
    +import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
    +import org.apache.flink.table.api.{TableEnvironment, TableException, ValidationException}
    +import org.apache.flink.table.api.scala._
    +import org.apache.flink.table.runtime.utils.StreamTestData
    +import org.apache.flink.table.utils.TableTestBase
    +import org.apache.flink.types.Row
    +import org.junit.Test
    +
    +class JoinValidationTest extends TableTestBase {
    +
    +  private val util = streamTestUtil()
    +  private val ds1 = util.addTable[(Int, Long, String)]("Table3",'a, 'b, 'c)
    +  private val ds2 = util.addTable[(Int, Long, Int, String, Long)]("Table5", 'd, 'e, 'f, 'g, 'h)
    +
    +  @Test(expected = classOf[ValidationException])
    +  def testJoinNonExistingKey(): Unit = {
    +    ds1.join(ds2)
    +      // must fail. Field 'foo does not exist
    +      .where('foo === 'e)
    +      .select('c, 'g)
    +  }
    +
    +  @Test(expected = classOf[ValidationException])
    +  def testJoinWithNonMatchingKeyTypes(): Unit = {
    +    ds1.join(ds2)
    +      // must fail. Field 'a is Int, and 'g is String
    +      .where('a === 'g)
    +      .select('c, 'g)
    +  }
    +
    +
    +  @Test(expected = classOf[ValidationException])
    +  def testJoinWithAmbiguousFields(): Unit = {
    +    ds1.join(ds2.select('d, 'e, 'f, 'g, 'h as 'c))
    +      // must fail. Both inputs share the same field 'c
    +      .where('a === 'd)
    +      .select('c, 'g)
    +  }
    +
    +  @Test(expected = classOf[TableException])
    +  def testNoEqualityJoinPredicate1(): Unit = {
    +    ds1.join(ds2)
    +      // must fail. No equality join predicate
    +      .where('d === 'f)
    +      .select('c, 'g)
    +      .toDataSet[Row]
    +  }
    +
    +  @Test(expected = classOf[TableException])
    +  def testNoEqualityJoinPredicate2(): Unit = {
    +    ds1.join(ds2)
    +      // must fail. No equality join predicate
    +      .where('a < 'd)
    +      .select('c, 'g)
    +      .toDataSet[Row]
    --- End diff --
    
    replace `toDataSet` call by `toRetractStream[Row]`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #4471: [FLINK-6094] [table] Implement stream-stream proctime non...

Posted by hequn8128 <gi...@git.apache.org>.
Github user hequn8128 commented on the issue:

    https://github.com/apache/flink/pull/4471
  
    Hi, @twalthr , thanks for your review.
    The pr has been updated according to your comments. It mainly contains the following changes:
    
    - Do some minor refactors in `UpdatingPlanChecker`, `NonWindowInnerJoin` and tests
    - As for indicator attributes
      - Event-time attributes are considered to be not supported. When executing the join, the join operator needs to make sure that no late data is emitted. Window join makes it possible by holding back watermarks, but non-window join is unbounded, so we don't know how much to hold back. 
      - Proctime attributes are supported to be outputted from join but can not exist in join predicate. It seems there is no easy way to support proctime attributes in join predicate. If we evaluate proctime in code generator, left proctime will always equals right proctime, which makes `left.proctime > right.proctime` always return false. Currently, users can cast proctime attributes to long type if they want to do predicate. What do you think?


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by shaoxuan-wang <gi...@git.apache.org>.
Github user shaoxuan-wang commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r140951770
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamJoinRule.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.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.flink.table.api.TableConfig
    +import org.apache.flink.table.calcite.FlinkTypeFactory
    +import org.apache.flink.table.plan.nodes.FlinkConventions
    +import org.apache.flink.table.plan.nodes.datastream.DataStreamJoin
    +import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
    +import org.apache.flink.table.plan.schema.RowSchema
    +import org.apache.flink.table.runtime.join.WindowJoinUtil
    +import scala.collection.JavaConverters._
    +
    +class DataStreamJoinRule
    +  extends ConverterRule(
    +    classOf[FlinkLogicalJoin],
    +    FlinkConventions.LOGICAL,
    +    FlinkConventions.DATASTREAM,
    +    "DataStreamJoinRule") {
    +
    +  override def matches(call: RelOptRuleCall): Boolean = {
    +    val join: FlinkLogicalJoin = call.rel(0).asInstanceOf[FlinkLogicalJoin]
    +    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)
    +
    +    // remaining predicate must not access time attributes
    +    val remainingPredsAccessTime = remainingPreds.isDefined &&
    +      WindowJoinUtil.accessesTimeAttribute(remainingPreds.get, join.getRowType)
    +
    +    // Check that no event-time attributes are in the input.
    +    val rowTimeAttrInOutput = join.getRowType.getFieldList.asScala
    +      .exists(f => FlinkTypeFactory.isRowtimeIndicatorType(f.getType))
    +
    +    if (!windowBounds.isDefined && !remainingPredsAccessTime && !rowTimeAttrInOutput) {
    --- End diff --
    
    @fhueske, we actually agree quite a lot on the concern of infinite size you have raised. The same problem does not only exist in joining, but also in other cases, for example GROUPBY, where the grouping-key and associated state can be unlimited in terms of the size that the state of Flink can not hold them all. IMO, there is not an easy way to completely eliminate this just through the validation of query planner/optimizer, so I think it is not a good idea to only allow the unbounded-joining after a certain operators, like non-windowed aggregation (in fact, as mentioned above, the grouping-key of aggregation may also be infinite, so this does not ensure the finite state for joining operator). 
    On the other hand, I think the finite state can only be ensured by the users by giving some hints/controls. We need instruct users to properly set those control knobs, such that their jobs will not run out of space. One hint we currently have is state ttl. (I think @hequn8128 has already added this for this unbounded joining). Maybe here we can add a check on state ttl to force users set a proper value. What do you think?
      


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132197433
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamJoinRule.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.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.flink.table.api.TableConfig
    +import org.apache.flink.table.calcite.FlinkTypeFactory
    +import org.apache.flink.table.plan.nodes.FlinkConventions
    +import org.apache.flink.table.plan.nodes.datastream.DataStreamJoin
    +import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
    +import org.apache.flink.table.plan.schema.RowSchema
    +import org.apache.flink.table.runtime.join.WindowJoinUtil
    +import scala.collection.JavaConverters._
    +
    +class DataStreamJoinRule
    +  extends ConverterRule(
    +    classOf[FlinkLogicalJoin],
    +    FlinkConventions.LOGICAL,
    +    FlinkConventions.DATASTREAM,
    +    "DataStreamJoinRule") {
    +
    +  override def matches(call: RelOptRuleCall): Boolean = {
    +    val join: FlinkLogicalJoin = call.rel(0).asInstanceOf[FlinkLogicalJoin]
    +    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)
    +
    +    // remaining predicate must not access time attributes
    +    val remainingPredsAccessTime = remainingPreds.isDefined &&
    +      WindowJoinUtil.accessesTimeAttribute(remainingPreds.get, join.getRowType)
    +
    +    // Check that no event-time attributes are in the input.
    +    val rowTimeAttrInOutput = join.getRowType.getFieldList.asScala
    +      .exists(f => FlinkTypeFactory.isRowtimeIndicatorType(f.getType))
    +
    +    if (!windowBounds.isDefined && !remainingPredsAccessTime && !rowTimeAttrInOutput) {
    --- End diff --
    
    I'm not sure if this is the right condition for this join implementation.
    I think we need to check that both inputs have a finite size, i.e., there is a bounded number of records in the table at a given point in time. This is important because we need to keep all active records of both inputs in state.
    A table which is derived from a stream can have a finite size if:
    
    - it is the result of a non-windowed aggregation (`GROUP BY user`)
    - it is derived by an update stream->table conversion
    - it is the tail of a stream (e.g., `WHERE rowtime > now() - INTERVAL '4' HOUR` or `ORDER BY rowtime DESC FETCH 100 ROWS ONLY`)
    - there might be more cases.
    
    The windowed join is different in that is can process unbounded tables because it can limit the valid data ranges with the window boundaries. So, just because a join cannot be computed with the windowed join does not automatically mean we can process it with this implementation.
    
    Right now, only non-windowed aggregation is supported and hence the only way to obtain a finite table from a stream. 
    
    In `DataSetSingleRowJoinRule.isSingleRow()` we check if an input rel produces only a single row by tracking back the input. I think we need similar logic here to ensure that the input relation is of finite size (in our case has a non-windowed `Aggregate`).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132227802
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/UpdatingPlanChecker.scala ---
    @@ -37,8 +37,11 @@ object UpdatingPlanChecker {
       /** Extracts the unique keys of the table produced by the plan. */
       def getUniqueKeyFields(plan: RelNode): Option[Array[String]] = {
         val keyExtractor = new UniqueKeyExtractor
    -    keyExtractor.go(plan)
    -    keyExtractor.keys
    +    if (!keyExtractor.visit(plan).isDefined) {
    --- End diff --
    
    Can be replaced by `keyExtractor.visit(plan).map(_.map(_._1).toArray)`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r152819531
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/JoinITCase.scala ---
    @@ -0,0 +1,262 @@
    +/*
    + * 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.table
    +
    +import org.apache.flink.api.scala._
    +import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
    +import org.apache.flink.table.api.{StreamQueryConfig, TableEnvironment}
    +import org.apache.flink.table.api.scala._
    +import org.apache.flink.table.runtime.utils.{StreamITCase, StreamingWithStateTestBase}
    +import org.junit.Assert._
    +import org.junit.Test
    +import org.apache.flink.api.common.time.Time
    +import org.apache.flink.streaming.api.TimeCharacteristic
    +import org.apache.flink.table.functions.aggfunctions.CountAggFunction
    +import org.apache.flink.table.runtime.stream.table.GroupWindowITCase.TimestampAndWatermarkWithOffset
    +import org.apache.flink.table.runtime.utils.JavaUserDefinedAggFunctions.{CountDistinct, WeightedAvg}
    +import org.apache.flink.types.Row
    +
    +import scala.collection.mutable
    +
    +class JoinITCase extends StreamingWithStateTestBase {
    +
    +  private val queryConfig = new StreamQueryConfig()
    +  queryConfig.withIdleStateRetentionTime(Time.hours(1), Time.hours(2))
    +
    +  @Test
    +  def testOutputWithPk(): Unit = {
    +    // data input
    +
    +    val data1 = List(
    +      (0, 0),
    +      (1, 0),
    +      (1, 1),
    +      (2, 2),
    +      (3, 3),
    +      (4, 4),
    +      (5, 4),
    +      (5, 5)
    +    )
    +
    +    val data2 = List(
    +      (1, 1),
    +      (2, 0),
    +      (2, 1),
    +      (2, 2),
    +      (3, 3),
    +      (4, 4),
    +      (5, 4),
    +      (5, 5),
    +      (6, 6)
    +    )
    +
    +    val env = StreamExecutionEnvironment.getExecutionEnvironment
    +    val tEnv = TableEnvironment.getTableEnvironment(env)
    +    StreamITCase.clear
    +    env.setStateBackend(getStateBackend)
    +    env.setParallelism(1)
    +
    +    val leftTable = env.fromCollection(data1).toTable(tEnv, 'a, 'b)
    +    val rightTable = env.fromCollection(data2).toTable(tEnv, 'bb, 'c)
    +
    +    val leftTableWithPk = leftTable
    +      .groupBy('a)
    +      .select('a, 'b.max as 'b)
    +
    +    val rightTableWithPk = rightTable
    +        .groupBy('bb)
    +        .select('bb, 'c.max as 'c)
    +
    +    leftTableWithPk
    +      .join(rightTableWithPk, 'b === 'bb)
    +      .select('a, 'b, 'c)
    +      .writeToSink(new TestUpsertSink(Array("a,b"), false), queryConfig)
    +
    +    env.execute()
    +    val results = RowCollector.getAndClearValues
    +    val retracted = RowCollector.upsertResults(results, Array(0)).sorted
    +
    +    val expected = Seq("1,1,1", "2,2,2", "3,3,3", "4,4,4", "5,5,5")
    +    assertEquals(expected, retracted)
    +
    +  }
    +
    +
    +  @Test
    +  def testOutputWithoutPk(): Unit = {
    +    // data input
    +
    +    val data1 = List(
    +      (0, 0),
    +      (1, 0),
    +      (1, 1),
    +      (2, 2),
    +      (3, 3),
    +      (4, 4),
    +      (5, 4),
    +      (5, 5)
    +    )
    +
    +    val data2 = List(
    +      (1, 1, 1),
    +      (1, 1, 1),
    +      (1, 1, 1),
    +      (1, 1, 1),
    +      (2, 2, 2),
    +      (3, 3, 3),
    +      (4, 4, 4),
    +      (5, 5, 5),
    +      (5, 5, 5),
    +      (6, 6, 6)
    +    )
    +
    +    val env = StreamExecutionEnvironment.getExecutionEnvironment
    +    val tEnv = TableEnvironment.getTableEnvironment(env)
    +    StreamITCase.clear
    +    env.setStateBackend(getStateBackend)
    +    env.setParallelism(1)
    +
    +    val leftTable = env.fromCollection(data1).toTable(tEnv, 'a, 'b)
    +    val rightTable = env.fromCollection(data2).toTable(tEnv, 'bb, 'c, 'd)
    +
    +    val leftTableWithPk = leftTable
    +      .groupBy('a)
    +      .select('a, 'b.max as 'b)
    +
    +    leftTableWithPk
    +      .join(rightTable, 'a === 'bb && ('a < 4 || 'a > 4))
    +      .select('a, 'b, 'c, 'd)
    +      .writeToSink(new TestRetractSink, queryConfig)
    +
    +    env.execute()
    +    val results = RowCollector.getAndClearValues
    +
    +    val retracted = RowCollector.retractResults(results).sorted
    +
    +    val expected = Seq("1,1,1,1", "1,1,1,1", "1,1,1,1", "1,1,1,1", "2,2,2,2", "3,3,3,3",
    +                       "5,5,5,5", "5,5,5,5")
    +    assertEquals(expected, retracted)
    +  }
    +
    +
    +  @Test
    +  def testJoinWithTimeAttributeOutput() {
    +
    +    val data1 = List(
    +      (1L, 1, "LEFT:Hi"),
    +      (2L, 2, "LEFT:Hello"),
    +      (4L, 2, "LEFT:Hello"),
    +      (8L, 3, "LEFT:Hello world"),
    +      (16L, 3, "LEFT:Hello world"))
    +
    +    val data2 = List(
    +      (1L, 1, "RIGHT:Hi"),
    +      (2L, 2, "RIGHT:Hello"),
    +      (4L, 2, "RIGHT:Hello"),
    +      (8L, 3, "RIGHT:Hello world"),
    +      (16L, 3, "RIGHT:Hello world"))
    +
    +    val env = StreamExecutionEnvironment.getExecutionEnvironment
    +    env.setParallelism(1)
    +    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
    +    val tEnv = TableEnvironment.getTableEnvironment(env)
    +    StreamITCase.testResults = mutable.MutableList()
    +
    +    val stream1 = env
    +      .fromCollection(data1)
    +      .assignTimestampsAndWatermarks(new TimestampAndWatermarkWithOffset[(Long, Int, String)](0L))
    +    val stream2 = env
    +      .fromCollection(data2)
    +      .assignTimestampsAndWatermarks(new TimestampAndWatermarkWithOffset[(Long, Int, String)](0L))
    +
    +    val table1 = stream1.toTable(tEnv, 'long_l, 'int_l, 'string_l, 'rowtime_l.rowtime)
    +    val table2 = stream2.toTable(tEnv, 'long_r, 'int_r, 'string_r)
    +    val countFun = new CountAggFunction
    +    val weightAvgFun = new WeightedAvg
    +    val countDistinct = new CountDistinct
    +
    +    val table = table1
    +      .join(table2, 'long_l === 'long_r)
    +      .select('long_l as 'long, 'int_r as 'int, 'string_r as 'string, 'rowtime_l as 'rowtime)
    +
    +    val windowedTable = table
    +      .window(Tumble over 5.milli on 'rowtime as 'w)
    +      .groupBy('w, 'string)
    +      .select('string, countFun('string), 'int.avg, weightAvgFun('long, 'int),
    +              weightAvgFun('int, 'int), 'int.min, 'int.max, 'int.sum, 'w.start, 'w.end,
    +              countDistinct('long))
    +
    +    val results = windowedTable.toAppendStream[Row]
    +    results.addSink(new StreamITCase.StringSink[Row])
    +    env.execute()
    +
    +    val expected = Seq(
    +      "RIGHT:Hello world,1,3,8,3,3,3,3,1970-01-01 00:00:00.005,1970-01-01 00:00:00.01,1",
    +      "RIGHT:Hello world,1,3,16,3,3,3,3,1970-01-01 00:00:00.015,1970-01-01 00:00:00.02,1",
    +      "RIGHT:Hello,2,2,3,2,2,2,4,1970-01-01 00:00:00.0,1970-01-01 00:00:00.005,2",
    +      "RIGHT:Hi,1,1,1,1,1,1,1,1970-01-01 00:00:00.0,1970-01-01 00:00:00.005,1")
    +    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
    +  }
    +
    +
    +  @Test(expected = classOf[UnsupportedOperationException])
    +  def testLeftOuterJoin(): Unit = {
    +    val env = StreamExecutionEnvironment.getExecutionEnvironment
    +    val tEnv = TableEnvironment.getTableEnvironment(env)
    +    StreamITCase.clear
    +    env.setStateBackend(getStateBackend)
    +    env.setParallelism(1)
    +
    +    val leftTable = env.fromCollection(List((1, 2))).toTable(tEnv, 'a, 'b)
    +    val rightTable = env.fromCollection(List((1, 2))).toTable(tEnv, 'bb, 'c)
    +
    +    leftTable.leftOuterJoin(rightTable, 'a ==='bb).toAppendStream[Row]
    +    env.execute()
    --- End diff --
    
    Remove `execute()` in all failing tests.


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132233953
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/UpdatingPlanChecker.scala ---
    @@ -90,40 +96,86 @@ object UpdatingPlanChecker {
                   // resolve names of input fields
                   .map(io => (inNames.get(io._1), io._2))
     
    -            // filter by input keys
    -            val outKeys = inOutNames.filter(io => keys.get.contains(io._1)).map(_._2)
    -            // check if all keys have been preserved
    -            if (outKeys.nonEmpty && outKeys.length == keys.get.length) {
    +            // filter by input keyAncestors
    +            val outKeyAncesters = inOutNames
    --- End diff --
    
    this can be simplified if we create a `Map` from `keysOfInput`:
    
    ```
    val inputKeysMap = inputKeys.get.toMap
    
    val outKeyAncesters = inOutNames
                  .filter(io => inputKeysMap.contains(io._1))
                  .map(io => (io._2, inputKeysMap(io._1)))
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132277678
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/UpdatingPlanChecker.scala ---
    @@ -90,40 +96,86 @@ object UpdatingPlanChecker {
                   // resolve names of input fields
                   .map(io => (inNames.get(io._1), io._2))
     
    -            // filter by input keys
    -            val outKeys = inOutNames.filter(io => keys.get.contains(io._1)).map(_._2)
    -            // check if all keys have been preserved
    -            if (outKeys.nonEmpty && outKeys.length == keys.get.length) {
    +            // filter by input keyAncestors
    +            val outKeyAncesters = inOutNames
    +              .filter(io => keyAncestors.get.map(e => e._1).contains(io._1))
    +              .map(io => (io._2, keyAncestors.get.find(ka => ka._1 == io._1).get._2))
    +
    +            // check if all keyAncestors have been preserved
    +            if (outKeyAncesters.nonEmpty &&
    +              outKeyAncesters.map(ka => ka._2).distinct.length ==
    +                keyAncestors.get.map(ka => ka._2).distinct.length) {
                   // all key have been preserved (but possibly renamed)
    -              keys = Some(outKeys.toArray)
    +              Some(outKeyAncesters.toList)
                 } else {
                   // some (or all) keys have been removed. Keys are no longer unique and removed
    -              keys = None
    +              None
                 }
    +          } else {
    +            None
               }
    +
             case _: DataStreamOverAggregate =>
    -          super.visit(node, ordinal, parent)
    -        // keys are always forwarded by Over aggregate
    +          // keyAncestors are always forwarded by Over aggregate
    +          visit(node.getInput(0))
             case a: DataStreamGroupAggregate =>
    -          // get grouping keys
    +          // get grouping keyAncestors
               val groupKeys = a.getRowType.getFieldNames.asScala.take(a.getGroupings.length)
    -          keys = Some(groupKeys.toArray)
    +          Some(groupKeys.map(e => (e, e)).toList)
             case w: DataStreamGroupWindowAggregate =>
    -          // get grouping keys
    +          // get grouping keyAncestors
               val groupKeys =
                 w.getRowType.getFieldNames.asScala.take(w.getGroupings.length).toArray
               // get window start and end time
               val windowStartEnd = w.getWindowProperties.map(_.name)
               // we have only a unique key if at least one window property is selected
               if (windowStartEnd.nonEmpty) {
    -            keys = Some(groupKeys ++ windowStartEnd)
    +            Some((groupKeys ++ windowStartEnd).map(e => (e, e)).toList)
    +          } else {
    +            None
    +          }
    +
    +        case j: DataStreamJoin =>
    +          val leftKeyAncestors = visit(j.getLeft)
    +          val rightKeyAncestors = visit(j.getRight)
    +          if (!leftKeyAncestors.isDefined || !rightKeyAncestors.isDefined) {
    +            None
    +          } else {
    +            // both left and right contain keys
    +            val leftJoinKeys =
    --- End diff --
    
    Is easier to compute with:
    
    ```
    val leftFieldNames = j.getLeft.getRowType.getFieldNames
    val leftJoinKeys: Seq[String] = j.getJoinInfo.leftKeys.asScala.map(leftFieldNames.get(_))
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132230387
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/UpdatingPlanChecker.scala ---
    @@ -56,17 +59,20 @@ object UpdatingPlanChecker {
       }
     
       /** Identifies unique key fields in the output of a RelNode. */
    -  private class UniqueKeyExtractor extends RelVisitor {
    -
    -    var keys: Option[Array[String]] = None
    +  private class UniqueKeyExtractor {
     
    -    override def visit(node: RelNode, ordinal: Int, parent: RelNode): Unit = {
    +    // visit() function will return a tuple, the first element of tuple is the key, the second is
    +    // the key's corresponding ancestor. Ancestors are used to identify same keys, for example:
    +    // select('pk as pk1, 'pk as pk2), both pk1 and pk2 have the same ancestor, i.e., pk.
    +    // A node having keys means: 1.it generates keys by itself 2.all ancestors from it's upstream
    +    // nodes have been preserved even though the ancestors have been duplicated.
    +    def visit(node: RelNode): Option[List[(String, String)]] = {
           node match {
             case c: DataStreamCalc =>
    -          super.visit(node, ordinal, parent)
    -          // check if input has keys
    -          if (keys.isDefined) {
    -            // track keys forward
    +          val keyAncestors = visit(node.getInput(0))
    --- End diff --
    
    `keyAncestors` -> `inputKeys`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132470085
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/UpdatingPlanCheckerTest.scala ---
    @@ -0,0 +1,203 @@
    +/*
    + * 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
    +
    +import org.apache.flink.table.api.Table
    +import org.apache.flink.table.plan.util.UpdatingPlanChecker
    +import org.apache.flink.table.utils.StreamTableTestUtil
    +import org.junit.Assert._
    +import org.apache.flink.table.api.scala._
    +import org.apache.flink.api.scala._
    +import org.junit.Test
    +
    +
    +class UpdatingPlanCheckerTest {
    +
    +  @Test
    +  def testSelect(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(String, Int)]('word, 'number)
    +    val resultTable = table.select('word, 'number)
    +
    +    util.verifyTableUniqueKey(resultTable, Nil)
    +  }
    +
    +  @Test
    +  def testGroupByWithoutKey(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(String, Int)]('word, 'number)
    +
    +    val resultTable = table
    +      .groupBy('word)
    +      .select('number.count)
    +
    +    util.verifyTableUniqueKey(resultTable, Nil)
    +  }
    +
    +  @Test
    +  def testGroupBy(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(String, Int)]('word, 'number)
    +
    +    val resultTable = table
    +      .groupBy('word)
    +      .select('word, 'number.count)
    +
    +    util.verifyTableUniqueKey(resultTable, Seq("word"))
    +  }
    +
    +  @Test
    +  def testGroupByWithDumplicateKey(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(String, Int)]('word, 'number)
    +
    +    val resultTable = table
    +      .groupBy('word)
    +      .select('word as 'word1, 'word as 'word2, 'number.count)
    +
    +    util.verifyTableUniqueKey(resultTable, Seq("word1", "word2"))
    +  }
    +
    +  //1. join key = left key = right key
    +  @Test
    +  def testJoinKeysEqualsleftAndRightKeys(): Unit = {
    --- End diff --
    
    Please add test cases for the changes that I proposed in `UpdatingPlanChecker`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r152817078
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/JoinITCase.scala ---
    @@ -30,6 +30,8 @@ import org.apache.flink.table.api.{TableEnvironment, Types}
     import org.apache.flink.table.expressions.Null
     import org.apache.flink.table.runtime.utils.{StreamITCase, StreamingWithStateTestBase}
     import org.apache.flink.types.Row
    +import org.hamcrest.CoreMatchers
    --- End diff --
    
    Remove unused import.


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r146104539
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamJoinRule.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.plan.rules.datastream
    +
    +import org.apache.calcite.plan.{RelOptRule, 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.rex.{RexCall, RexInputRef, RexNode}
    +import org.apache.flink.table.api.TableConfig
    +import org.apache.flink.table.plan.nodes.FlinkConventions
    +import org.apache.flink.table.plan.nodes.datastream.DataStreamJoin
    +import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
    +import org.apache.flink.table.plan.schema.{RowSchema, TimeIndicatorRelDataType}
    +import org.apache.flink.table.runtime.join.WindowJoinUtil
    +
    +import scala.collection.JavaConverters._
    +
    +class DataStreamJoinRule
    +  extends ConverterRule(
    +    classOf[FlinkLogicalJoin],
    +    FlinkConventions.LOGICAL,
    +    FlinkConventions.DATASTREAM,
    +    "DataStreamJoinRule") {
    +
    +  /**
    +    * Checks if an expression accesses a time attribute.
    +    *
    +    * @param expr The expression to check.
    +    * @param inputType The input type of the expression.
    +    * @return True, if the expression accesses a time attribute. False otherwise.
    +    */
    +  def accessesTimeAttribute(expr: RexNode, inputType: RelDataType): Boolean = {
    +    expr match {
    +      case i: RexInputRef =>
    +        val accessedType = inputType.getFieldList.get(i.getIndex).getType
    +        accessedType match {
    +          case _: TimeIndicatorRelDataType => true
    +          case _ => false
    +        }
    +      case c: RexCall =>
    +        c.operands.asScala.exists(accessesTimeAttribute(_, inputType))
    +      case _ => false
    +    }
    +  }
    +
    +  override def matches(call: RelOptRuleCall): Boolean = {
    +    val join: FlinkLogicalJoin = call.rel(0).asInstanceOf[FlinkLogicalJoin]
    +    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)
    +
    +    // remaining predicate must not access time attributes
    +    val remainingPredsAccessTime = remainingPreds.isDefined &&
    +      accessesTimeAttribute(remainingPreds.get, join.getRowType)
    --- End diff --
    
    Why this condition must be held? What if there exist a predicate that defines only one bound of the time-window?


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r148948528
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamJoinRule.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.plan.rules.datastream
    +
    +import org.apache.calcite.plan.{RelOptRule, 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.rex.{RexCall, RexInputRef, RexNode}
    +import org.apache.flink.table.api.TableConfig
    +import org.apache.flink.table.plan.nodes.FlinkConventions
    +import org.apache.flink.table.plan.nodes.datastream.DataStreamJoin
    +import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
    +import org.apache.flink.table.plan.schema.{RowSchema, TimeIndicatorRelDataType}
    +import org.apache.flink.table.runtime.join.WindowJoinUtil
    +
    +import scala.collection.JavaConverters._
    +
    +class DataStreamJoinRule
    +  extends ConverterRule(
    +    classOf[FlinkLogicalJoin],
    +    FlinkConventions.LOGICAL,
    +    FlinkConventions.DATASTREAM,
    +    "DataStreamJoinRule") {
    +
    +  /**
    +    * Checks if an expression accesses a time attribute.
    +    *
    +    * @param expr The expression to check.
    +    * @param inputType The input type of the expression.
    +    * @return True, if the expression accesses a time attribute. False otherwise.
    +    */
    +  def accessesTimeAttribute(expr: RexNode, inputType: RelDataType): Boolean = {
    +    expr match {
    +      case i: RexInputRef =>
    +        val accessedType = inputType.getFieldList.get(i.getIndex).getType
    +        accessedType match {
    +          case _: TimeIndicatorRelDataType => true
    +          case _ => false
    +        }
    +      case c: RexCall =>
    +        c.operands.asScala.exists(accessesTimeAttribute(_, inputType))
    +      case _ => false
    +    }
    +  }
    +
    +  override def matches(call: RelOptRuleCall): Boolean = {
    +    val join: FlinkLogicalJoin = call.rel(0).asInstanceOf[FlinkLogicalJoin]
    +    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)
    +
    +    // remaining predicate must not access time attributes
    +    val remainingPredsAccessTime = remainingPreds.isDefined &&
    +      accessesTimeAttribute(remainingPreds.get, join.getRowType)
    --- End diff --
    
    I'd suggest to treat the time attributes as common fields since "half a window" means nothing.


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r146104323
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamJoin.scala ---
    @@ -0,0 +1,211 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.table.plan.nodes.datastream
    +
    +import org.apache.calcite.plan._
    +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.join.DataStreamInnerJoin
    +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.
    +  */
    +class DataStreamJoin(
    +    cluster: RelOptCluster,
    +    traitSet: RelTraitSet,
    +    leftNode: RelNode,
    +    rightNode: RelNode,
    +    joinCondition: RexNode,
    +    joinInfo: JoinInfo,
    +    joinType: JoinRelType,
    +    leftSchema: RowSchema,
    +    rightSchema: RowSchema,
    +    schema: RowSchema,
    +    ruleDescription: String)
    +  extends BiRel(cluster, traitSet, leftNode, rightNode)
    +          with CommonJoin
    +          with DataStreamRel {
    +
    +  override def deriveRowType(): RelDataType = schema.relDataType
    +
    +  override def needsUpdatesAsRetraction: Boolean = true
    +
    +  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
    +    new DataStreamJoin(
    +      cluster,
    +      traitSet,
    +      inputs.get(0),
    +      inputs.get(1),
    +      joinCondition,
    +      joinInfo,
    +      joinType,
    +      leftSchema,
    +      rightSchema,
    +      schema,
    +      ruleDescription)
    +  }
    +
    +  def getJoinInfo: JoinInfo = joinInfo
    +
    +  def getJoinType: JoinRelType = joinType
    +
    +  override def toString: String = {
    +    joinToString(
    +      schema.relDataType,
    +      joinCondition,
    +      joinType,
    +      getExpressionString)
    +  }
    +
    +  override def explainTerms(pw: RelWriter): RelWriter = {
    +    joinExplainTerms(
    +      super.explainTerms(pw),
    +      schema.relDataType,
    +      joinCondition,
    +      joinType,
    +      getExpressionString)
    +  }
    +
    +  override def translateToPlan(
    +      tableEnv: StreamTableEnvironment,
    +      queryConfig: StreamQueryConfig): DataStream[CRow] = {
    +
    +    val config = tableEnv.getConfig
    +    val returnType = schema.typeInfo
    +    val keyPairs = joinInfo.pairs().toList
    +
    +    // get the equality keys
    +    val leftKeys = ArrayBuffer.empty[Int]
    +    val rightKeys = ArrayBuffer.empty[Int]
    +    if (keyPairs.isEmpty) {
    +      // if no equality keys => not supported
    +      throw TableException(
    +        "Joins should have at least one equality condition.\n" +
    +          s"\tLeft: ${left.toString},\n" +
    +          s"\tRight: ${right.toString},\n" +
    +          s"\tCondition: (${joinConditionToString(schema.relDataType,
    +             joinCondition, getExpressionString)})"
    +      )
    +    }
    +    else {
    +      // at least one equality expression
    +      val leftFields = left.getRowType.getFieldList
    +      val rightFields = right.getRowType.getFieldList
    +
    +      keyPairs.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 {
    +          throw TableException(
    +            "Equality join predicate on incompatible types.\n" +
    +              s"\tLeft: ${left.toString},\n" +
    +              s"\tRight: ${right.toString},\n" +
    +              s"\tCondition: (${joinConditionToString(schema.relDataType,
    +                joinCondition, getExpressionString)})"
    +          )
    +        }
    +      })
    +    }
    +
    +    val leftDataStream =
    +      left.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
    +    val rightDataStream =
    +      right.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
    +
    +    val (connectOperator, nullCheck) = joinType match {
    +      case JoinRelType.INNER => (leftDataStream.connect(rightDataStream), false)
    +      case _ => throw new UnsupportedOperationException(s"An Unsupported JoinType [ $joinType ]")
    +    }
    +
    +    if (nullCheck && !config.getNullCheck) {
    +      throw TableException("Null check in TableConfig must be enabled for outer joins.")
    --- End diff --
    
    The outer joins have not been implemented yet.


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132441331
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/JoinITCase.scala ---
    @@ -102,5 +103,51 @@ class JoinITCase extends StreamingWithStateTestBase {
         env.execute()
       }
     
    +  /** test process time non-window inner join **/
    +  @Test
    +  def testProcessTimeNonWindowInnerJoin(): Unit = {
    +    val env = StreamExecutionEnvironment.getExecutionEnvironment
    +    val tEnv = TableEnvironment.getTableEnvironment(env)
    +    env.setStateBackend(getStateBackend)
    +    StreamITCase.clear
    +    env.setParallelism(1)
    +
    +    val data1 = new mutable.MutableList[(Int, Long, String)]
    +    data1.+=((1, 1L, "Hi1"))
    +    data1.+=((1, 2L, "Hi2"))
    +    data1.+=((1, 2L, "Hi2"))
    +    data1.+=((1, 5L, "Hi3"))
    +    data1.+=((2, 7L, "Hi5"))
    +    data1.+=((1, 9L, "Hi6"))
    +    data1.+=((1, 8L, "Hi8"))
    +
    +    val data2 = new mutable.MutableList[(Int, Long, String)]
    +    data2.+=((1, 1L, "HiHi"))
    +    data2.+=((2, 2L, "HeHe"))
    +
    +    val t1 = env.fromCollection(data1).toTable(tEnv, 'a, 'b, 'c)
    +    val t2 = env.fromCollection(data2).toTable(tEnv, 'a, 'b, 'c)
    +
    +    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 " +
    --- End diff --
    
    Please format the query to make it easier to read:
    
    ```
    SELECT t2.a, t2.c, t1.c 
    FROM 
      T1 as t1 JOIN T2 as t2 
        ON t1.a = t2.a AND t1.b > t2.b
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132300560
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeNonWindowInnerJoin.scala ---
    @@ -0,0 +1,275 @@
    +/*
    + * 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.RichFlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.java.typeutils.{ResultTypeQueryable, TupleTypeInfo}
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.runtime.CRowWrappingMultiOuputCollector
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
    +
    +/**
    +  * Connect data for left stream and right stream. Only use for innerJoin.
    +  *
    +  * @param joiner           join function
    +  * @param leftType         the input type of left stream
    +  * @param rightType        the input type of right stream
    +  * @param resultType       the output type of join
    +  * @param queryConfig
    +  */
    +class ProcTimeNonWindowInnerJoin(
    +    joiner: RichFlatJoinFunction[Row, Row, Row],
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    resultType: TypeInformation[CRow],
    +    queryConfig: StreamQueryConfig) extends
    +  CoProcessFunction[CRow, CRow, CRow] with ResultTypeQueryable[CRow] {
    +
    +
    +  // state to hold left stream element
    +  private var leftState: MapState[Row, JTuple2[Int, Long]] = null
    +  // state to hold right stream element
    +  private var rightState: MapState[Row, JTuple2[Int, Long]] = null
    +  private var cRowWrapper: CRowWrappingMultiOuputCollector = null
    +
    +  private val minRetentionTime: Long = queryConfig.getMinIdleStateRetentionTime
    +  private val maxRetentionTime: Long = queryConfig.getMaxIdleStateRetentionTime
    +  private val stateCleaningEnabled: Boolean = minRetentionTime > 1
    +
    +  // state to record last timer of left stream, 0 means no timer
    +  private var timerState1: ValueState[Long] = _
    +  // state to record last timer of right stream, 0 means no timer
    +  private var timerState2: ValueState[Long] = _
    +
    +
    +  override def open(parameters: Configuration): Unit = {
    +    // initialize left and right state
    +    val tupleTypeInfo = new TupleTypeInfo[JTuple2[Int, Long]](Types.INT, Types.LONG)
    +    val leftStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "left", leftType, tupleTypeInfo)
    +    val rightStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "right", rightType, tupleTypeInfo)
    +    leftState = getRuntimeContext.getMapState(leftStateDescriptor)
    +    rightState = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // initialize timer state
    +    val valueStateDescriptor1 = new ValueStateDescriptor[Long]("timervaluestate1", classOf[Long])
    +    timerState1 = getRuntimeContext.getState(valueStateDescriptor1)
    +    val valueStateDescriptor2 = new ValueStateDescriptor[Long]("timervaluestate2", classOf[Long])
    +    timerState2 = getRuntimeContext.getState(valueStateDescriptor2)
    +
    +    cRowWrapper = new CRowWrappingMultiOuputCollector()
    +    joiner.setRuntimeContext(getRuntimeContext)
    +    joiner.open(parameters)
    +  }
    +
    +  /**
    +    * Process left stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx   The ctx to register timer or get current time
    +    * @param out   The collector for returning result values.
    +    *
    +    */
    +  override def processElement1(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, timerState1, leftState, rightState, true)
    +  }
    +
    +  /**
    +    * Process right stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx   The ctx to register timer or get current time
    +    * @param out   The collector for returning result values.
    +    *
    +    */
    +  override def processElement2(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, timerState2, rightState, leftState, false)
    +  }
    +
    +
    +  /**
    +    * Called when a processing timer trigger.
    +    * Expire left/right records which are expired in left and right state.
    +    *
    +    * @param timestamp The timestamp of the firing timer.
    +    * @param ctx       The ctx to register timer or get current time
    +    * @param out       The collector for returning result values.
    +    */
    +  override def onTimer(
    +      timestamp: Long,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +
    +    if (stateCleaningEnabled && timerState1.value == timestamp) {
    +      expireOutTimeRow(
    +        timestamp,
    +        leftState,
    +        timerState1,
    +        ctx
    +      )
    +    }
    +
    +    if (stateCleaningEnabled && timerState2.value == timestamp) {
    +      expireOutTimeRow(
    +        timestamp,
    +        rightState,
    +        timerState2,
    +        ctx
    +      )
    +    }
    +  }
    +
    +
    +  def getNewExpiredTime(
    +      curProcessTime: Long,
    +      oldExpiredTime: Long): Long = {
    +
    +    var newExpiredTime = oldExpiredTime
    +    if (stateCleaningEnabled) {
    +      if (-1 == oldExpiredTime || (curProcessTime + minRetentionTime) > oldExpiredTime) {
    +        newExpiredTime = curProcessTime + maxRetentionTime
    +      }
    +    }
    +    newExpiredTime
    +  }
    +
    +  /**
    +    * Puts or Retract an element from the input stream into state and search the other state to
    +    * output records meet the condition. Records will be expired in state if state retention time
    +    * has been specified.
    +    */
    +  def processElement(
    +      value: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow],
    +      timerState: ValueState[Long],
    +      currentSideState: MapState[Row, JTuple2[Int, Long]],
    +      otherSideState: MapState[Row, JTuple2[Int, Long]],
    +      isLeft: Boolean): Unit = {
    +
    +    cRowWrapper.out = out
    +    cRowWrapper.setChange(value.change)
    +
    +    val curProcessTime = ctx.timerService.currentProcessingTime
    +    var oldCnt = 0
    --- End diff --
    
    make this a `val` as follows:
    
    ```
    val (oldCnt, oldExpiredTime) = if (currentRowCntAndExpiredTime != null) {
      (currentRowCntAndExpiredTime.f0, currentRowCntAndExpiredTime.f1)
    } else {
      (0, -1L)
    }
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r152811691
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/DataStreamInnerJoin.scala ---
    @@ -0,0 +1,285 @@
    +/*
    + * 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._
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.java.typeutils.TupleTypeInfo
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.runtime.CRowWrappingMultiOuputCollector
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
    +import org.slf4j.LoggerFactory
    +import org.apache.flink.table.codegen.Compiler
    +
    +
    +/**
    +  * Connect data for left stream and right stream. Only use for innerJoin.
    +  *
    +  * @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
    +  */
    +class DataStreamInnerJoin(
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    resultType: TypeInformation[CRow],
    +    genJoinFuncName: String,
    +    genJoinFuncCode: String,
    +    queryConfig: StreamQueryConfig)
    +  extends CoProcessFunction[CRow, CRow, CRow]
    +          with Compiler[FlatJoinFunction[Row, Row, Row]] {
    +
    +  // state to hold left stream element
    +  private var leftState: MapState[Row, JTuple2[Int, Long]] = _
    +  // state to hold right stream element
    +  private var rightState: MapState[Row, JTuple2[Int, Long]] = _
    +  private var cRowWrapper: CRowWrappingMultiOuputCollector = _
    +
    +  private val minRetentionTime: Long = queryConfig.getMinIdleStateRetentionTime
    +  private val maxRetentionTime: Long = queryConfig.getMaxIdleStateRetentionTime
    +  private val stateCleaningEnabled: Boolean = minRetentionTime > 1
    +
    +  // state to record last timer of left stream, 0 means no timer
    +  private var leftTimer: ValueState[Long] = _
    +  // state to record last timer of right stream, 0 means no timer
    +  private var rightTimer: ValueState[Long] = _
    +
    +  // other condition function
    +  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
    +
    +  val LOG = LoggerFactory.getLogger(this.getClass)
    +
    +  override def open(parameters: Configuration): Unit = {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +                s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    joinFunction = clazz.newInstance()
    +
    +    // initialize left and right state, the first element of tuple2 indicates how many rows of
    +    // this row, while the second element represents the expired time of this row.
    +    val tupleTypeInfo = new TupleTypeInfo[JTuple2[Int, Long]](Types.INT, Types.LONG)
    +    val leftStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "left", leftType, tupleTypeInfo)
    +    val rightStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "right", rightType, tupleTypeInfo)
    +    leftState = getRuntimeContext.getMapState(leftStateDescriptor)
    +    rightState = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // initialize timer state
    +    val valueStateDescriptor1 = new ValueStateDescriptor[Long]("timervaluestate1", classOf[Long])
    +    leftTimer = getRuntimeContext.getState(valueStateDescriptor1)
    +    val valueStateDescriptor2 = new ValueStateDescriptor[Long]("timervaluestate2", classOf[Long])
    +    rightTimer = getRuntimeContext.getState(valueStateDescriptor2)
    +
    +    cRowWrapper = new CRowWrappingMultiOuputCollector()
    +  }
    +
    +  /**
    +    * Process left stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx    The ctx to register timer or get current time
    +    * @param out    The collector for returning result values.
    +    *
    +    */
    +  override def processElement1(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, leftTimer, leftState, rightState, true)
    +  }
    +
    +  /**
    +    * Process right stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx    The ctx to register timer or get current time
    +    * @param out    The collector for returning result values.
    +    *
    +    */
    +  override def processElement2(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, rightTimer, rightState, leftState, false)
    +  }
    +
    +
    +  /**
    +    * Called when a processing timer trigger.
    +    * Expire left/right records which are expired in left and right state.
    +    *
    +    * @param timestamp The timestamp of the firing timer.
    +    * @param ctx       The ctx to register timer or get current time
    +    * @param out       The collector for returning result values.
    +    */
    +  override def onTimer(
    +      timestamp: Long,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +
    +    if (stateCleaningEnabled && leftTimer.value == timestamp) {
    +      expireOutTimeRow(
    +        timestamp,
    +        leftState,
    +        leftTimer,
    +        ctx
    +      )
    +    }
    +
    +    if (stateCleaningEnabled && rightTimer.value == timestamp) {
    +      expireOutTimeRow(
    +        timestamp,
    +        rightState,
    +        rightTimer,
    +        ctx
    +      )
    +    }
    +  }
    +
    +
    +  def getNewExpiredTime(
    +      curProcessTime: Long,
    +      oldExpiredTime: Long): Long = {
    +
    +    if (stateCleaningEnabled && curProcessTime + minRetentionTime > oldExpiredTime) {
    +      curProcessTime + maxRetentionTime
    +    } else {
    +      oldExpiredTime
    +    }
    +  }
    +
    +  /**
    +    * Puts or Retract an element from the input stream into state and search the other state to
    +    * output records meet the condition. Records will be expired in state if state retention time
    +    * has been specified.
    +    */
    +  def processElement(
    +      value: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow],
    +      timerState: ValueState[Long],
    +      currentSideState: MapState[Row, JTuple2[Int, Long]],
    +      otherSideState: MapState[Row, JTuple2[Int, Long]],
    +      isLeft: Boolean): Unit = {
    +
    +    cRowWrapper.setCollector(out)
    +    cRowWrapper.setChange(value.change)
    +
    +    val curProcessTime = ctx.timerService.currentProcessingTime
    +    val oldCntAndExpiredTime = currentSideState.get(value.row)
    +    val cntAndExpiredTime = if (null == oldCntAndExpiredTime) {
    +      JTuple2.of(0, -1L)
    +    } else {
    +      oldCntAndExpiredTime
    +    }
    +
    +    cntAndExpiredTime.f1 = getNewExpiredTime(curProcessTime, cntAndExpiredTime.f1)
    +    if (stateCleaningEnabled && timerState.value() == 0) {
    +      timerState.update(cntAndExpiredTime.f1)
    +      ctx.timerService().registerProcessingTimeTimer(cntAndExpiredTime.f1)
    +    }
    +
    +    // update current side stream state
    +    if (!value.change) {
    +      cntAndExpiredTime.f0 = cntAndExpiredTime.f0 - 1
    +      if (cntAndExpiredTime.f0 <= 0) {
    +        currentSideState.remove(value.row)
    +      } else {
    +        currentSideState.put(value.row, cntAndExpiredTime)
    +      }
    +    } else {
    +      cntAndExpiredTime.f0 = cntAndExpiredTime.f0 + 1
    +      currentSideState.put(value.row, cntAndExpiredTime)
    +    }
    +
    +    val otherSideRowsIterator = otherSideState.keys().iterator()
    +    // join other side data
    +    while (otherSideRowsIterator.hasNext) {
    +      val otherSideRow = otherSideRowsIterator.next()
    +      val cntAndExpiredTime = otherSideState.get(otherSideRow)
    +      // join
    +      cRowWrapper.setTimes(cntAndExpiredTime.f0)
    +      if (isLeft) {
    +        joinFunction.join(value.row, otherSideRow, cRowWrapper)
    +      } else {
    +        joinFunction.join(otherSideRow, value.row, cRowWrapper)
    +      }
    +      // clear expired data. Note: clear after join to keep closer to the original semantics
    +      if (stateCleaningEnabled && curProcessTime >= cntAndExpiredTime.f1) {
    +        otherSideRowsIterator.remove()
    +      }
    +    }
    +  }
    +
    +
    +  /**
    +    * Removes records which are expired from the state. Registers a new timer if the state still
    +    * holds records after the clean-up.
    +    */
    +  private def expireOutTimeRow(
    +      curTime: Long,
    +      rowMapState: MapState[Row, JTuple2[Int, Long]],
    +      timerState: ValueState[Long],
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext): Unit = {
    +
    +    val keyIter = rowMapState.keys().iterator()
    --- End diff --
    
    Same question than above. Iterate over entries?


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r152761559
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/DataStreamInnerJoin.scala ---
    @@ -0,0 +1,287 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +
    +package org.apache.flink.table.runtime.join
    +
    +import org.apache.flink.api.common.functions.FlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.java.typeutils.TupleTypeInfo
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.runtime.CRowWrappingMultiOuputCollector
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
    +import org.slf4j.LoggerFactory
    +import org.apache.flink.table.codegen.Compiler
    +
    +
    +/**
    +  * Connect data for left stream and right stream. Only use for innerJoin.
    +  *
    +  * @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
    +  */
    +class DataStreamInnerJoin(
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    resultType: TypeInformation[CRow],
    +    genJoinFuncName: String,
    +    genJoinFuncCode: String,
    +    queryConfig: StreamQueryConfig)
    +  extends CoProcessFunction[CRow, CRow, CRow]
    +          with Compiler[FlatJoinFunction[Row, Row, Row]] {
    +
    +  // state to hold left stream element
    +  private var leftState: MapState[Row, JTuple2[Int, Long]] = _
    +  // state to hold right stream element
    +  private var rightState: MapState[Row, JTuple2[Int, Long]] = _
    +  private var cRowWrapper: CRowWrappingMultiOuputCollector = _
    +
    +  private val minRetentionTime: Long = queryConfig.getMinIdleStateRetentionTime
    +  private val maxRetentionTime: Long = queryConfig.getMaxIdleStateRetentionTime
    +  private val stateCleaningEnabled: Boolean = minRetentionTime > 1
    +
    +  // state to record last timer of left stream, 0 means no timer
    +  private var leftTimer: ValueState[Long] = _
    +  // state to record last timer of right stream, 0 means no timer
    +  private var rightTimer: ValueState[Long] = _
    +
    +  // other condition function
    +  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
    +
    +  val LOG = LoggerFactory.getLogger(this.getClass)
    +
    +  override def open(parameters: Configuration): Unit = {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +                s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    joinFunction = clazz.newInstance()
    +
    +    // initialize left and right state, the first element of tuple2 indicates how many rows of
    +    // this row, while the second element represents the expired time of this row.
    +    val tupleTypeInfo = new TupleTypeInfo[JTuple2[Int, Long]](Types.INT, Types.LONG)
    +    val leftStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "left", leftType, tupleTypeInfo)
    +    val rightStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "right", rightType, tupleTypeInfo)
    +    leftState = getRuntimeContext.getMapState(leftStateDescriptor)
    +    rightState = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // initialize timer state
    +    val valueStateDescriptor1 = new ValueStateDescriptor[Long]("timervaluestate1", classOf[Long])
    +    leftTimer = getRuntimeContext.getState(valueStateDescriptor1)
    +    val valueStateDescriptor2 = new ValueStateDescriptor[Long]("timervaluestate2", classOf[Long])
    +    rightTimer = getRuntimeContext.getState(valueStateDescriptor2)
    +
    +    cRowWrapper = new CRowWrappingMultiOuputCollector()
    +  }
    +
    +  /**
    +    * Process left stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx    The ctx to register timer or get current time
    +    * @param out    The collector for returning result values.
    +    *
    +    */
    +  override def processElement1(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, leftTimer, leftState, rightState, true)
    +  }
    +
    +  /**
    +    * Process right stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx    The ctx to register timer or get current time
    +    * @param out    The collector for returning result values.
    +    *
    +    */
    +  override def processElement2(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, rightTimer, rightState, leftState, false)
    +  }
    +
    +
    +  /**
    +    * Called when a processing timer trigger.
    +    * Expire left/right records which are expired in left and right state.
    +    *
    +    * @param timestamp The timestamp of the firing timer.
    +    * @param ctx       The ctx to register timer or get current time
    +    * @param out       The collector for returning result values.
    +    */
    +  override def onTimer(
    +      timestamp: Long,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +
    +    if (stateCleaningEnabled && leftTimer.value == timestamp) {
    --- End diff --
    
    +1


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132421076
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala ---
    @@ -232,4 +238,111 @@ class JoinHarnessTest extends HarnessTestBase{
         testHarness.close()
       }
     
    +  @Test
    +  def testProcTimeNonWindowInnerJoin() {
    +
    +    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 joinFun = new FlatJoinRunner[Row, Row, Row](
    +      "TestJoinFunction",
    +      funcCode,
    +      joinReturnType.rowType)
    +
    +    val joinProcessFunc = new ProcTimeNonWindowInnerJoin(
    +      joinFun,
    +      rT,
    +      rT,
    +      joinReturnType,
    +      queryConfig)
    +
    +    val operator: KeyedCoProcessOperator[Integer, CRow, CRow, CRow] =
    +      new KeyedCoProcessOperator[Integer, CRow, CRow, CRow](joinProcessFunc)
    +    val testHarness: KeyedTwoInputStreamOperatorTestHarness[Integer, CRow, CRow, CRow] =
    +      new KeyedTwoInputStreamOperatorTestHarness[Integer, CRow, CRow, CRow](
    +        operator,
    +        new TupleRowKeySelector[Integer](0),
    +        new TupleRowKeySelector[Integer](0),
    +        BasicTypeInfo.INT_TYPE_INFO,
    +        1, 1, 0)
    +
    +    testHarness.open()
    +
    +    // left stream input
    +    testHarness.setProcessingTime(1)
    +    testHarness.processElement1(new StreamRecord(
    +      CRow(Row.of(1: JInt, "aaa"), true), 1))
    --- End diff --
    
    If you are not using the `StreamRecord` timestamp field, you can omit it, i.e.,
    
    ```
    new StreamRecord(CRow(Row.of(1: JInt, "aaa"), true)))
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132284044
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeNonWindowInnerJoin.scala ---
    @@ -0,0 +1,275 @@
    +/*
    + * 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.RichFlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.java.typeutils.{ResultTypeQueryable, TupleTypeInfo}
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.runtime.CRowWrappingMultiOuputCollector
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
    +
    +/**
    +  * Connect data for left stream and right stream. Only use for innerJoin.
    +  *
    +  * @param joiner           join function
    +  * @param leftType         the input type of left stream
    +  * @param rightType        the input type of right stream
    +  * @param resultType       the output type of join
    +  * @param queryConfig
    +  */
    +class ProcTimeNonWindowInnerJoin(
    +    joiner: RichFlatJoinFunction[Row, Row, Row],
    --- End diff --
    
    We can directly compile and call the code-gen'd `FlatJoinFunction` and avoid an additional wrapping layer


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r152825553
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/UpdatingPlanChecker.scala ---
    @@ -116,14 +135,100 @@ object UpdatingPlanChecker {
               val windowStartEnd = w.getWindowProperties.map(_.name)
               // we have only a unique key if at least one window property is selected
               if (windowStartEnd.nonEmpty) {
    -            keys = Some(groupKeys ++ windowStartEnd)
    +            val smallestAttribute = windowStartEnd.sorted.head
    --- End diff --
    
    Replace with `windowStartEnd.min`. Can you explain this logic? Selecting only the first string looks a bit weird.


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r146105359
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/DataStreamInnerJoin.scala ---
    @@ -0,0 +1,287 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +
    +package org.apache.flink.table.runtime.join
    +
    +import org.apache.flink.api.common.functions.FlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.java.typeutils.TupleTypeInfo
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.runtime.CRowWrappingMultiOuputCollector
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
    +import org.slf4j.LoggerFactory
    +import org.apache.flink.table.codegen.Compiler
    +
    +
    +/**
    +  * Connect data for left stream and right stream. Only use for innerJoin.
    +  *
    +  * @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
    +  */
    +class DataStreamInnerJoin(
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    resultType: TypeInformation[CRow],
    +    genJoinFuncName: String,
    +    genJoinFuncCode: String,
    +    queryConfig: StreamQueryConfig)
    +  extends CoProcessFunction[CRow, CRow, CRow]
    +          with Compiler[FlatJoinFunction[Row, Row, Row]] {
    +
    +  // state to hold left stream element
    +  private var leftState: MapState[Row, JTuple2[Int, Long]] = _
    +  // state to hold right stream element
    +  private var rightState: MapState[Row, JTuple2[Int, Long]] = _
    +  private var cRowWrapper: CRowWrappingMultiOuputCollector = _
    +
    +  private val minRetentionTime: Long = queryConfig.getMinIdleStateRetentionTime
    +  private val maxRetentionTime: Long = queryConfig.getMaxIdleStateRetentionTime
    +  private val stateCleaningEnabled: Boolean = minRetentionTime > 1
    +
    +  // state to record last timer of left stream, 0 means no timer
    +  private var leftTimer: ValueState[Long] = _
    +  // state to record last timer of right stream, 0 means no timer
    +  private var rightTimer: ValueState[Long] = _
    +
    +  // other condition function
    +  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
    +
    +  val LOG = LoggerFactory.getLogger(this.getClass)
    +
    +  override def open(parameters: Configuration): Unit = {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +                s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    joinFunction = clazz.newInstance()
    +
    +    // initialize left and right state, the first element of tuple2 indicates how many rows of
    +    // this row, while the second element represents the expired time of this row.
    +    val tupleTypeInfo = new TupleTypeInfo[JTuple2[Int, Long]](Types.INT, Types.LONG)
    +    val leftStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "left", leftType, tupleTypeInfo)
    +    val rightStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "right", rightType, tupleTypeInfo)
    +    leftState = getRuntimeContext.getMapState(leftStateDescriptor)
    +    rightState = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // initialize timer state
    +    val valueStateDescriptor1 = new ValueStateDescriptor[Long]("timervaluestate1", classOf[Long])
    +    leftTimer = getRuntimeContext.getState(valueStateDescriptor1)
    +    val valueStateDescriptor2 = new ValueStateDescriptor[Long]("timervaluestate2", classOf[Long])
    +    rightTimer = getRuntimeContext.getState(valueStateDescriptor2)
    +
    +    cRowWrapper = new CRowWrappingMultiOuputCollector()
    +  }
    +
    +  /**
    +    * Process left stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx    The ctx to register timer or get current time
    +    * @param out    The collector for returning result values.
    +    *
    +    */
    +  override def processElement1(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, leftTimer, leftState, rightState, true)
    +  }
    +
    +  /**
    +    * Process right stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx    The ctx to register timer or get current time
    +    * @param out    The collector for returning result values.
    +    *
    +    */
    +  override def processElement2(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, rightTimer, rightState, leftState, false)
    +  }
    +
    +
    +  /**
    +    * Called when a processing timer trigger.
    +    * Expire left/right records which are expired in left and right state.
    +    *
    +    * @param timestamp The timestamp of the firing timer.
    +    * @param ctx       The ctx to register timer or get current time
    +    * @param out       The collector for returning result values.
    +    */
    +  override def onTimer(
    +      timestamp: Long,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +
    +    if (stateCleaningEnabled && leftTimer.value == timestamp) {
    +      expireOutTimeRow(
    +        timestamp,
    +        leftState,
    +        leftTimer,
    +        ctx
    +      )
    +    }
    +
    +    if (stateCleaningEnabled && rightTimer.value == timestamp) {
    +      expireOutTimeRow(
    +        timestamp,
    +        rightState,
    +        rightTimer,
    +        ctx
    +      )
    +    }
    +  }
    +
    +
    +  def getNewExpiredTime(
    +      curProcessTime: Long,
    +      oldExpiredTime: Long): Long = {
    +
    +    var newExpiredTime = oldExpiredTime
    +    if (stateCleaningEnabled) {
    +      if (curProcessTime + minRetentionTime > oldExpiredTime) {
    +        newExpiredTime = curProcessTime + maxRetentionTime
    +      }
    +    }
    +    newExpiredTime
    +  }
    --- End diff --
    
    ```
    if (curProcessTime + minRetentionTime > oldExpiredTime) {
      curProcessTime + maxRetentionTime
    } else {
      oldExpiredTime
    }
    ```


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r146105104
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/DataStreamInnerJoin.scala ---
    @@ -0,0 +1,287 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +
    +package org.apache.flink.table.runtime.join
    +
    +import org.apache.flink.api.common.functions.FlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.java.typeutils.TupleTypeInfo
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.runtime.CRowWrappingMultiOuputCollector
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
    +import org.slf4j.LoggerFactory
    +import org.apache.flink.table.codegen.Compiler
    +
    +
    +/**
    +  * Connect data for left stream and right stream. Only use for innerJoin.
    +  *
    +  * @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
    +  */
    +class DataStreamInnerJoin(
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    resultType: TypeInformation[CRow],
    +    genJoinFuncName: String,
    +    genJoinFuncCode: String,
    +    queryConfig: StreamQueryConfig)
    +  extends CoProcessFunction[CRow, CRow, CRow]
    +          with Compiler[FlatJoinFunction[Row, Row, Row]] {
    +
    +  // state to hold left stream element
    +  private var leftState: MapState[Row, JTuple2[Int, Long]] = _
    +  // state to hold right stream element
    +  private var rightState: MapState[Row, JTuple2[Int, Long]] = _
    +  private var cRowWrapper: CRowWrappingMultiOuputCollector = _
    +
    +  private val minRetentionTime: Long = queryConfig.getMinIdleStateRetentionTime
    +  private val maxRetentionTime: Long = queryConfig.getMaxIdleStateRetentionTime
    +  private val stateCleaningEnabled: Boolean = minRetentionTime > 1
    +
    +  // state to record last timer of left stream, 0 means no timer
    +  private var leftTimer: ValueState[Long] = _
    +  // state to record last timer of right stream, 0 means no timer
    +  private var rightTimer: ValueState[Long] = _
    +
    +  // other condition function
    +  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
    +
    +  val LOG = LoggerFactory.getLogger(this.getClass)
    +
    +  override def open(parameters: Configuration): Unit = {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +                s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    joinFunction = clazz.newInstance()
    +
    +    // initialize left and right state, the first element of tuple2 indicates how many rows of
    +    // this row, while the second element represents the expired time of this row.
    +    val tupleTypeInfo = new TupleTypeInfo[JTuple2[Int, Long]](Types.INT, Types.LONG)
    +    val leftStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "left", leftType, tupleTypeInfo)
    +    val rightStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "right", rightType, tupleTypeInfo)
    +    leftState = getRuntimeContext.getMapState(leftStateDescriptor)
    +    rightState = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // initialize timer state
    +    val valueStateDescriptor1 = new ValueStateDescriptor[Long]("timervaluestate1", classOf[Long])
    +    leftTimer = getRuntimeContext.getState(valueStateDescriptor1)
    +    val valueStateDescriptor2 = new ValueStateDescriptor[Long]("timervaluestate2", classOf[Long])
    +    rightTimer = getRuntimeContext.getState(valueStateDescriptor2)
    +
    +    cRowWrapper = new CRowWrappingMultiOuputCollector()
    +  }
    +
    +  /**
    +    * Process left stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx    The ctx to register timer or get current time
    +    * @param out    The collector for returning result values.
    +    *
    +    */
    +  override def processElement1(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, leftTimer, leftState, rightState, true)
    +  }
    +
    +  /**
    +    * Process right stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx    The ctx to register timer or get current time
    +    * @param out    The collector for returning result values.
    +    *
    +    */
    +  override def processElement2(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, rightTimer, rightState, leftState, false)
    +  }
    +
    +
    +  /**
    +    * Called when a processing timer trigger.
    +    * Expire left/right records which are expired in left and right state.
    +    *
    +    * @param timestamp The timestamp of the firing timer.
    +    * @param ctx       The ctx to register timer or get current time
    +    * @param out       The collector for returning result values.
    +    */
    +  override def onTimer(
    +      timestamp: Long,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +
    +    if (stateCleaningEnabled && leftTimer.value == timestamp) {
    +      expireOutTimeRow(
    +        timestamp,
    +        leftState,
    +        leftTimer,
    +        ctx
    +      )
    +    }
    +
    +    if (stateCleaningEnabled && rightTimer.value == timestamp) {
    +      expireOutTimeRow(
    +        timestamp,
    +        rightState,
    +        rightTimer,
    +        ctx
    +      )
    +    }
    +  }
    +
    +
    +  def getNewExpiredTime(
    +      curProcessTime: Long,
    +      oldExpiredTime: Long): Long = {
    +
    +    var newExpiredTime = oldExpiredTime
    +    if (stateCleaningEnabled) {
    +      if (curProcessTime + minRetentionTime > oldExpiredTime) {
    +        newExpiredTime = curProcessTime + maxRetentionTime
    +      }
    +    }
    +    newExpiredTime
    +  }
    +
    +  /**
    +    * Puts or Retract an element from the input stream into state and search the other state to
    +    * output records meet the condition. Records will be expired in state if state retention time
    +    * has been specified.
    +    */
    +  def processElement(
    +      value: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow],
    +      timerState: ValueState[Long],
    +      currentSideState: MapState[Row, JTuple2[Int, Long]],
    +      otherSideState: MapState[Row, JTuple2[Int, Long]],
    +      isLeft: Boolean): Unit = {
    +
    +    cRowWrapper.out = out
    +    cRowWrapper.setChange(value.change)
    --- End diff --
    
    It's a little bit strange to assign the `out` directly while set the `change` via a method.


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132211118
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamJoin.scala ---
    @@ -0,0 +1,212 @@
    +/*
    + * 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.`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.FlatJoinRunner
    +import org.apache.flink.table.runtime.join.ProcTimeNonWindowInnerJoin
    +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.
    +  */
    +class DataStreamJoin(
    +    cluster: RelOptCluster,
    +    traitSet: RelTraitSet,
    +    leftNode: RelNode,
    +    rightNode: RelNode,
    +    joinCondition: RexNode,
    +    joinInfo: JoinInfo,
    +    joinType: JoinRelType,
    +    leftSchema: RowSchema,
    +    rightSchema: RowSchema,
    +    schema: RowSchema,
    +    ruleDescription: String)
    +  extends BiRel(cluster, traitSet, leftNode, rightNode)
    +          with CommonJoin
    +          with DataStreamRel {
    +
    +  override def deriveRowType(): RelDataType = schema.logicalType
    +
    +  override def needsUpdatesAsRetraction: Boolean = true
    +
    +  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
    +    new DataStreamJoin(
    +      cluster,
    +      traitSet,
    +      inputs.get(0),
    +      inputs.get(1),
    +      joinCondition,
    +      joinInfo,
    +      joinType,
    +      leftSchema,
    +      rightSchema,
    +      schema,
    +      ruleDescription)
    +  }
    +
    +  def getJoinInfo: JoinInfo = joinInfo
    +
    +  override def toString: String = {
    +    joinToString(
    +      schema.logicalType,
    +      joinCondition,
    +      joinType,
    +      getExpressionString)
    +  }
    +
    +  override def explainTerms(pw: RelWriter): RelWriter = {
    +    joinExplainTerms(
    +      super.explainTerms(pw),
    +      schema.logicalType,
    +      joinCondition,
    +      joinType,
    +      getExpressionString)
    +  }
    +
    +  override def translateToPlan(
    +      tableEnv: StreamTableEnvironment,
    +      queryConfig: StreamQueryConfig): DataStream[CRow] = {
    +
    +    val config = tableEnv.getConfig
    +    val returnType = schema.physicalTypeInfo
    +    val keyPairs = joinInfo.pairs().toList
    +
    +    // get the equality keys
    +    val leftKeys = ArrayBuffer.empty[Int]
    +    val rightKeys = ArrayBuffer.empty[Int]
    +    if (keyPairs.isEmpty) {
    +      // if no equality keys => not supported
    +      throw TableException(
    +        "Joins should have at least one equality condition.\n" +
    +          s"\tLeft: ${left.toString},\n" +
    +          s"\tRight: ${right.toString},\n" +
    +          s"\tCondition: (${joinConditionToString(schema.logicalType,
    +             joinCondition, getExpressionString)})"
    +      )
    +    }
    +    else {
    +      // at least one equality expression
    +      val leftFields = left.getRowType.getFieldList
    +      val rightFields = right.getRowType.getFieldList
    +
    +      keyPairs.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 {
    +          throw TableException(
    +            "Equality join predicate on incompatible types.\n" +
    +              s"\tLeft: ${left.toString},\n" +
    +              s"\tRight: ${right.toString},\n" +
    +              s"\tCondition: (${joinConditionToString(schema.logicalType,
    +                joinCondition, getExpressionString)})"
    +          )
    +        }
    +      })
    +    }
    +
    +    val leftDataStream =
    +      left.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
    +    val rightDataStream =
    +      right.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
    +
    +    val (connectOperator, nullCheck) = joinType match {
    +      case JoinRelType.INNER => (leftDataStream.connect(rightDataStream), false)
    +      case _ => throw new UnsupportedOperationException(s"An Unsupported JoinType [ $joinType ]")
    +    }
    +
    +    if (nullCheck && !config.getNullCheck) {
    +      throw TableException("Null check in TableConfig must be enabled for outer joins.")
    +    }
    +
    +
    +    val generator = new FunctionCodeGenerator(
    +      config,
    +      nullCheck,
    +      leftSchema.physicalTypeInfo,
    +      Some(rightSchema.physicalTypeInfo))
    +    val conversion = generator.generateConverterResultExpression(
    +      schema.physicalTypeInfo,
    +      schema.physicalType.getFieldNames)
    --- End diff --
    
    `schema.physicalRowFieldNames`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #4471: [FLINK-6094] [table] Implement stream-stream proctime non...

Posted by hequn8128 <gi...@git.apache.org>.
Github user hequn8128 commented on the issue:

    https://github.com/apache/flink/pull/4471
  
    @fhueske  hi fabian, sorry for the late update, i will resolve the conflicts ASAP, a busy weekend :)


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132307133
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/UpdatingPlanCheckerTest.scala ---
    @@ -0,0 +1,203 @@
    +/*
    + * 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
    +
    +import org.apache.flink.table.api.Table
    +import org.apache.flink.table.plan.util.UpdatingPlanChecker
    +import org.apache.flink.table.utils.StreamTableTestUtil
    +import org.junit.Assert._
    +import org.apache.flink.table.api.scala._
    +import org.apache.flink.api.scala._
    +import org.junit.Test
    +
    +
    +class UpdatingPlanCheckerTest {
    +
    +  @Test
    +  def testSelect(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(String, Int)]('word, 'number)
    +    val resultTable = table.select('word, 'number)
    +
    +    util.verifyTableUniqueKey(resultTable, Nil)
    +  }
    +
    +  @Test
    +  def testGroupByWithoutKey(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(String, Int)]('word, 'number)
    +
    +    val resultTable = table
    +      .groupBy('word)
    +      .select('number.count)
    +
    +    util.verifyTableUniqueKey(resultTable, Nil)
    +  }
    +
    +  @Test
    +  def testGroupBy(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(String, Int)]('word, 'number)
    +
    +    val resultTable = table
    +      .groupBy('word)
    +      .select('word, 'number.count)
    +
    +    util.verifyTableUniqueKey(resultTable, Seq("word"))
    +  }
    +
    +  @Test
    +  def testGroupByWithDumplicateKey(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(String, Int)]('word, 'number)
    +
    +    val resultTable = table
    +      .groupBy('word)
    +      .select('word as 'word1, 'word as 'word2, 'number.count)
    +
    +    util.verifyTableUniqueKey(resultTable, Seq("word1", "word2"))
    +  }
    +
    --- End diff --
    
    Please add a groupBy test where one of the key fields is projected out.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r152760126
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/DataStreamInnerJoin.scala ---
    @@ -0,0 +1,285 @@
    +/*
    + * 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._
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.java.typeutils.TupleTypeInfo
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.runtime.CRowWrappingMultiOuputCollector
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
    +import org.slf4j.LoggerFactory
    +import org.apache.flink.table.codegen.Compiler
    +
    +
    +/**
    +  * Connect data for left stream and right stream. Only use for innerJoin.
    +  *
    +  * @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
    +  */
    +class DataStreamInnerJoin(
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    resultType: TypeInformation[CRow],
    +    genJoinFuncName: String,
    +    genJoinFuncCode: String,
    +    queryConfig: StreamQueryConfig)
    +  extends CoProcessFunction[CRow, CRow, CRow]
    +          with Compiler[FlatJoinFunction[Row, Row, Row]] {
    +
    +  // state to hold left stream element
    +  private var leftState: MapState[Row, JTuple2[Int, Long]] = _
    +  // state to hold right stream element
    +  private var rightState: MapState[Row, JTuple2[Int, Long]] = _
    +  private var cRowWrapper: CRowWrappingMultiOuputCollector = _
    +
    +  private val minRetentionTime: Long = queryConfig.getMinIdleStateRetentionTime
    +  private val maxRetentionTime: Long = queryConfig.getMaxIdleStateRetentionTime
    +  private val stateCleaningEnabled: Boolean = minRetentionTime > 1
    +
    +  // state to record last timer of left stream, 0 means no timer
    +  private var leftTimer: ValueState[Long] = _
    +  // state to record last timer of right stream, 0 means no timer
    +  private var rightTimer: ValueState[Long] = _
    +
    +  // other condition function
    +  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
    +
    +  val LOG = LoggerFactory.getLogger(this.getClass)
    --- End diff --
    
    Use the new `with Logging` interface.


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r152830710
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/UpdatingPlanChecker.scala ---
    @@ -116,14 +135,100 @@ object UpdatingPlanChecker {
               val windowStartEnd = w.getWindowProperties.map(_.name)
               // we have only a unique key if at least one window property is selected
               if (windowStartEnd.nonEmpty) {
    -            keys = Some(groupKeys ++ windowStartEnd)
    +            val smallestAttribute = windowStartEnd.sorted.head
    +            Some((groupKeys.map(e => (e, e)) ++ windowStartEnd.map((_, smallestAttribute))).toList)
    +          } else {
    +            None
    +          }
    +
    +        case j: DataStreamJoin =>
    +          val joinType = j.getJoinType
    +          joinType match {
    +            case JoinRelType.INNER => {
    +              // get key(s) for inner join
    +              val lInputKeys = visit(j.getLeft)
    +              val rInputKeys = visit(j.getRight)
    +              if (lInputKeys.isEmpty || rInputKeys.isEmpty) {
    +                None
    +              } else {
    +                // Output of inner join must have keys if left and right both contain key(s).
    +                // Key groups from both side will be merged by join equi-predicates
    +                val lFieldNames: Seq[String] = j.getLeft.getRowType.getFieldNames
    +                val rFieldNames: Seq[String] = j.getRight.getRowType.getFieldNames
    +                val lJoinKeys: Seq[String] = j.getJoinInfo.leftKeys.map(lFieldNames.get(_))
    +                val rJoinKeys: Seq[String] = j.getJoinInfo.rightKeys.map(rFieldNames.get(_))
    +
    +                getOutputKeysForInnerJoin(
    +                  lFieldNames ++ rFieldNames,
    +                  lInputKeys.get ++ rInputKeys.get,
    +                  lJoinKeys.zip(rJoinKeys).toList
    +                )
    +              }
    +            }
    +            case _ => throw new UnsupportedOperationException(
    +              s"An Unsupported JoinType [ $joinType ]")
               }
             case _: DataStreamRel =>
    -          // anything else does not forward keys or might duplicate key, so we can stop
    -          keys = None
    +          // anything else does not forward keys, so we can stop
    +          None
           }
         }
     
    -  }
     
    +    def getOutputKeysForInnerJoin(
    --- End diff --
    
    Please document this method.


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r152773431
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/DataStreamInnerJoin.scala ---
    @@ -0,0 +1,285 @@
    +/*
    + * 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._
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.java.typeutils.TupleTypeInfo
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.runtime.CRowWrappingMultiOuputCollector
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
    +import org.slf4j.LoggerFactory
    +import org.apache.flink.table.codegen.Compiler
    +
    +
    +/**
    +  * Connect data for left stream and right stream. Only use for innerJoin.
    +  *
    +  * @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
    +  */
    +class DataStreamInnerJoin(
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    resultType: TypeInformation[CRow],
    +    genJoinFuncName: String,
    +    genJoinFuncCode: String,
    +    queryConfig: StreamQueryConfig)
    +  extends CoProcessFunction[CRow, CRow, CRow]
    +          with Compiler[FlatJoinFunction[Row, Row, Row]] {
    +
    +  // state to hold left stream element
    +  private var leftState: MapState[Row, JTuple2[Int, Long]] = _
    +  // state to hold right stream element
    +  private var rightState: MapState[Row, JTuple2[Int, Long]] = _
    +  private var cRowWrapper: CRowWrappingMultiOuputCollector = _
    +
    +  private val minRetentionTime: Long = queryConfig.getMinIdleStateRetentionTime
    +  private val maxRetentionTime: Long = queryConfig.getMaxIdleStateRetentionTime
    +  private val stateCleaningEnabled: Boolean = minRetentionTime > 1
    +
    +  // state to record last timer of left stream, 0 means no timer
    +  private var leftTimer: ValueState[Long] = _
    +  // state to record last timer of right stream, 0 means no timer
    +  private var rightTimer: ValueState[Long] = _
    +
    +  // other condition function
    +  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
    +
    +  val LOG = LoggerFactory.getLogger(this.getClass)
    +
    +  override def open(parameters: Configuration): Unit = {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +                s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    joinFunction = clazz.newInstance()
    +
    +    // initialize left and right state, the first element of tuple2 indicates how many rows of
    +    // this row, while the second element represents the expired time of this row.
    +    val tupleTypeInfo = new TupleTypeInfo[JTuple2[Int, Long]](Types.INT, Types.LONG)
    +    val leftStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "left", leftType, tupleTypeInfo)
    +    val rightStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "right", rightType, tupleTypeInfo)
    +    leftState = getRuntimeContext.getMapState(leftStateDescriptor)
    +    rightState = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // initialize timer state
    +    val valueStateDescriptor1 = new ValueStateDescriptor[Long]("timervaluestate1", classOf[Long])
    +    leftTimer = getRuntimeContext.getState(valueStateDescriptor1)
    +    val valueStateDescriptor2 = new ValueStateDescriptor[Long]("timervaluestate2", classOf[Long])
    +    rightTimer = getRuntimeContext.getState(valueStateDescriptor2)
    +
    +    cRowWrapper = new CRowWrappingMultiOuputCollector()
    +  }
    +
    +  /**
    +    * Process left stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx    The ctx to register timer or get current time
    +    * @param out    The collector for returning result values.
    +    *
    +    */
    +  override def processElement1(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, leftTimer, leftState, rightState, true)
    +  }
    +
    +  /**
    +    * Process right stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx    The ctx to register timer or get current time
    +    * @param out    The collector for returning result values.
    +    *
    +    */
    +  override def processElement2(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, rightTimer, rightState, leftState, false)
    +  }
    +
    +
    +  /**
    +    * Called when a processing timer trigger.
    +    * Expire left/right records which are expired in left and right state.
    +    *
    +    * @param timestamp The timestamp of the firing timer.
    +    * @param ctx       The ctx to register timer or get current time
    +    * @param out       The collector for returning result values.
    +    */
    +  override def onTimer(
    +      timestamp: Long,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +
    +    if (stateCleaningEnabled && leftTimer.value == timestamp) {
    +      expireOutTimeRow(
    +        timestamp,
    +        leftState,
    +        leftTimer,
    +        ctx
    +      )
    +    }
    +
    +    if (stateCleaningEnabled && rightTimer.value == timestamp) {
    +      expireOutTimeRow(
    +        timestamp,
    +        rightState,
    +        rightTimer,
    +        ctx
    +      )
    +    }
    +  }
    +
    +
    +  def getNewExpiredTime(
    +      curProcessTime: Long,
    +      oldExpiredTime: Long): Long = {
    +
    +    if (stateCleaningEnabled && curProcessTime + minRetentionTime > oldExpiredTime) {
    +      curProcessTime + maxRetentionTime
    +    } else {
    +      oldExpiredTime
    +    }
    +  }
    +
    +  /**
    +    * Puts or Retract an element from the input stream into state and search the other state to
    +    * output records meet the condition. Records will be expired in state if state retention time
    +    * has been specified.
    +    */
    +  def processElement(
    +      value: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow],
    +      timerState: ValueState[Long],
    +      currentSideState: MapState[Row, JTuple2[Int, Long]],
    +      otherSideState: MapState[Row, JTuple2[Int, Long]],
    +      isLeft: Boolean): Unit = {
    +
    +    cRowWrapper.setCollector(out)
    +    cRowWrapper.setChange(value.change)
    +
    +    val curProcessTime = ctx.timerService.currentProcessingTime
    +    val oldCntAndExpiredTime = currentSideState.get(value.row)
    --- End diff --
    
    This only works given that https://issues.apache.org/jira/browse/FLINK-8139 is fixed.


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132305698
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/UpdatingPlanCheckerTest.scala ---
    @@ -0,0 +1,203 @@
    +/*
    + * 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
    +
    +import org.apache.flink.table.api.Table
    +import org.apache.flink.table.plan.util.UpdatingPlanChecker
    +import org.apache.flink.table.utils.StreamTableTestUtil
    +import org.junit.Assert._
    +import org.apache.flink.table.api.scala._
    +import org.apache.flink.api.scala._
    +import org.junit.Test
    +
    +
    +class UpdatingPlanCheckerTest {
    +
    +  @Test
    +  def testSelect(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(String, Int)]('word, 'number)
    +    val resultTable = table.select('word, 'number)
    +
    +    util.verifyTableUniqueKey(resultTable, Nil)
    +  }
    +
    +  @Test
    +  def testGroupByWithoutKey(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(String, Int)]('word, 'number)
    +
    +    val resultTable = table
    +      .groupBy('word)
    +      .select('number.count)
    +
    +    util.verifyTableUniqueKey(resultTable, Nil)
    +  }
    +
    +  @Test
    +  def testGroupBy(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(String, Int)]('word, 'number)
    +
    +    val resultTable = table
    +      .groupBy('word)
    +      .select('word, 'number.count)
    +
    +    util.verifyTableUniqueKey(resultTable, Seq("word"))
    +  }
    +
    +  @Test
    +  def testGroupByWithDumplicateKey(): Unit = {
    +    val util = new UpdatePlanChecnkerUtil()
    +    val table = util.addTable[(String, Int)]('word, 'number)
    +
    +    val resultTable = table
    +      .groupBy('word)
    +      .select('word as 'word1, 'word as 'word2, 'number.count)
    +
    +    util.verifyTableUniqueKey(resultTable, Seq("word1", "word2"))
    +  }
    +
    +  //1. join key = left key = right key
    +  @Test
    +  def testJoinKeysEqualsleftAndRightKeys(): Unit = {
    --- End diff --
    
    incorrect camel case -> make `Left` upper case


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by hequn8128 <gi...@git.apache.org>.
Github user hequn8128 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r148947768
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/DataStreamInnerJoin.scala ---
    @@ -0,0 +1,287 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +
    +package org.apache.flink.table.runtime.join
    +
    +import org.apache.flink.api.common.functions.FlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.java.typeutils.TupleTypeInfo
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.runtime.CRowWrappingMultiOuputCollector
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
    +import org.slf4j.LoggerFactory
    +import org.apache.flink.table.codegen.Compiler
    +
    +
    +/**
    +  * Connect data for left stream and right stream. Only use for innerJoin.
    +  *
    +  * @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
    +  */
    +class DataStreamInnerJoin(
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    resultType: TypeInformation[CRow],
    +    genJoinFuncName: String,
    +    genJoinFuncCode: String,
    +    queryConfig: StreamQueryConfig)
    +  extends CoProcessFunction[CRow, CRow, CRow]
    +          with Compiler[FlatJoinFunction[Row, Row, Row]] {
    +
    +  // state to hold left stream element
    +  private var leftState: MapState[Row, JTuple2[Int, Long]] = _
    +  // state to hold right stream element
    +  private var rightState: MapState[Row, JTuple2[Int, Long]] = _
    +  private var cRowWrapper: CRowWrappingMultiOuputCollector = _
    +
    +  private val minRetentionTime: Long = queryConfig.getMinIdleStateRetentionTime
    +  private val maxRetentionTime: Long = queryConfig.getMaxIdleStateRetentionTime
    +  private val stateCleaningEnabled: Boolean = minRetentionTime > 1
    +
    +  // state to record last timer of left stream, 0 means no timer
    +  private var leftTimer: ValueState[Long] = _
    +  // state to record last timer of right stream, 0 means no timer
    +  private var rightTimer: ValueState[Long] = _
    +
    +  // other condition function
    +  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
    +
    +  val LOG = LoggerFactory.getLogger(this.getClass)
    +
    +  override def open(parameters: Configuration): Unit = {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +                s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    joinFunction = clazz.newInstance()
    +
    +    // initialize left and right state, the first element of tuple2 indicates how many rows of
    +    // this row, while the second element represents the expired time of this row.
    +    val tupleTypeInfo = new TupleTypeInfo[JTuple2[Int, Long]](Types.INT, Types.LONG)
    +    val leftStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "left", leftType, tupleTypeInfo)
    +    val rightStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "right", rightType, tupleTypeInfo)
    +    leftState = getRuntimeContext.getMapState(leftStateDescriptor)
    +    rightState = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // initialize timer state
    +    val valueStateDescriptor1 = new ValueStateDescriptor[Long]("timervaluestate1", classOf[Long])
    +    leftTimer = getRuntimeContext.getState(valueStateDescriptor1)
    +    val valueStateDescriptor2 = new ValueStateDescriptor[Long]("timervaluestate2", classOf[Long])
    +    rightTimer = getRuntimeContext.getState(valueStateDescriptor2)
    +
    +    cRowWrapper = new CRowWrappingMultiOuputCollector()
    +  }
    +
    +  /**
    +    * Process left stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx    The ctx to register timer or get current time
    +    * @param out    The collector for returning result values.
    +    *
    +    */
    +  override def processElement1(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, leftTimer, leftState, rightState, true)
    +  }
    +
    +  /**
    +    * Process right stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx    The ctx to register timer or get current time
    +    * @param out    The collector for returning result values.
    +    *
    +    */
    +  override def processElement2(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, rightTimer, rightState, leftState, false)
    +  }
    +
    +
    +  /**
    +    * Called when a processing timer trigger.
    +    * Expire left/right records which are expired in left and right state.
    +    *
    +    * @param timestamp The timestamp of the firing timer.
    +    * @param ctx       The ctx to register timer or get current time
    +    * @param out       The collector for returning result values.
    +    */
    +  override def onTimer(
    +      timestamp: Long,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +
    +    if (stateCleaningEnabled && leftTimer.value == timestamp) {
    +      expireOutTimeRow(
    +        timestamp,
    +        leftState,
    +        leftTimer,
    +        ctx
    +      )
    +    }
    +
    +    if (stateCleaningEnabled && rightTimer.value == timestamp) {
    +      expireOutTimeRow(
    +        timestamp,
    +        rightState,
    +        rightTimer,
    +        ctx
    +      )
    +    }
    +  }
    +
    +
    +  def getNewExpiredTime(
    +      curProcessTime: Long,
    +      oldExpiredTime: Long): Long = {
    +
    +    var newExpiredTime = oldExpiredTime
    +    if (stateCleaningEnabled) {
    +      if (curProcessTime + minRetentionTime > oldExpiredTime) {
    +        newExpiredTime = curProcessTime + maxRetentionTime
    +      }
    +    }
    +    newExpiredTime
    +  }
    +
    +  /**
    +    * Puts or Retract an element from the input stream into state and search the other state to
    +    * output records meet the condition. Records will be expired in state if state retention time
    +    * has been specified.
    +    */
    +  def processElement(
    +      value: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow],
    +      timerState: ValueState[Long],
    +      currentSideState: MapState[Row, JTuple2[Int, Long]],
    +      otherSideState: MapState[Row, JTuple2[Int, Long]],
    +      isLeft: Boolean): Unit = {
    +
    +    cRowWrapper.out = out
    +    cRowWrapper.setChange(value.change)
    --- End diff --
    
    agree


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r152826859
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/UpdatingPlanChecker.scala ---
    @@ -116,14 +135,100 @@ object UpdatingPlanChecker {
               val windowStartEnd = w.getWindowProperties.map(_.name)
    --- End diff --
    
    This can not only be start and end but also `.rowtime` or `.proctime`.


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r152759465
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowWrappingMultiOuputCollector.scala ---
    @@ -0,0 +1,50 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.table.runtime
    +
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +
    +/**
    +  * The collector to wrap a [[Row]] into a [[CRow]] and collect it multiple times.
    +  */
    +class CRowWrappingMultiOuputCollector() extends Collector[Row] {
    --- End diff --
    
    `Output`


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r152817615
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/JoinITCase.scala ---
    @@ -414,6 +416,56 @@ class JoinITCase extends StreamingWithStateTestBase {
         StreamITCase.compareWithList(expected)
       }
     
    +  /** test process time non-window inner join **/
    +  @Test
    +  def testProcessTimeNonWindowInnerJoin(): Unit = {
    +    val env = StreamExecutionEnvironment.getExecutionEnvironment
    +    val tEnv = TableEnvironment.getTableEnvironment(env)
    +    env.setStateBackend(getStateBackend)
    +    StreamITCase.clear
    +    env.setParallelism(1)
    --- End diff --
    
    Use the default parallelism here.


---

[GitHub] flink issue #4471: [FLINK-6094] [table] Implement stream-stream proctime non...

Posted by hequn8128 <gi...@git.apache.org>.
Github user hequn8128 commented on the issue:

    https://github.com/apache/flink/pull/4471
  
    Hi @xccui , 
    Thanks for your review. I have updated the PR according to your comments.  @fhueske It would be great if you can also take a look.
    
    Thank you, Hequn.


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132298846
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeNonWindowInnerJoin.scala ---
    @@ -0,0 +1,275 @@
    +/*
    + * 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.RichFlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.java.typeutils.{ResultTypeQueryable, TupleTypeInfo}
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.runtime.CRowWrappingMultiOuputCollector
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
    +
    +/**
    +  * Connect data for left stream and right stream. Only use for innerJoin.
    +  *
    +  * @param joiner           join function
    +  * @param leftType         the input type of left stream
    +  * @param rightType        the input type of right stream
    +  * @param resultType       the output type of join
    +  * @param queryConfig
    +  */
    +class ProcTimeNonWindowInnerJoin(
    +    joiner: RichFlatJoinFunction[Row, Row, Row],
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    resultType: TypeInformation[CRow],
    +    queryConfig: StreamQueryConfig) extends
    +  CoProcessFunction[CRow, CRow, CRow] with ResultTypeQueryable[CRow] {
    +
    +
    +  // state to hold left stream element
    +  private var leftState: MapState[Row, JTuple2[Int, Long]] = null
    +  // state to hold right stream element
    +  private var rightState: MapState[Row, JTuple2[Int, Long]] = null
    +  private var cRowWrapper: CRowWrappingMultiOuputCollector = null
    +
    +  private val minRetentionTime: Long = queryConfig.getMinIdleStateRetentionTime
    +  private val maxRetentionTime: Long = queryConfig.getMaxIdleStateRetentionTime
    +  private val stateCleaningEnabled: Boolean = minRetentionTime > 1
    +
    +  // state to record last timer of left stream, 0 means no timer
    +  private var timerState1: ValueState[Long] = _
    +  // state to record last timer of right stream, 0 means no timer
    +  private var timerState2: ValueState[Long] = _
    +
    +
    +  override def open(parameters: Configuration): Unit = {
    +    // initialize left and right state
    +    val tupleTypeInfo = new TupleTypeInfo[JTuple2[Int, Long]](Types.INT, Types.LONG)
    +    val leftStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "left", leftType, tupleTypeInfo)
    +    val rightStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "right", rightType, tupleTypeInfo)
    +    leftState = getRuntimeContext.getMapState(leftStateDescriptor)
    +    rightState = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // initialize timer state
    +    val valueStateDescriptor1 = new ValueStateDescriptor[Long]("timervaluestate1", classOf[Long])
    +    timerState1 = getRuntimeContext.getState(valueStateDescriptor1)
    +    val valueStateDescriptor2 = new ValueStateDescriptor[Long]("timervaluestate2", classOf[Long])
    +    timerState2 = getRuntimeContext.getState(valueStateDescriptor2)
    +
    +    cRowWrapper = new CRowWrappingMultiOuputCollector()
    +    joiner.setRuntimeContext(getRuntimeContext)
    +    joiner.open(parameters)
    +  }
    +
    +  /**
    +    * Process left stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx   The ctx to register timer or get current time
    +    * @param out   The collector for returning result values.
    +    *
    +    */
    +  override def processElement1(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, timerState1, leftState, rightState, true)
    +  }
    +
    +  /**
    +    * Process right stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx   The ctx to register timer or get current time
    +    * @param out   The collector for returning result values.
    +    *
    +    */
    +  override def processElement2(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, timerState2, rightState, leftState, false)
    +  }
    +
    +
    +  /**
    +    * Called when a processing timer trigger.
    +    * Expire left/right records which are expired in left and right state.
    +    *
    +    * @param timestamp The timestamp of the firing timer.
    +    * @param ctx       The ctx to register timer or get current time
    +    * @param out       The collector for returning result values.
    +    */
    +  override def onTimer(
    +      timestamp: Long,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +
    +    if (stateCleaningEnabled && timerState1.value == timestamp) {
    +      expireOutTimeRow(
    +        timestamp,
    +        leftState,
    +        timerState1,
    +        ctx
    +      )
    +    }
    +
    +    if (stateCleaningEnabled && timerState2.value == timestamp) {
    +      expireOutTimeRow(
    +        timestamp,
    +        rightState,
    +        timerState2,
    +        ctx
    +      )
    +    }
    +  }
    +
    +
    +  def getNewExpiredTime(
    +      curProcessTime: Long,
    +      oldExpiredTime: Long): Long = {
    +
    +    var newExpiredTime = oldExpiredTime
    +    if (stateCleaningEnabled) {
    +      if (-1 == oldExpiredTime || (curProcessTime + minRetentionTime) > oldExpiredTime) {
    +        newExpiredTime = curProcessTime + maxRetentionTime
    +      }
    +    }
    +    newExpiredTime
    +  }
    +
    +  /**
    +    * Puts or Retract an element from the input stream into state and search the other state to
    +    * output records meet the condition. Records will be expired in state if state retention time
    +    * has been specified.
    +    */
    +  def processElement(
    +      value: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow],
    +      timerState: ValueState[Long],
    +      currentSideState: MapState[Row, JTuple2[Int, Long]],
    +      otherSideState: MapState[Row, JTuple2[Int, Long]],
    +      isLeft: Boolean): Unit = {
    +
    +    cRowWrapper.out = out
    +    cRowWrapper.setChange(value.change)
    +
    +    val curProcessTime = ctx.timerService.currentProcessingTime
    +    var oldCnt = 0
    +    var oldExpiredTime: Long = -1
    +
    +    val currentRowCntAndExpiredTime = currentSideState.get(value.row)
    +    if (currentRowCntAndExpiredTime != null) {
    +      oldCnt = currentRowCntAndExpiredTime.f0
    +      oldExpiredTime = currentRowCntAndExpiredTime.f1
    +    }
    +
    +    val newExpiredTime = getNewExpiredTime(curProcessTime, oldExpiredTime)
    +    if (stateCleaningEnabled && timerState.value() == 0) {
    +      timerState.update(newExpiredTime)
    +      ctx.timerService().registerProcessingTimeTimer(newExpiredTime)
    +    }
    +
    +    // update current side stream state
    +    if (!value.asInstanceOf[CRow].change) {
    +      oldCnt = oldCnt - 1
    +      if (oldCnt <= 0) {
    +        currentSideState.remove(value.row)
    +      } else {
    +        currentSideState.put(value.row, JTuple2.of(oldCnt, newExpiredTime))
    +      }
    +    } else {
    +      oldCnt = oldCnt + 1
    +      currentSideState.put(value.row, JTuple2.of(oldCnt, newExpiredTime))
    +    }
    +
    +    val otherSideRowsIterator = otherSideState.keys().iterator()
    +    // join other side data
    +    while (otherSideRowsIterator.hasNext) {
    +      val otherSideRow = otherSideRowsIterator.next()
    +      val cntAndExpiredTime = otherSideState.get(otherSideRow)
    +      // if other side record is expired
    +      if (stateCleaningEnabled && curProcessTime >= cntAndExpiredTime.f1) {
    +        otherSideRowsIterator.remove()
    +      }
    +      // if other side record is valid
    +      else {
    --- End diff --
    
    I think we should still join with the record that we just removed. The expiration is to avoid too large states and not to guarantee certain semantics (if we were strict about the semantics we would never expire). So if we have access to a row, we should join it and remove it afterwards if it is expired. 
    
    Thereby, we are a little bit closer to the original semantics of the query.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132210266
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamJoin.scala ---
    @@ -0,0 +1,212 @@
    +/*
    + * 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.`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.FlatJoinRunner
    +import org.apache.flink.table.runtime.join.ProcTimeNonWindowInnerJoin
    +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.
    +  */
    +class DataStreamJoin(
    +    cluster: RelOptCluster,
    +    traitSet: RelTraitSet,
    +    leftNode: RelNode,
    +    rightNode: RelNode,
    +    joinCondition: RexNode,
    +    joinInfo: JoinInfo,
    +    joinType: JoinRelType,
    +    leftSchema: RowSchema,
    +    rightSchema: RowSchema,
    +    schema: RowSchema,
    +    ruleDescription: String)
    +  extends BiRel(cluster, traitSet, leftNode, rightNode)
    +          with CommonJoin
    +          with DataStreamRel {
    +
    +  override def deriveRowType(): RelDataType = schema.logicalType
    +
    +  override def needsUpdatesAsRetraction: Boolean = true
    +
    +  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
    +    new DataStreamJoin(
    +      cluster,
    +      traitSet,
    +      inputs.get(0),
    +      inputs.get(1),
    +      joinCondition,
    +      joinInfo,
    +      joinType,
    +      leftSchema,
    +      rightSchema,
    +      schema,
    +      ruleDescription)
    +  }
    +
    +  def getJoinInfo: JoinInfo = joinInfo
    +
    +  override def toString: String = {
    +    joinToString(
    +      schema.logicalType,
    +      joinCondition,
    +      joinType,
    +      getExpressionString)
    +  }
    +
    +  override def explainTerms(pw: RelWriter): RelWriter = {
    +    joinExplainTerms(
    +      super.explainTerms(pw),
    +      schema.logicalType,
    +      joinCondition,
    +      joinType,
    +      getExpressionString)
    +  }
    +
    +  override def translateToPlan(
    +      tableEnv: StreamTableEnvironment,
    +      queryConfig: StreamQueryConfig): DataStream[CRow] = {
    +
    +    val config = tableEnv.getConfig
    +    val returnType = schema.physicalTypeInfo
    +    val keyPairs = joinInfo.pairs().toList
    +
    +    // get the equality keys
    +    val leftKeys = ArrayBuffer.empty[Int]
    +    val rightKeys = ArrayBuffer.empty[Int]
    +    if (keyPairs.isEmpty) {
    +      // if no equality keys => not supported
    +      throw TableException(
    +        "Joins should have at least one equality condition.\n" +
    +          s"\tLeft: ${left.toString},\n" +
    +          s"\tRight: ${right.toString},\n" +
    +          s"\tCondition: (${joinConditionToString(schema.logicalType,
    +             joinCondition, getExpressionString)})"
    +      )
    +    }
    +    else {
    +      // at least one equality expression
    +      val leftFields = left.getRowType.getFieldList
    +      val rightFields = right.getRowType.getFieldList
    +
    +      keyPairs.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 {
    +          throw TableException(
    +            "Equality join predicate on incompatible types.\n" +
    +              s"\tLeft: ${left.toString},\n" +
    +              s"\tRight: ${right.toString},\n" +
    +              s"\tCondition: (${joinConditionToString(schema.logicalType,
    +                joinCondition, getExpressionString)})"
    +          )
    +        }
    +      })
    +    }
    +
    +    val leftDataStream =
    +      left.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
    +    val rightDataStream =
    +      right.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
    +
    +    val (connectOperator, nullCheck) = joinType match {
    +      case JoinRelType.INNER => (leftDataStream.connect(rightDataStream), false)
    +      case _ => throw new UnsupportedOperationException(s"An Unsupported JoinType [ $joinType ]")
    +    }
    +
    +    if (nullCheck && !config.getNullCheck) {
    +      throw TableException("Null check in TableConfig must be enabled for outer joins.")
    +    }
    +
    +
    +    val generator = new FunctionCodeGenerator(
    +      config,
    +      nullCheck,
    +      leftSchema.physicalTypeInfo,
    +      Some(rightSchema.physicalTypeInfo))
    +    val conversion = generator.generateConverterResultExpression(
    +      schema.physicalTypeInfo,
    +      schema.physicalType.getFieldNames)
    +
    +
    +    var body = ""
    --- End diff --
    
    `var` should be avoided in Scala.
    
    You can do the following:
    
    ```
    val body = if (joinInfo.isEqui) {
      "code goes here"
    } else {
      "or here"
    }
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132424191
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeNonWindowInnerJoin.scala ---
    @@ -0,0 +1,275 @@
    +/*
    + * 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.RichFlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.java.typeutils.{ResultTypeQueryable, TupleTypeInfo}
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.runtime.CRowWrappingMultiOuputCollector
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
    +
    +/**
    +  * Connect data for left stream and right stream. Only use for innerJoin.
    +  *
    +  * @param joiner           join function
    +  * @param leftType         the input type of left stream
    +  * @param rightType        the input type of right stream
    +  * @param resultType       the output type of join
    +  * @param queryConfig
    +  */
    +class ProcTimeNonWindowInnerJoin(
    +    joiner: RichFlatJoinFunction[Row, Row, Row],
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    resultType: TypeInformation[CRow],
    +    queryConfig: StreamQueryConfig) extends
    +  CoProcessFunction[CRow, CRow, CRow] with ResultTypeQueryable[CRow] {
    +
    +
    +  // state to hold left stream element
    +  private var leftState: MapState[Row, JTuple2[Int, Long]] = null
    +  // state to hold right stream element
    +  private var rightState: MapState[Row, JTuple2[Int, Long]] = null
    +  private var cRowWrapper: CRowWrappingMultiOuputCollector = null
    +
    +  private val minRetentionTime: Long = queryConfig.getMinIdleStateRetentionTime
    +  private val maxRetentionTime: Long = queryConfig.getMaxIdleStateRetentionTime
    +  private val stateCleaningEnabled: Boolean = minRetentionTime > 1
    +
    +  // state to record last timer of left stream, 0 means no timer
    +  private var timerState1: ValueState[Long] = _
    +  // state to record last timer of right stream, 0 means no timer
    +  private var timerState2: ValueState[Long] = _
    +
    +
    +  override def open(parameters: Configuration): Unit = {
    +    // initialize left and right state
    +    val tupleTypeInfo = new TupleTypeInfo[JTuple2[Int, Long]](Types.INT, Types.LONG)
    +    val leftStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "left", leftType, tupleTypeInfo)
    +    val rightStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "right", rightType, tupleTypeInfo)
    +    leftState = getRuntimeContext.getMapState(leftStateDescriptor)
    +    rightState = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // initialize timer state
    +    val valueStateDescriptor1 = new ValueStateDescriptor[Long]("timervaluestate1", classOf[Long])
    +    timerState1 = getRuntimeContext.getState(valueStateDescriptor1)
    +    val valueStateDescriptor2 = new ValueStateDescriptor[Long]("timervaluestate2", classOf[Long])
    +    timerState2 = getRuntimeContext.getState(valueStateDescriptor2)
    +
    +    cRowWrapper = new CRowWrappingMultiOuputCollector()
    +    joiner.setRuntimeContext(getRuntimeContext)
    +    joiner.open(parameters)
    +  }
    +
    +  /**
    +    * Process left stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx   The ctx to register timer or get current time
    +    * @param out   The collector for returning result values.
    +    *
    +    */
    +  override def processElement1(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, timerState1, leftState, rightState, true)
    +  }
    +
    +  /**
    +    * Process right stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx   The ctx to register timer or get current time
    +    * @param out   The collector for returning result values.
    +    *
    +    */
    +  override def processElement2(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, timerState2, rightState, leftState, false)
    +  }
    +
    +
    +  /**
    +    * Called when a processing timer trigger.
    +    * Expire left/right records which are expired in left and right state.
    +    *
    +    * @param timestamp The timestamp of the firing timer.
    +    * @param ctx       The ctx to register timer or get current time
    +    * @param out       The collector for returning result values.
    +    */
    +  override def onTimer(
    +      timestamp: Long,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +
    +    if (stateCleaningEnabled && timerState1.value == timestamp) {
    +      expireOutTimeRow(
    +        timestamp,
    +        leftState,
    +        timerState1,
    +        ctx
    +      )
    +    }
    +
    +    if (stateCleaningEnabled && timerState2.value == timestamp) {
    +      expireOutTimeRow(
    +        timestamp,
    +        rightState,
    +        timerState2,
    +        ctx
    +      )
    +    }
    +  }
    +
    +
    +  def getNewExpiredTime(
    +      curProcessTime: Long,
    +      oldExpiredTime: Long): Long = {
    +
    +    var newExpiredTime = oldExpiredTime
    +    if (stateCleaningEnabled) {
    +      if (-1 == oldExpiredTime || (curProcessTime + minRetentionTime) > oldExpiredTime) {
    --- End diff --
    
    `-1 == oldExpiredTime` should always be true if `(curProcessTime + minRetentionTime) > oldExpiredTime`
    So we could remove it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r146104726
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/UpdatingPlanChecker.scala ---
    @@ -56,16 +58,20 @@ object UpdatingPlanChecker {
       }
     
       /** Identifies unique key fields in the output of a RelNode. */
    -  private class UniqueKeyExtractor extends RelVisitor {
    +  private class UniqueKeyExtractor {
    --- End diff --
    
    Are these changes on `UniqueKeyExtractor` necessary? I'd suggest to make it a separate issue (and for inputs retraction as well).


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r152745910
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamJoinRule.scala ---
    @@ -0,0 +1,108 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.table.plan.rules.datastream
    +
    +import org.apache.calcite.plan.{RelOptRule, 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.rex.{RexCall, RexInputRef, RexNode}
    +import org.apache.flink.table.api.TableConfig
    +import org.apache.flink.table.plan.nodes.FlinkConventions
    +import org.apache.flink.table.plan.nodes.datastream.DataStreamJoin
    +import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
    +import org.apache.flink.table.plan.schema.{RowSchema, TimeIndicatorRelDataType}
    +import org.apache.flink.table.runtime.join.WindowJoinUtil
    +
    +import scala.collection.JavaConverters._
    +
    +class DataStreamJoinRule
    +  extends ConverterRule(
    +    classOf[FlinkLogicalJoin],
    +    FlinkConventions.LOGICAL,
    +    FlinkConventions.DATASTREAM,
    +    "DataStreamJoinRule") {
    +
    +  /**
    +    * Checks if an expression accesses a time attribute.
    +    *
    +    * @param expr The expression to check.
    +    * @param inputType The input type of the expression.
    +    * @return True, if the expression accesses a time attribute. False otherwise.
    +    */
    +  def accessesTimeAttribute(expr: RexNode, inputType: RelDataType): Boolean = {
    +    expr match {
    +      case i: RexInputRef =>
    +        val accessedType = inputType.getFieldList.get(i.getIndex).getType
    +        accessedType match {
    --- End diff --
    
    Use `FlinkTypeFactory.isTimeIndicatorType`


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r146104524
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamJoinRule.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.plan.rules.datastream
    +
    +import org.apache.calcite.plan.{RelOptRule, 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.rex.{RexCall, RexInputRef, RexNode}
    +import org.apache.flink.table.api.TableConfig
    +import org.apache.flink.table.plan.nodes.FlinkConventions
    +import org.apache.flink.table.plan.nodes.datastream.DataStreamJoin
    +import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
    +import org.apache.flink.table.plan.schema.{RowSchema, TimeIndicatorRelDataType}
    +import org.apache.flink.table.runtime.join.WindowJoinUtil
    +
    +import scala.collection.JavaConverters._
    +
    +class DataStreamJoinRule
    +  extends ConverterRule(
    +    classOf[FlinkLogicalJoin],
    +    FlinkConventions.LOGICAL,
    +    FlinkConventions.DATASTREAM,
    +    "DataStreamJoinRule") {
    +
    +  /**
    +    * Checks if an expression accesses a time attribute.
    +    *
    +    * @param expr The expression to check.
    +    * @param inputType The input type of the expression.
    +    * @return True, if the expression accesses a time attribute. False otherwise.
    +    */
    +  def accessesTimeAttribute(expr: RexNode, inputType: RelDataType): Boolean = {
    +    expr match {
    +      case i: RexInputRef =>
    +        val accessedType = inputType.getFieldList.get(i.getIndex).getType
    +        accessedType match {
    +          case _: TimeIndicatorRelDataType => true
    +          case _ => false
    +        }
    +      case c: RexCall =>
    +        c.operands.asScala.exists(accessesTimeAttribute(_, inputType))
    +      case _ => false
    +    }
    +  }
    +
    +  override def matches(call: RelOptRuleCall): Boolean = {
    +    val join: FlinkLogicalJoin = call.rel(0).asInstanceOf[FlinkLogicalJoin]
    +    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)
    +
    +    // remaining predicate must not access time attributes
    +    val remainingPredsAccessTime = remainingPreds.isDefined &&
    +      accessesTimeAttribute(remainingPreds.get, join.getRowType)
    +
    +    if (!windowBounds.isDefined && !remainingPredsAccessTime) {
    --- End diff --
    
    Use this expression as the returned value directly.


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r152742299
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamJoin.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.nodes.datastream
    +
    +import org.apache.calcite.plan._
    +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.join.DataStreamInnerJoin
    +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.
    +  */
    +class DataStreamJoin(
    +    cluster: RelOptCluster,
    +    traitSet: RelTraitSet,
    +    leftNode: RelNode,
    +    rightNode: RelNode,
    +    joinCondition: RexNode,
    +    joinInfo: JoinInfo,
    +    joinType: JoinRelType,
    +    leftSchema: RowSchema,
    +    rightSchema: RowSchema,
    +    schema: RowSchema,
    +    ruleDescription: String)
    +  extends BiRel(cluster, traitSet, leftNode, rightNode)
    +          with CommonJoin
    +          with DataStreamRel {
    +
    +  override def deriveRowType(): RelDataType = schema.relDataType
    +
    +  override def needsUpdatesAsRetraction: Boolean = true
    +
    +  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
    +    new DataStreamJoin(
    +      cluster,
    +      traitSet,
    +      inputs.get(0),
    +      inputs.get(1),
    +      joinCondition,
    +      joinInfo,
    +      joinType,
    +      leftSchema,
    +      rightSchema,
    +      schema,
    +      ruleDescription)
    +  }
    +
    +  def getJoinInfo: JoinInfo = joinInfo
    +
    +  def getJoinType: JoinRelType = joinType
    +
    +  override def toString: String = {
    +    joinToString(
    +      schema.relDataType,
    +      joinCondition,
    +      joinType,
    +      getExpressionString)
    +  }
    +
    +  override def explainTerms(pw: RelWriter): RelWriter = {
    +    joinExplainTerms(
    +      super.explainTerms(pw),
    +      schema.relDataType,
    +      joinCondition,
    +      joinType,
    +      getExpressionString)
    +  }
    +
    +  override def translateToPlan(
    +      tableEnv: StreamTableEnvironment,
    +      queryConfig: StreamQueryConfig): DataStream[CRow] = {
    +
    +    val config = tableEnv.getConfig
    +    val returnType = schema.typeInfo
    +    val keyPairs = joinInfo.pairs().toList
    +
    +    // get the equality keys
    +    val leftKeys = ArrayBuffer.empty[Int]
    +    val rightKeys = ArrayBuffer.empty[Int]
    +    if (keyPairs.isEmpty) {
    --- End diff --
    
    Should we move the checks in the constructor to fail earlier?


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by hequn8128 <gi...@git.apache.org>.
Github user hequn8128 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r158489017
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/DataStreamInnerJoin.scala ---
    @@ -0,0 +1,285 @@
    +/*
    + * 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._
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.java.typeutils.TupleTypeInfo
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.runtime.CRowWrappingMultiOuputCollector
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
    +import org.slf4j.LoggerFactory
    +import org.apache.flink.table.codegen.Compiler
    +
    +
    +/**
    +  * Connect data for left stream and right stream. Only use for innerJoin.
    +  *
    +  * @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
    +  */
    +class DataStreamInnerJoin(
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    resultType: TypeInformation[CRow],
    +    genJoinFuncName: String,
    +    genJoinFuncCode: String,
    +    queryConfig: StreamQueryConfig)
    +  extends CoProcessFunction[CRow, CRow, CRow]
    +          with Compiler[FlatJoinFunction[Row, Row, Row]] {
    +
    +  // state to hold left stream element
    +  private var leftState: MapState[Row, JTuple2[Int, Long]] = _
    +  // state to hold right stream element
    +  private var rightState: MapState[Row, JTuple2[Int, Long]] = _
    +  private var cRowWrapper: CRowWrappingMultiOuputCollector = _
    +
    +  private val minRetentionTime: Long = queryConfig.getMinIdleStateRetentionTime
    +  private val maxRetentionTime: Long = queryConfig.getMaxIdleStateRetentionTime
    +  private val stateCleaningEnabled: Boolean = minRetentionTime > 1
    +
    +  // state to record last timer of left stream, 0 means no timer
    +  private var leftTimer: ValueState[Long] = _
    +  // state to record last timer of right stream, 0 means no timer
    +  private var rightTimer: ValueState[Long] = _
    +
    +  // other condition function
    +  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
    +
    +  val LOG = LoggerFactory.getLogger(this.getClass)
    +
    +  override def open(parameters: Configuration): Unit = {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +                s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    joinFunction = clazz.newInstance()
    +
    +    // initialize left and right state, the first element of tuple2 indicates how many rows of
    +    // this row, while the second element represents the expired time of this row.
    +    val tupleTypeInfo = new TupleTypeInfo[JTuple2[Int, Long]](Types.INT, Types.LONG)
    +    val leftStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "left", leftType, tupleTypeInfo)
    +    val rightStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "right", rightType, tupleTypeInfo)
    +    leftState = getRuntimeContext.getMapState(leftStateDescriptor)
    +    rightState = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // initialize timer state
    +    val valueStateDescriptor1 = new ValueStateDescriptor[Long]("timervaluestate1", classOf[Long])
    +    leftTimer = getRuntimeContext.getState(valueStateDescriptor1)
    +    val valueStateDescriptor2 = new ValueStateDescriptor[Long]("timervaluestate2", classOf[Long])
    +    rightTimer = getRuntimeContext.getState(valueStateDescriptor2)
    +
    +    cRowWrapper = new CRowWrappingMultiOuputCollector()
    +  }
    +
    +  /**
    +    * Process left stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx    The ctx to register timer or get current time
    +    * @param out    The collector for returning result values.
    +    *
    +    */
    +  override def processElement1(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, leftTimer, leftState, rightState, true)
    +  }
    +
    +  /**
    +    * Process right stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx    The ctx to register timer or get current time
    +    * @param out    The collector for returning result values.
    +    *
    +    */
    +  override def processElement2(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, rightTimer, rightState, leftState, false)
    +  }
    +
    +
    +  /**
    +    * Called when a processing timer trigger.
    +    * Expire left/right records which are expired in left and right state.
    +    *
    +    * @param timestamp The timestamp of the firing timer.
    +    * @param ctx       The ctx to register timer or get current time
    +    * @param out       The collector for returning result values.
    +    */
    +  override def onTimer(
    +      timestamp: Long,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +
    +    if (stateCleaningEnabled && leftTimer.value == timestamp) {
    +      expireOutTimeRow(
    +        timestamp,
    +        leftState,
    +        leftTimer,
    +        ctx
    +      )
    +    }
    +
    +    if (stateCleaningEnabled && rightTimer.value == timestamp) {
    +      expireOutTimeRow(
    +        timestamp,
    +        rightState,
    +        rightTimer,
    +        ctx
    +      )
    +    }
    +  }
    +
    +
    +  def getNewExpiredTime(
    +      curProcessTime: Long,
    +      oldExpiredTime: Long): Long = {
    +
    +    if (stateCleaningEnabled && curProcessTime + minRetentionTime > oldExpiredTime) {
    +      curProcessTime + maxRetentionTime
    +    } else {
    +      oldExpiredTime
    +    }
    +  }
    +
    +  /**
    +    * Puts or Retract an element from the input stream into state and search the other state to
    +    * output records meet the condition. Records will be expired in state if state retention time
    +    * has been specified.
    +    */
    +  def processElement(
    +      value: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow],
    +      timerState: ValueState[Long],
    +      currentSideState: MapState[Row, JTuple2[Int, Long]],
    +      otherSideState: MapState[Row, JTuple2[Int, Long]],
    +      isLeft: Boolean): Unit = {
    +
    +    cRowWrapper.setCollector(out)
    +    cRowWrapper.setChange(value.change)
    +
    +    val curProcessTime = ctx.timerService.currentProcessingTime
    +    val oldCntAndExpiredTime = currentSideState.get(value.row)
    --- End diff --
    
    Nice catch !


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132306504
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeNonWindowInnerJoin.scala ---
    @@ -0,0 +1,275 @@
    +/*
    + * 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.RichFlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.java.typeutils.{ResultTypeQueryable, TupleTypeInfo}
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.runtime.CRowWrappingMultiOuputCollector
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
    +
    +/**
    +  * Connect data for left stream and right stream. Only use for innerJoin.
    +  *
    +  * @param joiner           join function
    +  * @param leftType         the input type of left stream
    +  * @param rightType        the input type of right stream
    +  * @param resultType       the output type of join
    +  * @param queryConfig
    +  */
    +class ProcTimeNonWindowInnerJoin(
    --- End diff --
    
    I'd rename the class. It is not a processing time join (join semantics do not depend on time). Maybe just `DataStreamInnerJoin`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132468046
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/UpdatingPlanChecker.scala ---
    @@ -56,17 +59,20 @@ object UpdatingPlanChecker {
       }
     
       /** Identifies unique key fields in the output of a RelNode. */
    -  private class UniqueKeyExtractor extends RelVisitor {
    -
    -    var keys: Option[Array[String]] = None
    +  private class UniqueKeyExtractor {
     
    -    override def visit(node: RelNode, ordinal: Int, parent: RelNode): Unit = {
    +    // visit() function will return a tuple, the first element of tuple is the key, the second is
    +    // the key's corresponding ancestor. Ancestors are used to identify same keys, for example:
    --- End diff --
    
    I think a more common term than "ancestor" is "equivalence group". In principle, this is used to identify fields which are equivalent. I think we should not point to a field in the input of an operator but rather choose on of the fields in the current input as "id" for the equivalence group. For example, if we have a table `(a, b, c)` and do `select(a, a as x, b as y, b as z)` I would resolve these fields as `[(a, a), (x, a), (y, y), (z, y)]`, i.e, always use the lexicographic smallest attribute as the common group id. 
    
    IMO, this convention is easier to handle if we have to work with equivalence groups which are joined by equi-predicates.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by hequn8128 <gi...@git.apache.org>.
Github user hequn8128 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r148947760
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamJoinRule.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.plan.rules.datastream
    +
    +import org.apache.calcite.plan.{RelOptRule, 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.rex.{RexCall, RexInputRef, RexNode}
    +import org.apache.flink.table.api.TableConfig
    +import org.apache.flink.table.plan.nodes.FlinkConventions
    +import org.apache.flink.table.plan.nodes.datastream.DataStreamJoin
    +import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
    +import org.apache.flink.table.plan.schema.{RowSchema, TimeIndicatorRelDataType}
    +import org.apache.flink.table.runtime.join.WindowJoinUtil
    +
    +import scala.collection.JavaConverters._
    +
    +class DataStreamJoinRule
    +  extends ConverterRule(
    +    classOf[FlinkLogicalJoin],
    +    FlinkConventions.LOGICAL,
    +    FlinkConventions.DATASTREAM,
    +    "DataStreamJoinRule") {
    +
    +  /**
    +    * Checks if an expression accesses a time attribute.
    +    *
    +    * @param expr The expression to check.
    +    * @param inputType The input type of the expression.
    +    * @return True, if the expression accesses a time attribute. False otherwise.
    +    */
    +  def accessesTimeAttribute(expr: RexNode, inputType: RelDataType): Boolean = {
    +    expr match {
    +      case i: RexInputRef =>
    +        val accessedType = inputType.getFieldList.get(i.getIndex).getType
    +        accessedType match {
    +          case _: TimeIndicatorRelDataType => true
    +          case _ => false
    +        }
    +      case c: RexCall =>
    +        c.operands.asScala.exists(accessesTimeAttribute(_, inputType))
    +      case _ => false
    +    }
    +  }
    +
    +  override def matches(call: RelOptRuleCall): Boolean = {
    +    val join: FlinkLogicalJoin = call.rel(0).asInstanceOf[FlinkLogicalJoin]
    +    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)
    +
    +    // remaining predicate must not access time attributes
    +    val remainingPredsAccessTime = remainingPreds.isDefined &&
    +      accessesTimeAttribute(remainingPreds.get, join.getRowType)
    --- End diff --
    
    hi, we can implement this feature later, what do you think ?


---

[GitHub] flink issue #4471: [FLINK-6094] [table] Implement stream-stream proctime non...

Posted by hequn8128 <gi...@git.apache.org>.
Github user hequn8128 commented on the issue:

    https://github.com/apache/flink/pull/4471
  
    Hi @fhueske  , the pr has been updated according to your comments and also has been rebased to the latest master. The pr mainly includes the following changes:
    1. Refactor `UpdatingPlanChecker`. Change the unique key extractor logic of join and use the lexicographic smallest attribute as the common group id.
    2. Support time indicator attributes. In fact, we do not need to add special processing logic. Time indicator will be transmitted to the downstream.
    3. Add more test cases.
    4. Refactor stream join function. 


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132229636
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/UpdatingPlanChecker.scala ---
    @@ -56,17 +59,20 @@ object UpdatingPlanChecker {
       }
     
       /** Identifies unique key fields in the output of a RelNode. */
    -  private class UniqueKeyExtractor extends RelVisitor {
    -
    -    var keys: Option[Array[String]] = None
    +  private class UniqueKeyExtractor {
     
    -    override def visit(node: RelNode, ordinal: Int, parent: RelNode): Unit = {
    +    // visit() function will return a tuple, the first element of tuple is the key, the second is
    --- End diff --
    
    `the first element is the name of a key field, the second is a group name that is shared by all equivalent key fields.`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132303211
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/validation/JoinValidationTest.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.stream.table.validation
    +
    +import org.apache.flink.api.scala._
    +import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
    +import org.apache.flink.table.api.{TableEnvironment, TableException, ValidationException}
    +import org.apache.flink.table.api.scala._
    +import org.apache.flink.table.runtime.utils.StreamTestData
    +import org.apache.flink.table.utils.TableTestBase
    +import org.apache.flink.types.Row
    +import org.junit.Test
    +
    +class JoinValidationTest extends TableTestBase {
    +
    +  private val util = streamTestUtil()
    +  private val ds1 = util.addTable[(Int, Long, String)]("Table3",'a, 'b, 'c)
    +  private val ds2 = util.addTable[(Int, Long, Int, String, Long)]("Table5", 'd, 'e, 'f, 'g, 'h)
    +
    +  @Test(expected = classOf[ValidationException])
    +  def testJoinNonExistingKey(): Unit = {
    +    ds1.join(ds2)
    +      // must fail. Field 'foo does not exist
    +      .where('foo === 'e)
    +      .select('c, 'g)
    +  }
    +
    +  @Test(expected = classOf[ValidationException])
    +  def testJoinWithNonMatchingKeyTypes(): Unit = {
    +    ds1.join(ds2)
    +      // must fail. Field 'a is Int, and 'g is String
    +      .where('a === 'g)
    +      .select('c, 'g)
    +  }
    +
    +
    +  @Test(expected = classOf[ValidationException])
    +  def testJoinWithAmbiguousFields(): Unit = {
    +    ds1.join(ds2.select('d, 'e, 'f, 'g, 'h as 'c))
    +      // must fail. Both inputs share the same field 'c
    +      .where('a === 'd)
    +      .select('c, 'g)
    +  }
    +
    +  @Test(expected = classOf[TableException])
    +  def testNoEqualityJoinPredicate1(): Unit = {
    +    ds1.join(ds2)
    +      // must fail. No equality join predicate
    +      .where('d === 'f)
    +      .select('c, 'g)
    +      .toDataSet[Row]
    --- End diff --
    
    replace `toDataSet` call by `toRetractStream[Row]`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132300742
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeNonWindowInnerJoin.scala ---
    @@ -0,0 +1,275 @@
    +/*
    + * 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.RichFlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.java.typeutils.{ResultTypeQueryable, TupleTypeInfo}
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.runtime.CRowWrappingMultiOuputCollector
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
    +
    +/**
    +  * Connect data for left stream and right stream. Only use for innerJoin.
    +  *
    +  * @param joiner           join function
    +  * @param leftType         the input type of left stream
    +  * @param rightType        the input type of right stream
    +  * @param resultType       the output type of join
    +  * @param queryConfig
    +  */
    +class ProcTimeNonWindowInnerJoin(
    +    joiner: RichFlatJoinFunction[Row, Row, Row],
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    resultType: TypeInformation[CRow],
    +    queryConfig: StreamQueryConfig) extends
    +  CoProcessFunction[CRow, CRow, CRow] with ResultTypeQueryable[CRow] {
    +
    +
    +  // state to hold left stream element
    +  private var leftState: MapState[Row, JTuple2[Int, Long]] = null
    +  // state to hold right stream element
    +  private var rightState: MapState[Row, JTuple2[Int, Long]] = null
    +  private var cRowWrapper: CRowWrappingMultiOuputCollector = null
    +
    +  private val minRetentionTime: Long = queryConfig.getMinIdleStateRetentionTime
    +  private val maxRetentionTime: Long = queryConfig.getMaxIdleStateRetentionTime
    +  private val stateCleaningEnabled: Boolean = minRetentionTime > 1
    +
    +  // state to record last timer of left stream, 0 means no timer
    +  private var timerState1: ValueState[Long] = _
    +  // state to record last timer of right stream, 0 means no timer
    +  private var timerState2: ValueState[Long] = _
    +
    +
    +  override def open(parameters: Configuration): Unit = {
    +    // initialize left and right state
    +    val tupleTypeInfo = new TupleTypeInfo[JTuple2[Int, Long]](Types.INT, Types.LONG)
    +    val leftStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "left", leftType, tupleTypeInfo)
    +    val rightStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "right", rightType, tupleTypeInfo)
    +    leftState = getRuntimeContext.getMapState(leftStateDescriptor)
    +    rightState = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // initialize timer state
    +    val valueStateDescriptor1 = new ValueStateDescriptor[Long]("timervaluestate1", classOf[Long])
    +    timerState1 = getRuntimeContext.getState(valueStateDescriptor1)
    +    val valueStateDescriptor2 = new ValueStateDescriptor[Long]("timervaluestate2", classOf[Long])
    +    timerState2 = getRuntimeContext.getState(valueStateDescriptor2)
    +
    +    cRowWrapper = new CRowWrappingMultiOuputCollector()
    +    joiner.setRuntimeContext(getRuntimeContext)
    +    joiner.open(parameters)
    +  }
    +
    +  /**
    +    * Process left stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx   The ctx to register timer or get current time
    +    * @param out   The collector for returning result values.
    +    *
    +    */
    +  override def processElement1(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, timerState1, leftState, rightState, true)
    +  }
    +
    +  /**
    +    * Process right stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx   The ctx to register timer or get current time
    +    * @param out   The collector for returning result values.
    +    *
    +    */
    +  override def processElement2(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, timerState2, rightState, leftState, false)
    +  }
    +
    +
    +  /**
    +    * Called when a processing timer trigger.
    +    * Expire left/right records which are expired in left and right state.
    +    *
    +    * @param timestamp The timestamp of the firing timer.
    +    * @param ctx       The ctx to register timer or get current time
    +    * @param out       The collector for returning result values.
    +    */
    +  override def onTimer(
    +      timestamp: Long,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +
    +    if (stateCleaningEnabled && timerState1.value == timestamp) {
    +      expireOutTimeRow(
    +        timestamp,
    +        leftState,
    +        timerState1,
    +        ctx
    +      )
    +    }
    +
    +    if (stateCleaningEnabled && timerState2.value == timestamp) {
    +      expireOutTimeRow(
    +        timestamp,
    +        rightState,
    +        timerState2,
    +        ctx
    +      )
    +    }
    +  }
    +
    +
    +  def getNewExpiredTime(
    +      curProcessTime: Long,
    +      oldExpiredTime: Long): Long = {
    +
    +    var newExpiredTime = oldExpiredTime
    +    if (stateCleaningEnabled) {
    +      if (-1 == oldExpiredTime || (curProcessTime + minRetentionTime) > oldExpiredTime) {
    +        newExpiredTime = curProcessTime + maxRetentionTime
    +      }
    +    }
    +    newExpiredTime
    +  }
    +
    +  /**
    +    * Puts or Retract an element from the input stream into state and search the other state to
    +    * output records meet the condition. Records will be expired in state if state retention time
    +    * has been specified.
    +    */
    +  def processElement(
    +      value: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow],
    +      timerState: ValueState[Long],
    +      currentSideState: MapState[Row, JTuple2[Int, Long]],
    +      otherSideState: MapState[Row, JTuple2[Int, Long]],
    +      isLeft: Boolean): Unit = {
    +
    +    cRowWrapper.out = out
    +    cRowWrapper.setChange(value.change)
    +
    +    val curProcessTime = ctx.timerService.currentProcessingTime
    +    var oldCnt = 0
    +    var oldExpiredTime: Long = -1
    +
    +    val currentRowCntAndExpiredTime = currentSideState.get(value.row)
    +    if (currentRowCntAndExpiredTime != null) {
    +      oldCnt = currentRowCntAndExpiredTime.f0
    +      oldExpiredTime = currentRowCntAndExpiredTime.f1
    +    }
    +
    +    val newExpiredTime = getNewExpiredTime(curProcessTime, oldExpiredTime)
    +    if (stateCleaningEnabled && timerState.value() == 0) {
    +      timerState.update(newExpiredTime)
    +      ctx.timerService().registerProcessingTimeTimer(newExpiredTime)
    +    }
    +
    +    // update current side stream state
    +    if (!value.asInstanceOf[CRow].change) {
    +      oldCnt = oldCnt - 1
    +      if (oldCnt <= 0) {
    +        currentSideState.remove(value.row)
    +      } else {
    +        currentSideState.put(value.row, JTuple2.of(oldCnt, newExpiredTime))
    +      }
    +    } else {
    +      oldCnt = oldCnt + 1
    --- End diff --
    
    -> `val newCnt = oldCnt + 1`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by xccui <gi...@git.apache.org>.
Github user xccui commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r146105512
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/DataStreamInnerJoin.scala ---
    @@ -0,0 +1,287 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +
    +package org.apache.flink.table.runtime.join
    +
    +import org.apache.flink.api.common.functions.FlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.java.typeutils.TupleTypeInfo
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.runtime.CRowWrappingMultiOuputCollector
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
    +import org.slf4j.LoggerFactory
    +import org.apache.flink.table.codegen.Compiler
    +
    +
    +/**
    +  * Connect data for left stream and right stream. Only use for innerJoin.
    +  *
    +  * @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
    +  */
    +class DataStreamInnerJoin(
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    resultType: TypeInformation[CRow],
    +    genJoinFuncName: String,
    +    genJoinFuncCode: String,
    +    queryConfig: StreamQueryConfig)
    +  extends CoProcessFunction[CRow, CRow, CRow]
    +          with Compiler[FlatJoinFunction[Row, Row, Row]] {
    +
    +  // state to hold left stream element
    +  private var leftState: MapState[Row, JTuple2[Int, Long]] = _
    +  // state to hold right stream element
    +  private var rightState: MapState[Row, JTuple2[Int, Long]] = _
    +  private var cRowWrapper: CRowWrappingMultiOuputCollector = _
    +
    +  private val minRetentionTime: Long = queryConfig.getMinIdleStateRetentionTime
    +  private val maxRetentionTime: Long = queryConfig.getMaxIdleStateRetentionTime
    +  private val stateCleaningEnabled: Boolean = minRetentionTime > 1
    +
    +  // state to record last timer of left stream, 0 means no timer
    +  private var leftTimer: ValueState[Long] = _
    +  // state to record last timer of right stream, 0 means no timer
    +  private var rightTimer: ValueState[Long] = _
    +
    +  // other condition function
    +  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
    +
    +  val LOG = LoggerFactory.getLogger(this.getClass)
    +
    +  override def open(parameters: Configuration): Unit = {
    +    LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " +
    +                s"Code:\n$genJoinFuncCode")
    +    val clazz = compile(
    +      getRuntimeContext.getUserCodeClassLoader,
    +      genJoinFuncName,
    +      genJoinFuncCode)
    +    LOG.debug("Instantiating JoinFunction.")
    +    joinFunction = clazz.newInstance()
    +
    +    // initialize left and right state, the first element of tuple2 indicates how many rows of
    +    // this row, while the second element represents the expired time of this row.
    +    val tupleTypeInfo = new TupleTypeInfo[JTuple2[Int, Long]](Types.INT, Types.LONG)
    +    val leftStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "left", leftType, tupleTypeInfo)
    +    val rightStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "right", rightType, tupleTypeInfo)
    +    leftState = getRuntimeContext.getMapState(leftStateDescriptor)
    +    rightState = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // initialize timer state
    +    val valueStateDescriptor1 = new ValueStateDescriptor[Long]("timervaluestate1", classOf[Long])
    +    leftTimer = getRuntimeContext.getState(valueStateDescriptor1)
    +    val valueStateDescriptor2 = new ValueStateDescriptor[Long]("timervaluestate2", classOf[Long])
    +    rightTimer = getRuntimeContext.getState(valueStateDescriptor2)
    +
    +    cRowWrapper = new CRowWrappingMultiOuputCollector()
    +  }
    +
    +  /**
    +    * Process left stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx    The ctx to register timer or get current time
    +    * @param out    The collector for returning result values.
    +    *
    +    */
    +  override def processElement1(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, leftTimer, leftState, rightState, true)
    +  }
    +
    +  /**
    +    * Process right stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx    The ctx to register timer or get current time
    +    * @param out    The collector for returning result values.
    +    *
    +    */
    +  override def processElement2(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, rightTimer, rightState, leftState, false)
    +  }
    +
    +
    +  /**
    +    * Called when a processing timer trigger.
    +    * Expire left/right records which are expired in left and right state.
    +    *
    +    * @param timestamp The timestamp of the firing timer.
    +    * @param ctx       The ctx to register timer or get current time
    +    * @param out       The collector for returning result values.
    +    */
    +  override def onTimer(
    +      timestamp: Long,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +
    +    if (stateCleaningEnabled && leftTimer.value == timestamp) {
    +      expireOutTimeRow(
    +        timestamp,
    +        leftState,
    +        leftTimer,
    +        ctx
    +      )
    +    }
    +
    +    if (stateCleaningEnabled && rightTimer.value == timestamp) {
    +      expireOutTimeRow(
    +        timestamp,
    +        rightState,
    +        rightTimer,
    +        ctx
    +      )
    +    }
    +  }
    +
    +
    +  def getNewExpiredTime(
    +      curProcessTime: Long,
    +      oldExpiredTime: Long): Long = {
    +
    +    var newExpiredTime = oldExpiredTime
    +    if (stateCleaningEnabled) {
    +      if (curProcessTime + minRetentionTime > oldExpiredTime) {
    +        newExpiredTime = curProcessTime + maxRetentionTime
    +      }
    +    }
    +    newExpiredTime
    +  }
    +
    +  /**
    +    * Puts or Retract an element from the input stream into state and search the other state to
    +    * output records meet the condition. Records will be expired in state if state retention time
    +    * has been specified.
    +    */
    +  def processElement(
    +      value: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow],
    +      timerState: ValueState[Long],
    +      currentSideState: MapState[Row, JTuple2[Int, Long]],
    +      otherSideState: MapState[Row, JTuple2[Int, Long]],
    +      isLeft: Boolean): Unit = {
    +
    +    cRowWrapper.out = out
    +    cRowWrapper.setChange(value.change)
    +
    +    val curProcessTime = ctx.timerService.currentProcessingTime
    +    val oldCntAndExpiredTime = currentSideState.get(value.row)
    +    val cntAndExpiredTime = if (null == oldCntAndExpiredTime) {
    +      JTuple2.of(0, -1L)
    +    } else {
    +      oldCntAndExpiredTime
    +    }
    +
    +    cntAndExpiredTime.f1 = getNewExpiredTime(curProcessTime, cntAndExpiredTime.f1)
    +    if (stateCleaningEnabled && timerState.value() == 0) {
    +      timerState.update(cntAndExpiredTime.f1)
    +      ctx.timerService().registerProcessingTimeTimer(cntAndExpiredTime.f1)
    +    }
    +
    +    // update current side stream state
    +    if (!value.asInstanceOf[CRow].change) {
    --- End diff --
    
    Unnecessary casting.


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r143394582
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamJoinRule.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.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.flink.table.api.TableConfig
    +import org.apache.flink.table.calcite.FlinkTypeFactory
    +import org.apache.flink.table.plan.nodes.FlinkConventions
    +import org.apache.flink.table.plan.nodes.datastream.DataStreamJoin
    +import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
    +import org.apache.flink.table.plan.schema.RowSchema
    +import org.apache.flink.table.runtime.join.WindowJoinUtil
    +import scala.collection.JavaConverters._
    +
    +class DataStreamJoinRule
    +  extends ConverterRule(
    +    classOf[FlinkLogicalJoin],
    +    FlinkConventions.LOGICAL,
    +    FlinkConventions.DATASTREAM,
    +    "DataStreamJoinRule") {
    +
    +  override def matches(call: RelOptRuleCall): Boolean = {
    +    val join: FlinkLogicalJoin = call.rel(0).asInstanceOf[FlinkLogicalJoin]
    +    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)
    +
    +    // remaining predicate must not access time attributes
    +    val remainingPredsAccessTime = remainingPreds.isDefined &&
    +      WindowJoinUtil.accessesTimeAttribute(remainingPreds.get, join.getRowType)
    +
    +    // Check that no event-time attributes are in the input.
    +    val rowTimeAttrInOutput = join.getRowType.getFieldList.asScala
    +      .exists(f => FlinkTypeFactory.isRowtimeIndicatorType(f.getType))
    +
    +    if (!windowBounds.isDefined && !remainingPredsAccessTime && !rowTimeAttrInOutput) {
    --- End diff --
    
    Great, thanks! I'll do another pass over the PR once you updated it. Thank you.


---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r132300001
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeNonWindowInnerJoin.scala ---
    @@ -0,0 +1,275 @@
    +/*
    + * 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.RichFlatJoinFunction
    +import org.apache.flink.api.common.state._
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.java.typeutils.{ResultTypeQueryable, TupleTypeInfo}
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.api.{StreamQueryConfig, Types}
    +import org.apache.flink.table.runtime.CRowWrappingMultiOuputCollector
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
    +
    +/**
    +  * Connect data for left stream and right stream. Only use for innerJoin.
    +  *
    +  * @param joiner           join function
    +  * @param leftType         the input type of left stream
    +  * @param rightType        the input type of right stream
    +  * @param resultType       the output type of join
    +  * @param queryConfig
    +  */
    +class ProcTimeNonWindowInnerJoin(
    +    joiner: RichFlatJoinFunction[Row, Row, Row],
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    resultType: TypeInformation[CRow],
    +    queryConfig: StreamQueryConfig) extends
    +  CoProcessFunction[CRow, CRow, CRow] with ResultTypeQueryable[CRow] {
    +
    +
    +  // state to hold left stream element
    +  private var leftState: MapState[Row, JTuple2[Int, Long]] = null
    +  // state to hold right stream element
    +  private var rightState: MapState[Row, JTuple2[Int, Long]] = null
    +  private var cRowWrapper: CRowWrappingMultiOuputCollector = null
    +
    +  private val minRetentionTime: Long = queryConfig.getMinIdleStateRetentionTime
    +  private val maxRetentionTime: Long = queryConfig.getMaxIdleStateRetentionTime
    +  private val stateCleaningEnabled: Boolean = minRetentionTime > 1
    +
    +  // state to record last timer of left stream, 0 means no timer
    +  private var timerState1: ValueState[Long] = _
    +  // state to record last timer of right stream, 0 means no timer
    +  private var timerState2: ValueState[Long] = _
    +
    +
    +  override def open(parameters: Configuration): Unit = {
    +    // initialize left and right state
    +    val tupleTypeInfo = new TupleTypeInfo[JTuple2[Int, Long]](Types.INT, Types.LONG)
    +    val leftStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "left", leftType, tupleTypeInfo)
    +    val rightStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, Long]](
    +      "right", rightType, tupleTypeInfo)
    +    leftState = getRuntimeContext.getMapState(leftStateDescriptor)
    +    rightState = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // initialize timer state
    +    val valueStateDescriptor1 = new ValueStateDescriptor[Long]("timervaluestate1", classOf[Long])
    +    timerState1 = getRuntimeContext.getState(valueStateDescriptor1)
    +    val valueStateDescriptor2 = new ValueStateDescriptor[Long]("timervaluestate2", classOf[Long])
    +    timerState2 = getRuntimeContext.getState(valueStateDescriptor2)
    +
    +    cRowWrapper = new CRowWrappingMultiOuputCollector()
    +    joiner.setRuntimeContext(getRuntimeContext)
    +    joiner.open(parameters)
    +  }
    +
    +  /**
    +    * Process left stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx   The ctx to register timer or get current time
    +    * @param out   The collector for returning result values.
    +    *
    +    */
    +  override def processElement1(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, timerState1, leftState, rightState, true)
    +  }
    +
    +  /**
    +    * Process right stream records
    +    *
    +    * @param valueC The input value.
    +    * @param ctx   The ctx to register timer or get current time
    +    * @param out   The collector for returning result values.
    +    *
    +    */
    +  override def processElement2(
    +      valueC: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow]): Unit = {
    +
    +    processElement(valueC, ctx, out, timerState2, rightState, leftState, false)
    +  }
    +
    +
    +  /**
    +    * Called when a processing timer trigger.
    +    * Expire left/right records which are expired in left and right state.
    +    *
    +    * @param timestamp The timestamp of the firing timer.
    +    * @param ctx       The ctx to register timer or get current time
    +    * @param out       The collector for returning result values.
    +    */
    +  override def onTimer(
    +      timestamp: Long,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext,
    +      out: Collector[CRow]): Unit = {
    +
    +    if (stateCleaningEnabled && timerState1.value == timestamp) {
    +      expireOutTimeRow(
    +        timestamp,
    +        leftState,
    +        timerState1,
    +        ctx
    +      )
    +    }
    +
    +    if (stateCleaningEnabled && timerState2.value == timestamp) {
    +      expireOutTimeRow(
    +        timestamp,
    +        rightState,
    +        timerState2,
    +        ctx
    +      )
    +    }
    +  }
    +
    +
    +  def getNewExpiredTime(
    +      curProcessTime: Long,
    +      oldExpiredTime: Long): Long = {
    +
    +    var newExpiredTime = oldExpiredTime
    +    if (stateCleaningEnabled) {
    +      if (-1 == oldExpiredTime || (curProcessTime + minRetentionTime) > oldExpiredTime) {
    +        newExpiredTime = curProcessTime + maxRetentionTime
    +      }
    +    }
    +    newExpiredTime
    +  }
    +
    +  /**
    +    * Puts or Retract an element from the input stream into state and search the other state to
    +    * output records meet the condition. Records will be expired in state if state retention time
    +    * has been specified.
    +    */
    +  def processElement(
    +      value: CRow,
    +      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      out: Collector[CRow],
    +      timerState: ValueState[Long],
    +      currentSideState: MapState[Row, JTuple2[Int, Long]],
    +      otherSideState: MapState[Row, JTuple2[Int, Long]],
    +      isLeft: Boolean): Unit = {
    +
    +    cRowWrapper.out = out
    +    cRowWrapper.setChange(value.change)
    +
    +    val curProcessTime = ctx.timerService.currentProcessingTime
    +    var oldCnt = 0
    +    var oldExpiredTime: Long = -1
    +
    +    val currentRowCntAndExpiredTime = currentSideState.get(value.row)
    +    if (currentRowCntAndExpiredTime != null) {
    +      oldCnt = currentRowCntAndExpiredTime.f0
    +      oldExpiredTime = currentRowCntAndExpiredTime.f1
    +    }
    +
    +    val newExpiredTime = getNewExpiredTime(curProcessTime, oldExpiredTime)
    +    if (stateCleaningEnabled && timerState.value() == 0) {
    +      timerState.update(newExpiredTime)
    +      ctx.timerService().registerProcessingTimeTimer(newExpiredTime)
    +    }
    +
    +    // update current side stream state
    +    if (!value.asInstanceOf[CRow].change) {
    +      oldCnt = oldCnt - 1
    +      if (oldCnt <= 0) {
    +        currentSideState.remove(value.row)
    +      } else {
    +        currentSideState.put(value.row, JTuple2.of(oldCnt, newExpiredTime))
    --- End diff --
    
    We can reuse `currentRowCntAndExpiredTime` instead of creating a new `Tuple2`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4471: [FLINK-6094] [table] Implement stream-stream proct...

Posted by hequn8128 <gi...@git.apache.org>.
Github user hequn8128 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r148947758
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/UpdatingPlanChecker.scala ---
    @@ -56,16 +58,20 @@ object UpdatingPlanChecker {
       }
     
       /** Identifies unique key fields in the output of a RelNode. */
    -  private class UniqueKeyExtractor extends RelVisitor {
    +  private class UniqueKeyExtractor {
    --- End diff --
    
    Yes, it is necessary. Currently, output table must have keys if the table has changes (is not append-only) or it will lead to compilation exceptions . You can check this logic in `org.apache.flink.table.api.StreamTableEnvironment` 


---