[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2018-01-10 Thread ASF GitHub Bot (JIRA)

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

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

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.


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
> Fix For: 1.5.0
>
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2018-01-10 Thread ASF GitHub Bot (JIRA)

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

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

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?


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
> Fix For: 1.5.0
>
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2018-01-10 Thread ASF GitHub Bot (JIRA)

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

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

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.  : )


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
> Fix For: 1.5.0
>
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2018-01-09 Thread ASF GitHub Bot (JIRA)

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

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

Github user asfgit closed the pull request at:

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


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
> Fix For: 1.5.0
>
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2018-01-08 Thread ASF GitHub Bot (JIRA)

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

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

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 :)


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2018-01-08 Thread ASF GitHub Bot (JIRA)

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

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

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.


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-12-22 Thread ASF GitHub Bot (JIRA)

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

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

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 ?


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-12-22 Thread ASF GitHub Bot (JIRA)

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

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

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.


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-12-22 Thread ASF GitHub Bot (JIRA)

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

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

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?


> Implement stream-stream proctime non-window  inner join
> ---
>
>   

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-12-22 Thread ASF GitHub Bot (JIRA)

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

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

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 resultTypethe 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", 

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-12-22 Thread ASF GitHub Bot (JIRA)

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

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

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 resultTypethe 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 ?


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-12-22 Thread ASF GitHub Bot (JIRA)

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

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

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)`).


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-12-22 Thread ASF GitHub Bot (JIRA)

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

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

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?


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-12-01 Thread ASF GitHub Bot (JIRA)

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

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

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. 


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-11-23 Thread ASF GitHub Bot (JIRA)

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

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

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.


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-11-23 Thread ASF GitHub Bot (JIRA)

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

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

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 = 

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-11-23 Thread ASF GitHub Bot (JIRA)

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

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

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 = 

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-11-23 Thread ASF GitHub Bot (JIRA)

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

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

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.


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-11-23 Thread ASF GitHub Bot (JIRA)

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

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

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.


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-11-23 Thread ASF GitHub Bot (JIRA)

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

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

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.


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-11-23 Thread ASF GitHub Bot (JIRA)

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

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

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 resultTypethe 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`?


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-11-23 Thread ASF GitHub Bot (JIRA)

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

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

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`.


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-11-23 Thread ASF GitHub Bot (JIRA)

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

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

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?


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-11-23 Thread ASF GitHub Bot (JIRA)

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

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

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 resultTypethe 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", 

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-11-23 Thread ASF GitHub Bot (JIRA)

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

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

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`


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-11-23 Thread ASF GitHub Bot (JIRA)

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

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

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?


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan 

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-11-23 Thread ASF GitHub Bot (JIRA)

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

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

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?


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-11-23 Thread ASF GitHub Bot (JIRA)

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

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

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?


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-11-23 Thread ASF GitHub Bot (JIRA)

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

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

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 resultTypethe 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", 

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-11-23 Thread ASF GitHub Bot (JIRA)

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

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

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 = 

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-11-23 Thread ASF GitHub Bot (JIRA)

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

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

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`


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-11-23 Thread ASF GitHub Bot (JIRA)

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

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

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`


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-11-23 Thread ASF GitHub Bot (JIRA)

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

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

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.


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-11-23 Thread ASF GitHub Bot (JIRA)

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

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

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 resultTypethe 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", 

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-11-23 Thread ASF GitHub Bot (JIRA)

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

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

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 resultTypethe 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", 

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-11-23 Thread ASF GitHub Bot (JIRA)

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

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

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 resultTypethe 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.


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-11-23 Thread ASF GitHub Bot (JIRA)

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

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

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.


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-11-23 Thread ASF GitHub Bot (JIRA)

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

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

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 

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-11-23 Thread ASF GitHub Bot (JIRA)

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

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

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 resultTypethe 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", 

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-11-23 Thread ASF GitHub Bot (JIRA)

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

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

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 

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-11-23 Thread ASF GitHub Bot (JIRA)

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

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

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. 


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
>

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-11-06 Thread ASF GitHub Bot (JIRA)

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

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

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


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-11-04 Thread ASF GitHub Bot (JIRA)

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

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

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.


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-11-04 Thread ASF GitHub Bot (JIRA)

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

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

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 resultTypethe 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", 

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-11-04 Thread ASF GitHub Bot (JIRA)

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

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

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 resultTypethe 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", 

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-11-04 Thread ASF GitHub Bot (JIRA)

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

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

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 resultTypethe 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", 

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-11-04 Thread ASF GitHub Bot (JIRA)

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

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

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` 


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-11-04 Thread ASF GitHub Bot (JIRA)

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

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

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 ?


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-11-04 Thread ASF GitHub Bot (JIRA)

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

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

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.


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-10-21 Thread ASF GitHub Bot (JIRA)

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

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

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 

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-10-21 Thread ASF GitHub Bot (JIRA)

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

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

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 resultTypethe 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", 

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-10-21 Thread ASF GitHub Bot (JIRA)

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

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

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 resultTypethe 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", 

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-10-21 Thread ASF GitHub Bot (JIRA)

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

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

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 resultTypethe 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", 

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-10-21 Thread ASF GitHub Bot (JIRA)

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

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

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 

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-10-21 Thread ASF GitHub Bot (JIRA)

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

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

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.


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-10-21 Thread ASF GitHub Bot (JIRA)

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

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

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).


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-10-21 Thread ASF GitHub Bot (JIRA)

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

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

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?


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-10-21 Thread ASF GitHub Bot (JIRA)

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

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

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 resultTypethe 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", 

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-10-21 Thread ASF GitHub Bot (JIRA)

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

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

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.


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-10-20 Thread ASF GitHub Bot (JIRA)

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

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

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. 


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-10-15 Thread ASF GitHub Bot (JIRA)

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

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

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 :)


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-10-09 Thread ASF GitHub Bot (JIRA)

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

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

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.


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-10-06 Thread ASF GitHub Bot (JIRA)

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

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

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


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-10-04 Thread ASF GitHub Bot (JIRA)

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

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

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


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-09-25 Thread ASF GitHub Bot (JIRA)

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

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

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?
  


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn 

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-08-10 Thread ASF GitHub Bot (JIRA)

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

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

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]`


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-08-10 Thread ASF GitHub Bot (JIRA)

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

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

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 rightTypethe 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` for?


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-08-10 Thread ASF GitHub Bot (JIRA)

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

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

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.


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-08-10 Thread ASF GitHub Bot (JIRA)

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

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

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.


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-08-10 Thread ASF GitHub Bot (JIRA)

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

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

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 rightTypethe 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`?


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-08-10 Thread ASF GitHub Bot (JIRA)

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

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

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)`


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-08-10 Thread ASF GitHub Bot (JIRA)

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

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

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)))
```


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-08-10 Thread ASF GitHub Bot (JIRA)

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

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

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.


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-08-10 Thread ASF GitHub Bot (JIRA)

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

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

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.


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-08-10 Thread ASF GitHub Bot (JIRA)

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

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

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 {
+ 

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-08-10 Thread ASF GitHub Bot (JIRA)

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

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

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).


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> 

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-08-10 Thread ASF GitHub Bot (JIRA)

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

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

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.`


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-08-10 Thread ASF GitHub Bot (JIRA)

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

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

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 rightTypethe 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
+  

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-08-10 Thread ASF GitHub Bot (JIRA)

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

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

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 

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-08-10 Thread ASF GitHub Bot (JIRA)

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

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

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).


> Implement stream-stream proctime non-window  inner join

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-08-10 Thread ASF GitHub Bot (JIRA)

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

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

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 rightTypethe 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
+  

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-08-10 Thread ASF GitHub Bot (JIRA)

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

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

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 rightTypethe 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


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-08-10 Thread ASF GitHub Bot (JIRA)

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

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

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`


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-08-10 Thread ASF GitHub Bot (JIRA)

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

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

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 rightTypethe 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
+  

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-08-10 Thread ASF GitHub Bot (JIRA)

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

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

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.


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-08-10 Thread ASF GitHub Bot (JIRA)

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

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

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


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-08-10 Thread ASF GitHub Bot (JIRA)

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

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

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

