[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-13 Thread hongyuhong
Github user hongyuhong closed the pull request at:

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


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-12 Thread wuchong
Github user wuchong commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r127113760
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamWindowJoinRule.scala
 ---
@@ -20,28 +20,60 @@ 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.RexNode
+import org.apache.flink.table.api.{TableConfig, TableException}
 import org.apache.flink.table.plan.nodes.FlinkConventions
-import org.apache.flink.table.plan.nodes.datastream.DataStreamRowStreamJoin
+import org.apache.flink.table.plan.nodes.datastream.DataStreamWindowJoin
 import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
 import org.apache.flink.table.plan.schema.RowSchema
-import org.apache.flink.table.runtime.join.JoinUtil
+import org.apache.flink.table.runtime.join.WindowJoinUtil
 
-class DataStreamRowStreamJoinRule
+class DataStreamWindowJoinRule
   extends ConverterRule(
   classOf[FlinkLogicalJoin],
   FlinkConventions.LOGICAL,
   FlinkConventions.DATASTREAM,
   "DataStreamJoinRule") {
 
+  /** Time indicator type **/
+  private var timeType: RelDataType = _
+
+  /** left input lower boudary **/
+  private var leftLowerBoundary: Long = _
+
+  /** left input upper boudary **/
+  private var leftUpperBoundary: Long = _
+
+  /** remain join condition exclude equal condition and time condition **/
+  private var remainCondition: Option[RexNode] = _
+
   override def matches(call: RelOptRuleCall): Boolean = {
 val join: FlinkLogicalJoin = call.rel(0).asInstanceOf[FlinkLogicalJoin]
 
 val joinInfo = join.analyzeCondition
 
-JoinUtil.isStreamStreamJoin(
-  joinInfo.getRemaining(join.getCluster.getRexBuilder),
-  join.getRowType)
+try {
+  val leftRowSchema = new RowSchema(join.getLeft.getRowType)
+
+  val result =
+WindowJoinUtil.analyzeTimeBoundary(
+  joinInfo.getRemaining(join.getCluster.getRexBuilder),
+  leftRowSchema.logicalType.getFieldCount,
+  leftRowSchema.physicalType.getFieldCount,
+  join.getRowType,
+  join.getCluster.getRexBuilder,
+  TableConfig.DEFAULT)
+  timeType = result._1
+  leftLowerBoundary = result._2
+  leftUpperBoundary = result._3
+  remainCondition = result._4
+  true
--- End diff --

Hi @fhueske , I think a good reason to put the check in `translateToPlan` 
is for throwing a proper exception message, such as `Left join is not 
supported...`. If we put the check in rule's matches or throw the exception in 
rule's matches, it will throw a obscure message with the logical plan.

IMO, a meaningful message is important for users. 


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-12 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r126987096
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamWindowJoinRule.scala
 ---
@@ -20,28 +20,60 @@ 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.RexNode
+import org.apache.flink.table.api.{TableConfig, TableException}
 import org.apache.flink.table.plan.nodes.FlinkConventions
-import org.apache.flink.table.plan.nodes.datastream.DataStreamRowStreamJoin
+import org.apache.flink.table.plan.nodes.datastream.DataStreamWindowJoin
 import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
 import org.apache.flink.table.plan.schema.RowSchema
-import org.apache.flink.table.runtime.join.JoinUtil
+import org.apache.flink.table.runtime.join.WindowJoinUtil
 
-class DataStreamRowStreamJoinRule
+class DataStreamWindowJoinRule
   extends ConverterRule(
   classOf[FlinkLogicalJoin],
   FlinkConventions.LOGICAL,
   FlinkConventions.DATASTREAM,
   "DataStreamJoinRule") {
 
+  /** Time indicator type **/
+  private var timeType: RelDataType = _
+
+  /** left input lower boudary **/
+  private var leftLowerBoundary: Long = _
+
+  /** left input upper boudary **/
+  private var leftUpperBoundary: Long = _
+
+  /** remain join condition exclude equal condition and time condition **/
+  private var remainCondition: Option[RexNode] = _
+
   override def matches(call: RelOptRuleCall): Boolean = {
 val join: FlinkLogicalJoin = call.rel(0).asInstanceOf[FlinkLogicalJoin]
 
 val joinInfo = join.analyzeCondition
 
-JoinUtil.isStreamStreamJoin(
-  joinInfo.getRemaining(join.getCluster.getRexBuilder),
-  join.getRowType)
+try {
+  val leftRowSchema = new RowSchema(join.getLeft.getRowType)
+
+  val result =
+WindowJoinUtil.analyzeTimeBoundary(
+  joinInfo.getRemaining(join.getCluster.getRexBuilder),
+  leftRowSchema.logicalType.getFieldCount,
+  leftRowSchema.physicalType.getFieldCount,
+  join.getRowType,
+  join.getCluster.getRexBuilder,
+  TableConfig.DEFAULT)
+  timeType = result._1
+  leftLowerBoundary = result._2
+  leftUpperBoundary = result._3
+  remainCondition = result._4
+  true
--- End diff --

`DataStreamWindowJoin` should support outer joins, but at the moment it 
does not. 
Until it supports outer joins, I would not translate to 
`DataStreamWindowJoin`.


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-05 Thread hongyuhong
Github user hongyuhong commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125807639
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamWindowJoinRule.scala
 ---
@@ -20,28 +20,60 @@ 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.RexNode
+import org.apache.flink.table.api.{TableConfig, TableException}
 import org.apache.flink.table.plan.nodes.FlinkConventions
-import org.apache.flink.table.plan.nodes.datastream.DataStreamRowStreamJoin
+import org.apache.flink.table.plan.nodes.datastream.DataStreamWindowJoin
 import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
 import org.apache.flink.table.plan.schema.RowSchema
-import org.apache.flink.table.runtime.join.JoinUtil
+import org.apache.flink.table.runtime.join.WindowJoinUtil
 
-class DataStreamRowStreamJoinRule
+class DataStreamWindowJoinRule
   extends ConverterRule(
   classOf[FlinkLogicalJoin],
   FlinkConventions.LOGICAL,
   FlinkConventions.DATASTREAM,
   "DataStreamJoinRule") {
 
+  /** Time indicator type **/
+  private var timeType: RelDataType = _
+
+  /** left input lower boudary **/
+  private var leftLowerBoundary: Long = _
+
+  /** left input upper boudary **/
+  private var leftUpperBoundary: Long = _
+
+  /** remain join condition exclude equal condition and time condition **/
+  private var remainCondition: Option[RexNode] = _
+
   override def matches(call: RelOptRuleCall): Boolean = {
 val join: FlinkLogicalJoin = call.rel(0).asInstanceOf[FlinkLogicalJoin]
 
 val joinInfo = join.analyzeCondition
 
-JoinUtil.isStreamStreamJoin(
-  joinInfo.getRemaining(join.getCluster.getRexBuilder),
-  join.getRowType)
+try {
+  val leftRowSchema = new RowSchema(join.getLeft.getRowType)
+
+  val result =
+WindowJoinUtil.analyzeTimeBoundary(
+  joinInfo.getRemaining(join.getCluster.getRexBuilder),
+  leftRowSchema.logicalType.getFieldCount,
+  leftRowSchema.physicalType.getFieldCount,
+  join.getRowType,
+  join.getCluster.getRexBuilder,
+  TableConfig.DEFAULT)
+  timeType = result._1
+  leftLowerBoundary = result._2
+  leftUpperBoundary = result._3
+  remainCondition = result._4
+  true
--- End diff --

Should we check the type here?  I think the DataStreamWindowJoin can 
support inner and outer join, so i check the jointype in translateToPlan for 
convenient expansion.


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-04 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125448648
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeWindowInnerJoin.scala
 ---
@@ -0,0 +1,326 @@
+/*
+ * 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 java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftLowerBound
+  *the left stream lower bound, and -leftLowerBound is the right 
stream upper bound
+  * @param leftUpperBound
+  *the left stream upper bound, and -leftUpperBound is the right 
stream lower bound
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeWindowInnerJoin(
+private val leftLowerBound: Long,
+private val leftUpperBound: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** 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] = _
+
+  private val leftStreamWinSize: Long = if (leftLowerBound < 0) 
-leftLowerBound else 0
+  private val rightStreamWinSize: Long = if (leftUpperBound > 0) 
leftUpperBound else 0
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+
+  override def open(config: Configuration) {
+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()
+
+listToRemove = new util.ArrayList[Long]()
+cRowWrapper = new CRowWrappingCollector()
+
+// initialize row state
+val rowListTypeInfo1: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](element1Type)
+val mapStateDescriptor1: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]]("row1mapstate",
+BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-04 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125459202
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala
 ---
@@ -300,22 +271,27 @@ object JoinUtil {
   }
 }
 
-val literalRex = replaceTimeFieldWithLiteral(rexNode)
+val literalLeftRex = replaceTimeFieldWithLiteral(leftRexNode)
+val literalRightRex = replaceTimeFieldWithLiteral(rightRexNode)
 
 val exprReducer = new ExpressionReducer(config)
 val originList = new util.ArrayList[RexNode]()
-originList.add(literalRex)
+originList.add(literalLeftRex)
+originList.add(literalRightRex)
 val reduceList = new util.ArrayList[RexNode]()
 exprReducer.reduce(rexBuilder, originList, reduceList)
 
-reduceList.get(0) match {
+val literals = reduceList.map(f => f match {
   case call: RexCall =>
--- End diff --

Which case is this? 
Isn't a blind cast of the first operand to `RexLiteral` dangerous and could 
result in a `ClassCastException`?


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-04 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125463323
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala
 ---
@@ -0,0 +1,361 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object WindowJoinUtil {
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   join condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+* @return  timetype, left lower boundary, right lower boundary, remain 
condition
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType
+  )
+
+if (timeTerms.size != 2) {
+  throw new TableException("A time-based stream join requires exactly 
" +
+"two join predicates that bound the time in both directions.")
+}
+
+// extract time offset from the time indicator conditon
+val streamTimeOffsets =
+  timeTerms.map(x => extractTimeOffsetFromCondition(x._3, x._2, 
rexBuilder, config))
+
+val (leftLowerBound, leftUpperBound) =
+  streamTimeOffsets match {
+case Seq((x, true), (y, false)) => (x, y)
+case Seq((x, false), (y, true)) => (y, x)
+case _ =>
+  throw new TableException(
+"Time-based join conditions must reference the time attribute 
of both input tables.")
+  }
+
+// compose the remain condition list into one condition
+val remainCondition =
+  remainTerms match {
+case Seq() => None
+case _ =>
+  // turn the logical field index to physical field index
+  def transInputRef(expr: RexNode): RexNode = {
--- End diff --

Can we use `RowSchema.mapRexNode()` to compute the mapping?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-04 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125461422
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala
 ---
@@ -210,77 +187,71 @@ object JoinUtil {
   case _ => Seq()
 }
   case c: RexCall =>
-c.operands.map(analyzeSingleConditionTerm(_, leftFieldCount, 
inputType)).reduce(_++_)
+c.operands.map(extractTimeIndicatorAccesses(_, leftFieldCount, 
inputType)).reduce(_++_)
   case _ => Seq()
 }
   }
 
   /**
-* Extract time offset and determain which table the offset belong to
+* Extract time offset and determain it's the lower bound of left 
stream or the upper bound
--- End diff --

```
Computes the absolute bound on the left operand of a comparison expression 
and 
whether the bound is an upper or lower bound.
```


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-04 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125482464
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala
 ---
@@ -87,44 +98,38 @@ class DataStreamRowStreamJoin(
 
 val config = tableEnv.getConfig
 
+val leftIsAccRetract = DataStreamRetractionRules.isAccRetract(left)
--- End diff --

Retractions are just one (of two) ways to encode updates in a stream. This 
join implementation does not support any kind of updates (which is fine). 
However, we have to check if the input streams produce updates or not. The 
`StreamTableEnvironment` has a private class `AppendOnlyValidator`. We should 
move `AppendOnlyValidator` and `UniqueKeyExtractor` into a class `UpdateUtils` 
together with the methods `isAppendOnly()` and `getUniqueKeyFields()`. 
`isAppendOnly()` checks if an operator produces updates or not and should be 
used here.


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-04 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125458518
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala
 ---
@@ -0,0 +1,361 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object WindowJoinUtil {
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   join condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+* @return  timetype, left lower boundary, right lower boundary, remain 
condition
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType
+  )
+
+if (timeTerms.size != 2) {
+  throw new TableException("A time-based stream join requires exactly 
" +
+"two join predicates that bound the time in both directions.")
+}
+
+// extract time offset from the time indicator conditon
+val streamTimeOffsets =
+  timeTerms.map(x => extractTimeOffsetFromCondition(x._3, x._2, 
rexBuilder, config))
+
+val (leftLowerBound, leftUpperBound) =
+  streamTimeOffsets match {
+case Seq((x, true), (y, false)) => (x, y)
+case Seq((x, false), (y, true)) => (y, x)
+case _ =>
+  throw new TableException(
+"Time-based join conditions must reference the time attribute 
of both input tables.")
+  }
+
+// compose the remain condition list into one condition
+val remainCondition =
+  remainTerms match {
+case Seq() => None
+case _ =>
+  // turn the logical field index to physical field index
+  def transInputRef(expr: RexNode): RexNode = {
+expr match {
+  case c: RexCall =>
+val newOps = c.operands.map(transInputRef(_))
+rexBuilder.makeCall(c.getType, c.getOperator, newOps)
+  case i: RexInputRef if i.getIndex >= leftLogicalFieldCnt =>
+

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-04 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125448784
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeWindowInnerJoin.scala
 ---
@@ -0,0 +1,326 @@
+/*
+ * 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 java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftLowerBound
+  *the left stream lower bound, and -leftLowerBound is the right 
stream upper bound
+  * @param leftUpperBound
+  *the left stream upper bound, and -leftUpperBound is the right 
stream lower bound
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeWindowInnerJoin(
+private val leftLowerBound: Long,
+private val leftUpperBound: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** 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] = _
+
+  private val leftStreamWinSize: Long = if (leftLowerBound < 0) 
-leftLowerBound else 0
+  private val rightStreamWinSize: Long = if (leftUpperBound > 0) 
leftUpperBound else 0
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+
+  override def open(config: Configuration) {
+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()
+
+listToRemove = new util.ArrayList[Long]()
+cRowWrapper = new CRowWrappingCollector()
+
+// initialize row state
+val rowListTypeInfo1: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](element1Type)
+val mapStateDescriptor1: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]]("row1mapstate",
+BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-04 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125457556
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala
 ---
@@ -210,77 +187,71 @@ object JoinUtil {
   case _ => Seq()
 }
   case c: RexCall =>
-c.operands.map(analyzeSingleConditionTerm(_, leftFieldCount, 
inputType)).reduce(_++_)
+c.operands.map(extractTimeIndicatorAccesses(_, leftFieldCount, 
inputType)).reduce(_++_)
   case _ => Seq()
 }
   }
 
   /**
-* Extract time offset and determain which table the offset belong to
+* Extract time offset and determain it's the lower bound of left 
stream or the upper bound
+*
+* @return window boundary, is left lower bound
 */
   def extractTimeOffsetFromCondition(
   timeTerm: RexNode,
   isLeftExprBelongLeftTable: Boolean,
   rexBuilder: RexBuilder,
-  config: TableConfig) = {
+  config: TableConfig): (Long, Boolean) = {
 
 val timeCall: RexCall = timeTerm.asInstanceOf[RexCall]
-val leftLiteral =
-  reduceTimeExpression(
-timeCall.operands.get(0),
-rexBuilder,
-config)
 
-val rightLiteral =
-  reduceTimeExpression(
-timeCall.operands.get(1),
-rexBuilder,
-config)
-
-val (tmpTimeOffset: Long, isLeftTableTimeOffset: Boolean) =
+val isLeftLowerBound: Boolean =
   timeTerm.getKind match {
-// e.g a.proctime > b.proctime - 5 sec, we need to store stream a.
-// the left expr(a) belong to left table, so the offset belong to 
left table
+// e.g a.proctime > b.proctime - 5 sec, then it's the lower bound 
of a and the value is -5
+// e.g b.proctime > a.proctime - 5 sec, then it's not the lower 
bound of a but upper bound
 case kind @ (SqlKind.GREATER_THAN | SqlKind.GREATER_THAN_OR_EQUAL) 
=>
-  (rightLiteral - leftLiteral, isLeftExprBelongLeftTable)
-// e.g a.proctime < b.proctime + 5 sec, we need to store stream b.
+  isLeftExprBelongLeftTable
+// e.g a.proctime < b.proctime + 5 sec, the the upper bound of a 
is 5
 case kind @ (SqlKind.LESS_THAN | SqlKind.LESS_THAN_OR_EQUAL) =>
-  (leftLiteral - rightLiteral, !isLeftExprBelongLeftTable)
-case _ => 0
+  !isLeftExprBelongLeftTable
+case _ =>
+  throw new TableException("Unsupport time-condition.")
--- End diff --

Unsupport -> Unsupported


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-04 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125478938
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala
 ---
@@ -160,37 +181,52 @@ class JoinHarnessTest extends HarnessTestBase{
 
 testHarness.setProcessingTime(1)
 testHarness.processElement1(new StreamRecord(
-  CRow(Row.of(1: JInt, "aaa"), true), 1))
+  CRow(Row.of(1: JInt, "aaa1"), true), 1))
 testHarness.setProcessingTime(2)
 testHarness.processElement1(new StreamRecord(
-  CRow(Row.of(2: JInt, "bbb"), true), 2))
+  CRow(Row.of(2: JInt, "aaa2"), true), 2))
 testHarness.setProcessingTime(3)
 testHarness.processElement1(new StreamRecord(
-  CRow(Row.of(1: JInt, "aaa2"), true), 3))
+  CRow(Row.of(1: JInt, "aaa3"), true), 3))
+assert(testHarness.numKeyedStateEntries() == 4)
+assert(testHarness.numProcessingTimeTimers() == 2)
 
+// Do not store b elemets
--- End diff --

elemets -> elements


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-04 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125464505
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/JoinTest.scala
 ---
@@ -67,27 +66,47 @@ class JoinTest extends TableTestBase {
 streamUtil.verifySql(sqlQuery, expected)
   }
 
+  @Test(expected = classOf[TableException])
--- End diff --

Please add a brief comment why the query is expected to fail.


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-04 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125446795
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeWindowInnerJoin.scala
 ---
@@ -238,13 +247,13 @@ class ProcTimeInnerJoin(
 
 }
 
-// loop the the other stream elments
+// loop the the other stream elements
 val oppositeKeyIter = oppoRowMapState.keys().iterator()
 while (oppositeKeyIter.hasNext) {
   val eleTime = oppositeKeyIter.next()
-  if (eleTime < oppoExpiredTime) {
+  if (eleTime < oppoLowerTime) {
 listToRemove.add(eleTime)
-  } else {
+  } else if (eleTime <= oppoUpperTime){
--- End diff --

+space between `) {`


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-04 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125454546
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala
 ---
@@ -191,8 +166,10 @@ object JoinUtil {
 
   /**
* analysis if condition term has time indicator
--- End diff --

```
Extracts all time indicator attributes that are accessed in an expression.
```


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-04 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125447106
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeWindowInnerJoin.scala
 ---
@@ -0,0 +1,326 @@
+/*
+ * 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 java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftLowerBound
+  *the left stream lower bound, and -leftLowerBound is the right 
stream upper bound
+  * @param leftUpperBound
+  *the left stream upper bound, and -leftUpperBound is the right 
stream lower bound
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeWindowInnerJoin(
+private val leftLowerBound: Long,
+private val leftUpperBound: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** 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] = _
+
+  private val leftStreamWinSize: Long = if (leftLowerBound < 0) 
-leftLowerBound else 0
+  private val rightStreamWinSize: Long = if (leftUpperBound > 0) 
leftUpperBound else 0
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+
+  override def open(config: Configuration) {
+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()
+
+listToRemove = new util.ArrayList[Long]()
+cRowWrapper = new CRowWrappingCollector()
+
+// initialize row state
+val rowListTypeInfo1: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](element1Type)
+val mapStateDescriptor1: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]]("row1mapstate",
+BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-04 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125440019
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamWindowJoinRule.scala
 ---
@@ -20,28 +20,60 @@ 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.RexNode
+import org.apache.flink.table.api.{TableConfig, TableException}
 import org.apache.flink.table.plan.nodes.FlinkConventions
-import org.apache.flink.table.plan.nodes.datastream.DataStreamRowStreamJoin
+import org.apache.flink.table.plan.nodes.datastream.DataStreamWindowJoin
 import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
 import org.apache.flink.table.plan.schema.RowSchema
-import org.apache.flink.table.runtime.join.JoinUtil
+import org.apache.flink.table.runtime.join.WindowJoinUtil
 
-class DataStreamRowStreamJoinRule
+class DataStreamWindowJoinRule
   extends ConverterRule(
   classOf[FlinkLogicalJoin],
   FlinkConventions.LOGICAL,
   FlinkConventions.DATASTREAM,
   "DataStreamJoinRule") {
 
+  /** Time indicator type **/
+  private var timeType: RelDataType = _
+
+  /** left input lower boudary **/
+  private var leftLowerBoundary: Long = _
+
+  /** left input upper boudary **/
+  private var leftUpperBoundary: Long = _
+
+  /** remain join condition exclude equal condition and time condition **/
+  private var remainCondition: Option[RexNode] = _
+
   override def matches(call: RelOptRuleCall): Boolean = {
 val join: FlinkLogicalJoin = call.rel(0).asInstanceOf[FlinkLogicalJoin]
 
 val joinInfo = join.analyzeCondition
 
-JoinUtil.isStreamStreamJoin(
-  joinInfo.getRemaining(join.getCluster.getRexBuilder),
-  join.getRowType)
+try {
+  val leftRowSchema = new RowSchema(join.getLeft.getRowType)
+
+  val result =
+WindowJoinUtil.analyzeTimeBoundary(
+  joinInfo.getRemaining(join.getCluster.getRexBuilder),
+  leftRowSchema.logicalType.getFieldCount,
+  leftRowSchema.physicalType.getFieldCount,
+  join.getRowType,
+  join.getCluster.getRexBuilder,
+  TableConfig.DEFAULT)
+  timeType = result._1
+  leftLowerBoundary = result._2
+  leftUpperBoundary = result._3
+  remainCondition = result._4
+  true
--- End diff --

Also check for `join.getJoinType == JoinType.INNER`


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-04 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125465060
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/JoinTest.scala
 ---
@@ -101,22 +120,22 @@ class JoinTest extends TableTestBase {
 verifyTimeBoundary(
--- End diff --

I'd add another case where both bounds are negative.


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-04 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125421732
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamWindowJoinRule.scala
 ---
@@ -20,28 +20,60 @@ 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.RexNode
+import org.apache.flink.table.api.{TableConfig, TableException}
 import org.apache.flink.table.plan.nodes.FlinkConventions
-import org.apache.flink.table.plan.nodes.datastream.DataStreamRowStreamJoin
+import org.apache.flink.table.plan.nodes.datastream.DataStreamWindowJoin
 import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
 import org.apache.flink.table.plan.schema.RowSchema
-import org.apache.flink.table.runtime.join.JoinUtil
+import org.apache.flink.table.runtime.join.WindowJoinUtil
 
-class DataStreamRowStreamJoinRule
+class DataStreamWindowJoinRule
   extends ConverterRule(
   classOf[FlinkLogicalJoin],
   FlinkConventions.LOGICAL,
   FlinkConventions.DATASTREAM,
   "DataStreamJoinRule") {
 
+  /** Time indicator type **/
+  private var timeType: RelDataType = _
--- End diff --

I'm not sure if we can store these values as local variables. It kind 
assumes that the rule is called in a certain way (first check `matches()` and 
immediately after that `convert()`. The rule might also be called concurrently 
from different threads. I'd rather run the same analysis again in `convert()`.


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-04 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125452889
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala
 ---
@@ -0,0 +1,361 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object WindowJoinUtil {
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   join condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+* @return  timetype, left lower boundary, right lower boundary, remain 
condition
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType
+  )
+
+if (timeTerms.size != 2) {
+  throw new TableException("A time-based stream join requires exactly 
" +
+"two join predicates that bound the time in both directions.")
+}
+
+// extract time offset from the time indicator conditon
+val streamTimeOffsets =
+  timeTerms.map(x => extractTimeOffsetFromCondition(x._3, x._2, 
rexBuilder, config))
+
+val (leftLowerBound, leftUpperBound) =
+  streamTimeOffsets match {
+case Seq((x, true), (y, false)) => (x, y)
+case Seq((x, false), (y, true)) => (y, x)
+case _ =>
+  throw new TableException(
+"Time-based join conditions must reference the time attribute 
of both input tables.")
+  }
+
+// compose the remain condition list into one condition
+val remainCondition =
+  remainTerms match {
+case Seq() => None
+case _ =>
+  // turn the logical field index to physical field index
+  def transInputRef(expr: RexNode): RexNode = {
+expr match {
+  case c: RexCall =>
+val newOps = c.operands.map(transInputRef(_))
+rexBuilder.makeCall(c.getType, c.getOperator, newOps)
+  case i: RexInputRef if i.getIndex >= leftLogicalFieldCnt =>
+

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-04 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125443593
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeWindowInnerJoin.scala
 ---
@@ -0,0 +1,326 @@
+/*
+ * 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 java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftLowerBound
+  *the left stream lower bound, and -leftLowerBound is the right 
stream upper bound
+  * @param leftUpperBound
+  *the left stream upper bound, and -leftUpperBound is the right 
stream lower bound
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeWindowInnerJoin(
+private val leftLowerBound: Long,
+private val leftUpperBound: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** 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] = _
+
+  private val leftStreamWinSize: Long = if (leftLowerBound < 0) 
-leftLowerBound else 0
+  private val rightStreamWinSize: Long = if (leftUpperBound > 0) 
leftUpperBound else 0
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+
+  override def open(config: Configuration) {
+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()
+
+listToRemove = new util.ArrayList[Long]()
+cRowWrapper = new CRowWrappingCollector()
+
+// initialize row state
+val rowListTypeInfo1: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](element1Type)
+val mapStateDescriptor1: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]]("row1mapstate",
+BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-04 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125456339
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala
 ---
@@ -162,24 +135,26 @@ object JoinUtil {
   (l._1 ++ r._1, l._2 ++ r._2)
 }
   case c: RexCall =>
-val timeIndicators = analyzeSingleConditionTerm(c, leftFieldCount, 
inputType)
+val timeIndicators = extractTimeIndicatorAccesses(c, 
leftFieldCount, inputType)
 timeIndicators match {
   case Seq() =>
 (Seq(), Seq(c))
   case Seq(v1, v2) =>
 if (v1._1 != v2._1) {
-  throw new TableException("The time indicators for each input 
should be the same.")
+  throw new TableException(
+"Both time attributes in a join condition must be of the 
same type.")
 }
 if (v1._2 == v2._2) {
-  throw new TableException("Both input's time indicators is 
needed.")
+  throw new TableException("Time-based join conditions " +
+"must reference the time attribute of both input tables.")
 }
 (Seq((v1._1, v1._2, c)), Seq())
--- End diff --

I think the way the `boolean` field is derived is a bit dangerous. We 
assume here that the `RexCall` is a comparison where the first operand is the 
left term and the second operand is the right term but it could also be any 
other call (with an arbitrary number of operands) that returns a boolean. 
Although we later check that the `RexCall` is a comparison, I think we should 
do this here.


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-04 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125446548
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeWindowInnerJoin.scala
 ---
@@ -238,13 +247,13 @@ class ProcTimeInnerJoin(
 
 }
 
-// loop the the other stream elments
+// loop the the other stream elements
--- End diff --

"the the"


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-04 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125457882
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala
 ---
@@ -210,77 +187,71 @@ object JoinUtil {
   case _ => Seq()
 }
   case c: RexCall =>
-c.operands.map(analyzeSingleConditionTerm(_, leftFieldCount, 
inputType)).reduce(_++_)
+c.operands.map(extractTimeIndicatorAccesses(_, leftFieldCount, 
inputType)).reduce(_++_)
   case _ => Seq()
 }
   }
 
   /**
-* Extract time offset and determain which table the offset belong to
+* Extract time offset and determain it's the lower bound of left 
stream or the upper bound
+*
+* @return window boundary, is left lower bound
 */
   def extractTimeOffsetFromCondition(
   timeTerm: RexNode,
   isLeftExprBelongLeftTable: Boolean,
   rexBuilder: RexBuilder,
-  config: TableConfig) = {
+  config: TableConfig): (Long, Boolean) = {
 
 val timeCall: RexCall = timeTerm.asInstanceOf[RexCall]
-val leftLiteral =
-  reduceTimeExpression(
-timeCall.operands.get(0),
-rexBuilder,
-config)
 
-val rightLiteral =
-  reduceTimeExpression(
-timeCall.operands.get(1),
-rexBuilder,
-config)
-
-val (tmpTimeOffset: Long, isLeftTableTimeOffset: Boolean) =
+val isLeftLowerBound: Boolean =
   timeTerm.getKind match {
-// e.g a.proctime > b.proctime - 5 sec, we need to store stream a.
-// the left expr(a) belong to left table, so the offset belong to 
left table
+// e.g a.proctime > b.proctime - 5 sec, then it's the lower bound 
of a and the value is -5
+// e.g b.proctime > a.proctime - 5 sec, then it's not the lower 
bound of a but upper bound
 case kind @ (SqlKind.GREATER_THAN | SqlKind.GREATER_THAN_OR_EQUAL) 
=>
-  (rightLiteral - leftLiteral, isLeftExprBelongLeftTable)
-// e.g a.proctime < b.proctime + 5 sec, we need to store stream b.
+  isLeftExprBelongLeftTable
+// e.g a.proctime < b.proctime + 5 sec, the the upper bound of a 
is 5
 case kind @ (SqlKind.LESS_THAN | SqlKind.LESS_THAN_OR_EQUAL) =>
-  (leftLiteral - rightLiteral, !isLeftExprBelongLeftTable)
-case _ => 0
+  !isLeftExprBelongLeftTable
+case _ =>
+  throw new TableException("Unsupport time-condition.")
   }
 
-val timeOffset =
-  // only preceding offset need to store records
-  if (tmpTimeOffset < 0)
-// determain the boudary value
+val (leftLiteral, rightLiteral) =
+  reduceTimeExpression(
+timeCall.operands.get(0),
+timeCall.operands.get(1),
+rexBuilder,
+config)
+val tmpTimeOffset: Long =
+  if (isLeftExprBelongLeftTable) rightLiteral - leftLiteral else 
leftLiteral - rightLiteral
+
+val boundary =
+  tmpTimeOffset.signum * (
 if (timeTerm.getKind == SqlKind.LESS_THAN || timeTerm.getKind == 
SqlKind.GREATER_THAN) {
-  -tmpTimeOffset - 1
+  tmpTimeOffset.abs - 1
 } else {
-  -tmpTimeOffset
-}
-  else 0
+  tmpTimeOffset.abs
+})
 
-(timeOffset, isLeftTableTimeOffset)
+(boundary, isLeftLowerBound)
   }
 
   /**
 * Calcute the time boundary. Replace the rowtime/proctime with zero 
literal.
--- End diff --

```
Calculates the time boundary by replacing the time attribute by a zero 
literal and reducing the expression.
```


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-04 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125448467
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeWindowInnerJoin.scala
 ---
@@ -0,0 +1,326 @@
+/*
+ * 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 java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftLowerBound
+  *the left stream lower bound, and -leftLowerBound is the right 
stream upper bound
+  * @param leftUpperBound
+  *the left stream upper bound, and -leftUpperBound is the right 
stream lower bound
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeWindowInnerJoin(
+private val leftLowerBound: Long,
+private val leftUpperBound: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** 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] = _
+
+  private val leftStreamWinSize: Long = if (leftLowerBound < 0) 
-leftLowerBound else 0
+  private val rightStreamWinSize: Long = if (leftUpperBound > 0) 
leftUpperBound else 0
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+
+  override def open(config: Configuration) {
+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()
+
+listToRemove = new util.ArrayList[Long]()
+cRowWrapper = new CRowWrappingCollector()
+
+// initialize row state
+val rowListTypeInfo1: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](element1Type)
+val mapStateDescriptor1: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]]("row1mapstate",
+BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-04 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125427060
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala
 ---
@@ -87,44 +98,38 @@ class DataStreamRowStreamJoin(
 
 val config = tableEnv.getConfig
 
+val leftIsAccRetract = DataStreamRetractionRules.isAccRetract(left)
+val rightIsAccRetract = DataStreamRetractionRules.isAccRetract(right)
+if (leftIsAccRetract || rightIsAccRetract) {
+  throw new TableException(
+"Retraction on stream window join is not supported yet.")
--- End diff --

```
Windowed stream join does not support updates.
```


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-04 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125423693
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamWindowJoinRule.scala
 ---
@@ -20,28 +20,60 @@ 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.RexNode
+import org.apache.flink.table.api.{TableConfig, TableException}
 import org.apache.flink.table.plan.nodes.FlinkConventions
-import org.apache.flink.table.plan.nodes.datastream.DataStreamRowStreamJoin
+import org.apache.flink.table.plan.nodes.datastream.DataStreamWindowJoin
 import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
 import org.apache.flink.table.plan.schema.RowSchema
-import org.apache.flink.table.runtime.join.JoinUtil
+import org.apache.flink.table.runtime.join.WindowJoinUtil
 
-class DataStreamRowStreamJoinRule
+class DataStreamWindowJoinRule
   extends ConverterRule(
   classOf[FlinkLogicalJoin],
   FlinkConventions.LOGICAL,
   FlinkConventions.DATASTREAM,
   "DataStreamJoinRule") {
 
+  /** Time indicator type **/
+  private var timeType: RelDataType = _
+
+  /** left input lower boudary **/
+  private var leftLowerBoundary: Long = _
+
+  /** left input upper boudary **/
+  private var leftUpperBoundary: Long = _
+
+  /** remain join condition exclude equal condition and time condition **/
+  private var remainCondition: Option[RexNode] = _
+
   override def matches(call: RelOptRuleCall): Boolean = {
 val join: FlinkLogicalJoin = call.rel(0).asInstanceOf[FlinkLogicalJoin]
 
 val joinInfo = join.analyzeCondition
 
-JoinUtil.isStreamStreamJoin(
-  joinInfo.getRemaining(join.getCluster.getRexBuilder),
-  join.getRowType)
+try {
+  val leftRowSchema = new RowSchema(join.getLeft.getRowType)
+
+  val result =
+WindowJoinUtil.analyzeTimeBoundary(
+  joinInfo.getRemaining(join.getCluster.getRexBuilder),
+  leftRowSchema.logicalType.getFieldCount,
+  leftRowSchema.physicalType.getFieldCount,
+  join.getRowType,
+  join.getCluster.getRexBuilder,
+  TableConfig.DEFAULT)
+  timeType = result._1
--- End diff --

This can be done more concisely as:

```
(timeType, leftLowerBoundary, leftUpperBoundary, remainCondition) =
  WindowJoinUtil.analyzeTimeBoundary(...)
```


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-04 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125456634
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala
 ---
@@ -162,24 +135,26 @@ object JoinUtil {
   (l._1 ++ r._1, l._2 ++ r._2)
 }
   case c: RexCall =>
-val timeIndicators = analyzeSingleConditionTerm(c, leftFieldCount, 
inputType)
+val timeIndicators = extractTimeIndicatorAccesses(c, 
leftFieldCount, inputType)
 timeIndicators match {
   case Seq() =>
 (Seq(), Seq(c))
   case Seq(v1, v2) =>
 if (v1._1 != v2._1) {
-  throw new TableException("The time indicators for each input 
should be the same.")
+  throw new TableException(
+"Both time attributes in a join condition must be of the 
same type.")
 }
 if (v1._2 == v2._2) {
-  throw new TableException("Both input's time indicators is 
needed.")
+  throw new TableException("Time-based join conditions " +
+"must reference the time attribute of both input tables.")
 }
 (Seq((v1._1, v1._2, c)), Seq())
--- End diff --

If we check the `RexCall` type here, we can analyze both sides of the 
condition independently and also handle cases where both attributes are on the 
same side of the condition.


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-04 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125457951
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala
 ---
@@ -210,77 +187,71 @@ object JoinUtil {
   case _ => Seq()
 }
   case c: RexCall =>
-c.operands.map(analyzeSingleConditionTerm(_, leftFieldCount, 
inputType)).reduce(_++_)
+c.operands.map(extractTimeIndicatorAccesses(_, leftFieldCount, 
inputType)).reduce(_++_)
   case _ => Seq()
 }
   }
 
   /**
-* Extract time offset and determain which table the offset belong to
+* Extract time offset and determain it's the lower bound of left 
stream or the upper bound
+*
+* @return window boundary, is left lower bound
 */
   def extractTimeOffsetFromCondition(
   timeTerm: RexNode,
   isLeftExprBelongLeftTable: Boolean,
   rexBuilder: RexBuilder,
-  config: TableConfig) = {
+  config: TableConfig): (Long, Boolean) = {
 
 val timeCall: RexCall = timeTerm.asInstanceOf[RexCall]
-val leftLiteral =
-  reduceTimeExpression(
-timeCall.operands.get(0),
-rexBuilder,
-config)
 
-val rightLiteral =
-  reduceTimeExpression(
-timeCall.operands.get(1),
-rexBuilder,
-config)
-
-val (tmpTimeOffset: Long, isLeftTableTimeOffset: Boolean) =
+val isLeftLowerBound: Boolean =
   timeTerm.getKind match {
-// e.g a.proctime > b.proctime - 5 sec, we need to store stream a.
-// the left expr(a) belong to left table, so the offset belong to 
left table
+// e.g a.proctime > b.proctime - 5 sec, then it's the lower bound 
of a and the value is -5
+// e.g b.proctime > a.proctime - 5 sec, then it's not the lower 
bound of a but upper bound
 case kind @ (SqlKind.GREATER_THAN | SqlKind.GREATER_THAN_OR_EQUAL) 
=>
-  (rightLiteral - leftLiteral, isLeftExprBelongLeftTable)
-// e.g a.proctime < b.proctime + 5 sec, we need to store stream b.
+  isLeftExprBelongLeftTable
+// e.g a.proctime < b.proctime + 5 sec, the the upper bound of a 
is 5
 case kind @ (SqlKind.LESS_THAN | SqlKind.LESS_THAN_OR_EQUAL) =>
-  (leftLiteral - rightLiteral, !isLeftExprBelongLeftTable)
-case _ => 0
+  !isLeftExprBelongLeftTable
+case _ =>
+  throw new TableException("Unsupport time-condition.")
   }
 
-val timeOffset =
-  // only preceding offset need to store records
-  if (tmpTimeOffset < 0)
-// determain the boudary value
+val (leftLiteral, rightLiteral) =
+  reduceTimeExpression(
+timeCall.operands.get(0),
+timeCall.operands.get(1),
+rexBuilder,
+config)
+val tmpTimeOffset: Long =
+  if (isLeftExprBelongLeftTable) rightLiteral - leftLiteral else 
leftLiteral - rightLiteral
+
+val boundary =
+  tmpTimeOffset.signum * (
 if (timeTerm.getKind == SqlKind.LESS_THAN || timeTerm.getKind == 
SqlKind.GREATER_THAN) {
-  -tmpTimeOffset - 1
+  tmpTimeOffset.abs - 1
 } else {
-  -tmpTimeOffset
-}
-  else 0
+  tmpTimeOffset.abs
+})
 
-(timeOffset, isLeftTableTimeOffset)
+(boundary, isLeftLowerBound)
   }
 
   /**
 * Calcute the time boundary. Replace the rowtime/proctime with zero 
literal.
 * For example:
-*  a.proctime - inteval '1' second > b.proctime - interval '1' second 
- interval '2' second
-*  |---left--|   
|---right---\
-* then the boundary of a is right - left:
-*  ((0 - 1000) - 2000) - (0 - 1000) = -2000(-preceding, +following)
+* b.proctime - interval '1' second - interval '2' second will be 
translate to
--- End diff --

will be translate -> will be translated


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-04 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125417060
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala
 ---
@@ -44,15 +46,19 @@ class DataStreamRowStreamJoin(
 leftSchema: RowSchema,
 rightSchema: RowSchema,
 schema: RowSchema,
+timeType: RelDataType,
--- End diff --

Can be a boolean flag `isRowTime`


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-02 Thread hongyuhong
Github user hongyuhong commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125205215
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
+  condition: RexNode,
+  inputType: RelDataType) = {
+
+def isExistTumble(expr: RexNode): Boolean = {
+  expr match {
+case c: RexCall =>
+  c.getOperator match {
+case _: SqlFloorFunction =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case SqlStdOperatorTable.TUMBLE =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case _ =>
+  c.getOperands.map(isExistTumble(_)).exists(_ == true)
+  }
+case _ => false
+  }
+}
+
+val isExistTimeIndicator = analyzeSingleConditionTerm(condition, 0, 
inputType).size > 0
+val isExistTumbleExpr = isExistTumble(condition)
+
+!isExistTumbleExpr && isExistTimeIndicator
+  }
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType
+  )
+
+if (timeTerms.size != 2) {
+  throw new TableException("There only can and must have 2 time 
conditions.")
+}
+
+// extract time offset from the time indicator conditon
+val 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-02 Thread hongyuhong
Github user hongyuhong commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125203460
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/JoinTest.scala
 ---
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.api.scala.stream.sql
+
+import org.apache.calcite.rel.logical.LogicalJoin
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.runtime.join.JoinUtil
+import org.apache.flink.table.utils.TableTestUtil._
+import org.apache.flink.table.utils.{StreamTableTestUtil, TableTestBase}
+import org.junit.Assert._
+import org.junit.Test
+
+class JoinTest extends TableTestBase {
+  private val streamUtil: StreamTableTestUtil = streamTestUtil()
+  streamUtil.addTable[(Int, String, Long)]("MyTable", 'a, 'b, 'c.rowtime, 
'proctime.proctime)
+  streamUtil.addTable[(Int, String, Long)]("MyTable2", 'a, 'b, 'c.rowtime, 
'proctime.proctime)
+
+  @Test
+  def testProcessingTimeInnerJoin() = {
+
+val sqlQuery = "SELECT t1.a, t2.b " +
+  "FROM MyTable as t1 join MyTable2 as t2 on t1.a = t2.a and " +
+  "t1.proctime between t2.proctime - interval '1' hour and t2.proctime 
+ interval '1' hour"
+val expected =
+  unaryNode(
+"DataStreamCalc",
+binaryNode(
+  "DataStreamRowStreamJoin",
+  unaryNode(
+"DataStreamCalc",
+streamTableNode(0),
+term("select", "a", "proctime")
+  ),
+  unaryNode(
+"DataStreamCalc",
+streamTableNode(1),
+term("select", "a", "b", "proctime")
+  ),
+  term("condition",
+"AND(=(a, a0), >=(TIME_MATERIALIZATION(proctime), " +
+  "-(TIME_MATERIALIZATION(proctime0), 360)), " +
+  "<=(TIME_MATERIALIZATION(proctime), " +
+  "DATETIME_PLUS(TIME_MATERIALIZATION(proctime0), 360)))"),
+  term("select", "a, proctime, a0, b, proctime0"),
+  term("joinType", "InnerJoin")
+),
+term("select", "a", "b")
+  )
+
+streamUtil.verifySql(sqlQuery, expected)
+  }
+
+
+  @Test
+  def testJoinTimeBoundary(): Unit = {
+verifyTimeBoundary(
+  "t1.proctime between t2.proctime - interval '1' hour " +
+"and t2.proctime + interval '1' hour",
+  360,
+  360,
+  "proctime")
+
+verifyTimeBoundary(
+  "t1.proctime > t2.proctime - interval '1' second and " +
+"t1.proctime < t2.proctime + interval '1' second",
+  999,
+  999,
+  "proctime")
+
+verifyTimeBoundary(
+  "t1.c >= t2.c - interval '1' second and " +
+"t1.c <= t2.c + interval '1' second",
+  1000,
+  1000,
+  "rowtime")
+
+verifyTimeBoundary(
+  "t1.c >= t2.c and " +
+"t1.c <= t2.c + interval '1' second",
+  0,
+  1000,
+  "rowtime")
+
+verifyTimeBoundary(
+  "t1.c >= t2.c + interval '1' second and " +
+"t1.c <= t2.c + interval '10' second",
+  0,
+  1,
+  "rowtime")
+
+verifyTimeBoundary(
+  "t2.c - interval '1' second <= t1.c and " +
+"t2.c + interval '10' second >= t1.c",
+  1000,
+  1,
+  "rowtime")
+
+verifyTimeBoundary(
+  "t1.c - interval '2' second >= t2.c + interval '1' second -" +
+"interval '10' second and " +
+"t1.c <= t2.c + interval '10' second",
+  7000,
+  1,
+  "rowtime")
+  }
+
+  def verifyTimeBoundary(
--- End diff --

Yes, it's better to check unsupported case here, but i think 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-02 Thread hongyuhong
Github user hongyuhong commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125203253
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeInnerJoin.scala
 ---
@@ -0,0 +1,316 @@
+/*
+ * 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 java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftStreamWindowSizethe left stream window size
+  * @param rightStreamWindowSizethe right stream window size
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeInnerJoin(
+private val leftStreamWindowSize: Long,
+private val rightStreamWindowSize: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** 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] = _
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+  override def open(config: Configuration) {
+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()
+
+listToRemove = new util.ArrayList[Long]()
+cRowWrapper = new CRowWrappingCollector()
+
+// initialize row state
+val rowListTypeInfo1: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](element1Type)
+val mapStateDescriptor1: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]]("row1mapstate",
+BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], 
rowListTypeInfo1)
+row1MapState = getRuntimeContext.getMapState(mapStateDescriptor1)
+
+val rowListTypeInfo2: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](element2Type)
+val mapStateDescriptor2: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-02 Thread hongyuhong
Github user hongyuhong commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125203271
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeInnerJoin.scala
 ---
@@ -0,0 +1,316 @@
+/*
+ * 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 java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftStreamWindowSizethe left stream window size
+  * @param rightStreamWindowSizethe right stream window size
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeInnerJoin(
+private val leftStreamWindowSize: Long,
+private val rightStreamWindowSize: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** 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] = _
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+  override def open(config: Configuration) {
+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()
+
+listToRemove = new util.ArrayList[Long]()
+cRowWrapper = new CRowWrappingCollector()
+
+// initialize row state
+val rowListTypeInfo1: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](element1Type)
+val mapStateDescriptor1: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]]("row1mapstate",
+BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], 
rowListTypeInfo1)
+row1MapState = getRuntimeContext.getMapState(mapStateDescriptor1)
+
+val rowListTypeInfo2: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](element2Type)
+val mapStateDescriptor2: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-02 Thread hongyuhong
Github user hongyuhong commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125202985
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
+  condition: RexNode,
+  inputType: RelDataType) = {
+
+def isExistTumble(expr: RexNode): Boolean = {
+  expr match {
+case c: RexCall =>
+  c.getOperator match {
+case _: SqlFloorFunction =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case SqlStdOperatorTable.TUMBLE =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case _ =>
+  c.getOperands.map(isExistTumble(_)).exists(_ == true)
+  }
+case _ => false
+  }
+}
+
+val isExistTimeIndicator = analyzeSingleConditionTerm(condition, 0, 
inputType).size > 0
+val isExistTumbleExpr = isExistTumble(condition)
+
+!isExistTumbleExpr && isExistTimeIndicator
+  }
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType
+  )
+
+if (timeTerms.size != 2) {
+  throw new TableException("There only can and must have 2 time 
conditions.")
+}
+
+// extract time offset from the time indicator conditon
+val 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-07-02 Thread hongyuhong
Github user hongyuhong commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r125202807
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
+  condition: RexNode,
+  inputType: RelDataType) = {
+
+def isExistTumble(expr: RexNode): Boolean = {
+  expr match {
+case c: RexCall =>
+  c.getOperator match {
+case _: SqlFloorFunction =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case SqlStdOperatorTable.TUMBLE =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case _ =>
+  c.getOperands.map(isExistTumble(_)).exists(_ == true)
+  }
+case _ => false
+  }
+}
+
+val isExistTimeIndicator = analyzeSingleConditionTerm(condition, 0, 
inputType).size > 0
+val isExistTumbleExpr = isExistTumble(condition)
+
+!isExistTumbleExpr && isExistTimeIndicator
+  }
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType
+  )
+
+if (timeTerms.size != 2) {
+  throw new TableException("There only can and must have 2 time 
conditions.")
+}
+
+// extract time offset from the time indicator conditon
+val 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123016738
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
+  condition: RexNode,
+  inputType: RelDataType) = {
+
+def isExistTumble(expr: RexNode): Boolean = {
+  expr match {
+case c: RexCall =>
+  c.getOperator match {
+case _: SqlFloorFunction =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case SqlStdOperatorTable.TUMBLE =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case _ =>
+  c.getOperands.map(isExistTumble(_)).exists(_ == true)
+  }
+case _ => false
+  }
+}
+
+val isExistTimeIndicator = analyzeSingleConditionTerm(condition, 0, 
inputType).size > 0
+val isExistTumbleExpr = isExistTumble(condition)
+
+!isExistTumbleExpr && isExistTimeIndicator
+  }
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType
+  )
+
+if (timeTerms.size != 2) {
+  throw new TableException("There only can and must have 2 time 
conditions.")
+}
+
+// extract time offset from the time indicator conditon
+val 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123009399
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
--- End diff --

The check is only approximate, i.e., the stream join operator might not be 
able to execute the query even if this check is passed.

For example it only checks if there is at least one time indicator in the 
condition. However, we would need to check that there are exactly two 
conjunctive terms that have time indicator attributes on both sides and define 
bounds to both sides. Basically the complete analysis that we later do in the 
join. I think we can do this analysis already in the rule and pass the result 
of the analysis to the join.


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123228354
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeInnerJoin.scala
 ---
@@ -0,0 +1,316 @@
+/*
+ * 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 java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftStreamWindowSizethe left stream window size
+  * @param rightStreamWindowSizethe right stream window size
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeInnerJoin(
+private val leftStreamWindowSize: Long,
+private val rightStreamWindowSize: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** 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] = _
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+  override def open(config: Configuration) {
+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()
+
+listToRemove = new util.ArrayList[Long]()
+cRowWrapper = new CRowWrappingCollector()
+
+// initialize row state
+val rowListTypeInfo1: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](element1Type)
+val mapStateDescriptor1: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]]("row1mapstate",
+BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], 
rowListTypeInfo1)
+row1MapState = getRuntimeContext.getMapState(mapStateDescriptor1)
+
+val rowListTypeInfo2: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](element2Type)
+val mapStateDescriptor2: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123031759
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
+  condition: RexNode,
+  inputType: RelDataType) = {
+
+def isExistTumble(expr: RexNode): Boolean = {
+  expr match {
+case c: RexCall =>
+  c.getOperator match {
+case _: SqlFloorFunction =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case SqlStdOperatorTable.TUMBLE =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case _ =>
+  c.getOperands.map(isExistTumble(_)).exists(_ == true)
+  }
+case _ => false
+  }
+}
+
+val isExistTimeIndicator = analyzeSingleConditionTerm(condition, 0, 
inputType).size > 0
+val isExistTumbleExpr = isExistTumble(condition)
+
+!isExistTumbleExpr && isExistTimeIndicator
+  }
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType
+  )
+
+if (timeTerms.size != 2) {
+  throw new TableException("There only can and must have 2 time 
conditions.")
+}
+
+// extract time offset from the time indicator conditon
+val 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123035300
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
+  condition: RexNode,
+  inputType: RelDataType) = {
+
+def isExistTumble(expr: RexNode): Boolean = {
+  expr match {
+case c: RexCall =>
+  c.getOperator match {
+case _: SqlFloorFunction =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case SqlStdOperatorTable.TUMBLE =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case _ =>
+  c.getOperands.map(isExistTumble(_)).exists(_ == true)
+  }
+case _ => false
+  }
+}
+
+val isExistTimeIndicator = analyzeSingleConditionTerm(condition, 0, 
inputType).size > 0
+val isExistTumbleExpr = isExistTumble(condition)
+
+!isExistTumbleExpr && isExistTimeIndicator
+  }
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType
+  )
+
+if (timeTerms.size != 2) {
+  throw new TableException("There only can and must have 2 time 
conditions.")
+}
+
+// extract time offset from the time indicator conditon
+val 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123228716
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeInnerJoin.scala
 ---
@@ -0,0 +1,316 @@
+/*
+ * 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 java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftStreamWindowSizethe left stream window size
+  * @param rightStreamWindowSizethe right stream window size
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeInnerJoin(
+private val leftStreamWindowSize: Long,
+private val rightStreamWindowSize: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** 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] = _
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+  override def open(config: Configuration) {
+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()
+
+listToRemove = new util.ArrayList[Long]()
+cRowWrapper = new CRowWrappingCollector()
+
+// initialize row state
+val rowListTypeInfo1: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](element1Type)
+val mapStateDescriptor1: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]]("row1mapstate",
+BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], 
rowListTypeInfo1)
+row1MapState = getRuntimeContext.getMapState(mapStateDescriptor1)
+
+val rowListTypeInfo2: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](element2Type)
+val mapStateDescriptor2: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123030850
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
+  condition: RexNode,
+  inputType: RelDataType) = {
+
+def isExistTumble(expr: RexNode): Boolean = {
+  expr match {
+case c: RexCall =>
+  c.getOperator match {
+case _: SqlFloorFunction =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case SqlStdOperatorTable.TUMBLE =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case _ =>
+  c.getOperands.map(isExistTumble(_)).exists(_ == true)
+  }
+case _ => false
+  }
+}
+
+val isExistTimeIndicator = analyzeSingleConditionTerm(condition, 0, 
inputType).size > 0
+val isExistTumbleExpr = isExistTumble(condition)
+
+!isExistTumbleExpr && isExistTimeIndicator
+  }
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType
+  )
+
+if (timeTerms.size != 2) {
+  throw new TableException("There only can and must have 2 time 
conditions.")
+}
+
+// extract time offset from the time indicator conditon
+val 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123226695
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeInnerJoin.scala
 ---
@@ -0,0 +1,316 @@
+/*
+ * 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 java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftStreamWindowSizethe left stream window size
+  * @param rightStreamWindowSizethe right stream window size
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeInnerJoin(
+private val leftStreamWindowSize: Long,
+private val rightStreamWindowSize: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** 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] = _
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+  override def open(config: Configuration) {
+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()
+
+listToRemove = new util.ArrayList[Long]()
+cRowWrapper = new CRowWrappingCollector()
+
+// initialize row state
+val rowListTypeInfo1: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](element1Type)
+val mapStateDescriptor1: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]]("row1mapstate",
+BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], 
rowListTypeInfo1)
+row1MapState = getRuntimeContext.getMapState(mapStateDescriptor1)
+
+val rowListTypeInfo2: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](element2Type)
+val mapStateDescriptor2: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123236585
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala
 ---
@@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.harness
+
+import java.util.concurrent.ConcurrentLinkedQueue
+import java.lang.{Integer => JInt}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.streaming.api.operators.co.KeyedCoProcessOperator
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord
+import 
org.apache.flink.streaming.util.{KeyedTwoInputStreamOperatorTestHarness, 
TwoInputStreamOperatorTestHarness}
+import org.apache.flink.table.codegen.GeneratedFunction
+import 
org.apache.flink.table.runtime.harness.HarnessTestBase.{RowResultSortComparator,
 TupleRowKeySelector}
+import org.apache.flink.table.runtime.join.ProcTimeInnerJoin
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.junit.Test
+
+
+class JoinHarnessTest extends HarnessTestBase{
+
+  private val rT = new RowTypeInfo(Array[TypeInformation[_]](
+INT_TYPE_INFO,
+STRING_TYPE_INFO),
+Array("a", "b"))
+
+
+  val funcCode: String =
+"""
+  |public class TestJoinFunction
+  |  extends 
org.apache.flink.api.common.functions.RichFlatJoinFunction {
+  |  transient org.apache.flink.types.Row out =
+  |new org.apache.flink.types.Row(4);
+  |  public TestJoinFunction() throws Exception {}
+  |
+  |  @Override
+  |  public void open(org.apache.flink.configuration.Configuration 
parameters)
+  |  throws Exception {}
+  |
+  |  @Override
+  |  public void join(Object _in1, Object _in2, 
org.apache.flink.util.Collector c)
+  |   throws Exception {
+  |   org.apache.flink.types.Row in1 = (org.apache.flink.types.Row) 
_in1;
+  |   org.apache.flink.types.Row in2 = (org.apache.flink.types.Row) 
_in2;
+  |
+  |   out.setField(0, in1.getField(0));
+  |   out.setField(1, in1.getField(1));
+  |   out.setField(2, in2.getField(0));
+  |   out.setField(3, in2.getField(1));
+  |
+  |   c.collect(out);
+  |
+  |  }
+  |
+  |  @Override
+  |  public void close() throws Exception {}
+  |}
+""".stripMargin
+
+  @Test
+  def testProcTimeJoin() {
+
+val joinProcessFunc = new ProcTimeInnerJoin(10, 20, rT, rT, 
"TestJoinFunction", funcCode)
+
+val operator: KeyedCoProcessOperator[Integer, CRow, CRow, CRow] =
+  new KeyedCoProcessOperator[Integer, CRow, CRow, 
CRow](joinProcessFunc)
+val testHarness: TwoInputStreamOperatorTestHarness[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()
+
+testHarness.setProcessingTime(1)
+testHarness.processElement1(new StreamRecord(
+  CRow(Row.of(1: JInt, "aaa"), true), 1))
+testHarness.setProcessingTime(2)
+testHarness.processElement1(new StreamRecord(
+  CRow(Row.of(2: JInt, "bbb"), true), 2))
+testHarness.setProcessingTime(3)
+testHarness.processElement1(new StreamRecord(
+  CRow(Row.of(1: JInt, "aaa2"), true), 3))
+
+testHarness.processElement2(new StreamRecord(
+  CRow(Row.of(1: JInt, "Hi1"), true), 3))
+

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123017357
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
+  condition: RexNode,
+  inputType: RelDataType) = {
+
+def isExistTumble(expr: RexNode): Boolean = {
+  expr match {
+case c: RexCall =>
+  c.getOperator match {
+case _: SqlFloorFunction =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
--- End diff --

replace `_size > 0` by `_.nonEmpty`


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123020714
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
+  condition: RexNode,
+  inputType: RelDataType) = {
+
+def isExistTumble(expr: RexNode): Boolean = {
+  expr match {
+case c: RexCall =>
+  c.getOperator match {
+case _: SqlFloorFunction =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case SqlStdOperatorTable.TUMBLE =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case _ =>
+  c.getOperands.map(isExistTumble(_)).exists(_ == true)
+  }
+case _ => false
+  }
+}
+
+val isExistTimeIndicator = analyzeSingleConditionTerm(condition, 0, 
inputType).size > 0
+val isExistTumbleExpr = isExistTumble(condition)
+
+!isExistTumbleExpr && isExistTimeIndicator
+  }
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType
+  )
+
+if (timeTerms.size != 2) {
+  throw new TableException("There only can and must have 2 time 
conditions.")
+}
+
+// extract time offset from the time indicator conditon
+val 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123021563
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
+  condition: RexNode,
+  inputType: RelDataType) = {
+
+def isExistTumble(expr: RexNode): Boolean = {
+  expr match {
+case c: RexCall =>
+  c.getOperator match {
+case _: SqlFloorFunction =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case SqlStdOperatorTable.TUMBLE =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case _ =>
+  c.getOperands.map(isExistTumble(_)).exists(_ == true)
+  }
+case _ => false
+  }
+}
+
+val isExistTimeIndicator = analyzeSingleConditionTerm(condition, 0, 
inputType).size > 0
+val isExistTumbleExpr = isExistTumble(condition)
+
+!isExistTumbleExpr && isExistTimeIndicator
+  }
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType
+  )
+
+if (timeTerms.size != 2) {
+  throw new TableException("There only can and must have 2 time 
conditions.")
+}
+
+// extract time offset from the time indicator conditon
+val 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123233427
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/JoinTest.scala
 ---
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.api.scala.stream.sql
+
+import org.apache.calcite.rel.logical.LogicalJoin
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.runtime.join.JoinUtil
+import org.apache.flink.table.utils.TableTestUtil._
+import org.apache.flink.table.utils.{StreamTableTestUtil, TableTestBase}
+import org.junit.Assert._
+import org.junit.Test
+
+class JoinTest extends TableTestBase {
+  private val streamUtil: StreamTableTestUtil = streamTestUtil()
+  streamUtil.addTable[(Int, String, Long)]("MyTable", 'a, 'b, 'c.rowtime, 
'proctime.proctime)
+  streamUtil.addTable[(Int, String, Long)]("MyTable2", 'a, 'b, 'c.rowtime, 
'proctime.proctime)
+
+  @Test
+  def testProcessingTimeInnerJoin() = {
+
+val sqlQuery = "SELECT t1.a, t2.b " +
+  "FROM MyTable as t1 join MyTable2 as t2 on t1.a = t2.a and " +
+  "t1.proctime between t2.proctime - interval '1' hour and t2.proctime 
+ interval '1' hour"
+val expected =
+  unaryNode(
+"DataStreamCalc",
+binaryNode(
+  "DataStreamRowStreamJoin",
+  unaryNode(
+"DataStreamCalc",
+streamTableNode(0),
+term("select", "a", "proctime")
+  ),
+  unaryNode(
+"DataStreamCalc",
+streamTableNode(1),
+term("select", "a", "b", "proctime")
+  ),
+  term("condition",
+"AND(=(a, a0), >=(TIME_MATERIALIZATION(proctime), " +
+  "-(TIME_MATERIALIZATION(proctime0), 360)), " +
+  "<=(TIME_MATERIALIZATION(proctime), " +
+  "DATETIME_PLUS(TIME_MATERIALIZATION(proctime0), 360)))"),
+  term("select", "a, proctime, a0, b, proctime0"),
+  term("joinType", "InnerJoin")
+),
+term("select", "a", "b")
+  )
+
+streamUtil.verifySql(sqlQuery, expected)
+  }
+
+
+  @Test
+  def testJoinTimeBoundary(): Unit = {
+verifyTimeBoundary(
+  "t1.proctime between t2.proctime - interval '1' hour " +
+"and t2.proctime + interval '1' hour",
+  360,
+  360,
+  "proctime")
+
+verifyTimeBoundary(
+  "t1.proctime > t2.proctime - interval '1' second and " +
+"t1.proctime < t2.proctime + interval '1' second",
+  999,
+  999,
+  "proctime")
+
+verifyTimeBoundary(
+  "t1.c >= t2.c - interval '1' second and " +
+"t1.c <= t2.c + interval '1' second",
+  1000,
+  1000,
+  "rowtime")
+
+verifyTimeBoundary(
+  "t1.c >= t2.c and " +
+"t1.c <= t2.c + interval '1' second",
+  0,
+  1000,
+  "rowtime")
+
+verifyTimeBoundary(
+  "t1.c >= t2.c + interval '1' second and " +
+"t1.c <= t2.c + interval '10' second",
+  0,
+  1,
+  "rowtime")
+
+verifyTimeBoundary(
+  "t2.c - interval '1' second <= t1.c and " +
+"t2.c + interval '10' second >= t1.c",
+  1000,
+  1,
+  "rowtime")
+
+verifyTimeBoundary(
+  "t1.c - interval '2' second >= t2.c + interval '1' second -" +
+"interval '10' second and " +
+"t1.c <= t2.c + interval '10' second",
+  7000,
+  1,
+  "rowtime")
+  }
+
+  def verifyTimeBoundary(
--- End diff --

I would move this also into a dedicated test class. I don't 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123226087
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeInnerJoin.scala
 ---
@@ -0,0 +1,316 @@
+/*
+ * 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 java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftStreamWindowSizethe left stream window size
+  * @param rightStreamWindowSizethe right stream window size
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeInnerJoin(
+private val leftStreamWindowSize: Long,
+private val rightStreamWindowSize: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** 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] = _
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+  override def open(config: Configuration) {
+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()
+
+listToRemove = new util.ArrayList[Long]()
+cRowWrapper = new CRowWrappingCollector()
+
+// initialize row state
+val rowListTypeInfo1: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](element1Type)
+val mapStateDescriptor1: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]]("row1mapstate",
+BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], 
rowListTypeInfo1)
+row1MapState = getRuntimeContext.getMapState(mapStateDescriptor1)
+
+val rowListTypeInfo2: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](element2Type)
+val mapStateDescriptor2: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123020909
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
+  condition: RexNode,
+  inputType: RelDataType) = {
+
+def isExistTumble(expr: RexNode): Boolean = {
+  expr match {
+case c: RexCall =>
+  c.getOperator match {
+case _: SqlFloorFunction =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case SqlStdOperatorTable.TUMBLE =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case _ =>
+  c.getOperands.map(isExistTumble(_)).exists(_ == true)
+  }
+case _ => false
+  }
+}
+
+val isExistTimeIndicator = analyzeSingleConditionTerm(condition, 0, 
inputType).size > 0
+val isExistTumbleExpr = isExistTumble(condition)
+
+!isExistTumbleExpr && isExistTimeIndicator
+  }
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType
+  )
+
+if (timeTerms.size != 2) {
+  throw new TableException("There only can and must have 2 time 
conditions.")
+}
+
+// extract time offset from the time indicator conditon
+val 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123226130
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeInnerJoin.scala
 ---
@@ -0,0 +1,316 @@
+/*
+ * 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 java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftStreamWindowSizethe left stream window size
+  * @param rightStreamWindowSizethe right stream window size
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeInnerJoin(
+private val leftStreamWindowSize: Long,
+private val rightStreamWindowSize: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** 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] = _
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+  override def open(config: Configuration) {
+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()
+
+listToRemove = new util.ArrayList[Long]()
+cRowWrapper = new CRowWrappingCollector()
+
+// initialize row state
+val rowListTypeInfo1: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](element1Type)
+val mapStateDescriptor1: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]]("row1mapstate",
+BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], 
rowListTypeInfo1)
+row1MapState = getRuntimeContext.getMapState(mapStateDescriptor1)
+
+val rowListTypeInfo2: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](element2Type)
+val mapStateDescriptor2: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123225232
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeInnerJoin.scala
 ---
@@ -0,0 +1,316 @@
+/*
+ * 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 java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftStreamWindowSizethe left stream window size
+  * @param rightStreamWindowSizethe right stream window size
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeInnerJoin(
--- End diff --

Does it make sense to split the implementation into two operators:
1. both streams need to be buffered (`l.ptime > r.ptime - 10.secs AND 
l.ptime < r.ptime + 5.secs`)
2. only one stream needs to be buffered (`l.ptime > r.ptime - 10.secs AND 
l.ptime < r.ptime - 5.secs`)


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123232793
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/JoinTest.scala
 ---
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.api.scala.stream.sql
+
+import org.apache.calcite.rel.logical.LogicalJoin
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.runtime.join.JoinUtil
+import org.apache.flink.table.utils.TableTestUtil._
+import org.apache.flink.table.utils.{StreamTableTestUtil, TableTestBase}
+import org.junit.Assert._
+import org.junit.Test
+
+class JoinTest extends TableTestBase {
+  private val streamUtil: StreamTableTestUtil = streamTestUtil()
+  streamUtil.addTable[(Int, String, Long)]("MyTable", 'a, 'b, 'c.rowtime, 
'proctime.proctime)
+  streamUtil.addTable[(Int, String, Long)]("MyTable2", 'a, 'b, 'c.rowtime, 
'proctime.proctime)
+
+  @Test
+  def testProcessingTimeInnerJoin() = {
+
+val sqlQuery = "SELECT t1.a, t2.b " +
+  "FROM MyTable as t1 join MyTable2 as t2 on t1.a = t2.a and " +
+  "t1.proctime between t2.proctime - interval '1' hour and t2.proctime 
+ interval '1' hour"
+val expected =
+  unaryNode(
+"DataStreamCalc",
+binaryNode(
+  "DataStreamRowStreamJoin",
+  unaryNode(
+"DataStreamCalc",
+streamTableNode(0),
+term("select", "a", "proctime")
+  ),
+  unaryNode(
+"DataStreamCalc",
+streamTableNode(1),
+term("select", "a", "b", "proctime")
+  ),
+  term("condition",
+"AND(=(a, a0), >=(TIME_MATERIALIZATION(proctime), " +
+  "-(TIME_MATERIALIZATION(proctime0), 360)), " +
+  "<=(TIME_MATERIALIZATION(proctime), " +
+  "DATETIME_PLUS(TIME_MATERIALIZATION(proctime0), 360)))"),
+  term("select", "a, proctime, a0, b, proctime0"),
+  term("joinType", "InnerJoin")
+),
+term("select", "a", "b")
+  )
+
+streamUtil.verifySql(sqlQuery, expected)
+  }
+
+
+  @Test
+  def testJoinTimeBoundary(): Unit = {
+verifyTimeBoundary(
+  "t1.proctime between t2.proctime - interval '1' hour " +
+"and t2.proctime + interval '1' hour",
+  360,
+  360,
+  "proctime")
+
+verifyTimeBoundary(
+  "t1.proctime > t2.proctime - interval '1' second and " +
+"t1.proctime < t2.proctime + interval '1' second",
+  999,
+  999,
+  "proctime")
+
+verifyTimeBoundary(
+  "t1.c >= t2.c - interval '1' second and " +
+"t1.c <= t2.c + interval '1' second",
+  1000,
+  1000,
+  "rowtime")
+
+verifyTimeBoundary(
+  "t1.c >= t2.c and " +
+"t1.c <= t2.c + interval '1' second",
+  0,
+  1000,
+  "rowtime")
+
+verifyTimeBoundary(
+  "t1.c >= t2.c + interval '1' second and " +
+"t1.c <= t2.c + interval '10' second",
+  0,
+  1,
+  "rowtime")
+
+verifyTimeBoundary(
+  "t2.c - interval '1' second <= t1.c and " +
+"t2.c + interval '10' second >= t1.c",
+  1000,
+  1,
+  "rowtime")
+
+verifyTimeBoundary(
+  "t1.c - interval '2' second >= t2.c + interval '1' second -" +
+"interval '10' second and " +
+"t1.c <= t2.c + interval '10' second",
+  7000,
+  1,
+  "rowtime")
+  }
+
+  def verifyTimeBoundary(
--- End diff --

I like this test, but it would also be good to check that all 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123229245
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeInnerJoin.scala
 ---
@@ -0,0 +1,316 @@
+/*
+ * 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 java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftStreamWindowSizethe left stream window size
+  * @param rightStreamWindowSizethe right stream window size
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeInnerJoin(
+private val leftStreamWindowSize: Long,
+private val rightStreamWindowSize: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** 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] = _
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+  override def open(config: Configuration) {
+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()
+
+listToRemove = new util.ArrayList[Long]()
+cRowWrapper = new CRowWrappingCollector()
+
+// initialize row state
+val rowListTypeInfo1: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](element1Type)
+val mapStateDescriptor1: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]]("row1mapstate",
+BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], 
rowListTypeInfo1)
+row1MapState = getRuntimeContext.getMapState(mapStateDescriptor1)
+
+val rowListTypeInfo2: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](element2Type)
+val mapStateDescriptor2: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123227488
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeInnerJoin.scala
 ---
@@ -0,0 +1,316 @@
+/*
+ * 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 java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftStreamWindowSizethe left stream window size
+  * @param rightStreamWindowSizethe right stream window size
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeInnerJoin(
+private val leftStreamWindowSize: Long,
+private val rightStreamWindowSize: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** 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] = _
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+  override def open(config: Configuration) {
+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()
+
+listToRemove = new util.ArrayList[Long]()
+cRowWrapper = new CRowWrappingCollector()
+
+// initialize row state
+val rowListTypeInfo1: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](element1Type)
+val mapStateDescriptor1: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]]("row1mapstate",
+BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], 
rowListTypeInfo1)
+row1MapState = getRuntimeContext.getMapState(mapStateDescriptor1)
+
+val rowListTypeInfo2: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](element2Type)
+val mapStateDescriptor2: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123238837
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala
 ---
@@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.harness
+
+import java.util.concurrent.ConcurrentLinkedQueue
+import java.lang.{Integer => JInt}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.streaming.api.operators.co.KeyedCoProcessOperator
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord
+import 
org.apache.flink.streaming.util.{KeyedTwoInputStreamOperatorTestHarness, 
TwoInputStreamOperatorTestHarness}
+import org.apache.flink.table.codegen.GeneratedFunction
+import 
org.apache.flink.table.runtime.harness.HarnessTestBase.{RowResultSortComparator,
 TupleRowKeySelector}
+import org.apache.flink.table.runtime.join.ProcTimeInnerJoin
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.junit.Test
+
+
+class JoinHarnessTest extends HarnessTestBase{
+
+  private val rT = new RowTypeInfo(Array[TypeInformation[_]](
+INT_TYPE_INFO,
+STRING_TYPE_INFO),
+Array("a", "b"))
+
+
+  val funcCode: String =
+"""
+  |public class TestJoinFunction
+  |  extends 
org.apache.flink.api.common.functions.RichFlatJoinFunction {
+  |  transient org.apache.flink.types.Row out =
+  |new org.apache.flink.types.Row(4);
+  |  public TestJoinFunction() throws Exception {}
+  |
+  |  @Override
+  |  public void open(org.apache.flink.configuration.Configuration 
parameters)
+  |  throws Exception {}
+  |
+  |  @Override
+  |  public void join(Object _in1, Object _in2, 
org.apache.flink.util.Collector c)
+  |   throws Exception {
+  |   org.apache.flink.types.Row in1 = (org.apache.flink.types.Row) 
_in1;
+  |   org.apache.flink.types.Row in2 = (org.apache.flink.types.Row) 
_in2;
+  |
+  |   out.setField(0, in1.getField(0));
+  |   out.setField(1, in1.getField(1));
+  |   out.setField(2, in2.getField(0));
+  |   out.setField(3, in2.getField(1));
+  |
+  |   c.collect(out);
+  |
+  |  }
+  |
+  |  @Override
+  |  public void close() throws Exception {}
+  |}
+""".stripMargin
+
+  @Test
+  def testProcTimeJoin() {
+
+val joinProcessFunc = new ProcTimeInnerJoin(10, 20, rT, rT, 
"TestJoinFunction", funcCode)
+
+val operator: KeyedCoProcessOperator[Integer, CRow, CRow, CRow] =
+  new KeyedCoProcessOperator[Integer, CRow, CRow, 
CRow](joinProcessFunc)
+val testHarness: TwoInputStreamOperatorTestHarness[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()
+
+testHarness.setProcessingTime(1)
+testHarness.processElement1(new StreamRecord(
+  CRow(Row.of(1: JInt, "aaa"), true), 1))
+testHarness.setProcessingTime(2)
+testHarness.processElement1(new StreamRecord(
+  CRow(Row.of(2: JInt, "bbb"), true), 2))
+testHarness.setProcessingTime(3)
+testHarness.processElement1(new StreamRecord(
+  CRow(Row.of(1: JInt, "aaa2"), true), 3))
+
+testHarness.processElement2(new StreamRecord(
+  CRow(Row.of(1: JInt, "Hi1"), true), 3))
+

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123205673
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
+  condition: RexNode,
+  inputType: RelDataType) = {
+
+def isExistTumble(expr: RexNode): Boolean = {
+  expr match {
+case c: RexCall =>
+  c.getOperator match {
+case _: SqlFloorFunction =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case SqlStdOperatorTable.TUMBLE =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case _ =>
+  c.getOperands.map(isExistTumble(_)).exists(_ == true)
+  }
+case _ => false
+  }
+}
+
+val isExistTimeIndicator = analyzeSingleConditionTerm(condition, 0, 
inputType).size > 0
+val isExistTumbleExpr = isExistTumble(condition)
+
+!isExistTumbleExpr && isExistTimeIndicator
+  }
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType
+  )
+
+if (timeTerms.size != 2) {
+  throw new TableException("There only can and must have 2 time 
conditions.")
+}
+
+// extract time offset from the time indicator conditon
+val 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123238701
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala
 ---
@@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.harness
+
+import java.util.concurrent.ConcurrentLinkedQueue
+import java.lang.{Integer => JInt}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.streaming.api.operators.co.KeyedCoProcessOperator
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord
+import 
org.apache.flink.streaming.util.{KeyedTwoInputStreamOperatorTestHarness, 
TwoInputStreamOperatorTestHarness}
+import org.apache.flink.table.codegen.GeneratedFunction
+import 
org.apache.flink.table.runtime.harness.HarnessTestBase.{RowResultSortComparator,
 TupleRowKeySelector}
+import org.apache.flink.table.runtime.join.ProcTimeInnerJoin
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.junit.Test
+
+
+class JoinHarnessTest extends HarnessTestBase{
+
+  private val rT = new RowTypeInfo(Array[TypeInformation[_]](
+INT_TYPE_INFO,
+STRING_TYPE_INFO),
+Array("a", "b"))
+
+
+  val funcCode: String =
+"""
+  |public class TestJoinFunction
+  |  extends 
org.apache.flink.api.common.functions.RichFlatJoinFunction {
+  |  transient org.apache.flink.types.Row out =
+  |new org.apache.flink.types.Row(4);
+  |  public TestJoinFunction() throws Exception {}
+  |
+  |  @Override
+  |  public void open(org.apache.flink.configuration.Configuration 
parameters)
+  |  throws Exception {}
+  |
+  |  @Override
+  |  public void join(Object _in1, Object _in2, 
org.apache.flink.util.Collector c)
+  |   throws Exception {
+  |   org.apache.flink.types.Row in1 = (org.apache.flink.types.Row) 
_in1;
+  |   org.apache.flink.types.Row in2 = (org.apache.flink.types.Row) 
_in2;
+  |
+  |   out.setField(0, in1.getField(0));
+  |   out.setField(1, in1.getField(1));
+  |   out.setField(2, in2.getField(0));
+  |   out.setField(3, in2.getField(1));
+  |
+  |   c.collect(out);
+  |
+  |  }
+  |
+  |  @Override
+  |  public void close() throws Exception {}
+  |}
+""".stripMargin
+
+  @Test
+  def testProcTimeJoin() {
--- End diff --

Please add comments for the scenarios that this test covers.


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123229427
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeInnerJoin.scala
 ---
@@ -0,0 +1,316 @@
+/*
+ * 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 java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftStreamWindowSizethe left stream window size
+  * @param rightStreamWindowSizethe right stream window size
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeInnerJoin(
+private val leftStreamWindowSize: Long,
+private val rightStreamWindowSize: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** 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] = _
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+  override def open(config: Configuration) {
+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()
+
+listToRemove = new util.ArrayList[Long]()
+cRowWrapper = new CRowWrappingCollector()
+
+// initialize row state
+val rowListTypeInfo1: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](element1Type)
+val mapStateDescriptor1: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]]("row1mapstate",
+BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], 
rowListTypeInfo1)
+row1MapState = getRuntimeContext.getMapState(mapStateDescriptor1)
+
+val rowListTypeInfo2: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](element2Type)
+val mapStateDescriptor2: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123222750
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamRowStreamJoin.scala
 ---
@@ -0,0 +1,186 @@
+/*
+ * 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.core.{JoinInfo, JoinRelType}
+import org.apache.calcite.rel.{BiRel, RelNode, RelWriter}
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.api.java.functions.NullByteKeySelector
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.table.api.{StreamQueryConfig, 
StreamTableEnvironment, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.plan.nodes.CommonJoin
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.table.runtime.join.{JoinUtil, ProcTimeInnerJoin}
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+/**
+  * Flink RelNode which matches along with JoinOperator and its related 
operations.
+  */
+class DataStreamRowStreamJoin(
--- End diff --

Rename to `DataStreamWindowJoin`?


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123037106
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
+  condition: RexNode,
+  inputType: RelDataType) = {
+
+def isExistTumble(expr: RexNode): Boolean = {
+  expr match {
+case c: RexCall =>
+  c.getOperator match {
+case _: SqlFloorFunction =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case SqlStdOperatorTable.TUMBLE =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case _ =>
+  c.getOperands.map(isExistTumble(_)).exists(_ == true)
+  }
+case _ => false
+  }
+}
+
+val isExistTimeIndicator = analyzeSingleConditionTerm(condition, 0, 
inputType).size > 0
+val isExistTumbleExpr = isExistTumble(condition)
+
+!isExistTumbleExpr && isExistTimeIndicator
+  }
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType
+  )
+
+if (timeTerms.size != 2) {
+  throw new TableException("There only can and must have 2 time 
conditions.")
+}
+
+// extract time offset from the time indicator conditon
+val 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123025909
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
+  condition: RexNode,
+  inputType: RelDataType) = {
+
+def isExistTumble(expr: RexNode): Boolean = {
+  expr match {
+case c: RexCall =>
+  c.getOperator match {
+case _: SqlFloorFunction =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case SqlStdOperatorTable.TUMBLE =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case _ =>
+  c.getOperands.map(isExistTumble(_)).exists(_ == true)
+  }
+case _ => false
+  }
+}
+
+val isExistTimeIndicator = analyzeSingleConditionTerm(condition, 0, 
inputType).size > 0
+val isExistTumbleExpr = isExistTumble(condition)
+
+!isExistTumbleExpr && isExistTimeIndicator
+  }
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType
+  )
+
+if (timeTerms.size != 2) {
+  throw new TableException("There only can and must have 2 time 
conditions.")
+}
+
+// extract time offset from the time indicator conditon
+val 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123030370
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
+  condition: RexNode,
+  inputType: RelDataType) = {
+
+def isExistTumble(expr: RexNode): Boolean = {
+  expr match {
+case c: RexCall =>
+  c.getOperator match {
+case _: SqlFloorFunction =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case SqlStdOperatorTable.TUMBLE =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case _ =>
+  c.getOperands.map(isExistTumble(_)).exists(_ == true)
+  }
+case _ => false
+  }
+}
+
+val isExistTimeIndicator = analyzeSingleConditionTerm(condition, 0, 
inputType).size > 0
+val isExistTumbleExpr = isExistTumble(condition)
+
+!isExistTumbleExpr && isExistTimeIndicator
+  }
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType
+  )
+
+if (timeTerms.size != 2) {
+  throw new TableException("There only can and must have 2 time 
conditions.")
+}
+
+// extract time offset from the time indicator conditon
+val 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123209008
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeInnerJoin.scala
 ---
@@ -0,0 +1,316 @@
+/*
+ * 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 java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.CRowWrappingCollector
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftStreamWindowSizethe left stream window size
+  * @param rightStreamWindowSizethe right stream window size
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param genJoinFuncNamethe function code of other non-equi condition
+  * @param genJoinFuncCodethe function name of other non-equi condition
+  *
+  */
+class ProcTimeInnerJoin(
+private val leftStreamWindowSize: Long,
+private val rightStreamWindowSize: Long,
+private val element1Type: TypeInformation[Row],
+private val element2Type: TypeInformation[Row],
+private val genJoinFuncName: String,
+private val genJoinFuncCode: String)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]{
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  /** other condition function **/
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  /** tmp list to store expired records **/
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** 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] = _
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+  override def open(config: Configuration) {
--- End diff --

add a new line before the method


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123231612
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/JoinITCase.scala
 ---
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.stream.sql
+
+import org.apache.flink.api.scala._
+import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.api.scala.stream.utils.{StreamITCase, 
StreamingWithStateTestBase}
+import org.apache.flink.table.api.{TableEnvironment, TableException}
+import org.apache.flink.types.Row
+import org.junit.Assert._
+import org.junit._
+
+import scala.collection.mutable
+
+class JoinITCase extends StreamingWithStateTestBase {
+
+  val data = List(
+(1L, 1, "Hello"),
+(2L, 2, "Hello"),
+(3L, 3, "Hello"),
+(4L, 4, "Hello"),
+(5L, 5, "Hello"),
+(6L, 6, "Hello"),
+(7L, 7, "Hello World"),
+(8L, 8, "Hello World"),
+(20L, 20, "Hello World"))
+
+  /**
+* both stream should have boundary
+*/
+  @Test(expected = classOf[TableException])
+  def testJoinException0(): Unit = {
--- End diff --

We should not need an ITCase for these checks. The problem is that the 
validation is done during translation. If we would move the correctness checks 
into the optimizer, we don't need to translate the program.


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123231931
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/JoinITCase.scala
 ---
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.stream.sql
+
+import org.apache.flink.api.scala._
+import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.api.scala.stream.utils.{StreamITCase, 
StreamingWithStateTestBase}
+import org.apache.flink.table.api.{TableEnvironment, TableException}
+import org.apache.flink.types.Row
+import org.junit.Assert._
+import org.junit._
+
+import scala.collection.mutable
+
+class JoinITCase extends StreamingWithStateTestBase {
+
+  val data = List(
+(1L, 1, "Hello"),
+(2L, 2, "Hello"),
+(3L, 3, "Hello"),
+(4L, 4, "Hello"),
+(5L, 5, "Hello"),
+(6L, 6, "Hello"),
+(7L, 7, "Hello World"),
+(8L, 8, "Hello World"),
+(20L, 20, "Hello World"))
+
+  /**
+* both stream should have boundary
+*/
+  @Test(expected = classOf[TableException])
+  def testJoinException0(): Unit = {
+val env = StreamExecutionEnvironment.getExecutionEnvironment
+env.setStateBackend(getStateBackend)
+val tEnv = TableEnvironment.getTableEnvironment(env)
+StreamITCase.testResults = mutable.MutableList()
+
+val t1 = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c, 
'proctime.proctime)
+val t2 = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c, 
'proctime.proctime)
+
+tEnv.registerTable("T1", t1)
+tEnv.registerTable("T2", t2)
+
+val sqlQuery = "SELECT t2.a, t2.c, t1.c from T1 as t1 join T2 as t2 on 
t1.a = t2.a"
+
+val result = tEnv.sql(sqlQuery).toDataStream[Row]
+result.addSink(new StreamITCase.StringSink)
+env.execute()
+  }
+
+  /**
+* both stream should have boundary
+*/
+  @Test(expected = classOf[TableException])
+  def testJoinException1(): Unit = {
+val env = StreamExecutionEnvironment.getExecutionEnvironment
+env.setStateBackend(getStateBackend)
+val tEnv = TableEnvironment.getTableEnvironment(env)
+StreamITCase.testResults = mutable.MutableList()
+
+val t1 = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c, 
'proctime.proctime)
+val t2 = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c, 
'proctime.proctime)
+
+tEnv.registerTable("T1", t1)
+tEnv.registerTable("T2", t2)
+
+val sqlQuery = "SELECT t2.a, t2.c, t1.c from T1 as t1 join T2 as t2 on 
t1.a = t2.a " +
+  "and t1.proctime > t2.proctime - interval '5' second"
+
+val result = tEnv.sql(sqlQuery).toDataStream[Row]
+result.addSink(new StreamITCase.StringSink)
+env.execute()
+  }
+
+  /**
+* both stream should use same time indicator
+*/
+  @Test(expected = classOf[TableException])
+  def testJoinException2(): Unit = {
+val env = StreamExecutionEnvironment.getExecutionEnvironment
+env.setStateBackend(getStateBackend)
+val tEnv = TableEnvironment.getTableEnvironment(env)
+StreamITCase.testResults = mutable.MutableList()
+
+val t1 = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c, 
'proctime.proctime)
+val t2 = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c, 
'rowtime.rowtime)
+
+tEnv.registerTable("T1", t1)
+tEnv.registerTable("T2", t2)
+
+val sqlQuery = "SELECT t2.a, t2.c, t1.c from T1 as t1 join T2 as t2 on 
t1.a = t2.a " +
+  "and t1.proctime > t2.rowtime - interval '5' second "
+
+val result = tEnv.sql(sqlQuery).toDataStream[Row]
+result.addSink(new StreamITCase.StringSink)
+env.execute()
+  }
+
+
+  /** 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123189670
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/JoinITCase.scala
 ---
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.stream.sql
+
+import org.apache.flink.api.scala._
+import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.api.scala.stream.utils.{StreamITCase, 
StreamingWithStateTestBase}
+import org.apache.flink.table.api.{TableEnvironment, TableException}
+import org.apache.flink.types.Row
+import org.junit.Assert._
+import org.junit._
+
+import scala.collection.mutable
+
+class JoinITCase extends StreamingWithStateTestBase {
+
+  val data = List(
+(1L, 1, "Hello"),
+(2L, 2, "Hello"),
+(3L, 3, "Hello"),
+(4L, 4, "Hello"),
+(5L, 5, "Hello"),
+(6L, 6, "Hello"),
+(7L, 7, "Hello World"),
+(8L, 8, "Hello World"),
+(20L, 20, "Hello World"))
+
+  /**
+* both stream should have boundary
+*/
+  @Test(expected = classOf[TableException])
+  def testJoinException0(): Unit = {
+val env = StreamExecutionEnvironment.getExecutionEnvironment
+env.setStateBackend(getStateBackend)
+val tEnv = TableEnvironment.getTableEnvironment(env)
+StreamITCase.testResults = mutable.MutableList()
+
+val t1 = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c, 
'proctime.proctime)
+val t2 = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c, 
'proctime.proctime)
+
+tEnv.registerTable("T1", t1)
+tEnv.registerTable("T2", t2)
+
+val sqlQuery = "SELECT t2.a, t2.c, t1.c from T1 as t1 join T2 as t2 on 
t1.a = t2.a"
+
+val result = tEnv.sql(sqlQuery).toDataStream[Row]
+result.addSink(new StreamITCase.StringSink)
+env.execute()
+  }
+
+  /**
+* both stream should have boundary
+*/
+  @Test(expected = classOf[TableException])
+  def testJoinException1(): Unit = {
+val env = StreamExecutionEnvironment.getExecutionEnvironment
+env.setStateBackend(getStateBackend)
+val tEnv = TableEnvironment.getTableEnvironment(env)
+StreamITCase.testResults = mutable.MutableList()
+
+val t1 = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c, 
'proctime.proctime)
+val t2 = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c, 
'proctime.proctime)
+
+tEnv.registerTable("T1", t1)
+tEnv.registerTable("T2", t2)
+
+val sqlQuery = "SELECT t2.a, t2.c, t1.c from T1 as t1 join T2 as t2 on 
t1.a = t2.a " +
+  "and t1.proctime > t2.proctime - interval '5' second"
+
+val result = tEnv.sql(sqlQuery).toDataStream[Row]
+result.addSink(new StreamITCase.StringSink)
+env.execute()
+  }
+
+  /**
+* both stream should use same time indicator
+*/
+  @Test(expected = classOf[TableException])
+  def testJoinException2(): Unit = {
+val env = StreamExecutionEnvironment.getExecutionEnvironment
+env.setStateBackend(getStateBackend)
+val tEnv = TableEnvironment.getTableEnvironment(env)
+StreamITCase.testResults = mutable.MutableList()
+
+val t1 = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c, 
'proctime.proctime)
+val t2 = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c, 
'rowtime.rowtime)
+
+tEnv.registerTable("T1", t1)
+tEnv.registerTable("T2", t2)
+
+val sqlQuery = "SELECT t2.a, t2.c, t1.c from T1 as t1 join T2 as t2 on 
t1.a = t2.a " +
+  "and t1.proctime > t2.rowtime - interval '5' second "
+
+val result = tEnv.sql(sqlQuery).toDataStream[Row]
+result.addSink(new StreamITCase.StringSink)
+env.execute()
+  }
+
+
+  /** 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r122843078
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamRowStreamJoin.scala
 ---
@@ -0,0 +1,186 @@
+/*
+ * 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.core.{JoinInfo, JoinRelType}
+import org.apache.calcite.rel.{BiRel, RelNode, RelWriter}
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.api.java.functions.NullByteKeySelector
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.table.api.{StreamQueryConfig, 
StreamTableEnvironment, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.plan.nodes.CommonJoin
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.table.runtime.join.{JoinUtil, ProcTimeInnerJoin}
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+/**
+  * Flink RelNode which matches along with JoinOperator and its related 
operations.
+  */
+class DataStreamRowStreamJoin(
+cluster: RelOptCluster,
+traitSet: RelTraitSet,
+leftNode: RelNode,
+rightNode: RelNode,
+joinCondition: RexNode,
+joinType: JoinRelType,
+leftSchema: RowSchema,
+rightSchema: RowSchema,
+schema: RowSchema,
+ruleDescription: String)
+  extends BiRel(cluster, traitSet, leftNode, rightNode)
+  with CommonJoin
+  with DataStreamRel {
+
+  override def deriveRowType() = schema.logicalType
+
+  override def copy(traitSet: RelTraitSet, inputs: 
java.util.List[RelNode]): RelNode = {
+new DataStreamRowStreamJoin(
+  cluster,
+  traitSet,
+  inputs.get(0),
+  inputs.get(1),
+  joinCondition,
+  joinType,
+  leftSchema,
+  rightSchema,
+  schema,
+  ruleDescription)
+  }
+
+  override def toString: String = {
+
+s"${joinTypeToString(joinType)}" +
+  s"(condition: (${joinConditionToString(schema.logicalType,
+joinCondition, getExpressionString)}), " +
+  s"select: (${joinSelectionToString(schema.logicalType)}))"
+  }
+
+  override def explainTerms(pw: RelWriter): RelWriter = {
+super.explainTerms(pw)
+  .item("condition", joinConditionToString(schema.logicalType,
+joinCondition, getExpressionString))
+  .item("select", joinSelectionToString(schema.logicalType))
+  .item("joinType", joinTypeToString(joinType))
+  }
+
+  override def translateToPlan(
+  tableEnv: StreamTableEnvironment,
+  queryConfig: StreamQueryConfig): DataStream[CRow] = {
+
+val config = tableEnv.getConfig
+
+// get the equality keys and other condition
+val joinInfo = JoinInfo.of(leftNode, rightNode, joinCondition)
+val leftKeys = joinInfo.leftKeys.toIntArray
+val rightKeys = joinInfo.rightKeys.toIntArray
+val otherCondition = joinInfo.getRemaining(cluster.getRexBuilder)
+
+// analyze time boundary and time predicate type(proctime/rowtime)
+val (timeType, leftStreamWindowSize, rightStreamWindowSize, 
remainCondition) =
+  JoinUtil.analyzeTimeBoundary(
--- End diff --

I think we should move the analysis to the rule. Otherwise, we might end up 
with a plan that cannot be translated. It is the rule's responsibility to 
ensure that the translated plan can be executed.

The rule can then pass the analyzed time predicate parameters (time type, 
bounds) to the `DataStreamRowStreamJoin`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r122838733
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala
 ---
@@ -162,8 +162,25 @@ class RelTimeIndicatorConverter(rexBuilder: 
RexBuilder) extends RelShuttle {
 LogicalProject.create(input, projects, fieldNames)
   }
 
-  override def visit(join: LogicalJoin): RelNode =
-throw new TableException("Logical join in a stream environment is not 
supported yet.")
+  override def visit(join: LogicalJoin): RelNode = {
+val left = join.getLeft.accept(this)
+val right = join.getRight.accept(this)
+
+// check if input field contains time indicator type
+// materialize field if no time indicator is present anymore
+// if input field is already materialized, change to timestamp type
+val inputFields = left.getRowType.getFieldList.map(_.getType) ++
+  right.getRowType.getFieldList.map(_.getType)
+val materializer = new RexTimeIndicatorMaterializer(
+  rexBuilder,
+  inputFields)
+
+val condition = join.getCondition.accept(materializer)
--- End diff --

I think we do not need to materialize time indicators for join predicates. 
If the time indicators are used in valid time-based join predicates we do not 
code-gen the predicate and if they the time-based join predicate is not valid, 
the query will fail anyway.


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123020420
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
+  condition: RexNode,
+  inputType: RelDataType) = {
+
+def isExistTumble(expr: RexNode): Boolean = {
+  expr match {
+case c: RexCall =>
+  c.getOperator match {
+case _: SqlFloorFunction =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case SqlStdOperatorTable.TUMBLE =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case _ =>
+  c.getOperands.map(isExistTumble(_)).exists(_ == true)
+  }
+case _ => false
+  }
+}
+
+val isExistTimeIndicator = analyzeSingleConditionTerm(condition, 0, 
inputType).size > 0
+val isExistTumbleExpr = isExistTumble(condition)
+
+!isExistTumbleExpr && isExistTimeIndicator
+  }
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType
+  )
+
+if (timeTerms.size != 2) {
+  throw new TableException("There only can and must have 2 time 
conditions.")
+}
+
+// extract time offset from the time indicator conditon
+val 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123029455
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
+  condition: RexNode,
+  inputType: RelDataType) = {
+
+def isExistTumble(expr: RexNode): Boolean = {
+  expr match {
+case c: RexCall =>
+  c.getOperator match {
+case _: SqlFloorFunction =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case SqlStdOperatorTable.TUMBLE =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case _ =>
+  c.getOperands.map(isExistTumble(_)).exists(_ == true)
+  }
+case _ => false
+  }
+}
+
+val isExistTimeIndicator = analyzeSingleConditionTerm(condition, 0, 
inputType).size > 0
+val isExistTumbleExpr = isExistTumble(condition)
+
+!isExistTumbleExpr && isExistTimeIndicator
+  }
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType
+  )
+
+if (timeTerms.size != 2) {
+  throw new TableException("There only can and must have 2 time 
conditions.")
+}
+
+// extract time offset from the time indicator conditon
+val 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123026855
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
+  condition: RexNode,
+  inputType: RelDataType) = {
+
+def isExistTumble(expr: RexNode): Boolean = {
+  expr match {
+case c: RexCall =>
+  c.getOperator match {
+case _: SqlFloorFunction =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case SqlStdOperatorTable.TUMBLE =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case _ =>
+  c.getOperands.map(isExistTumble(_)).exists(_ == true)
+  }
+case _ => false
+  }
+}
+
+val isExistTimeIndicator = analyzeSingleConditionTerm(condition, 0, 
inputType).size > 0
+val isExistTumbleExpr = isExistTumble(condition)
+
+!isExistTumbleExpr && isExistTimeIndicator
+  }
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType
+  )
+
+if (timeTerms.size != 2) {
+  throw new TableException("There only can and must have 2 time 
conditions.")
+}
+
+// extract time offset from the time indicator conditon
+val 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123023871
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
+  condition: RexNode,
+  inputType: RelDataType) = {
+
+def isExistTumble(expr: RexNode): Boolean = {
+  expr match {
+case c: RexCall =>
+  c.getOperator match {
+case _: SqlFloorFunction =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case SqlStdOperatorTable.TUMBLE =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case _ =>
+  c.getOperands.map(isExistTumble(_)).exists(_ == true)
+  }
+case _ => false
+  }
+}
+
+val isExistTimeIndicator = analyzeSingleConditionTerm(condition, 0, 
inputType).size > 0
+val isExistTumbleExpr = isExistTumble(condition)
+
+!isExistTumbleExpr && isExistTimeIndicator
+  }
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType
+  )
+
+if (timeTerms.size != 2) {
+  throw new TableException("There only can and must have 2 time 
conditions.")
--- End diff --

"A time-based stream join requires exactly two join predicates that bound 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123004606
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
+  condition: RexNode,
+  inputType: RelDataType) = {
+
+def isExistTumble(expr: RexNode): Boolean = {
+  expr match {
+case c: RexCall =>
+  c.getOperator match {
+case _: SqlFloorFunction =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case SqlStdOperatorTable.TUMBLE =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case _ =>
+  c.getOperands.map(isExistTumble(_)).exists(_ == true)
--- End diff --

can be simplified to `c.getOperands.exists(isExistTumble(_))`


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r122842288
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonJoin.scala
 ---
@@ -0,0 +1,51 @@
+/*
+ * 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
+
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex.RexNode
+
+import scala.collection.JavaConverters._
+
+trait CommonJoin {
+
+  private[flink] def joinSelectionToString(inputType: RelDataType): String 
= {
+inputType.getFieldNames.asScala.toList.mkString(", ")
+  }
+
+  private[flink] def joinConditionToString(
+inputType: RelDataType,
+joinCondition: RexNode,
+expression: (RexNode, List[String], Option[List[RexNode]]) => String): 
String = {
+
+val inFields = inputType.getFieldNames.asScala.toList
+expression(joinCondition, inFields, None)
+  }
+
+  private[flink] def joinTypeToString(joinType: JoinRelType) = {
+joinType match {
+  case JoinRelType.INNER => "InnerJoin"
+  case JoinRelType.LEFT=> "LeftOuterJoin"
+  case JoinRelType.RIGHT => "RightOuterJoin"
+  case JoinRelType.FULL => "FullOuterJoin"
+}
+  }
+
--- End diff --

add `explainTerms` and `toString`


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123003230
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
--- End diff --

remove "other"?


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r123028155
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,385 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlKind
+import org.apache.calcite.sql.fun.{SqlFloorFunction, SqlStdOperatorTable}
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
+import org.apache.flink.table.plan.schema.{RowSchema, 
TimeIndicatorRelDataType}
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * An util class to help analyze and build join code .
+  */
+object JoinUtil {
+
+  /**
+* check if the join case is stream join stream
+*
+* @param  condition   other condtion include time-condition
+* @param  inputType   left and right connect stream type
+*/
+  private[flink] def isStreamStreamJoin(
+  condition: RexNode,
+  inputType: RelDataType) = {
+
+def isExistTumble(expr: RexNode): Boolean = {
+  expr match {
+case c: RexCall =>
+  c.getOperator match {
+case _: SqlFloorFunction =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case SqlStdOperatorTable.TUMBLE =>
+  c.getOperands.map(analyzeSingleConditionTerm(_, 0, 
inputType)).exists(_.size > 0)
+case _ =>
+  c.getOperands.map(isExistTumble(_)).exists(_ == true)
+  }
+case _ => false
+  }
+}
+
+val isExistTimeIndicator = analyzeSingleConditionTerm(condition, 0, 
inputType).size > 0
+val isExistTumbleExpr = isExistTumble(condition)
+
+!isExistTumbleExpr && isExistTimeIndicator
+  }
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return remain condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftLogicalFieldCnt left stream logical field num
+* @param  leftPhysicalFieldCnt left stream physical field num
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = {
+
+// Converts the condition to conjunctive normal form (CNF)
+val cnfCondition = RexUtil.toCnf(rexBuilder, condition)
+
+// split the condition into time indicator condition and other 
condition
+val (timeTerms, remainTerms) =
+  splitJoinCondition(
+cnfCondition,
+leftLogicalFieldCnt,
+inputType
+  )
+
+if (timeTerms.size != 2) {
+  throw new TableException("There only can and must have 2 time 
conditions.")
+}
+
+// extract time offset from the time indicator conditon
+val 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-06-21 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r122841979
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonJoin.scala
 ---
@@ -0,0 +1,51 @@
+/*
+ * 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
+
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex.RexNode
+
+import scala.collection.JavaConverters._
+
+trait CommonJoin {
--- End diff --

The `DataSetJoin` should also extend from this class. 


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-05-31 Thread rtudoran
Github user rtudoran commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r119358128
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamRowStreamJoin.scala
 ---
@@ -0,0 +1,184 @@
+/*
+ * 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.core.{JoinInfo, JoinRelType}
+import org.apache.calcite.rel.{BiRel, RelNode, RelWriter}
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.api.java.functions.NullByteKeySelector
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.table.api.{StreamTableEnvironment, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.plan.nodes.CommonJoin
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.table.runtime.join.{JoinUtil, ProcTimeInnerJoin}
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+/**
+  * Flink RelNode which matches along with JoinOperator and its related 
operations.
+  */
+class DataStreamRowStreamJoin(
+cluster: RelOptCluster,
+traitSet: RelTraitSet,
+leftNode: RelNode,
+rightNode: RelNode,
+joinCondition: RexNode,
+joinType: JoinRelType,
+leftSchema: RowSchema,
+rightSchema: RowSchema,
+schema: RowSchema,
+ruleDescription: String)
+  extends BiRel(cluster, traitSet, leftNode, rightNode)
+  with CommonJoin
+  with DataStreamRel {
+
+  override def deriveRowType() = schema.logicalType
+
+  override def copy(traitSet: RelTraitSet, inputs: 
java.util.List[RelNode]): RelNode = {
+new DataStreamRowStreamJoin(
+  cluster,
+  traitSet,
+  inputs.get(0),
+  inputs.get(1),
+  joinCondition,
+  joinType,
+  leftSchema,
+  rightSchema,
+  schema,
+  ruleDescription)
+  }
+
+  override def toString: String = {
+
+s"${joinTypeToString(joinType)}" +
+  s"(condition: (${joinConditionToString(schema.logicalType,
+joinCondition, getExpressionString)}), " +
+  s"select: (${joinSelectionToString(schema.logicalType)}))"
+  }
+
+  override def explainTerms(pw: RelWriter): RelWriter = {
+super.explainTerms(pw)
+  .item("condition", joinConditionToString(schema.logicalType,
+joinCondition, getExpressionString))
+  .item("select", joinSelectionToString(schema.logicalType))
+  .item("joinType", joinTypeToString(joinType))
+  }
+
+  override def translateToPlan(tableEnv: StreamTableEnvironment): 
DataStream[CRow] = {
+
+val config = tableEnv.getConfig
+
+// get the equality keys and other condition
+val joinInfo = JoinInfo.of(leftNode, rightNode, joinCondition)
+val leftKeys = joinInfo.leftKeys.toIntArray
+val rightKeys = joinInfo.rightKeys.toIntArray
+val otherCondition = joinInfo.getRemaining(cluster.getRexBuilder)
+
+// analyze time boundary and time predicate type(proctime/rowtime)
+val (timeType, leftStreamWindowSize, rightStreamWindowSize, 
remainCondition) =
+  JoinUtil.analyzeTimeBoundary(
+otherCondition,
+leftSchema.logicalType.getFieldCount,
+leftSchema.physicalType.getFieldCount,
+schema.logicalType,
+cluster.getRexBuilder,
+config)
+
+val leftDataStream = 
left.asInstanceOf[DataStreamRel].translateToPlan(tableEnv)
+val rightDataStream = 
right.asInstanceOf[DataStreamRel].translateToPlan(tableEnv)
--- End diff --

pass the query config along the way 
val leftDataStream = 
left.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
val rightDataStream = 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-05-31 Thread rtudoran
Github user rtudoran commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r119357940
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamRowStreamJoin.scala
 ---
@@ -0,0 +1,184 @@
+/*
+ * 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.core.{JoinInfo, JoinRelType}
+import org.apache.calcite.rel.{BiRel, RelNode, RelWriter}
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.api.java.functions.NullByteKeySelector
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.table.api.{StreamTableEnvironment, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.plan.nodes.CommonJoin
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.table.runtime.join.{JoinUtil, ProcTimeInnerJoin}
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+/**
+  * Flink RelNode which matches along with JoinOperator and its related 
operations.
+  */
+class DataStreamRowStreamJoin(
+cluster: RelOptCluster,
+traitSet: RelTraitSet,
+leftNode: RelNode,
+rightNode: RelNode,
+joinCondition: RexNode,
+joinType: JoinRelType,
+leftSchema: RowSchema,
+rightSchema: RowSchema,
+schema: RowSchema,
+ruleDescription: String)
+  extends BiRel(cluster, traitSet, leftNode, rightNode)
+  with CommonJoin
+  with DataStreamRel {
+
+  override def deriveRowType() = schema.logicalType
+
+  override def copy(traitSet: RelTraitSet, inputs: 
java.util.List[RelNode]): RelNode = {
+new DataStreamRowStreamJoin(
+  cluster,
+  traitSet,
+  inputs.get(0),
+  inputs.get(1),
+  joinCondition,
+  joinType,
+  leftSchema,
+  rightSchema,
+  schema,
+  ruleDescription)
+  }
+
+  override def toString: String = {
+
+s"${joinTypeToString(joinType)}" +
+  s"(condition: (${joinConditionToString(schema.logicalType,
+joinCondition, getExpressionString)}), " +
+  s"select: (${joinSelectionToString(schema.logicalType)}))"
+  }
+
+  override def explainTerms(pw: RelWriter): RelWriter = {
+super.explainTerms(pw)
+  .item("condition", joinConditionToString(schema.logicalType,
+joinCondition, getExpressionString))
+  .item("select", joinSelectionToString(schema.logicalType))
+  .item("joinType", joinTypeToString(joinType))
+  }
+
+  override def translateToPlan(tableEnv: StreamTableEnvironment): 
DataStream[CRow] = {
--- End diff --

update to the new interface 
override def translateToPlan(
  tableEnv: StreamTableEnvironment,
  queryConfig: StreamQueryConfig): DataStream[CRow] = {


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-05-21 Thread rtudoran
Github user rtudoran commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r117651897
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeInnerJoin.scala
 ---
@@ -0,0 +1,323 @@
+/*
+ * 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 java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.RichFilterFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftStreamWindowSizethe left stream window size
+  * @param rightStreamWindowSizethe right stream window size
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param filterFuncthe function of other non-equi condition include 
time condition
+  *
+  */
+class ProcTimeInnerJoin(
+  private val leftStreamWindowSize: Long,
+  private val rightStreamWindowSize: Long,
+  private val element1Type: TypeInformation[CRow],
+  private val element2Type: TypeInformation[CRow],
+  private val filterFunc: RichFilterFunction[Row])
+  extends CoProcessFunction[CRow, CRow, CRow] {
+
+  private var outputC: CRow = _
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** state to record last timer of left stream, 0 means no timer **/
+  private var timerState1: ValueState[Long] = _
--- End diff --

I understand and it make sense to me. As it reduces the overhead i think it 
is ok. @fhueske - do you have other opinion? 


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-05-18 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r117205051
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamJoinRule.scala
 ---
@@ -0,0 +1,69 @@
+/*
+ * 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.plan.nodes.datastream.DataStreamJoin
+import org.apache.flink.table.plan.nodes.FlinkConventions
+import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
+import org.apache.flink.table.plan.schema.RowSchema
+
+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
--- End diff --

Yes, that's what I was thinking of. It would separate the implementation of 
the different joins.


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-05-18 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r117202009
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,488 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+import java.util.EnumSet
+
+import org.apache.calcite.avatica.util.TimeUnit
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.fun.SqlStdOperatorTable
+import org.apache.calcite.sql.parser.SqlParserPos
+import org.apache.calcite.sql.{SqlIntervalQualifier, SqlKind}
+import org.apache.flink.api.common.functions.FilterFunction
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
+import org.apache.flink.table.runtime.FilterRunner
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable.ArrayBuffer
+
+
+object JoinUtil {
+
+  /**
+* Analyze join condition to get equi-conditon and other condition
+* @param  joinNode   logicaljoin node
+* @param  expression the function to generate condition string
+*/
+  private[flink] def analyzeJoinCondition(
+joinNode: FlinkLogicalJoin,
+expression: (RexNode, List[String], Option[List[RexNode]]) => String) 
= {
+
+val joinInfo = joinNode.analyzeCondition()
+val keyPairs = joinInfo.pairs.toList
+val otherCondition =
+  if(joinInfo.isEqui) null
+  else joinInfo.getRemaining(joinNode.getCluster.getRexBuilder)
+
+val leftKeys = ArrayBuffer.empty[Int]
+val rightKeys = ArrayBuffer.empty[Int]
+if (!keyPairs.isEmpty) {
+  val leftFields = joinNode.getLeft.getRowType.getFieldList
+  val rightFields = joinNode.getRight.getRowType.getFieldList
+
+  keyPairs.foreach(pair => {
+val leftKeyType = 
leftFields.get(pair.source).getType.getSqlTypeName
+val rightKeyType = 
rightFields.get(pair.target).getType.getSqlTypeName
+
+// check if keys are compatible
+if (leftKeyType == rightKeyType) {
+  // add key pair
+  leftKeys.append(pair.source)
+  rightKeys.append(pair.target)
+} else {
+  throw TableException(
+"Equality join predicate on incompatible types.\n" +
+  s"\tLeft: ${joinNode.getLeft.toString},\n" +
+  s"\tRight: ${joinNode.getRight.toString},\n" +
+  s"\tCondition: (${expression(joinNode.getCondition,
+joinNode.getRowType.getFieldNames.toList, None)})"
+  )
+}
+  })
+}
+(leftKeys.toArray, rightKeys.toArray, otherCondition)
+  }
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return condition without time-condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftFieldCount left stream fields count
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
--- End diff --

Yes, the logic of greater and smaller / left and right side, etc. needs to 
go into step 5. Maybe have a look at Calcite's `RexUtil` which might have some 
helpful methods.


---
If your project is 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-05-18 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r117200048
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeInnerJoin.scala
 ---
@@ -0,0 +1,323 @@
+/*
+ * 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 java.util
+import java.util.{List => JList}
+
+import org.apache.flink.api.common.functions.RichFilterFunction
+import org.apache.flink.api.common.state._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to support stream join stream, currently just 
support inner-join
+  *
+  * @param leftStreamWindowSizethe left stream window size
+  * @param rightStreamWindowSizethe right stream window size
+  * @param element1Type  the input type of left stream
+  * @param element2Type  the input type of right stream
+  * @param filterFuncthe function of other non-equi condition include 
time condition
+  *
+  */
+class ProcTimeInnerJoin(
+  private val leftStreamWindowSize: Long,
+  private val rightStreamWindowSize: Long,
+  private val element1Type: TypeInformation[CRow],
+  private val element2Type: TypeInformation[CRow],
+  private val filterFunc: RichFilterFunction[Row])
+  extends CoProcessFunction[CRow, CRow, CRow] {
+
+  private var outputC: CRow = _
+  private var listToRemove: JList[Long] = _
+
+  /** state to hold left stream element **/
+  private var row1MapState: MapState[Long, JList[Row]] = _
+  /** state to hold right stream element **/
+  private var row2MapState: MapState[Long, JList[Row]] = _
+
+  /** 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(config: Configuration) {
+outputC = new CRow(new Row(element1Type.getArity + 
element2Type.getArity), true)
+filterFunc.setRuntimeContext(getRuntimeContext)
+filterFunc.open(config)
+
+listToRemove = new util.ArrayList[Long]()
+
+// initialize row state
+val rowListTypeInfo1: TypeInformation[JList[Row]] =
+new ListTypeInfo[Row](element1Type.asInstanceOf[CRowTypeInfo].rowType)
+val mapStateDescriptor1: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]]("row1mapstate",
+BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], 
rowListTypeInfo1)
+row1MapState = getRuntimeContext.getMapState(mapStateDescriptor1)
+
+val rowListTypeInfo2: TypeInformation[JList[Row]] =
+  new 
ListTypeInfo[Row](element2Type.asInstanceOf[CRowTypeInfo].rowType)
+val mapStateDescriptor2: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]]("row2mapstate",
+BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], 
rowListTypeInfo2)
+row2MapState = getRuntimeContext.getMapState(mapStateDescriptor2)
+
+// initialize timer state
+val valueStateDescriptor1: ValueStateDescriptor[Long] =
+  new ValueStateDescriptor[Long]("timervaluestate1", classOf[Long])
+timerState1 = getRuntimeContext.getState(valueStateDescriptor1)
+
+val valueStateDescriptor2: ValueStateDescriptor[Long] =
+  new ValueStateDescriptor[Long]("timervaluestate2", classOf[Long])
+timerState2 = 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-05-18 Thread hongyuhong
Github user hongyuhong commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r117176330
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamJoinRule.scala
 ---
@@ -0,0 +1,69 @@
+/*
+ * 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.plan.nodes.datastream.DataStreamJoin
+import org.apache.flink.table.plan.nodes.FlinkConventions
+import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
+import org.apache.flink.table.plan.schema.RowSchema
+
+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
--- End diff --

Hi @fhueske , you mean that we should distinguish different join cases, for 
example, this issue belong to stream-stream join, and there exist other case 
like stream-table join, window-window join(tumble), each case has different 
relnode and each case deal with inner/outer , rowtime/protime in their relnode, 
so maybe we can not call this relnode DataStreamJoin but need more detail, is 
that right? 
And if each case have their own rule, then we need to check the case in 
matchs() function, is it?


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-05-18 Thread hongyuhong
Github user hongyuhong commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r117170779
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,488 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+import java.util.EnumSet
+
+import org.apache.calcite.avatica.util.TimeUnit
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.fun.SqlStdOperatorTable
+import org.apache.calcite.sql.parser.SqlParserPos
+import org.apache.calcite.sql.{SqlIntervalQualifier, SqlKind}
+import org.apache.flink.api.common.functions.FilterFunction
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
+import org.apache.flink.table.runtime.FilterRunner
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable.ArrayBuffer
+
+
+object JoinUtil {
+
+  /**
+* Analyze join condition to get equi-conditon and other condition
+* @param  joinNode   logicaljoin node
+* @param  expression the function to generate condition string
+*/
+  private[flink] def analyzeJoinCondition(
+joinNode: FlinkLogicalJoin,
+expression: (RexNode, List[String], Option[List[RexNode]]) => String) 
= {
+
+val joinInfo = joinNode.analyzeCondition()
+val keyPairs = joinInfo.pairs.toList
+val otherCondition =
+  if(joinInfo.isEqui) null
+  else joinInfo.getRemaining(joinNode.getCluster.getRexBuilder)
+
+val leftKeys = ArrayBuffer.empty[Int]
+val rightKeys = ArrayBuffer.empty[Int]
+if (!keyPairs.isEmpty) {
+  val leftFields = joinNode.getLeft.getRowType.getFieldList
+  val rightFields = joinNode.getRight.getRowType.getFieldList
+
+  keyPairs.foreach(pair => {
+val leftKeyType = 
leftFields.get(pair.source).getType.getSqlTypeName
+val rightKeyType = 
rightFields.get(pair.target).getType.getSqlTypeName
+
+// check if keys are compatible
+if (leftKeyType == rightKeyType) {
+  // add key pair
+  leftKeys.append(pair.source)
+  rightKeys.append(pair.target)
+} else {
+  throw TableException(
+"Equality join predicate on incompatible types.\n" +
+  s"\tLeft: ${joinNode.getLeft.toString},\n" +
+  s"\tRight: ${joinNode.getRight.toString},\n" +
+  s"\tCondition: (${expression(joinNode.getCondition,
+joinNode.getRowType.getFieldNames.toList, None)})"
+  )
+}
+  })
+}
+(leftKeys.toArray, rightKeys.toArray, otherCondition)
+  }
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return condition without time-condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftFieldCount left stream fields count
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
--- End diff --

Hi @fhueske, i think this solution is more clearer exactly. I will rewrite 
the logic asap. But i think we still need to distinguish between 
greater/smaller at step 5, if the condition is 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-05-17 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r117033348
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,468 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+import java.util.EnumSet
+
+import org.apache.calcite.avatica.util.TimeUnit
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.fun.SqlStdOperatorTable
+import org.apache.calcite.sql.parser.SqlParserPos
+import org.apache.calcite.sql.{SqlIntervalQualifier, SqlKind}
+import org.apache.flink.api.common.functions.{FilterFunction, 
FlatJoinFunction}
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenException, CodeGenerator, 
ExpressionReducer}
+import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable.ArrayBuffer
+
+
+object JoinUtil {
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return condition without time-condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftFieldCount left stream fields count
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, RexNode) = {
+// analyze the time-conditon to get greate and less condition,
+// make sure left stream field in the left of the condition
+// e.g b.proctime > a.proctime - 1 will be translate to a.proctime - 1 
< b.proctime
+val greateConditions = new util.ArrayList[TimeSingleCondition]()
+val lessConditions = new util.ArrayList[TimeSingleCondition]()
+analyzeTimeCondition(condition, greateConditions,
+  lessConditions, leftLogicalFieldCnt, inputType)
+if (greateConditions.size != lessConditions.size
+|| greateConditions.size > 1
+|| greateConditions.size == 0) {
+  throw TableException(
+"Equality join time conditon should have proctime or rowtime 
indicator."
+  )
+}
+
+val greatCond = greateConditions.get(0)
+val lessCond = lessConditions.get(0)
+if (greatCond.timeType != lessCond.timeType) {
+  throw TableException(
+"Equality join time conditon should all use proctime or all use 
rowtime."
+  )
+}
+
+var leftStreamWindowSize: Long = 0
+var rightStreamWindowSize: Long = 0
+
+// only a.proctime > b.proctime - interval '1' hour need to store a 
stream
+val timeLiteral: RexLiteral =
+reduceTimeExpression(greatCond.rightExpr, greatCond.leftExpr, 
rexBuilder, config)
+leftStreamWindowSize = timeLiteral.getValue2.asInstanceOf[Long]
+// only need to store past records
+if (leftStreamWindowSize < 0) {
+  leftStreamWindowSize = -leftStreamWindowSize
+  if (!greatCond.isEqual) {
+leftStreamWindowSize -= 1
+  }
+} else {
+  leftStreamWindowSize = 0
+}
+
+// only 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-05-17 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r117033149
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,468 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+import java.util.EnumSet
+
+import org.apache.calcite.avatica.util.TimeUnit
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.fun.SqlStdOperatorTable
+import org.apache.calcite.sql.parser.SqlParserPos
+import org.apache.calcite.sql.{SqlIntervalQualifier, SqlKind}
+import org.apache.flink.api.common.functions.{FilterFunction, 
FlatJoinFunction}
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenException, CodeGenerator, 
ExpressionReducer}
+import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable.ArrayBuffer
+
+
+object JoinUtil {
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return condition without time-condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftFieldCount left stream fields count
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, RexNode) = {
+// analyze the time-conditon to get greate and less condition,
+// make sure left stream field in the left of the condition
+// e.g b.proctime > a.proctime - 1 will be translate to a.proctime - 1 
< b.proctime
+val greateConditions = new util.ArrayList[TimeSingleCondition]()
+val lessConditions = new util.ArrayList[TimeSingleCondition]()
+analyzeTimeCondition(condition, greateConditions,
+  lessConditions, leftLogicalFieldCnt, inputType)
+if (greateConditions.size != lessConditions.size
+|| greateConditions.size > 1
+|| greateConditions.size == 0) {
+  throw TableException(
+"Equality join time conditon should have proctime or rowtime 
indicator."
+  )
+}
+
+val greatCond = greateConditions.get(0)
+val lessCond = lessConditions.get(0)
+if (greatCond.timeType != lessCond.timeType) {
+  throw TableException(
+"Equality join time conditon should all use proctime or all use 
rowtime."
+  )
+}
+
+var leftStreamWindowSize: Long = 0
+var rightStreamWindowSize: Long = 0
+
+// only a.proctime > b.proctime - interval '1' hour need to store a 
stream
+val timeLiteral: RexLiteral =
+reduceTimeExpression(greatCond.rightExpr, greatCond.leftExpr, 
rexBuilder, config)
+leftStreamWindowSize = timeLiteral.getValue2.asInstanceOf[Long]
+// only need to store past records
+if (leftStreamWindowSize < 0) {
+  leftStreamWindowSize = -leftStreamWindowSize
+  if (!greatCond.isEqual) {
+leftStreamWindowSize -= 1
+  }
+} else {
+  leftStreamWindowSize = 0
+}
+
+// only 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-05-17 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r117006282
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,488 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+import java.util.EnumSet
+
+import org.apache.calcite.avatica.util.TimeUnit
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.fun.SqlStdOperatorTable
+import org.apache.calcite.sql.parser.SqlParserPos
+import org.apache.calcite.sql.{SqlIntervalQualifier, SqlKind}
+import org.apache.flink.api.common.functions.FilterFunction
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer}
+import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
+import org.apache.flink.table.runtime.FilterRunner
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable.ArrayBuffer
+
+
+object JoinUtil {
+
+  /**
+* Analyze join condition to get equi-conditon and other condition
+* @param  joinNode   logicaljoin node
+* @param  expression the function to generate condition string
+*/
+  private[flink] def analyzeJoinCondition(
+joinNode: FlinkLogicalJoin,
+expression: (RexNode, List[String], Option[List[RexNode]]) => String) 
= {
+
+val joinInfo = joinNode.analyzeCondition()
+val keyPairs = joinInfo.pairs.toList
+val otherCondition =
+  if(joinInfo.isEqui) null
+  else joinInfo.getRemaining(joinNode.getCluster.getRexBuilder)
+
+val leftKeys = ArrayBuffer.empty[Int]
+val rightKeys = ArrayBuffer.empty[Int]
+if (!keyPairs.isEmpty) {
+  val leftFields = joinNode.getLeft.getRowType.getFieldList
+  val rightFields = joinNode.getRight.getRowType.getFieldList
+
+  keyPairs.foreach(pair => {
+val leftKeyType = 
leftFields.get(pair.source).getType.getSqlTypeName
+val rightKeyType = 
rightFields.get(pair.target).getType.getSqlTypeName
+
+// check if keys are compatible
+if (leftKeyType == rightKeyType) {
+  // add key pair
+  leftKeys.append(pair.source)
+  rightKeys.append(pair.target)
+} else {
+  throw TableException(
+"Equality join predicate on incompatible types.\n" +
+  s"\tLeft: ${joinNode.getLeft.toString},\n" +
+  s"\tRight: ${joinNode.getRight.toString},\n" +
+  s"\tCondition: (${expression(joinNode.getCondition,
+joinNode.getRowType.getFieldNames.toList, None)})"
+  )
+}
+  })
+}
+(leftKeys.toArray, rightKeys.toArray, otherCondition)
+  }
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return condition without time-condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftFieldCount left stream fields count
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
--- End diff --

I think the logic of this function is correct. However, I find it a bit 
hard to follow because it starts with many conditions.
What do you think about the following approach:

1. convert 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-05-17 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r117037366
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,468 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+import java.util.EnumSet
+
+import org.apache.calcite.avatica.util.TimeUnit
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.fun.SqlStdOperatorTable
+import org.apache.calcite.sql.parser.SqlParserPos
+import org.apache.calcite.sql.{SqlIntervalQualifier, SqlKind}
+import org.apache.flink.api.common.functions.{FilterFunction, 
FlatJoinFunction}
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenException, CodeGenerator, 
ExpressionReducer}
+import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable.ArrayBuffer
+
+
+object JoinUtil {
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return condition without time-condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftFieldCount left stream fields count
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
--- End diff --

This method needs some good unit tests.


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-05-17 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r117027410
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,468 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+import java.util.EnumSet
+
+import org.apache.calcite.avatica.util.TimeUnit
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.fun.SqlStdOperatorTable
+import org.apache.calcite.sql.parser.SqlParserPos
+import org.apache.calcite.sql.{SqlIntervalQualifier, SqlKind}
+import org.apache.flink.api.common.functions.{FilterFunction, 
FlatJoinFunction}
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenException, CodeGenerator, 
ExpressionReducer}
+import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable.ArrayBuffer
+
+
+object JoinUtil {
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return condition without time-condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftFieldCount left stream fields count
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, RexNode) = {
+// analyze the time-conditon to get greate and less condition,
+// make sure left stream field in the left of the condition
+// e.g b.proctime > a.proctime - 1 will be translate to a.proctime - 1 
< b.proctime
+val greateConditions = new util.ArrayList[TimeSingleCondition]()
+val lessConditions = new util.ArrayList[TimeSingleCondition]()
+analyzeTimeCondition(condition, greateConditions,
+  lessConditions, leftLogicalFieldCnt, inputType)
+if (greateConditions.size != lessConditions.size
+|| greateConditions.size > 1
+|| greateConditions.size == 0) {
+  throw TableException(
+"Equality join time conditon should have proctime or rowtime 
indicator."
+  )
+}
+
+val greatCond = greateConditions.get(0)
+val lessCond = lessConditions.get(0)
+if (greatCond.timeType != lessCond.timeType) {
+  throw TableException(
+"Equality join time conditon should all use proctime or all use 
rowtime."
+  )
+}
+
+var leftStreamWindowSize: Long = 0
+var rightStreamWindowSize: Long = 0
+
+// only a.proctime > b.proctime - interval '1' hour need to store a 
stream
+val timeLiteral: RexLiteral =
+reduceTimeExpression(greatCond.rightExpr, greatCond.leftExpr, 
rexBuilder, config)
+leftStreamWindowSize = timeLiteral.getValue2.asInstanceOf[Long]
+// only need to store past records
+if (leftStreamWindowSize < 0) {
+  leftStreamWindowSize = -leftStreamWindowSize
+  if (!greatCond.isEqual) {
+leftStreamWindowSize -= 1
+  }
+} else {
+  leftStreamWindowSize = 0
+}
+
+// only 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-05-17 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r117002279
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,468 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+import java.util.EnumSet
+
+import org.apache.calcite.avatica.util.TimeUnit
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.fun.SqlStdOperatorTable
+import org.apache.calcite.sql.parser.SqlParserPos
+import org.apache.calcite.sql.{SqlIntervalQualifier, SqlKind}
+import org.apache.flink.api.common.functions.{FilterFunction, 
FlatJoinFunction}
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenException, CodeGenerator, 
ExpressionReducer}
+import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable.ArrayBuffer
+
+
+object JoinUtil {
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return condition without time-condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftFieldCount left stream fields count
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
--- End diff --

As a first step, the `condition` should be converted into CNF (conjunctive 
normal form) for normalization. Calcite offers the `RexUtil.toCnf()` method for 
that.


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-05-17 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r117031586
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,468 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+import java.util.EnumSet
+
+import org.apache.calcite.avatica.util.TimeUnit
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.fun.SqlStdOperatorTable
+import org.apache.calcite.sql.parser.SqlParserPos
+import org.apache.calcite.sql.{SqlIntervalQualifier, SqlKind}
+import org.apache.flink.api.common.functions.{FilterFunction, 
FlatJoinFunction}
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenException, CodeGenerator, 
ExpressionReducer}
+import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable.ArrayBuffer
+
+
+object JoinUtil {
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return condition without time-condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftFieldCount left stream fields count
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, RexNode) = {
+// analyze the time-conditon to get greate and less condition,
+// make sure left stream field in the left of the condition
+// e.g b.proctime > a.proctime - 1 will be translate to a.proctime - 1 
< b.proctime
+val greateConditions = new util.ArrayList[TimeSingleCondition]()
+val lessConditions = new util.ArrayList[TimeSingleCondition]()
+analyzeTimeCondition(condition, greateConditions,
+  lessConditions, leftLogicalFieldCnt, inputType)
+if (greateConditions.size != lessConditions.size
+|| greateConditions.size > 1
+|| greateConditions.size == 0) {
+  throw TableException(
+"Equality join time conditon should have proctime or rowtime 
indicator."
+  )
+}
+
+val greatCond = greateConditions.get(0)
+val lessCond = lessConditions.get(0)
+if (greatCond.timeType != lessCond.timeType) {
+  throw TableException(
+"Equality join time conditon should all use proctime or all use 
rowtime."
+  )
+}
+
+var leftStreamWindowSize: Long = 0
+var rightStreamWindowSize: Long = 0
+
+// only a.proctime > b.proctime - interval '1' hour need to store a 
stream
+val timeLiteral: RexLiteral =
+reduceTimeExpression(greatCond.rightExpr, greatCond.leftExpr, 
rexBuilder, config)
+leftStreamWindowSize = timeLiteral.getValue2.asInstanceOf[Long]
+// only need to store past records
+if (leftStreamWindowSize < 0) {
+  leftStreamWindowSize = -leftStreamWindowSize
+  if (!greatCond.isEqual) {
+leftStreamWindowSize -= 1
+  }
+} else {
+  leftStreamWindowSize = 0
+}
+
+// only 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-05-17 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r116791596
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,468 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+import java.util.EnumSet
+
+import org.apache.calcite.avatica.util.TimeUnit
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.fun.SqlStdOperatorTable
+import org.apache.calcite.sql.parser.SqlParserPos
+import org.apache.calcite.sql.{SqlIntervalQualifier, SqlKind}
+import org.apache.flink.api.common.functions.{FilterFunction, 
FlatJoinFunction}
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenException, CodeGenerator, 
ExpressionReducer}
+import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable.ArrayBuffer
+
+
+object JoinUtil {
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return condition without time-condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftFieldCount left stream fields count
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, RexNode) = {
+// analyze the time-conditon to get greate and less condition,
+// make sure left stream field in the left of the condition
+// e.g b.proctime > a.proctime - 1 will be translate to a.proctime - 1 
< b.proctime
+val greateConditions = new util.ArrayList[TimeSingleCondition]()
+val lessConditions = new util.ArrayList[TimeSingleCondition]()
+analyzeTimeCondition(condition, greateConditions,
--- End diff --

Please wrap the arguments of a function as follows (if it does not fit in a 
single line):
```
analyzeTimeCondition(
  condition, 
  greateConditions,
  lessConditions, 
  leftLogicalFieldCnt, 
  inputType)
```


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-05-17 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r117004436
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,468 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+import java.util.EnumSet
+
+import org.apache.calcite.avatica.util.TimeUnit
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.fun.SqlStdOperatorTable
+import org.apache.calcite.sql.parser.SqlParserPos
+import org.apache.calcite.sql.{SqlIntervalQualifier, SqlKind}
+import org.apache.flink.api.common.functions.{FilterFunction, 
FlatJoinFunction}
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenException, CodeGenerator, 
ExpressionReducer}
+import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable.ArrayBuffer
+
+
+object JoinUtil {
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return condition without time-condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftFieldCount left stream fields count
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
--- End diff --

I think this would make the code in this class a lot simpler, because we 
would not need to recursively dig into the condition. We can iterate over all 
conjunctive conditions and check for each if it is a valid time bound condition 
and either remove it or not. 


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-05-17 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r116790511
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,468 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+import java.util.EnumSet
+
+import org.apache.calcite.avatica.util.TimeUnit
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.fun.SqlStdOperatorTable
+import org.apache.calcite.sql.parser.SqlParserPos
+import org.apache.calcite.sql.{SqlIntervalQualifier, SqlKind}
+import org.apache.flink.api.common.functions.{FilterFunction, 
FlatJoinFunction}
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenException, CodeGenerator, 
ExpressionReducer}
+import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable.ArrayBuffer
+
+
+object JoinUtil {
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return condition without time-condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftFieldCount left stream fields count
--- End diff --

-> `leftLogicalFieldCnt`

Add `rightLogicalFieldCnt`


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


[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-05-17 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r116796524
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,468 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+import java.util.EnumSet
+
+import org.apache.calcite.avatica.util.TimeUnit
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.fun.SqlStdOperatorTable
+import org.apache.calcite.sql.parser.SqlParserPos
+import org.apache.calcite.sql.{SqlIntervalQualifier, SqlKind}
+import org.apache.flink.api.common.functions.{FilterFunction, 
FlatJoinFunction}
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.{CodeGenException, CodeGenerator, 
ExpressionReducer}
+import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable.ArrayBuffer
+
+
+object JoinUtil {
+
+  /**
+* Analyze time-condtion to get time boundary for each stream and get 
the time type
+* and return condition without time-condition.
+*
+* @param  condition   other condtion include time-condition
+* @param  leftFieldCount left stream fields count
+* @param  inputType   left and right connect stream type
+* @param  rexBuilder   util to build rexNode
+* @param  config  table environment config
+*/
+  private[flink] def analyzeTimeBoundary(
+  condition: RexNode,
+  leftLogicalFieldCnt: Int,
+  leftPhysicalFieldCnt: Int,
+  inputType: RelDataType,
+  rexBuilder: RexBuilder,
+  config: TableConfig): (RelDataType, Long, Long, RexNode) = {
+// analyze the time-conditon to get greate and less condition,
+// make sure left stream field in the left of the condition
+// e.g b.proctime > a.proctime - 1 will be translate to a.proctime - 1 
< b.proctime
+val greateConditions = new util.ArrayList[TimeSingleCondition]()
+val lessConditions = new util.ArrayList[TimeSingleCondition]()
+analyzeTimeCondition(condition, greateConditions,
+  lessConditions, leftLogicalFieldCnt, inputType)
+if (greateConditions.size != lessConditions.size
+|| greateConditions.size > 1
+|| greateConditions.size == 0) {
+  throw TableException(
+"Equality join time conditon should have proctime or rowtime 
indicator."
+  )
+}
+
+val greatCond = greateConditions.get(0)
+val lessCond = lessConditions.get(0)
+if (greatCond.timeType != lessCond.timeType) {
+  throw TableException(
+"Equality join time conditon should all use proctime or all use 
rowtime."
+  )
+}
+
+var leftStreamWindowSize: Long = 0
+var rightStreamWindowSize: Long = 0
+
+// only a.proctime > b.proctime - interval '1' hour need to store a 
stream
+val timeLiteral: RexLiteral =
+reduceTimeExpression(greatCond.rightExpr, greatCond.leftExpr, 
rexBuilder, config)
+leftStreamWindowSize = timeLiteral.getValue2.asInstanceOf[Long]
+// only need to store past records
+if (leftStreamWindowSize < 0) {
+  leftStreamWindowSize = -leftStreamWindowSize
+  if (!greatCond.isEqual) {
+leftStreamWindowSize -= 1
+  }
+} else {
+  leftStreamWindowSize = 0
+}
+
+// only 

[GitHub] flink pull request #3715: [FLINK-6232][Table]Support proctime inner equi...

2017-05-17 Thread fhueske
Github user fhueske commented on a diff in the pull request:

https://github.com/apache/flink/pull/3715#discussion_r116847715
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/JoinUtil.scala
 ---
@@ -0,0 +1,468 @@
+/*
+ * 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 java.math.{BigDecimal => JBigDecimal}
+import java.util
+import java.util.EnumSet
+
+import org.apache.calcite.avatica.util.TimeUnit
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.fun.SqlStdOperatorTable
+import org.apache.calcite.sql.parser.SqlParserPos
+import org.apache.calcite.sql.{SqlIntervalQualifier, SqlKind}
+import org.apache.flink.api.common.functions.{FilterFunction, 
FlatJoinFunction}
--- End diff --

please clean up unused imports


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


  1   2   >