[jira] [Created] (FLINK-7591) Support partition by and order by of MATCH_RECOGNIZE

2017-09-05 Thread Dian Fu (JIRA)
Dian Fu created FLINK-7591:
--

 Summary: Support partition by and order by of MATCH_RECOGNIZE
 Key: FLINK-7591
 URL: https://issues.apache.org/jira/browse/FLINK-7591
 Project: Flink
  Issue Type: Sub-task
  Components: CEP
Reporter: Dian Fu
Assignee: Dian Fu


FLINK-7062 adds the basic functionality of MATCH_RECOGNIZE. This JIRA adds the 
support of {{PARTITION BY}} and {{ORDER BY}} clauses.



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


[jira] [Commented] (FLINK-7491) Support COLLECT Aggregate function in Flink SQL

2017-09-05 Thread Shuyi Chen (JIRA)

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

Shuyi Chen commented on FLINK-7491:
---

Thanks a lot, [~fhueske].

> Support COLLECT Aggregate function in Flink SQL
> ---
>
> Key: FLINK-7491
> URL: https://issues.apache.org/jira/browse/FLINK-7491
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: Shuyi Chen
>Assignee: Shuyi Chen
>




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


[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137168799
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+  

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

2017-09-05 Thread xccui
Github user xccui commented on a diff in the pull request:

https://github.com/apache/flink/pull/4625#discussion_r137168799
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+  if (allowedLateness < 0) {
+throw new IllegalArgumentException("The allowed lateness must be 
non-negative.")
+  }
+
+
+  /**
+* For holding back watermarks.
+*
+* @return the maximum delay 

[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137163327
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+  

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

2017-09-05 Thread xccui
Github user xccui commented on a diff in the pull request:

https://github.com/apache/flink/pull/4625#discussion_r137163327
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+  if (allowedLateness < 0) {
+throw new IllegalArgumentException("The allowed lateness must be 
non-negative.")
+  }
+
+
+  /**
+* For holding back watermarks.
+*
+* @return the maximum delay 

[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137158818
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
--- End diff --

This variable is used to store a suggestion value for performing 
backpressure *in the future*. We could cache less records if one of 

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

2017-09-05 Thread xccui
Github user xccui commented on a diff in the pull request:

https://github.com/apache/flink/pull/4625#discussion_r137158818
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
--- End diff --

This variable is used to store a suggestion value for performing 
backpressure *in the future*. We could cache less records if one of the stream 
is held back with this suggestion. It's just like moving the cache from Flink 
to upstream components (e.g., Kafka).


---


[jira] [Commented] (FLINK-7310) always use HybridMemorySegment

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

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

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

Github user KurtYoung commented on the issue:

https://github.com/apache/flink/pull/4445
  
I would bet on deserialization for it. And why sorter suffers more 
regression than hash join is that sorter will cause more deserializations 
during compare records than hash join.

Despite the regression we will face, i think it's still worthy since we can 
avoid an extra copy from network to runtime. It's better if we can take the 
extra copy into account during benchmark, but it's ok we don't have it. 

+1 to merge this.


> always use HybridMemorySegment
> --
>
> Key: FLINK-7310
> URL: https://issues.apache.org/jira/browse/FLINK-7310
> Project: Flink
>  Issue Type: Sub-task
>  Components: Core
>Affects Versions: 1.4.0
>Reporter: Nico Kruber
>Assignee: Nico Kruber
>
> For future changes to the network buffers (sending our own off-heap buffers 
> through to netty), we cannot use {{HeapMemorySegment}} anymore and need to 
> rely on {{HybridMemorySegment}} instead.
> We should thus drop any code that loads the {{HeapMemorySegment}} (it is 
> still available if needed) in favour of the {{HybridMemorySegment}} which is 
> able to work on both heap and off-heap memory.
> FYI: For the performance penalty of this change compared to using 
> {{HeapMemorySegment}} alone, see this interesting blob article (from 2015):
> https://flink.apache.org/news/2015/09/16/off-heap-memory.html



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


[GitHub] flink issue #4445: [FLINK-7310][core] always use the HybridMemorySegment

2017-09-05 Thread KurtYoung
Github user KurtYoung commented on the issue:

https://github.com/apache/flink/pull/4445
  
I would bet on deserialization for it. And why sorter suffers more 
regression than hash join is that sorter will cause more deserializations 
during compare records than hash join.

Despite the regression we will face, i think it's still worthy since we can 
avoid an extra copy from network to runtime. It's better if we can take the 
extra copy into account during benchmark, but it's ok we don't have it. 

+1 to merge this.


---


[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137150128
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala
 ---
@@ -115,10 +118,15 @@ object WindowJoinUtil {
   case _ =>
 Some(otherPreds.reduceLeft((l, r) => 
RelOptUtil.andJoinFilters(rexBuilder, l, r)))
 }
-
-val bounds = Some(WindowBounds(timePreds.head.isEventTime, 
leftLowerBound, leftUpperBound))
-
-(bounds, remainCondition)
+if (timePreds.head.leftInputOnLeftSide) {
--- End diff --

We should also subtract the `leftLogicalFieldCnt` for the later index?


> Support rowtime inner equi-join between two streams in the SQL API
> --
>
> Key: FLINK-6233
> URL: https://issues.apache.org/jira/browse/FLINK-6233
> Project: Flink
>  Issue Type: Sub-task
>  Components: Table API & SQL
>Reporter: hongyuhong
>Assignee: Xingcan Cui
>
> The goal of this issue is to add support for inner equi-join on proc time 
> streams to the SQL interface.
> Queries similar to the following should be supported:
> {code}
> SELECT o.rowtime , o.productId, o.orderId, s.rowtime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL '1' HOUR;
> {code}
> The following restrictions should initially apply:
> * The join hint only support inner join
> * The ON clause should include equi-join condition
> * The time-condition {{o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL 
> '1' HOUR}} only can use rowtime that is a system attribute, the time 
> condition only support bounded time range like {{o.rowtime BETWEEN s.rowtime 
> - INTERVAL '1' HOUR AND s.rowtime + INTERVAL '1' HOUR}}, not support 
> unbounded like {{o.rowtime  s.rowtime}} ,  and  should include both two 
> stream's rowtime attribute, {{o.rowtime between rowtime () and rowtime () + 
> 1}} should also not be supported.
> An row-time streams join will not be able to handle late data, because this 
> would mean in insert a row into a sorted order shift all other computations. 
> This would be too expensive to maintain. Therefore, we will throw an error if 
> a user tries to use an row-time stream join with late data handling.
> This issue includes:
> * Design of the DataStream operator to deal with stream join
> * Translation from Calcite's RelNode representation (LogicalJoin). 



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


[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

2017-09-05 Thread xccui
Github user xccui commented on a diff in the pull request:

https://github.com/apache/flink/pull/4625#discussion_r137150128
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala
 ---
@@ -115,10 +118,15 @@ object WindowJoinUtil {
   case _ =>
 Some(otherPreds.reduceLeft((l, r) => 
RelOptUtil.andJoinFilters(rexBuilder, l, r)))
 }
-
-val bounds = Some(WindowBounds(timePreds.head.isEventTime, 
leftLowerBound, leftUpperBound))
-
-(bounds, remainCondition)
+if (timePreds.head.leftInputOnLeftSide) {
--- End diff --

We should also subtract the `leftLogicalFieldCnt` for the later index?


---


[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137146303
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala
 ---
@@ -184,4 +195,54 @@ class DataStreamWindowJoin(
 .returns(returnTypeInfo)
 }
   }
+
+  def createRowTimeInnerJoinFunction(
+leftDataStream: DataStream[CRow],
+rightDataStream: DataStream[CRow],
+joinFunctionName: String,
+joinFunctionCode: String,
+leftKeys: Array[Int],
+rightKeys: Array[Int]): DataStream[CRow] = {
+
+val returnTypeInfo = CRowTypeInfo(schema.typeInfo)
+
+val rowTimeInnerJoinFunc = new TimeBoundedStreamInnerJoin(
+  leftLowerBound,
+  leftUpperBound,
+  0L,
+  leftSchema.typeInfo,
+  rightSchema.typeInfo,
+  joinFunctionName,
+  joinFunctionCode,
+  leftTimeIdx,
+  rightTimeIdx,
+  JoinTimeIndicator.ROWTIME
+)
+
+if (!leftKeys.isEmpty) {
+  leftDataStream
+.connect(rightDataStream)
+.keyBy(leftKeys, rightKeys)
+.transform(
+  "rowTimeInnerJoinFunc",
+  returnTypeInfo,
+  new KeyedCoProcessOperatorWithWatermarkDelay[CRow, CRow, CRow, 
CRow](
+rowTimeInnerJoinFunc,
+rowTimeInnerJoinFunc.getMaxOutputDelay)
--- End diff --

I think the "watermark delay" is considered from the operator level while 
the "output delay" is named from the function level. So how about keep this 
name?


> Support rowtime inner equi-join between two streams in the SQL API
> --
>
> Key: FLINK-6233
> URL: https://issues.apache.org/jira/browse/FLINK-6233
> Project: Flink
>  Issue Type: Sub-task
>  Components: Table API & SQL
>Reporter: hongyuhong
>Assignee: Xingcan Cui
>
> The goal of this issue is to add support for inner equi-join on proc time 
> streams to the SQL interface.
> Queries similar to the following should be supported:
> {code}
> SELECT o.rowtime , o.productId, o.orderId, s.rowtime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL '1' HOUR;
> {code}
> The following restrictions should initially apply:
> * The join hint only support inner join
> * The ON clause should include equi-join condition
> * The time-condition {{o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL 
> '1' HOUR}} only can use rowtime that is a system attribute, the time 
> condition only support bounded time range like {{o.rowtime BETWEEN s.rowtime 
> - INTERVAL '1' HOUR AND s.rowtime + INTERVAL '1' HOUR}}, not support 
> unbounded like {{o.rowtime  s.rowtime}} ,  and  should include both two 
> stream's rowtime attribute, {{o.rowtime between rowtime () and rowtime () + 
> 1}} should also not be supported.
> An row-time streams join will not be able to handle late data, because this 
> would mean in insert a row into a sorted order shift all other computations. 
> This would be too expensive to maintain. Therefore, we will throw an error if 
> a user tries to use an row-time stream join with late data handling.
> This issue includes:
> * Design of the DataStream operator to deal with stream join
> * Translation from Calcite's RelNode representation (LogicalJoin). 



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


[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

2017-09-05 Thread xccui
Github user xccui commented on a diff in the pull request:

https://github.com/apache/flink/pull/4625#discussion_r137146303
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala
 ---
@@ -184,4 +195,54 @@ class DataStreamWindowJoin(
 .returns(returnTypeInfo)
 }
   }
+
+  def createRowTimeInnerJoinFunction(
+leftDataStream: DataStream[CRow],
+rightDataStream: DataStream[CRow],
+joinFunctionName: String,
+joinFunctionCode: String,
+leftKeys: Array[Int],
+rightKeys: Array[Int]): DataStream[CRow] = {
+
+val returnTypeInfo = CRowTypeInfo(schema.typeInfo)
+
+val rowTimeInnerJoinFunc = new TimeBoundedStreamInnerJoin(
+  leftLowerBound,
+  leftUpperBound,
+  0L,
+  leftSchema.typeInfo,
+  rightSchema.typeInfo,
+  joinFunctionName,
+  joinFunctionCode,
+  leftTimeIdx,
+  rightTimeIdx,
+  JoinTimeIndicator.ROWTIME
+)
+
+if (!leftKeys.isEmpty) {
+  leftDataStream
+.connect(rightDataStream)
+.keyBy(leftKeys, rightKeys)
+.transform(
+  "rowTimeInnerJoinFunc",
+  returnTypeInfo,
+  new KeyedCoProcessOperatorWithWatermarkDelay[CRow, CRow, CRow, 
CRow](
+rowTimeInnerJoinFunc,
+rowTimeInnerJoinFunc.getMaxOutputDelay)
--- End diff --

I think the "watermark delay" is considered from the operator level while 
the "output delay" is named from the function level. So how about keep this 
name?


---


[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137144634
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala
 ---
@@ -184,4 +195,54 @@ class DataStreamWindowJoin(
 .returns(returnTypeInfo)
 }
   }
+
+  def createRowTimeInnerJoinFunction(
+leftDataStream: DataStream[CRow],
+rightDataStream: DataStream[CRow],
+joinFunctionName: String,
+joinFunctionCode: String,
+leftKeys: Array[Int],
+rightKeys: Array[Int]): DataStream[CRow] = {
+
+val returnTypeInfo = CRowTypeInfo(schema.typeInfo)
+
+val rowTimeInnerJoinFunc = new TimeBoundedStreamInnerJoin(
+  leftLowerBound,
+  leftUpperBound,
+  0L,
+  leftSchema.typeInfo,
+  rightSchema.typeInfo,
+  joinFunctionName,
+  joinFunctionCode,
+  leftTimeIdx,
+  rightTimeIdx,
+  JoinTimeIndicator.ROWTIME
+)
+
+if (!leftKeys.isEmpty) {
+  leftDataStream
+.connect(rightDataStream)
+.keyBy(leftKeys, rightKeys)
+.transform(
+  "rowTimeInnerJoinFunc",
--- End diff --

I'd like to call this kind of join "time-bounded join" instead of "window 
join". When referring to window join, the users may think of tumbling-window or 
sliding-window, while they are actually not the same. However, as the 
“window-join” name has been widely used, I can also accept it. Do you have any 
idea about that?


> Support rowtime inner equi-join between two streams in the SQL API
> --
>
> Key: FLINK-6233
> URL: https://issues.apache.org/jira/browse/FLINK-6233
> Project: Flink
>  Issue Type: Sub-task
>  Components: Table API & SQL
>Reporter: hongyuhong
>Assignee: Xingcan Cui
>
> The goal of this issue is to add support for inner equi-join on proc time 
> streams to the SQL interface.
> Queries similar to the following should be supported:
> {code}
> SELECT o.rowtime , o.productId, o.orderId, s.rowtime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL '1' HOUR;
> {code}
> The following restrictions should initially apply:
> * The join hint only support inner join
> * The ON clause should include equi-join condition
> * The time-condition {{o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL 
> '1' HOUR}} only can use rowtime that is a system attribute, the time 
> condition only support bounded time range like {{o.rowtime BETWEEN s.rowtime 
> - INTERVAL '1' HOUR AND s.rowtime + INTERVAL '1' HOUR}}, not support 
> unbounded like {{o.rowtime  s.rowtime}} ,  and  should include both two 
> stream's rowtime attribute, {{o.rowtime between rowtime () and rowtime () + 
> 1}} should also not be supported.
> An row-time streams join will not be able to handle late data, because this 
> would mean in insert a row into a sorted order shift all other computations. 
> This would be too expensive to maintain. Therefore, we will throw an error if 
> a user tries to use an row-time stream join with late data handling.
> This issue includes:
> * Design of the DataStream operator to deal with stream join
> * Translation from Calcite's RelNode representation (LogicalJoin). 



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


[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

2017-09-05 Thread xccui
Github user xccui commented on a diff in the pull request:

https://github.com/apache/flink/pull/4625#discussion_r137144634
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala
 ---
@@ -184,4 +195,54 @@ class DataStreamWindowJoin(
 .returns(returnTypeInfo)
 }
   }
+
+  def createRowTimeInnerJoinFunction(
+leftDataStream: DataStream[CRow],
+rightDataStream: DataStream[CRow],
+joinFunctionName: String,
+joinFunctionCode: String,
+leftKeys: Array[Int],
+rightKeys: Array[Int]): DataStream[CRow] = {
+
+val returnTypeInfo = CRowTypeInfo(schema.typeInfo)
+
+val rowTimeInnerJoinFunc = new TimeBoundedStreamInnerJoin(
+  leftLowerBound,
+  leftUpperBound,
+  0L,
+  leftSchema.typeInfo,
+  rightSchema.typeInfo,
+  joinFunctionName,
+  joinFunctionCode,
+  leftTimeIdx,
+  rightTimeIdx,
+  JoinTimeIndicator.ROWTIME
+)
+
+if (!leftKeys.isEmpty) {
+  leftDataStream
+.connect(rightDataStream)
+.keyBy(leftKeys, rightKeys)
+.transform(
+  "rowTimeInnerJoinFunc",
--- End diff --

I'd like to call this kind of join "time-bounded join" instead of "window 
join". When referring to window join, the users may think of tumbling-window or 
sliding-window, while they are actually not the same. However, as the 
“window-join” name has been widely used, I can also accept it. Do you have 
any idea about that?


---


[jira] [Commented] (FLINK-7439) Support variable arguments for UDTF in SQL

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

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

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

Github user sunjincheng121 commented on the issue:

https://github.com/apache/flink/pull/4536
  
Thanks for the updating, looks good to me.
+1 to merged.


> Support variable arguments for UDTF in SQL
> --
>
> Key: FLINK-7439
> URL: https://issues.apache.org/jira/browse/FLINK-7439
> Project: Flink
>  Issue Type: Improvement
>  Components: Table API & SQL
>Reporter: Jark Wu
>Assignee: Jark Wu
>
> Currently, both UDF and UDAF support variable parameters, but UDTF not. 
> FLINK-5882 supports variable UDTF for Table API only, but missed SQL.



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


[GitHub] flink issue #4536: [FLINK-7439] [table] Support variable arguments for UDTF ...

2017-09-05 Thread sunjincheng121
Github user sunjincheng121 commented on the issue:

https://github.com/apache/flink/pull/4536
  
Thanks for the updating, looks good to me.
+1 to merged.


---


[jira] [Commented] (FLINK-6465) support FIRST_VALUE on Table API & SQL

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

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

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

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

https://github.com/apache/flink/pull/4556#discussion_r137142537
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/FirstValueAggFunctionWithRetract.scala
 ---
@@ -0,0 +1,121 @@
+/*
+ * 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.functions.aggfunctions
+
+import java.math.BigDecimal
+import java.util.{ArrayList => JArray, List => JList}
+
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.tuple.{Tuple1 => JTuple1}
+import org.apache.flink.api.java.typeutils.TupleTypeInfo
+import org.apache.flink.table.functions.AggregateFunction
+
+/** The initial accumulator for first value with retraction aggregate 
function */
+class FirstValueWithRetractAccumulator[T] extends JTuple1[JList[T]]
--- End diff --

 FLINK-7206 is done, I'll  using `DataView` for improve the implementation.


> support FIRST_VALUE on Table API & SQL
> --
>
> Key: FLINK-6465
> URL: https://issues.apache.org/jira/browse/FLINK-6465
> Project: Flink
>  Issue Type: Sub-task
>  Components: Table API & SQL
>Reporter: Hequn Cheng
>Assignee: sunjincheng
>
> {{FIRST_VALUE}} is a OVER WINDOW function. In this JIRA. will add 
> {{FIRST_VALUE}} function support on TableAPI & SQL.
> *Syntax:*
> FIRST_VALUE ( [scalar_expression ] )   
> OVER ( [ partition_by_clause ] order_by_clause [ rows_range_clause ] )  
> [About OVER 
> WINDOWS|https://ci.apache.org/projects/flink/flink-docs-release-1.4/dev/table/tableApi.html#over-windows]
> scalar_expression
> Is the value to be returned. scalar_expression can be a column, or other 
> arbitrary expression that results in a single value. Other analytic functions 
> are not permitted.
> *NOTE:*
> *  {{FIRST_VALUE}} if used for OVER WINDOW, e.g.: 
> {code}
> SELECT A, B, FIRST_VALUE(C) OVER (ORDER BY E) AS firstValue FROM tab
> {code}
> * OVER WINDOW's retraction is expensive(currently not supported yet), and 
> this JIRA. does not implement Retract logic of {{FIRST_VALUE}}.



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


[GitHub] flink pull request #4556: [FLINK-6465][table]support FIRST_VALUE on Table AP...

2017-09-05 Thread sunjincheng121
Github user sunjincheng121 commented on a diff in the pull request:

https://github.com/apache/flink/pull/4556#discussion_r137142537
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/FirstValueAggFunctionWithRetract.scala
 ---
@@ -0,0 +1,121 @@
+/*
+ * 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.functions.aggfunctions
+
+import java.math.BigDecimal
+import java.util.{ArrayList => JArray, List => JList}
+
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.tuple.{Tuple1 => JTuple1}
+import org.apache.flink.api.java.typeutils.TupleTypeInfo
+import org.apache.flink.table.functions.AggregateFunction
+
+/** The initial accumulator for first value with retraction aggregate 
function */
+class FirstValueWithRetractAccumulator[T] extends JTuple1[JList[T]]
--- End diff --

 FLINK-7206 is done, I'll  using `DataView` for improve the implementation.


---


[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

2017-09-05 Thread xccui
Github user xccui commented on a diff in the pull request:

https://github.com/apache/flink/pull/4625#discussion_r137137915
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamWindowJoinRule.scala
 ---
@@ -55,8 +55,10 @@ class DataStreamWindowJoinRule
 
 if (windowBounds.isDefined) {
   if (windowBounds.get.isEventTime) {
-// we cannot handle event-time window joins yet
-false
+val procTimeAttrInOutput = join.getRowType.getFieldList.asScala
+  .exists(f => FlinkTypeFactory.isProctimeIndicatorType(f.getType))
+
+!remainingPredsAccessTime && !procTimeAttrInOutput
--- End diff --

Shall we also keep the rowtime attributes in the outputs of proctime join?


---


[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137137915
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamWindowJoinRule.scala
 ---
@@ -55,8 +55,10 @@ class DataStreamWindowJoinRule
 
 if (windowBounds.isDefined) {
   if (windowBounds.get.isEventTime) {
-// we cannot handle event-time window joins yet
-false
+val procTimeAttrInOutput = join.getRowType.getFieldList.asScala
+  .exists(f => FlinkTypeFactory.isProctimeIndicatorType(f.getType))
+
+!remainingPredsAccessTime && !procTimeAttrInOutput
--- End diff --

Shall we also keep the rowtime attributes in the outputs of proctime join?


> Support rowtime inner equi-join between two streams in the SQL API
> --
>
> Key: FLINK-6233
> URL: https://issues.apache.org/jira/browse/FLINK-6233
> Project: Flink
>  Issue Type: Sub-task
>  Components: Table API & SQL
>Reporter: hongyuhong
>Assignee: Xingcan Cui
>
> The goal of this issue is to add support for inner equi-join on proc time 
> streams to the SQL interface.
> Queries similar to the following should be supported:
> {code}
> SELECT o.rowtime , o.productId, o.orderId, s.rowtime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL '1' HOUR;
> {code}
> The following restrictions should initially apply:
> * The join hint only support inner join
> * The ON clause should include equi-join condition
> * The time-condition {{o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL 
> '1' HOUR}} only can use rowtime that is a system attribute, the time 
> condition only support bounded time range like {{o.rowtime BETWEEN s.rowtime 
> - INTERVAL '1' HOUR AND s.rowtime + INTERVAL '1' HOUR}}, not support 
> unbounded like {{o.rowtime  s.rowtime}} ,  and  should include both two 
> stream's rowtime attribute, {{o.rowtime between rowtime () and rowtime () + 
> 1}} should also not be supported.
> An row-time streams join will not be able to handle late data, because this 
> would mean in insert a row into a sorted order shift all other computations. 
> This would be too expensive to maintain. Therefore, we will throw an error if 
> a user tries to use an row-time stream join with late data handling.
> This issue includes:
> * Design of the DataStream operator to deal with stream join
> * Translation from Calcite's RelNode representation (LogicalJoin). 



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


[GitHub] flink issue #4625: [FLINK-6233] [table] Support time-bounded stream inner jo...

2017-09-05 Thread xccui
Github user xccui commented on the issue:

https://github.com/apache/flink/pull/4625
  
Thanks for the review, @fhueske. This PR is a little rough when I 
committed. I'll address your comments and submit a refined version as soon as 
possible.

Best, Xingcan


---


[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

Github user xccui commented on the issue:

https://github.com/apache/flink/pull/4625
  
Thanks for the review, @fhueske. This PR is a little rough when I 
committed. I'll address your comments and submit a refined version as soon as 
possible.

Best, Xingcan


> Support rowtime inner equi-join between two streams in the SQL API
> --
>
> Key: FLINK-6233
> URL: https://issues.apache.org/jira/browse/FLINK-6233
> Project: Flink
>  Issue Type: Sub-task
>  Components: Table API & SQL
>Reporter: hongyuhong
>Assignee: Xingcan Cui
>
> The goal of this issue is to add support for inner equi-join on proc time 
> streams to the SQL interface.
> Queries similar to the following should be supported:
> {code}
> SELECT o.rowtime , o.productId, o.orderId, s.rowtime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL '1' HOUR;
> {code}
> The following restrictions should initially apply:
> * The join hint only support inner join
> * The ON clause should include equi-join condition
> * The time-condition {{o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL 
> '1' HOUR}} only can use rowtime that is a system attribute, the time 
> condition only support bounded time range like {{o.rowtime BETWEEN s.rowtime 
> - INTERVAL '1' HOUR AND s.rowtime + INTERVAL '1' HOUR}}, not support 
> unbounded like {{o.rowtime  s.rowtime}} ,  and  should include both two 
> stream's rowtime attribute, {{o.rowtime between rowtime () and rowtime () + 
> 1}} should also not be supported.
> An row-time streams join will not be able to handle late data, because this 
> would mean in insert a row into a sorted order shift all other computations. 
> This would be too expensive to maintain. Therefore, we will throw an error if 
> a user tries to use an row-time stream join with late data handling.
> This issue includes:
> * Design of the DataStream operator to deal with stream join
> * Translation from Calcite's RelNode representation (LogicalJoin). 



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


[jira] [Updated] (FLINK-7590) Flink failed to flush and close the file system output stream for checkpointing because of s3 read timeout

2017-09-05 Thread Bowen Li (JIRA)

 [ 
https://issues.apache.org/jira/browse/FLINK-7590?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Bowen Li updated FLINK-7590:

Summary: Flink failed to flush and close the file system output stream for 
checkpointing because of s3 read timeout  (was: Flink failed to flush and close 
the file system output stream for checkpointing)

> Flink failed to flush and close the file system output stream for 
> checkpointing because of s3 read timeout
> --
>
> Key: FLINK-7590
> URL: https://issues.apache.org/jira/browse/FLINK-7590
> Project: Flink
>  Issue Type: Bug
>  Components: State Backends, Checkpointing
>Affects Versions: 1.3.2
>Reporter: Bowen Li
> Fix For: 1.4.0, 1.3.3
>
>
> Flink job failed once over the weekend because of the following issue. It 
> picked itself up afterwards and has been running well. But the issue might 
> worth taking a look at.
> {code:java}
> 2017-09-03 13:18:38,998 INFO  
> org.apache.flink.runtime.executiongraph.ExecutionGraph- reduce 
> (14/18) (c97256badc87e995d456e7a13cec5de9) switched from RUNNING to FAILED.
> AsynchronousException{java.lang.Exception: Could not materialize checkpoint 
> 163 for operator reduce (14/18).}
>   at 
> org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:970)
>   at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>   at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>   at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>   at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>   at java.lang.Thread.run(Thread.java:748)
> Caused by: java.lang.Exception: Could not materialize checkpoint 163 for 
> operator reduce (14/18).
>   ... 6 more
> Caused by: java.util.concurrent.ExecutionException: java.io.IOException: 
> Could not flush and close the file system output stream to 
> s3://xxx/chk-163/dcb9e1df-78e0-444a-9646-7701b25c1aaa in order to obtain the 
> stream state handle
>   at java.util.concurrent.FutureTask.report(FutureTask.java:122)
>   at java.util.concurrent.FutureTask.get(FutureTask.java:192)
>   at 
> org.apache.flink.util.FutureUtil.runIfNotDoneAndGet(FutureUtil.java:43)
>   at 
> org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:897)
>   ... 5 more
>   Suppressed: java.lang.Exception: Could not properly cancel managed 
> keyed state future.
>   at 
> org.apache.flink.streaming.api.operators.OperatorSnapshotResult.cancel(OperatorSnapshotResult.java:90)
>   at 
> org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.cleanup(StreamTask.java:1023)
>   at 
> org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:961)
>   ... 5 more
>   Caused by: java.util.concurrent.ExecutionException: 
> java.io.IOException: Could not flush and close the file system output stream 
> to s3://xxx/chk-163/dcb9e1df-78e0-444a-9646-7701b25c1aaa in order to obtain 
> the stream state handle
>   at java.util.concurrent.FutureTask.report(FutureTask.java:122)
>   at java.util.concurrent.FutureTask.get(FutureTask.java:192)
>   at 
> org.apache.flink.util.FutureUtil.runIfNotDoneAndGet(FutureUtil.java:43)
>   at 
> org.apache.flink.runtime.state.StateUtil.discardStateFuture(StateUtil.java:85)
>   at 
> org.apache.flink.streaming.api.operators.OperatorSnapshotResult.cancel(OperatorSnapshotResult.java:88)
>   ... 7 more
>   Caused by: java.io.IOException: Could not flush and close the file 
> system output stream to s3://xxx/chk-163/dcb9e1df-78e0-444a-9646-7701b25c1aaa 
> in order to obtain the stream state handle
>   at 
> org.apache.flink.runtime.state.filesystem.FsCheckpointStreamFactory$FsCheckpointStateOutputStream.closeAndGetHandle(FsCheckpointStreamFactory.java:336)
>   at 
> org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$RocksDBFullSnapshotOperation.closeSnapshotStreamAndGetHandle(RocksDBKeyedStateBackend.java:693)
>   at 
> org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$RocksDBFullSnapshotOperation.closeCheckpointStream(RocksDBKeyedStateBackend.java:531)
>   at 
> org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$3.performOperation(RocksDBKeyedStateBackend.java:420)
>   at 
> org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$3.performOperation(RocksDBKeyedStateBackend.java:399)
>   at 
> 

[jira] [Created] (FLINK-7590) Flink failed to flush and close the file system output stream for checkpointing

2017-09-05 Thread Bowen Li (JIRA)
Bowen Li created FLINK-7590:
---

 Summary: Flink failed to flush and close the file system output 
stream for checkpointing
 Key: FLINK-7590
 URL: https://issues.apache.org/jira/browse/FLINK-7590
 Project: Flink
  Issue Type: Bug
  Components: State Backends, Checkpointing
Affects Versions: 1.3.2
Reporter: Bowen Li
 Fix For: 1.4.0, 1.3.3


Flink job failed once over the weekend because of the following issue. It 
picked itself up afterwards and has been running well. But the issue might 
worth taking a look at.

{code:java}
2017-09-03 13:18:38,998 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph- reduce (14/18) 
(c97256badc87e995d456e7a13cec5de9) switched from RUNNING to FAILED.
AsynchronousException{java.lang.Exception: Could not materialize checkpoint 163 
for operator reduce (14/18).}
at 
org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:970)
at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at java.lang.Thread.run(Thread.java:748)
Caused by: java.lang.Exception: Could not materialize checkpoint 163 for 
operator reduce (14/18).
... 6 more
Caused by: java.util.concurrent.ExecutionException: java.io.IOException: Could 
not flush and close the file system output stream to 
s3://xxx/chk-163/dcb9e1df-78e0-444a-9646-7701b25c1aaa in order to obtain the 
stream state handle
at java.util.concurrent.FutureTask.report(FutureTask.java:122)
at java.util.concurrent.FutureTask.get(FutureTask.java:192)
at 
org.apache.flink.util.FutureUtil.runIfNotDoneAndGet(FutureUtil.java:43)
at 
org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:897)
... 5 more
Suppressed: java.lang.Exception: Could not properly cancel managed 
keyed state future.
at 
org.apache.flink.streaming.api.operators.OperatorSnapshotResult.cancel(OperatorSnapshotResult.java:90)
at 
org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.cleanup(StreamTask.java:1023)
at 
org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:961)
... 5 more
Caused by: java.util.concurrent.ExecutionException: 
java.io.IOException: Could not flush and close the file system output stream to 
s3://xxx/chk-163/dcb9e1df-78e0-444a-9646-7701b25c1aaa in order to obtain the 
stream state handle
at java.util.concurrent.FutureTask.report(FutureTask.java:122)
at java.util.concurrent.FutureTask.get(FutureTask.java:192)
at 
org.apache.flink.util.FutureUtil.runIfNotDoneAndGet(FutureUtil.java:43)
at 
org.apache.flink.runtime.state.StateUtil.discardStateFuture(StateUtil.java:85)
at 
org.apache.flink.streaming.api.operators.OperatorSnapshotResult.cancel(OperatorSnapshotResult.java:88)
... 7 more
Caused by: java.io.IOException: Could not flush and close the file 
system output stream to s3://xxx/chk-163/dcb9e1df-78e0-444a-9646-7701b25c1aaa 
in order to obtain the stream state handle
at 
org.apache.flink.runtime.state.filesystem.FsCheckpointStreamFactory$FsCheckpointStateOutputStream.closeAndGetHandle(FsCheckpointStreamFactory.java:336)
at 
org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$RocksDBFullSnapshotOperation.closeSnapshotStreamAndGetHandle(RocksDBKeyedStateBackend.java:693)
at 
org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$RocksDBFullSnapshotOperation.closeCheckpointStream(RocksDBKeyedStateBackend.java:531)
at 
org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$3.performOperation(RocksDBKeyedStateBackend.java:420)
at 
org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$3.performOperation(RocksDBKeyedStateBackend.java:399)
at 
org.apache.flink.runtime.io.async.AbstractAsyncIOCallable.call(AbstractAsyncIOCallable.java:72)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at 
org.apache.flink.util.FutureUtil.runIfNotDoneAndGet(FutureUtil.java:40)
at 
org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:897)
... 5 more
Caused by: com.amazonaws.AmazonClientException: Unable to unmarshall 
response (Failed to parse XML document with handler class 

[jira] [Created] (FLINK-7589) org.apache.http.ConnectionClosedException: Premature end of Content-Length delimited message body (expected: 159764230; received: 64638536)

2017-09-05 Thread Bowen Li (JIRA)
Bowen Li created FLINK-7589:
---

 Summary: org.apache.http.ConnectionClosedException: Premature end 
of Content-Length delimited message body (expected: 159764230; received: 
64638536)
 Key: FLINK-7589
 URL: https://issues.apache.org/jira/browse/FLINK-7589
 Project: Flink
  Issue Type: Bug
Affects Versions: 1.3.2
Reporter: Bowen Li
 Fix For: 1.4.0, 1.3.3



When I tried to resume a Flink job from a savepoint with different parallelism, 
I ran into this error. And the resume failed.


{code:java}
2017-09-05 21:53:57,317 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph- returnsivs -> 
Sink: xxx (7/12) (0da16ec908fc7b9b16a5c2cf1aa92947) switched from RUNNING to 
FAILED.
org.apache.http.ConnectionClosedException: Premature end of Content-Length 
delimited message body (expected: 159764230; received: 64638536
at 
org.apache.http.impl.io.ContentLengthInputStream.read(ContentLengthInputStream.java:180)
at 
org.apache.http.conn.EofSensorInputStream.read(EofSensorInputStream.java:137)
at java.io.FilterInputStream.read(FilterInputStream.java:133)
at java.io.FilterInputStream.read(FilterInputStream.java:133)
at java.io.FilterInputStream.read(FilterInputStream.java:133)
at 
com.amazonaws.util.ContentLengthValidationInputStream.read(ContentLengthValidationInputStream.java:77)
at java.io.FilterInputStream.read(FilterInputStream.java:133)
at org.apache.hadoop.fs.s3a.S3AInputStream.read(S3AInputStream.java:160)
at java.io.DataInputStream.read(DataInputStream.java:149)
at 
org.apache.flink.runtime.fs.hdfs.HadoopDataInputStream.read(HadoopDataInputStream.java:72)
at 
org.apache.flink.core.fs.FSDataInputStreamWrapper.read(FSDataInputStreamWrapper.java:61)
at 
org.apache.flink.runtime.util.NonClosingStreamDecorator.read(NonClosingStreamDecorator.java:47)
at java.io.DataInputStream.readFully(DataInputStream.java:195)
at java.io.DataInputStream.readLong(DataInputStream.java:416)
at 
org.apache.flink.api.common.typeutils.base.LongSerializer.deserialize(LongSerializer.java:68)
at 
org.apache.flink.streaming.api.operators.InternalTimer$TimerSerializer.deserialize(InternalTimer.java:156)
at 
org.apache.flink.streaming.api.operators.HeapInternalTimerService.restoreTimersForKeyGroup(HeapInternalTimerService.java:345)
at 
org.apache.flink.streaming.api.operators.InternalTimeServiceManager.restoreStateForKeyGroup(InternalTimeServiceManager.java:141)
at 
org.apache.flink.streaming.api.operators.AbstractStreamOperator.initializeState(AbstractStreamOperator.java:496)
at 
org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.initializeState(AbstractUdfStreamOperator.java:104)
at 
org.apache.flink.streaming.api.operators.AbstractStreamOperator.initializeState(AbstractStreamOperator.java:251)
at 
org.apache.flink.streaming.runtime.tasks.StreamTask.initializeOperators(StreamTask.java:676)
at 
org.apache.flink.streaming.runtime.tasks.StreamTask.initializeState(StreamTask.java:663)
at 
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:252)
at org.apache.flink.runtime.taskmanager.Task.run(Task.java:702)
at java.lang.Thread.run(Thread.java:748)
{code}




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


[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137096261
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+ 

[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137110049
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+ 

[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137111384
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+ 

[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137096753
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+ 

[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137123724
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala
 ---
@@ -232,4 +232,164 @@ class JoinHarnessTest extends HarnessTestBase{
 testHarness.close()
   }
 
+  /** a.proctime >= b.proctime - 10 and a.proctime <= b.proctime + 20 **/
+  @Test
+  def testNewNormalProcTimeJoin() {
--- End diff --

We would also need harness tests for the event-time join.
In addition we need:
- plan translation tests (see 
`org.apache.flink.table.api.stream.sql.JoinTest`)
- some end-to-end ITCases (see 
`org.apache.flink.table.runtime.stream.sql.JoinITCase` but with result 
validation)

Moreover, the processing time restriction for windowed joins should be 
removed from the documentation.


> Support rowtime inner equi-join between two streams in the SQL API
> --
>
> Key: FLINK-6233
> URL: https://issues.apache.org/jira/browse/FLINK-6233
> Project: Flink
>  Issue Type: Sub-task
>  Components: Table API & SQL
>Reporter: hongyuhong
>Assignee: Xingcan Cui
>
> The goal of this issue is to add support for inner equi-join on proc time 
> streams to the SQL interface.
> Queries similar to the following should be supported:
> {code}
> SELECT o.rowtime , o.productId, o.orderId, s.rowtime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL '1' HOUR;
> {code}
> The following restrictions should initially apply:
> * The join hint only support inner join
> * The ON clause should include equi-join condition
> * The time-condition {{o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL 
> '1' HOUR}} only can use rowtime that is a system attribute, the time 
> condition only support bounded time range like {{o.rowtime BETWEEN s.rowtime 
> - INTERVAL '1' HOUR AND s.rowtime + INTERVAL '1' HOUR}}, not support 
> unbounded like {{o.rowtime  s.rowtime}} ,  and  should include both two 
> stream's rowtime attribute, {{o.rowtime between rowtime () and rowtime () + 
> 1}} should also not be supported.
> An row-time streams join will not be able to handle late data, because this 
> would mean in insert a row into a sorted order shift all other computations. 
> This would be too expensive to maintain. Therefore, we will throw an error if 
> a user tries to use an row-time stream join with late data handling.
> This issue includes:
> * Design of the DataStream operator to deal with stream join
> * Translation from Calcite's RelNode representation (LogicalJoin). 



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


[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137012982
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala
 ---
@@ -184,4 +195,54 @@ class DataStreamWindowJoin(
 .returns(returnTypeInfo)
 }
   }
+
+  def createRowTimeInnerJoinFunction(
+leftDataStream: DataStream[CRow],
+rightDataStream: DataStream[CRow],
+joinFunctionName: String,
+joinFunctionCode: String,
+leftKeys: Array[Int],
+rightKeys: Array[Int]): DataStream[CRow] = {
+
+val returnTypeInfo = CRowTypeInfo(schema.typeInfo)
+
+val rowTimeInnerJoinFunc = new TimeBoundedStreamInnerJoin(
+  leftLowerBound,
+  leftUpperBound,
+  0L,
+  leftSchema.typeInfo,
+  rightSchema.typeInfo,
+  joinFunctionName,
+  joinFunctionCode,
+  leftTimeIdx,
+  rightTimeIdx,
+  JoinTimeIndicator.ROWTIME
--- End diff --

Rather have two subclasses than dealing with everything in a single class?


> Support rowtime inner equi-join between two streams in the SQL API
> --
>
> Key: FLINK-6233
> URL: https://issues.apache.org/jira/browse/FLINK-6233
> Project: Flink
>  Issue Type: Sub-task
>  Components: Table API & SQL
>Reporter: hongyuhong
>Assignee: Xingcan Cui
>
> The goal of this issue is to add support for inner equi-join on proc time 
> streams to the SQL interface.
> Queries similar to the following should be supported:
> {code}
> SELECT o.rowtime , o.productId, o.orderId, s.rowtime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL '1' HOUR;
> {code}
> The following restrictions should initially apply:
> * The join hint only support inner join
> * The ON clause should include equi-join condition
> * The time-condition {{o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL 
> '1' HOUR}} only can use rowtime that is a system attribute, the time 
> condition only support bounded time range like {{o.rowtime BETWEEN s.rowtime 
> - INTERVAL '1' HOUR AND s.rowtime + INTERVAL '1' HOUR}}, not support 
> unbounded like {{o.rowtime  s.rowtime}} ,  and  should include both two 
> stream's rowtime attribute, {{o.rowtime between rowtime () and rowtime () + 
> 1}} should also not be supported.
> An row-time streams join will not be able to handle late data, because this 
> would mean in insert a row into a sorted order shift all other computations. 
> This would be too expensive to maintain. Therefore, we will throw an error if 
> a user tries to use an row-time stream join with late data handling.
> This issue includes:
> * Design of the DataStream operator to deal with stream join
> * Translation from Calcite's RelNode representation (LogicalJoin). 



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


[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137117454
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+ 

[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137101377
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+ 

[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137118783
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+ 

[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137099832
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+ 

[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137043440
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+ 

[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137093911
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+ 

[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137032484
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala
 ---
@@ -225,7 +233,7 @@ object WindowJoinUtil {
   case t: TimeIndicatorRelDataType =>
 // time attribute access. Remember time type and side of input
 val isLeftInput = idx < leftFieldCount
-Seq(TimeAttributeAccess(t.isEventTime, isLeftInput))
+Seq(TimeAttributeAccess(t.isEventTime, isLeftInput, idx))
--- End diff --

I would do the subtraction here:

```
if (idx < leftFieldCount) {
  Seq(TimeAttributeAccess(t.isEventTime, true, idx))
} else {
  Seq(TimeAttributeAccess(t.isEventTime, false, idx - leftFieldCount))
}
```



> Support rowtime inner equi-join between two streams in the SQL API
> --
>
> Key: FLINK-6233
> URL: https://issues.apache.org/jira/browse/FLINK-6233
> Project: Flink
>  Issue Type: Sub-task
>  Components: Table API & SQL
>Reporter: hongyuhong
>Assignee: Xingcan Cui
>
> The goal of this issue is to add support for inner equi-join on proc time 
> streams to the SQL interface.
> Queries similar to the following should be supported:
> {code}
> SELECT o.rowtime , o.productId, o.orderId, s.rowtime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL '1' HOUR;
> {code}
> The following restrictions should initially apply:
> * The join hint only support inner join
> * The ON clause should include equi-join condition
> * The time-condition {{o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL 
> '1' HOUR}} only can use rowtime that is a system attribute, the time 
> condition only support bounded time range like {{o.rowtime BETWEEN s.rowtime 
> - INTERVAL '1' HOUR AND s.rowtime + INTERVAL '1' HOUR}}, not support 
> unbounded like {{o.rowtime  s.rowtime}} ,  and  should include both two 
> stream's rowtime attribute, {{o.rowtime between rowtime () and rowtime () + 
> 1}} should also not be supported.
> An row-time streams join will not be able to handle late data, because this 
> would mean in insert a row into a sorted order shift all other computations. 
> This would be too expensive to maintain. Therefore, we will throw an error if 
> a user tries to use an row-time stream join with late data handling.
> This issue includes:
> * Design of the DataStream operator to deal with stream join
> * Translation from Calcite's RelNode representation (LogicalJoin). 



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


[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137119180
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+ 

[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137020297
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala
 ---
@@ -39,14 +39,17 @@ import scala.collection.JavaConverters._
   */
 object WindowJoinUtil {
 
-  case class WindowBounds(isEventTime: Boolean, leftLowerBound: Long, 
leftUpperBound: Long)
+  case class WindowBounds(isEventTime: Boolean, leftLowerBound: Long, 
leftUpperBound: Long,
--- End diff --

change formatting to
```
case class WindowBounds(
  isEventTime: Boolean,
  leftLowerBound: Long,
  ...,
  rightTimeIdx: Int)
```
similar to `TimePredicate`


> Support rowtime inner equi-join between two streams in the SQL API
> --
>
> Key: FLINK-6233
> URL: https://issues.apache.org/jira/browse/FLINK-6233
> Project: Flink
>  Issue Type: Sub-task
>  Components: Table API & SQL
>Reporter: hongyuhong
>Assignee: Xingcan Cui
>
> The goal of this issue is to add support for inner equi-join on proc time 
> streams to the SQL interface.
> Queries similar to the following should be supported:
> {code}
> SELECT o.rowtime , o.productId, o.orderId, s.rowtime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL '1' HOUR;
> {code}
> The following restrictions should initially apply:
> * The join hint only support inner join
> * The ON clause should include equi-join condition
> * The time-condition {{o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL 
> '1' HOUR}} only can use rowtime that is a system attribute, the time 
> condition only support bounded time range like {{o.rowtime BETWEEN s.rowtime 
> - INTERVAL '1' HOUR AND s.rowtime + INTERVAL '1' HOUR}}, not support 
> unbounded like {{o.rowtime  s.rowtime}} ,  and  should include both two 
> stream's rowtime attribute, {{o.rowtime between rowtime () and rowtime () + 
> 1}} should also not be supported.
> An row-time streams join will not be able to handle late data, because this 
> would mean in insert a row into a sorted order shift all other computations. 
> This would be too expensive to maintain. Therefore, we will throw an error if 
> a user tries to use an row-time stream join with late data handling.
> This issue includes:
> * Design of the DataStream operator to deal with stream join
> * Translation from Calcite's RelNode representation (LogicalJoin). 



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


[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137014165
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala
 ---
@@ -184,4 +195,54 @@ class DataStreamWindowJoin(
 .returns(returnTypeInfo)
 }
   }
+
+  def createRowTimeInnerJoinFunction(
+leftDataStream: DataStream[CRow],
+rightDataStream: DataStream[CRow],
+joinFunctionName: String,
+joinFunctionCode: String,
+leftKeys: Array[Int],
+rightKeys: Array[Int]): DataStream[CRow] = {
+
+val returnTypeInfo = CRowTypeInfo(schema.typeInfo)
+
+val rowTimeInnerJoinFunc = new TimeBoundedStreamInnerJoin(
+  leftLowerBound,
+  leftUpperBound,
+  0L,
+  leftSchema.typeInfo,
+  rightSchema.typeInfo,
+  joinFunctionName,
+  joinFunctionCode,
+  leftTimeIdx,
+  rightTimeIdx,
+  JoinTimeIndicator.ROWTIME
+)
+
+if (!leftKeys.isEmpty) {
+  leftDataStream
+.connect(rightDataStream)
+.keyBy(leftKeys, rightKeys)
+.transform(
+  "rowTimeInnerJoinFunc",
+  returnTypeInfo,
+  new KeyedCoProcessOperatorWithWatermarkDelay[CRow, CRow, CRow, 
CRow](
+rowTimeInnerJoinFunc,
+rowTimeInnerJoinFunc.getMaxOutputDelay)
--- End diff --

rename method to `getMaxWatermarkDelay`?


> Support rowtime inner equi-join between two streams in the SQL API
> --
>
> Key: FLINK-6233
> URL: https://issues.apache.org/jira/browse/FLINK-6233
> Project: Flink
>  Issue Type: Sub-task
>  Components: Table API & SQL
>Reporter: hongyuhong
>Assignee: Xingcan Cui
>
> The goal of this issue is to add support for inner equi-join on proc time 
> streams to the SQL interface.
> Queries similar to the following should be supported:
> {code}
> SELECT o.rowtime , o.productId, o.orderId, s.rowtime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL '1' HOUR;
> {code}
> The following restrictions should initially apply:
> * The join hint only support inner join
> * The ON clause should include equi-join condition
> * The time-condition {{o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL 
> '1' HOUR}} only can use rowtime that is a system attribute, the time 
> condition only support bounded time range like {{o.rowtime BETWEEN s.rowtime 
> - INTERVAL '1' HOUR AND s.rowtime + INTERVAL '1' HOUR}}, not support 
> unbounded like {{o.rowtime  s.rowtime}} ,  and  should include both two 
> stream's rowtime attribute, {{o.rowtime between rowtime () and rowtime () + 
> 1}} should also not be supported.
> An row-time streams join will not be able to handle late data, because this 
> would mean in insert a row into a sorted order shift all other computations. 
> This would be too expensive to maintain. Therefore, we will throw an error if 
> a user tries to use an row-time stream join with late data handling.
> This issue includes:
> * Design of the DataStream operator to deal with stream join
> * Translation from Calcite's RelNode representation (LogicalJoin). 



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


[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137040050
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+ 

[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137039045
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+ 

[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137120634
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+ 

[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137092951
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+ 

[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137117103
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+ 

[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137110758
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+ 

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

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

https://github.com/apache/flink/pull/4625#discussion_r137110758
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+  if (allowedLateness < 0) {
+throw new IllegalArgumentException("The allowed lateness must be 
non-negative.")
+  }
+
+
+  /**
+* For holding back watermarks.
+*
+* @return the maximum delay 

[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137100457
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+ 

[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137113304
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+ 

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

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

https://github.com/apache/flink/pull/4625#discussion_r137099832
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+  if (allowedLateness < 0) {
+throw new IllegalArgumentException("The allowed lateness must be 
non-negative.")
+  }
+
+
+  /**
+* For holding back watermarks.
+*
+* @return the maximum delay 

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

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

https://github.com/apache/flink/pull/4625#discussion_r137040219
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+  if (allowedLateness < 0) {
+throw new IllegalArgumentException("The allowed lateness must be 
non-negative.")
+  }
+
+
+  /**
+* For holding back watermarks.
+*
+* @return the maximum delay 

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

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

https://github.com/apache/flink/pull/4625#discussion_r137095404
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+  if (allowedLateness < 0) {
+throw new IllegalArgumentException("The allowed lateness must be 
non-negative.")
+  }
+
+
+  /**
+* For holding back watermarks.
+*
+* @return the maximum delay 

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

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

https://github.com/apache/flink/pull/4625#discussion_r137098896
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+  if (allowedLateness < 0) {
+throw new IllegalArgumentException("The allowed lateness must be 
non-negative.")
+  }
+
+
+  /**
+* For holding back watermarks.
+*
+* @return the maximum delay 

[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137100691
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+ 

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

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

https://github.com/apache/flink/pull/4625#discussion_r137096261
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+  if (allowedLateness < 0) {
+throw new IllegalArgumentException("The allowed lateness must be 
non-negative.")
+  }
+
+
+  /**
+* For holding back watermarks.
+*
+* @return the maximum delay 

[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137013584
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala
 ---
@@ -184,4 +195,54 @@ class DataStreamWindowJoin(
 .returns(returnTypeInfo)
 }
   }
+
+  def createRowTimeInnerJoinFunction(
+leftDataStream: DataStream[CRow],
+rightDataStream: DataStream[CRow],
+joinFunctionName: String,
+joinFunctionCode: String,
+leftKeys: Array[Int],
+rightKeys: Array[Int]): DataStream[CRow] = {
+
+val returnTypeInfo = CRowTypeInfo(schema.typeInfo)
+
+val rowTimeInnerJoinFunc = new TimeBoundedStreamInnerJoin(
+  leftLowerBound,
+  leftUpperBound,
+  0L,
+  leftSchema.typeInfo,
+  rightSchema.typeInfo,
+  joinFunctionName,
+  joinFunctionCode,
+  leftTimeIdx,
+  rightTimeIdx,
+  JoinTimeIndicator.ROWTIME
+)
+
+if (!leftKeys.isEmpty) {
+  leftDataStream
+.connect(rightDataStream)
+.keyBy(leftKeys, rightKeys)
+.transform(
+  "rowTimeInnerJoinFunc",
--- End diff --

Change to `"InnerRowtimeWindowJoin",`?


> Support rowtime inner equi-join between two streams in the SQL API
> --
>
> Key: FLINK-6233
> URL: https://issues.apache.org/jira/browse/FLINK-6233
> Project: Flink
>  Issue Type: Sub-task
>  Components: Table API & SQL
>Reporter: hongyuhong
>Assignee: Xingcan Cui
>
> The goal of this issue is to add support for inner equi-join on proc time 
> streams to the SQL interface.
> Queries similar to the following should be supported:
> {code}
> SELECT o.rowtime , o.productId, o.orderId, s.rowtime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL '1' HOUR;
> {code}
> The following restrictions should initially apply:
> * The join hint only support inner join
> * The ON clause should include equi-join condition
> * The time-condition {{o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL 
> '1' HOUR}} only can use rowtime that is a system attribute, the time 
> condition only support bounded time range like {{o.rowtime BETWEEN s.rowtime 
> - INTERVAL '1' HOUR AND s.rowtime + INTERVAL '1' HOUR}}, not support 
> unbounded like {{o.rowtime  s.rowtime}} ,  and  should include both two 
> stream's rowtime attribute, {{o.rowtime between rowtime () and rowtime () + 
> 1}} should also not be supported.
> An row-time streams join will not be able to handle late data, because this 
> would mean in insert a row into a sorted order shift all other computations. 
> This would be too expensive to maintain. Therefore, we will throw an error if 
> a user tries to use an row-time stream join with late data handling.
> This issue includes:
> * Design of the DataStream operator to deal with stream join
> * Translation from Calcite's RelNode representation (LogicalJoin). 



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


[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137112342
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+ 

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

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

https://github.com/apache/flink/pull/4625#discussion_r137112342
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+  if (allowedLateness < 0) {
+throw new IllegalArgumentException("The allowed lateness must be 
non-negative.")
+  }
+
+
+  /**
+* For holding back watermarks.
+*
+* @return the maximum delay 

[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137040696
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+ 

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

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

https://github.com/apache/flink/pull/4625#discussion_r137039075
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+  if (allowedLateness < 0) {
+throw new IllegalArgumentException("The allowed lateness must be 
non-negative.")
+  }
+
+
+  /**
+* For holding back watermarks.
+*
+* @return the maximum delay 

[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137040311
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+ 

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

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

https://github.com/apache/flink/pull/4625#discussion_r137040050
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+  if (allowedLateness < 0) {
+throw new IllegalArgumentException("The allowed lateness must be 
non-negative.")
+  }
+
+
+  /**
+* For holding back watermarks.
+*
+* @return the maximum delay 

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

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

https://github.com/apache/flink/pull/4625#discussion_r137038607
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
--- End diff --

What is the `backPressureSuggestion` for?


---


[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

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

https://github.com/apache/flink/pull/4625#discussion_r137117454
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+  if (allowedLateness < 0) {
+throw new IllegalArgumentException("The allowed lateness must be 
non-negative.")
+  }
+
+
+  /**
+* For holding back watermarks.
+*
+* @return the maximum delay 

[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137095404
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+ 

[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137038607
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
--- End diff --

What is the `backPressureSuggestion` for?


> Support rowtime inner equi-join between two streams in the SQL API
> 

[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137040783
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+ 

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

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

https://github.com/apache/flink/pull/4625#discussion_r137031980
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala
 ---
@@ -115,10 +118,15 @@ object WindowJoinUtil {
   case _ =>
 Some(otherPreds.reduceLeft((l, r) => 
RelOptUtil.andJoinFilters(rexBuilder, l, r)))
 }
-
-val bounds = Some(WindowBounds(timePreds.head.isEventTime, 
leftLowerBound, leftUpperBound))
-
-(bounds, remainCondition)
+if (timePreds.head.leftInputOnLeftSide) {
--- End diff --

Please replace the condition as follows:
```
  val bounds = if (timePreds.head.leftInputOnLeftSide) {
Some(WindowBounds(
  timePreds.head.isEventTime,
  leftLowerBound,
  leftUpperBound,
  timePreds.head.leftTimeIdx,
  timePreds.head.rightTimeIdx))
  } else {
Some(WindowBounds(
  timePreds.head.isEventTime,
  leftLowerBound,
  leftUpperBound,
  timePreds.head.rightTimeIdx,
  timePreds.head.leftTimeIdx))
  }

  (bounds, remainCondition)
```

We should not get the right index from the second time predicate because 
the tables might be accessed in inverse order there (we would need to check 
`leftInputOnLeftSide` again).


---


[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

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

https://github.com/apache/flink/pull/4625#discussion_r137035608
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
--- End diff --

this might be understood as both `BETWEEN` predicates could be evaluated at 
the same time which is not the case.


---


[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137012563
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala
 ---
@@ -184,4 +195,54 @@ class DataStreamWindowJoin(
 .returns(returnTypeInfo)
 }
   }
+
+  def createRowTimeInnerJoinFunction(
+leftDataStream: DataStream[CRow],
+rightDataStream: DataStream[CRow],
+joinFunctionName: String,
+joinFunctionCode: String,
+leftKeys: Array[Int],
+rightKeys: Array[Int]): DataStream[CRow] = {
+
+val returnTypeInfo = CRowTypeInfo(schema.typeInfo)
+
+val rowTimeInnerJoinFunc = new TimeBoundedStreamInnerJoin(
+  leftLowerBound,
+  leftUpperBound,
+  0L,
--- End diff --

change to `allowedLateness = 0L,`


> Support rowtime inner equi-join between two streams in the SQL API
> --
>
> Key: FLINK-6233
> URL: https://issues.apache.org/jira/browse/FLINK-6233
> Project: Flink
>  Issue Type: Sub-task
>  Components: Table API & SQL
>Reporter: hongyuhong
>Assignee: Xingcan Cui
>
> The goal of this issue is to add support for inner equi-join on proc time 
> streams to the SQL interface.
> Queries similar to the following should be supported:
> {code}
> SELECT o.rowtime , o.productId, o.orderId, s.rowtime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL '1' HOUR;
> {code}
> The following restrictions should initially apply:
> * The join hint only support inner join
> * The ON clause should include equi-join condition
> * The time-condition {{o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL 
> '1' HOUR}} only can use rowtime that is a system attribute, the time 
> condition only support bounded time range like {{o.rowtime BETWEEN s.rowtime 
> - INTERVAL '1' HOUR AND s.rowtime + INTERVAL '1' HOUR}}, not support 
> unbounded like {{o.rowtime  s.rowtime}} ,  and  should include both two 
> stream's rowtime attribute, {{o.rowtime between rowtime () and rowtime () + 
> 1}} should also not be supported.
> An row-time streams join will not be able to handle late data, because this 
> would mean in insert a row into a sorted order shift all other computations. 
> This would be too expensive to maintain. Therefore, we will throw an error if 
> a user tries to use an row-time stream join with late data handling.
> This issue includes:
> * Design of the DataStream operator to deal with stream join
> * Translation from Calcite's RelNode representation (LogicalJoin). 



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


[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137040219
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+ 

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

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

https://github.com/apache/flink/pull/4625#discussion_r137013584
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala
 ---
@@ -184,4 +195,54 @@ class DataStreamWindowJoin(
 .returns(returnTypeInfo)
 }
   }
+
+  def createRowTimeInnerJoinFunction(
+leftDataStream: DataStream[CRow],
+rightDataStream: DataStream[CRow],
+joinFunctionName: String,
+joinFunctionCode: String,
+leftKeys: Array[Int],
+rightKeys: Array[Int]): DataStream[CRow] = {
+
+val returnTypeInfo = CRowTypeInfo(schema.typeInfo)
+
+val rowTimeInnerJoinFunc = new TimeBoundedStreamInnerJoin(
+  leftLowerBound,
+  leftUpperBound,
+  0L,
+  leftSchema.typeInfo,
+  rightSchema.typeInfo,
+  joinFunctionName,
+  joinFunctionCode,
+  leftTimeIdx,
+  rightTimeIdx,
+  JoinTimeIndicator.ROWTIME
+)
+
+if (!leftKeys.isEmpty) {
+  leftDataStream
+.connect(rightDataStream)
+.keyBy(leftKeys, rightKeys)
+.transform(
+  "rowTimeInnerJoinFunc",
--- End diff --

Change to `"InnerRowtimeWindowJoin",`?


---


[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137035608
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
--- End diff --

this might be understood as both `BETWEEN` predicates could be evaluated at 
the same time which is not the case.


> Support rowtime inner equi-join between two streams in the SQL API
> --
>
> Key: FLINK-6233
> URL: https://issues.apache.org/jira/browse/FLINK-6233
> Project: Flink
>  Issue Type: Sub-task
>  Components: Table API & SQL
>Reporter: hongyuhong
>Assignee: Xingcan Cui
>
> The goal of this issue is to add support for inner equi-join on proc time 
> streams to the SQL interface.
> Queries similar to the following should be supported:
> {code}
> SELECT o.rowtime , o.productId, o.orderId, s.rowtime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL '1' HOUR;
> {code}
> The following restrictions should initially apply:
> * The join hint only support inner join
> * The ON clause should include equi-join condition
> * The time-condition {{o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL 
> '1' HOUR}} only can use rowtime that is a system attribute, the time 
> condition only support bounded time range like {{o.rowtime BETWEEN s.rowtime 
> - INTERVAL '1' HOUR AND s.rowtime + INTERVAL '1' HOUR}}, not support 
> unbounded like {{o.rowtime  s.rowtime}} ,  and  should include both two 
> stream's rowtime attribute, {{o.rowtime between rowtime () and rowtime () + 
> 1}} should also not be supported.
> An row-time streams join will not be able to handle late data, because this 
> would mean in insert a row into a sorted order shift all other computations. 
> This would be too expensive to maintain. Therefore, we will throw an error if 
> a user tries to use an row-time stream join with late data handling.
> This issue includes:
> * Design of the DataStream operator to deal with stream join
> * Translation from Calcite's RelNode representation (LogicalJoin). 



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


[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

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

https://github.com/apache/flink/pull/4625#discussion_r137038433
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+  if (allowedLateness < 0) {
+throw new IllegalArgumentException("The allowed lateness must be 
non-negative.")
+  }
+
+
+  /**
+* For holding back watermarks.
+*
+* @return the maximum delay 

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

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

https://github.com/apache/flink/pull/4625#discussion_r137041142
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+  if (allowedLateness < 0) {
+throw new IllegalArgumentException("The allowed lateness must be 
non-negative.")
+  }
+
+
+  /**
+* For holding back watermarks.
+*
+* @return the maximum delay 

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

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

https://github.com/apache/flink/pull/4625#discussion_r137037638
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
--- End diff --

please describe the parameters


---


[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137014641
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala
 ---
@@ -184,4 +195,54 @@ class DataStreamWindowJoin(
 .returns(returnTypeInfo)
 }
   }
+
+  def createRowTimeInnerJoinFunction(
+leftDataStream: DataStream[CRow],
+rightDataStream: DataStream[CRow],
+joinFunctionName: String,
+joinFunctionCode: String,
+leftKeys: Array[Int],
+rightKeys: Array[Int]): DataStream[CRow] = {
+
+val returnTypeInfo = CRowTypeInfo(schema.typeInfo)
+
+val rowTimeInnerJoinFunc = new TimeBoundedStreamInnerJoin(
+  leftLowerBound,
+  leftUpperBound,
+  0L,
+  leftSchema.typeInfo,
+  rightSchema.typeInfo,
+  joinFunctionName,
+  joinFunctionCode,
+  leftTimeIdx,
+  rightTimeIdx,
+  JoinTimeIndicator.ROWTIME
+)
+
+if (!leftKeys.isEmpty) {
+  leftDataStream
+.connect(rightDataStream)
+.keyBy(leftKeys, rightKeys)
+.transform(
+  "rowTimeInnerJoinFunc",
+  returnTypeInfo,
+  new KeyedCoProcessOperatorWithWatermarkDelay[CRow, CRow, CRow, 
CRow](
+rowTimeInnerJoinFunc,
+rowTimeInnerJoinFunc.getMaxOutputDelay)
+).returns(returnTypeInfo)
--- End diff --

The `returns()` call might be redundant as the `returnTypeInfo` is already 
set in the operator.


> Support rowtime inner equi-join between two streams in the SQL API
> --
>
> Key: FLINK-6233
> URL: https://issues.apache.org/jira/browse/FLINK-6233
> Project: Flink
>  Issue Type: Sub-task
>  Components: Table API & SQL
>Reporter: hongyuhong
>Assignee: Xingcan Cui
>
> The goal of this issue is to add support for inner equi-join on proc time 
> streams to the SQL interface.
> Queries similar to the following should be supported:
> {code}
> SELECT o.rowtime , o.productId, o.orderId, s.rowtime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL '1' HOUR;
> {code}
> The following restrictions should initially apply:
> * The join hint only support inner join
> * The ON clause should include equi-join condition
> * The time-condition {{o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL 
> '1' HOUR}} only can use rowtime that is a system attribute, the time 
> condition only support bounded time range like {{o.rowtime BETWEEN s.rowtime 
> - INTERVAL '1' HOUR AND s.rowtime + INTERVAL '1' HOUR}}, not support 
> unbounded like {{o.rowtime  s.rowtime}} ,  and  should include both two 
> stream's rowtime attribute, {{o.rowtime between rowtime () and rowtime () + 
> 1}} should also not be supported.
> An row-time streams join will not be able to handle late data, because this 
> would mean in insert a row into a sorted order shift all other computations. 
> This would be too expensive to maintain. Therefore, we will throw an error if 
> a user tries to use an row-time stream join with late data handling.
> This issue includes:
> * Design of the DataStream operator to deal with stream join
> * Translation from Calcite's RelNode representation (LogicalJoin). 



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


[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

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

https://github.com/apache/flink/pull/4625#discussion_r137100691
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+  if (allowedLateness < 0) {
+throw new IllegalArgumentException("The allowed lateness must be 
non-negative.")
+  }
+
+
+  /**
+* For holding back watermarks.
+*
+* @return the maximum delay 

[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137043506
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+ 

[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137108027
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+ 

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

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

https://github.com/apache/flink/pull/4625#discussion_r137120634
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+  if (allowedLateness < 0) {
+throw new IllegalArgumentException("The allowed lateness must be 
non-negative.")
+  }
+
+
+  /**
+* For holding back watermarks.
+*
+* @return the maximum delay 

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

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

https://github.com/apache/flink/pull/4625#discussion_r137108027
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+  if (allowedLateness < 0) {
+throw new IllegalArgumentException("The allowed lateness must be 
non-negative.")
+  }
+
+
+  /**
+* For holding back watermarks.
+*
+* @return the maximum delay 

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

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

https://github.com/apache/flink/pull/4625#discussion_r137014641
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala
 ---
@@ -184,4 +195,54 @@ class DataStreamWindowJoin(
 .returns(returnTypeInfo)
 }
   }
+
+  def createRowTimeInnerJoinFunction(
+leftDataStream: DataStream[CRow],
+rightDataStream: DataStream[CRow],
+joinFunctionName: String,
+joinFunctionCode: String,
+leftKeys: Array[Int],
+rightKeys: Array[Int]): DataStream[CRow] = {
+
+val returnTypeInfo = CRowTypeInfo(schema.typeInfo)
+
+val rowTimeInnerJoinFunc = new TimeBoundedStreamInnerJoin(
+  leftLowerBound,
+  leftUpperBound,
+  0L,
+  leftSchema.typeInfo,
+  rightSchema.typeInfo,
+  joinFunctionName,
+  joinFunctionCode,
+  leftTimeIdx,
+  rightTimeIdx,
+  JoinTimeIndicator.ROWTIME
+)
+
+if (!leftKeys.isEmpty) {
+  leftDataStream
+.connect(rightDataStream)
+.keyBy(leftKeys, rightKeys)
+.transform(
+  "rowTimeInnerJoinFunc",
+  returnTypeInfo,
+  new KeyedCoProcessOperatorWithWatermarkDelay[CRow, CRow, CRow, 
CRow](
+rowTimeInnerJoinFunc,
+rowTimeInnerJoinFunc.getMaxOutputDelay)
+).returns(returnTypeInfo)
--- End diff --

The `returns()` call might be redundant as the `returnTypeInfo` is already 
set in the operator.


---


[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

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

https://github.com/apache/flink/pull/4625#discussion_r137113304
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+  if (allowedLateness < 0) {
+throw new IllegalArgumentException("The allowed lateness must be 
non-negative.")
+  }
+
+
+  /**
+* For holding back watermarks.
+*
+* @return the maximum delay 

[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137038433
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+ 

[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137098896
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+ 

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

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

https://github.com/apache/flink/pull/4625#discussion_r137040311
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+  if (allowedLateness < 0) {
+throw new IllegalArgumentException("The allowed lateness must be 
non-negative.")
+  }
+
+
+  /**
+* For holding back watermarks.
+*
+* @return the maximum delay 

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

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

https://github.com/apache/flink/pull/4625#discussion_r137010384
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamWindowJoinRule.scala
 ---
@@ -55,8 +55,10 @@ class DataStreamWindowJoinRule
 
 if (windowBounds.isDefined) {
   if (windowBounds.get.isEventTime) {
-// we cannot handle event-time window joins yet
-false
+val procTimeAttrInOutput = join.getRowType.getFieldList.asScala
+  .exists(f => FlinkTypeFactory.isProctimeIndicatorType(f.getType))
+
+!remainingPredsAccessTime && !procTimeAttrInOutput
--- End diff --

change to `!remainingPredsAccessTime`. It is OK to keep processing time 
attributes in the join output.


---


[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

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

https://github.com/apache/flink/pull/4625#discussion_r137020297
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala
 ---
@@ -39,14 +39,17 @@ import scala.collection.JavaConverters._
   */
 object WindowJoinUtil {
 
-  case class WindowBounds(isEventTime: Boolean, leftLowerBound: Long, 
leftUpperBound: Long)
+  case class WindowBounds(isEventTime: Boolean, leftLowerBound: Long, 
leftUpperBound: Long,
--- End diff --

change formatting to
```
case class WindowBounds(
  isEventTime: Boolean,
  leftLowerBound: Long,
  ...,
  rightTimeIdx: Int)
```
similar to `TimePredicate`


---


[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137102330
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+ 

[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137042184
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+ 

[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137031980
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala
 ---
@@ -115,10 +118,15 @@ object WindowJoinUtil {
   case _ =>
 Some(otherPreds.reduceLeft((l, r) => 
RelOptUtil.andJoinFilters(rexBuilder, l, r)))
 }
-
-val bounds = Some(WindowBounds(timePreds.head.isEventTime, 
leftLowerBound, leftUpperBound))
-
-(bounds, remainCondition)
+if (timePreds.head.leftInputOnLeftSide) {
--- End diff --

Please replace the condition as follows:
```
  val bounds = if (timePreds.head.leftInputOnLeftSide) {
Some(WindowBounds(
  timePreds.head.isEventTime,
  leftLowerBound,
  leftUpperBound,
  timePreds.head.leftTimeIdx,
  timePreds.head.rightTimeIdx))
  } else {
Some(WindowBounds(
  timePreds.head.isEventTime,
  leftLowerBound,
  leftUpperBound,
  timePreds.head.rightTimeIdx,
  timePreds.head.leftTimeIdx))
  }

  (bounds, remainCondition)
```

We should not get the right index from the second time predicate because 
the tables might be accessed in inverse order there (we would need to check 
`leftInputOnLeftSide` again).


> Support rowtime inner equi-join between two streams in the SQL API
> --
>
> Key: FLINK-6233
> URL: https://issues.apache.org/jira/browse/FLINK-6233
> Project: Flink
>  Issue Type: Sub-task
>  Components: Table API & SQL
>Reporter: hongyuhong
>Assignee: Xingcan Cui
>
> The goal of this issue is to add support for inner equi-join on proc time 
> streams to the SQL interface.
> Queries similar to the following should be supported:
> {code}
> SELECT o.rowtime , o.productId, o.orderId, s.rowtime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL '1' HOUR;
> {code}
> The following restrictions should initially apply:
> * The join hint only support inner join
> * The ON clause should include equi-join condition
> * The time-condition {{o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL 
> '1' HOUR}} only can use rowtime that is a system attribute, the time 
> condition only support bounded time range like {{o.rowtime BETWEEN s.rowtime 
> - INTERVAL '1' HOUR AND s.rowtime + INTERVAL '1' HOUR}}, not support 
> unbounded like {{o.rowtime  s.rowtime}} ,  and  should include both two 
> stream's rowtime attribute, {{o.rowtime between rowtime () and rowtime () + 
> 1}} should also not be supported.
> An row-time streams join will not be able to handle late data, because this 
> would mean in insert a row into a sorted order shift all other computations. 
> This would be too expensive to maintain. Therefore, we will throw an error if 
> a user tries to use an row-time stream join with late data handling.
> This issue includes:
> * Design of the DataStream operator to deal with stream join
> * Translation from Calcite's RelNode representation (LogicalJoin). 



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


[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137037638
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
--- End diff --

please describe the parameters


> Support rowtime inner equi-join between two streams in the SQL API
> --
>
> Key: FLINK-6233
> URL: https://issues.apache.org/jira/browse/FLINK-6233
> Project: Flink
>  Issue Type: Sub-task
>  Components: Table API & SQL
>Reporter: hongyuhong
>Assignee: Xingcan Cui
>
> The goal of this issue is to add support for inner equi-join on proc time 
> streams to the SQL interface.
> Queries similar to the following should be supported:
> {code}
> SELECT o.rowtime , o.productId, o.orderId, s.rowtime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL '1' HOUR;
> {code}
> The following restrictions should initially apply:
> * The join hint only support inner join
> * The ON clause should include equi-join condition
> * The time-condition {{o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL 
> '1' HOUR}} only can use rowtime that is a system attribute, the time 
> condition only support bounded time range like {{o.rowtime BETWEEN s.rowtime 
> - INTERVAL '1' HOUR AND s.rowtime + INTERVAL '1' HOUR}}, not support 
> unbounded like {{o.rowtime  s.rowtime}} ,  and  should include both two 
> stream's rowtime attribute, {{o.rowtime between rowtime () and rowtime () + 
> 1}} should also not be supported.
> An row-time streams join will not be able to handle late data, because this 
> would mean in insert a row into a sorted order shift all other computations. 
> This would be too expensive to maintain. Therefore, we will throw an error if 
> a user tries to use an row-time stream join with late data handling.
> This issue includes:
> * Design of the DataStream operator to deal with stream join
> * Translation from Calcite's RelNode representation (LogicalJoin). 



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


[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137041142
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+ 

[GitHub] flink pull request #4625: [FLINK-6233] [table] Support time-bounded stream i...

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

https://github.com/apache/flink/pull/4625#discussion_r137092951
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+  if (allowedLateness < 0) {
+throw new IllegalArgumentException("The allowed lateness must be 
non-negative.")
+  }
+
+
+  /**
+* For holding back watermarks.
+*
+* @return the maximum delay 

[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137010384
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamWindowJoinRule.scala
 ---
@@ -55,8 +55,10 @@ class DataStreamWindowJoinRule
 
 if (windowBounds.isDefined) {
   if (windowBounds.get.isEventTime) {
-// we cannot handle event-time window joins yet
-false
+val procTimeAttrInOutput = join.getRowType.getFieldList.asScala
+  .exists(f => FlinkTypeFactory.isProctimeIndicatorType(f.getType))
+
+!remainingPredsAccessTime && !procTimeAttrInOutput
--- End diff --

change to `!remainingPredsAccessTime`. It is OK to keep processing time 
attributes in the join output.


> Support rowtime inner equi-join between two streams in the SQL API
> --
>
> Key: FLINK-6233
> URL: https://issues.apache.org/jira/browse/FLINK-6233
> Project: Flink
>  Issue Type: Sub-task
>  Components: Table API & SQL
>Reporter: hongyuhong
>Assignee: Xingcan Cui
>
> The goal of this issue is to add support for inner equi-join on proc time 
> streams to the SQL interface.
> Queries similar to the following should be supported:
> {code}
> SELECT o.rowtime , o.productId, o.orderId, s.rowtime AS shipTime 
> FROM Orders AS o 
> JOIN Shipments AS s 
> ON o.orderId = s.orderId 
> AND o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL '1' HOUR;
> {code}
> The following restrictions should initially apply:
> * The join hint only support inner join
> * The ON clause should include equi-join condition
> * The time-condition {{o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL 
> '1' HOUR}} only can use rowtime that is a system attribute, the time 
> condition only support bounded time range like {{o.rowtime BETWEEN s.rowtime 
> - INTERVAL '1' HOUR AND s.rowtime + INTERVAL '1' HOUR}}, not support 
> unbounded like {{o.rowtime  s.rowtime}} ,  and  should include both two 
> stream's rowtime attribute, {{o.rowtime between rowtime () and rowtime () + 
> 1}} should also not be supported.
> An row-time streams join will not be able to handle late data, because this 
> would mean in insert a row into a sorted order shift all other computations. 
> This would be too expensive to maintain. Therefore, we will throw an error if 
> a user tries to use an row-time stream join with late data handling.
> This issue includes:
> * Design of the DataStream operator to deal with stream join
> * Translation from Calcite's RelNode representation (LogicalJoin). 



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


[jira] [Commented] (FLINK-6233) Support rowtime inner equi-join between two streams in the SQL API

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

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

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

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

https://github.com/apache/flink/pull/4625#discussion_r137039075
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala
 ---
@@ -0,0 +1,533 @@
+/*
+ * 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.text.SimpleDateFormat
+import java.util
+import java.util.Map.Entry
+import java.util.{Date, 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.join.JoinTimeIndicator.JoinTimeIndicator
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+
+/**
+  * A CoProcessFunction to execute time-bounded stream inner-join.
+  *
+  * Sample criteria:
+  *
+  * L.time between R.time + X and R.time + Y
+  * or AND R.time between L.time - Y and L.time - X
+  *
+  * @param leftLowerBound  X
+  * @param leftUpperBound  Y
+  * @param allowedLateness the lateness allowed for the two streams
+  * @param leftTypethe input type of left stream
+  * @param rightType   the input type of right stream
+  * @param genJoinFuncName the function code of other non-equi conditions
+  * @param genJoinFuncCode the function name of other non-equi conditions
+  * @param timeIndicator   indicate whether joining on proctime or rowtime
+  *
+  */
+class TimeBoundedStreamInnerJoin(
+  private val leftLowerBound: Long,
+  private val leftUpperBound: Long,
+  private val allowedLateness: Long,
+  private val leftType: TypeInformation[Row],
+  private val rightType: TypeInformation[Row],
+  private val genJoinFuncName: String,
+  private val genJoinFuncCode: String,
+  private val leftTimeIdx: Int,
+  private val rightTimeIdx: Int,
+  private val timeIndicator: JoinTimeIndicator)
+  extends CoProcessFunction[CRow, CRow, CRow]
+with Compiler[FlatJoinFunction[Row, Row, Row]]
+with Logging {
+
+  private var cRowWrapper: CRowWrappingCollector = _
+
+  // the join function for other conditions
+  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
+
+  // cache to store the left stream records
+  private var leftCache: MapState[Long, JList[Row]] = _
+  // cache to store right stream records
+  private var rightCache: MapState[Long, JList[Row]] = _
+
+  // state to record the timer on the left stream. 0 means no timer set
+  private var leftTimerState: ValueState[Long] = _
+  // state to record the timer on the right stream. 0 means no timer set
+  private var rightTimerState: ValueState[Long] = _
+
+  private val leftRelativeSize: Long = -leftLowerBound
+  private val rightRelativeSize: Long = leftUpperBound
+
+  private val relativeWindowSize = rightRelativeSize + leftRelativeSize
+
+  private var leftOperatorTime: Long = 0L
+  private var rightOperatorTime: Long = 0L
+
+  private var backPressureSuggestion: Long = 0L
+
+  if (relativeWindowSize <= 0) {
+LOG.warn("The relative window size is non-positive, please check the 
join conditions.")
+  }
+
+ 

  1   2   3   >