https://github.com/apache/flink/pull/4471#discussion_r13231
  
--- 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 rightTypethe 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
+  

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-08-10 Thread ASF GitHub Bot (JIRA)

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

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

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 

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-08-10 Thread ASF GitHub Bot (JIRA)

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

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

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 rightTypethe 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
+  

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-08-10 Thread ASF GitHub Bot (JIRA)

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

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

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 rightTypethe 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
+  

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-08-10 Thread ASF GitHub Bot (JIRA)

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

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

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 

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-08-10 Thread ASF GitHub Bot (JIRA)

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

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

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


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-08-10 Thread ASF GitHub Bot (JIRA)

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

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

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 rightTypethe input type of right stream
+  * @param resultType   the output type of join
+  * @param queryConfig
--- End diff --

Please add parameter description


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-08-10 Thread ASF GitHub Bot (JIRA)

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

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

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).`?


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-08-10 Thread ASF GitHub Bot (JIRA)

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

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

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 {
+ 

[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-08-10 Thread ASF GitHub Bot (JIRA)

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

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

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]`


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-08-10 Thread ASF GitHub Bot (JIRA)

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

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

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(_))
```


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-08-10 Thread ASF GitHub Bot (JIRA)

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

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

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`


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-08-10 Thread ASF GitHub Bot (JIRA)

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

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

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`


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


[jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join

2017-08-10 Thread ASF GitHub Bot (JIRA)

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

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

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
```


> Implement stream-stream proctime non-window  inner join
> ---
>
> Key: FLINK-6094
> URL: https://issues.apache.org/jira/browse/FLINK-6094
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shaoxuan Wang
>Assignee: Hequn Cheng
>
> This includes:
> 1.Implement stream-stream proctime non-window  inner join
> 2.Implement the retract process logic for join



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


  1   2   >