[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116524126
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala
 ---
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.types.Row
+import org.apache.calcite.rel.`type`._
+import org.apache.calcite.rel.RelCollation
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.table.functions.AggregateFunction
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.flink.table.api.TableException
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.calcite.sql.`type`.SqlTypeName._
+import java.util.{ List => JList, ArrayList }
+import org.apache.flink.api.common.typeinfo.{ SqlTimeTypeInfo, 
TypeInformation }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import java.sql.Timestamp
+import org.apache.calcite.rel.RelFieldCollation
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import java.util.Comparator
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import 
java.lang.{Byte=>JByte,Integer=>JInt,Long=>JLong,Double=>JDouble,Short=>JShort,String=>JString,Float=>JFloat}
+import java.math.{BigDecimal=>JBigDecimal}
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.api.common.operators.Order
+import org.apache.calcite.rex.{RexLiteral, RexNode}
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.api.common.typeinfo.AtomicType
+import org.apache.flink.api.java.typeutils.runtime.RowComparator
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+import scala.collection.JavaConverters._
+
+/**
+ * Class represents a collection of helper methods to build the sort logic.
+ * It encapsulates as well the implementation for ordering and generic 
interfaces
+ */
+
+object SortUtil {
+
+  
+  /**
+   * Function creates 
[org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
+   * elements based on rowtime and potentially other fields
+   * @param collationSort The Sort collation list
+   * @param inputType input row type
+   * @param execCfg table environment execution configuration
+   * @return org.apache.flink.streaming.api.functions.ProcessFunction
+   */
+  private[flink] def createRowTimeSortFunction(
+collationSort: RelCollation,
+inputType: RelDataType,
+inputTypeInfo: TypeInformation[Row],
+execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
+
+val keySortFields = getSortFieldIndexList(collationSort)
+val keySortDirections = getSortFieldDirectionList(collationSort)
+
+   //drop time from comparison as we sort on time in the states and 
result emission
+val keyIndexesNoTime = keySortFields.slice(1, keySortFields.size)
+val keyDirectionsNoTime = keySortDirections.slice(1, 
keySortDirections.size)
+val booleanOrderings = getSortFieldDirectionBooleanList(collationSort)
+val booleanDirectionsNoTime = booleanOrderings.slice(1, 
booleanOrderings.size)
+
+val fieldComps = createFieldComparators(inputType, 
+keyIndexesNoTime, keyDirectionsNoTime, execCfg)
+val fieldCompsRefs = 
fieldComps.asInstanceOf[Array[TypeComparator[AnyRef]]]
+
+val rowComp = createRowComparator(inputType,
+   

[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116519646
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala
 ---
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.types.Row
+import org.apache.calcite.rel.`type`._
+import org.apache.calcite.rel.RelCollation
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.table.functions.AggregateFunction
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.flink.table.api.TableException
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.calcite.sql.`type`.SqlTypeName._
+import java.util.{ List => JList, ArrayList }
+import org.apache.flink.api.common.typeinfo.{ SqlTimeTypeInfo, 
TypeInformation }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import java.sql.Timestamp
+import org.apache.calcite.rel.RelFieldCollation
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import java.util.Comparator
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import 
java.lang.{Byte=>JByte,Integer=>JInt,Long=>JLong,Double=>JDouble,Short=>JShort,String=>JString,Float=>JFloat}
+import java.math.{BigDecimal=>JBigDecimal}
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.api.common.operators.Order
+import org.apache.calcite.rex.{RexLiteral, RexNode}
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.api.common.typeinfo.AtomicType
+import org.apache.flink.api.java.typeutils.runtime.RowComparator
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+import scala.collection.JavaConverters._
+
+/**
+ * Class represents a collection of helper methods to build the sort logic.
+ * It encapsulates as well the implementation for ordering and generic 
interfaces
+ */
+
+object SortUtil {
+
+  
+  /**
+   * Function creates 
[org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
+   * elements based on rowtime and potentially other fields
+   * @param collationSort The Sort collation list
+   * @param inputType input row type
+   * @param execCfg table environment execution configuration
+   * @return org.apache.flink.streaming.api.functions.ProcessFunction
+   */
+  private[flink] def createRowTimeSortFunction(
+collationSort: RelCollation,
+inputType: RelDataType,
+inputTypeInfo: TypeInformation[Row],
+execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
+
+val keySortFields = getSortFieldIndexList(collationSort)
+val keySortDirections = getSortFieldDirectionList(collationSort)
+
+   //drop time from comparison as we sort on time in the states and 
result emission
--- End diff --

align comment


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


[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116480310
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeSortProcessFunction.scala
 ---
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.api.common.state.{ ListState, ListStateDescriptor }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.runtime.state.{ FunctionInitializationContext, 
FunctionSnapshotContext }
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.types.Row
+import org.apache.flink.util.{ Collector, Preconditions }
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import scala.util.control.Breaks._
+import org.apache.flink.api.java.tuple.{ Tuple2 => JTuple2 }
+import org.apache.flink.api.common.state.MapState
+import org.apache.flink.api.common.state.MapStateDescriptor
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import java.util.Comparator
+import java.util.ArrayList
+import java.util.Collections
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+/**
+ * Process Function used for the aggregate in bounded proctime sort 
without offset/fetch
+ * [[org.apache.flink.streaming.api.datastream.DataStream]]
+ *
+ * @param fieldCount Is used to indicate fields in the current element to 
forward
+ * @param inputType It is used to mark the type of the incoming data
+ * @param rowComparator the [[java.util.Comparator]] is used for this sort 
aggregation
+ */
+class ProcTimeSortProcessFunction(
+  private val fieldCount: Int,
+  private val inputRowType: CRowTypeInfo,
+  private val rowComparator: CollectionRowComparator)
+extends ProcessFunction[CRow, CRow] {
+
+  Preconditions.checkNotNull(rowComparator)
+
+  private var stateEventsBuffer: ListState[Row] = _
+  private val sortArray: ArrayList[Row] = new ArrayList[Row]
+  
+  private var outputC: CRow = _
+  
+  override def open(config: Configuration) {
+val sortDescriptor = new ListStateDescriptor[Row]("sortState",
+inputRowType.asInstanceOf[CRowTypeInfo].rowType)
+stateEventsBuffer = getRuntimeContext.getListState(sortDescriptor)
+
+  }
+
+  override def processElement(
+inputC: CRow,
+ctx: ProcessFunction[CRow, CRow]#Context,
+out: Collector[CRow]): Unit = {
+
+val input = inputC.row
+
+if( outputC == null) {
+  outputC = new CRow(input, true)
+}
+
+val currentTime = ctx.timerService.currentProcessingTime
+//buffer the event incoming event
+  
+//we accumulate the events as they arrive within the given proctime
--- End diff --

remove this comment?


> Support Limit/Top(Sort) for Stream SQL
> --
>
> Key: FLINK-6075
> URL: https://issues.apache.org/jira/browse/FLINK-6075
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: radu
>  Labels: features
> Attachments: sort.png
>
>
> These will be split in 3 separated JIRA issues. However, the design is the 
> same only the processing function differs in terms of the output. Hence, the 
> design is the same for all of them.
> Time 

[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116537830
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SortITCase.scala
 ---
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.stream.sql
+
+import org.apache.flink.api.scala._
+import 
org.apache.flink.table.api.scala.stream.sql.SortITCase.{EventTimeSourceFunction,StringRowSelectorSink}
+import org.apache.flink.streaming.api.functions.source.SourceFunction
+import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
+import org.apache.flink.table.api.TableEnvironment
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.api.scala.stream.utils.{StreamITCase, 
StreamTestData, StreamingWithStateTestBase}
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.types.Row
+import org.junit.Assert._
+import org.junit._
+import org.apache.flink.streaming.api.TimeCharacteristic
+import 
org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext
+import org.apache.flink.streaming.api.watermark.Watermark
+import scala.collection.mutable
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction
+
+class SortITCase extends StreamingWithStateTestBase {
+
+  @Test
+  def testEventTimeOrderBy(): Unit = {
+val data = Seq(
+  Left((1500L, (1L, 15, "Hello"))),
+  Left((1600L, (1L, 16, "Hello"))),
+  Left((1000L, (1L, 1, "Hello"))),
+  Left((2000L, (2L, 2, "Hello"))),
+  Right(1000L),
+  Left((2000L, (2L, 2, "Hello"))),
+  Left((2000L, (2L, 3, "Hello"))),
+  Left((3000L, (3L, 3, "Hello"))),
+  Right(2000L),
+  Left((4000L, (4L, 4, "Hello"))),
+  Right(3000L),
+  Left((5000L, (5L, 5, "Hello"))),
+  Right(5000L),
+  Left((6000L, (6L, 65, "Hello"))),
+  Left((6000L, (6L, 6, "Hello"))),
+  Right(7000L),
+  Left((9000L, (6L, 9, "Hello"))),
+  Left((8500L, (6L, 18, "Hello"))),
+  Left((9000L, (6L, 7, "Hello"))),
+  Right(1L),
+  Left((1L, (7L, 7, "Hello World"))),
+  Left((11000L, (7L, 77, "Hello World"))),
+  Left((11000L, (7L, 17, "Hello World"))),
+  Right(12000L),
+  Left((14000L, (7L, 18, "Hello World"))),
+  Right(14000L),
+  Left((15000L, (8L, 8, "Hello World"))),
+  Right(17000L),
+  Left((2L, (20L, 20, "Hello World"))), 
+  Right(19000L))
+
+val env = StreamExecutionEnvironment.getExecutionEnvironment
--- End diff --

set the state backend to ensure correct serialization


> Support Limit/Top(Sort) for Stream SQL
> --
>
> Key: FLINK-6075
> URL: https://issues.apache.org/jira/browse/FLINK-6075
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: radu
>  Labels: features
> Attachments: sort.png
>
>
> These will be split in 3 separated JIRA issues. However, the design is the 
> same only the processing function differs in terms of the output. Hence, the 
> design is the same for all of them.
> Time target: Proc Time
> **SQL targeted query examples:**
> *Sort example*
> Q1)` SELECT a FROM stream1 GROUP BY HOP(proctime, INTERVAL '1' HOUR, INTERVAL 
> '3' HOUR) ORDER BY b` 
> Comment: window is defined using GROUP BY
> Comment: ASC or DESC keywords can be placed to mark the ordering 

[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116471426
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSort.scala
 ---
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.plan.{ RelOptCluster, RelTraitSet }
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.{ RelNode, RelWriter, SingleRel }
+import org.apache.flink.api.java.tuple.Tuple
+import org.apache.flink.streaming.api.datastream.{ AllWindowedStream, 
DataStream, KeyedStream, WindowedStream }
+import org.apache.flink.streaming.api.windowing.assigners._
+import org.apache.flink.streaming.api.windowing.time.Time
+import org.apache.flink.streaming.api.windowing.windows.{ Window => 
DataStreamWindow }
+import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.expressions._
+import org.apache.flink.table.plan.logical._
+import org.apache.flink.table.plan.nodes.CommonAggregate
+import org.apache.flink.table.runtime.aggregate.AggregateUtil._
+import org.apache.flink.table.runtime.aggregate._
+import org.apache.flink.table.typeutils.TypeCheckUtils.isTimeInterval
+import org.apache.flink.table.typeutils.{ RowIntervalTypeInfo, 
TimeIntervalTypeInfo }
+import org.apache.flink.types.Row
+import org.apache.calcite.sql.SqlAggFunction
+import org.apache.flink.table.plan.nodes.datastream.DataStreamRel
+import org.apache.calcite.sql.fun.SqlSingleValueAggFunction
+import org.apache.flink.api.common.functions.RichMapFunction
+import org.apache.flink.api.common.functions.RichFlatMapFunction
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.util.Collector
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.calcite.rel.core.Sort
+import org.apache.flink.api.java.functions.NullByteKeySelector
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import org.apache.flink.table.runtime.aggregate.SortUtil._
+import org.apache.calcite.rel.RelCollation
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+import org.apache.flink.table.api.{StreamQueryConfig, 
StreamTableEnvironment, TableException}
+import org.apache.flink.table.plan.schema.RowSchema
+
+/**
+  * Flink RelNode which matches along with Sort Rule.
+  *
+ */
+class DataStreamSort(
+  sortCollation: RelCollation,
+  sortOffset: RexNode,
+  sortFetch: RexNode,
+  cluster: RelOptCluster,
+  traitSet: RelTraitSet,
+  inputNode: RelNode,
+  schema: RowSchema,
+  inputSchema: RowSchema,
+  description: String)
+extends SingleRel(cluster, traitSet, inputNode) with DataStreamRel {
+
+  override def deriveRowType(): RelDataType = schema.logicalType
+
+  override def copy(traitSet: RelTraitSet, inputs: 
java.util.List[RelNode]): RelNode = {
+new DataStreamSort(
+  sortCollation,
+  sortOffset,
+  sortFetch,
+  cluster,
+  traitSet,
+  inputs.get(0),
+  schema,
+  inputSchema,
+  description)
+  }
+
+  override def toString: String = {
+s"Sort(by: ($SortUtil.getSortFieldToString(sortCollation, 
rowRelDataType))," +
+  " offset: $SortUtil.getOffsetToString(sortOffset)," +
+  " fetch: $SortUtil.getFetchToString(sortFetch, sortOffset))"
+  }
+  
+  override def explainTerms(pw: RelWriter) : RelWriter = {
+
+//need to identify time between others order fields. Time needs to be 

[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116495976
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala
 ---
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.types.Row
+import org.apache.calcite.rel.`type`._
+import org.apache.calcite.rel.RelCollation
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.table.functions.AggregateFunction
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.flink.table.api.TableException
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.calcite.sql.`type`.SqlTypeName._
+import java.util.{ List => JList, ArrayList }
+import org.apache.flink.api.common.typeinfo.{ SqlTimeTypeInfo, 
TypeInformation }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import java.sql.Timestamp
+import org.apache.calcite.rel.RelFieldCollation
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import java.util.Comparator
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import 
java.lang.{Byte=>JByte,Integer=>JInt,Long=>JLong,Double=>JDouble,Short=>JShort,String=>JString,Float=>JFloat}
+import java.math.{BigDecimal=>JBigDecimal}
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.api.common.operators.Order
+import org.apache.calcite.rex.{RexLiteral, RexNode}
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.api.common.typeinfo.AtomicType
+import org.apache.flink.api.java.typeutils.runtime.RowComparator
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+import scala.collection.JavaConverters._
+
+/**
+ * Class represents a collection of helper methods to build the sort logic.
+ * It encapsulates as well the implementation for ordering and generic 
interfaces
+ */
+
--- End diff --

rm newline


> Support Limit/Top(Sort) for Stream SQL
> --
>
> Key: FLINK-6075
> URL: https://issues.apache.org/jira/browse/FLINK-6075
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: radu
>  Labels: features
> Attachments: sort.png
>
>
> These will be split in 3 separated JIRA issues. However, the design is the 
> same only the processing function differs in terms of the output. Hence, the 
> design is the same for all of them.
> Time target: Proc Time
> **SQL targeted query examples:**
> *Sort example*
> Q1)` SELECT a FROM stream1 GROUP BY HOP(proctime, INTERVAL '1' HOUR, INTERVAL 
> '3' HOUR) ORDER BY b` 
> Comment: window is defined using GROUP BY
> Comment: ASC or DESC keywords can be placed to mark the ordering type
> *Limit example*
> Q2) `SELECT a FROM stream1 WHERE rowtime BETWEEN current_timestamp - INTERVAL 
> '1' HOUR AND current_timestamp ORDER BY b LIMIT 10`
> Comment: window is defined using time ranges in the WHERE clause
> Comment: window is row triggered
> *Top example*
> Q3) `SELECT sum(a) OVER (ORDER BY proctime RANGE INTERVAL '1' HOUR PRECEDING 
> LIMIT 10) FROM stream1`  
> Comment: limit over the contents of the sliding window
> General Comments:
> -All these SQL clauses are supported only over windows (bounded collections 
> of data). 
> -Each of the 3 operators will be supported with each of the types of 
> expressing the 

[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116478528
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeSortProcessFunction.scala
 ---
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.api.common.state.{ ListState, ListStateDescriptor }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.runtime.state.{ FunctionInitializationContext, 
FunctionSnapshotContext }
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.types.Row
+import org.apache.flink.util.{ Collector, Preconditions }
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import scala.util.control.Breaks._
+import org.apache.flink.api.java.tuple.{ Tuple2 => JTuple2 }
+import org.apache.flink.api.common.state.MapState
+import org.apache.flink.api.common.state.MapStateDescriptor
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import java.util.Comparator
+import java.util.ArrayList
+import java.util.Collections
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+/**
+ * Process Function used for the aggregate in bounded proctime sort 
without offset/fetch
+ * [[org.apache.flink.streaming.api.datastream.DataStream]]
+ *
+ * @param fieldCount Is used to indicate fields in the current element to 
forward
+ * @param inputType It is used to mark the type of the incoming data
--- End diff --

`inputType` -> `inputRowType`


> Support Limit/Top(Sort) for Stream SQL
> --
>
> Key: FLINK-6075
> URL: https://issues.apache.org/jira/browse/FLINK-6075
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: radu
>  Labels: features
> Attachments: sort.png
>
>
> These will be split in 3 separated JIRA issues. However, the design is the 
> same only the processing function differs in terms of the output. Hence, the 
> design is the same for all of them.
> Time target: Proc Time
> **SQL targeted query examples:**
> *Sort example*
> Q1)` SELECT a FROM stream1 GROUP BY HOP(proctime, INTERVAL '1' HOUR, INTERVAL 
> '3' HOUR) ORDER BY b` 
> Comment: window is defined using GROUP BY
> Comment: ASC or DESC keywords can be placed to mark the ordering type
> *Limit example*
> Q2) `SELECT a FROM stream1 WHERE rowtime BETWEEN current_timestamp - INTERVAL 
> '1' HOUR AND current_timestamp ORDER BY b LIMIT 10`
> Comment: window is defined using time ranges in the WHERE clause
> Comment: window is row triggered
> *Top example*
> Q3) `SELECT sum(a) OVER (ORDER BY proctime RANGE INTERVAL '1' HOUR PRECEDING 
> LIMIT 10) FROM stream1`  
> Comment: limit over the contents of the sliding window
> General Comments:
> -All these SQL clauses are supported only over windows (bounded collections 
> of data). 
> -Each of the 3 operators will be supported with each of the types of 
> expressing the windows. 
> **Description**
> The 3 operations (limit, top and sort) are similar in behavior as they all 
> require a sorted collection of the data on which the logic will be applied 
> (i.e., select a subset of the items or the entire sorted set). These 
> functions would make sense in the streaming context only in the context of a 
> window. Without defining a window the functions could never emit as 

[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116528193
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala
 ---
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.types.Row
+import org.apache.calcite.rel.`type`._
+import org.apache.calcite.rel.RelCollation
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.table.functions.AggregateFunction
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.flink.table.api.TableException
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.calcite.sql.`type`.SqlTypeName._
+import java.util.{ List => JList, ArrayList }
+import org.apache.flink.api.common.typeinfo.{ SqlTimeTypeInfo, 
TypeInformation }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import java.sql.Timestamp
+import org.apache.calcite.rel.RelFieldCollation
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import java.util.Comparator
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import 
java.lang.{Byte=>JByte,Integer=>JInt,Long=>JLong,Double=>JDouble,Short=>JShort,String=>JString,Float=>JFloat}
+import java.math.{BigDecimal=>JBigDecimal}
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.api.common.operators.Order
+import org.apache.calcite.rex.{RexLiteral, RexNode}
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.api.common.typeinfo.AtomicType
+import org.apache.flink.api.java.typeutils.runtime.RowComparator
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+import scala.collection.JavaConverters._
+
+/**
+ * Class represents a collection of helper methods to build the sort logic.
+ * It encapsulates as well the implementation for ordering and generic 
interfaces
+ */
+
+object SortUtil {
+
+  
+  /**
+   * Function creates 
[org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
+   * elements based on rowtime and potentially other fields
+   * @param collationSort The Sort collation list
+   * @param inputType input row type
+   * @param execCfg table environment execution configuration
+   * @return org.apache.flink.streaming.api.functions.ProcessFunction
+   */
+  private[flink] def createRowTimeSortFunction(
+collationSort: RelCollation,
+inputType: RelDataType,
+inputTypeInfo: TypeInformation[Row],
+execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
+
+val keySortFields = getSortFieldIndexList(collationSort)
+val keySortDirections = getSortFieldDirectionList(collationSort)
+
+   //drop time from comparison as we sort on time in the states and 
result emission
+val keyIndexesNoTime = keySortFields.slice(1, keySortFields.size)
+val keyDirectionsNoTime = keySortDirections.slice(1, 
keySortDirections.size)
+val booleanOrderings = getSortFieldDirectionBooleanList(collationSort)
+val booleanDirectionsNoTime = booleanOrderings.slice(1, 
booleanOrderings.size)
+
+val fieldComps = createFieldComparators(inputType, 
+keyIndexesNoTime, keyDirectionsNoTime, execCfg)
+val fieldCompsRefs = 
fieldComps.asInstanceOf[Array[TypeComparator[AnyRef]]]
+
+val rowComp = createRowComparator(inputType,
+keyIndexesNoTime, fieldCompsRefs, booleanDirectionsNoTime)
+val collectionRowComparator = new CollectionRowComparator(rowComp)
+
+val inputCRowType = CRowTypeInfo(inputTypeInfo)
+ 
+new 

[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116470110
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSort.scala
 ---
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.plan.{ RelOptCluster, RelTraitSet }
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.{ RelNode, RelWriter, SingleRel }
+import org.apache.flink.api.java.tuple.Tuple
+import org.apache.flink.streaming.api.datastream.{ AllWindowedStream, 
DataStream, KeyedStream, WindowedStream }
+import org.apache.flink.streaming.api.windowing.assigners._
+import org.apache.flink.streaming.api.windowing.time.Time
+import org.apache.flink.streaming.api.windowing.windows.{ Window => 
DataStreamWindow }
+import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.expressions._
+import org.apache.flink.table.plan.logical._
+import org.apache.flink.table.plan.nodes.CommonAggregate
+import org.apache.flink.table.runtime.aggregate.AggregateUtil._
+import org.apache.flink.table.runtime.aggregate._
+import org.apache.flink.table.typeutils.TypeCheckUtils.isTimeInterval
+import org.apache.flink.table.typeutils.{ RowIntervalTypeInfo, 
TimeIntervalTypeInfo }
+import org.apache.flink.types.Row
+import org.apache.calcite.sql.SqlAggFunction
+import org.apache.flink.table.plan.nodes.datastream.DataStreamRel
+import org.apache.calcite.sql.fun.SqlSingleValueAggFunction
+import org.apache.flink.api.common.functions.RichMapFunction
+import org.apache.flink.api.common.functions.RichFlatMapFunction
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.util.Collector
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.calcite.rel.core.Sort
+import org.apache.flink.api.java.functions.NullByteKeySelector
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import org.apache.flink.table.runtime.aggregate.SortUtil._
+import org.apache.calcite.rel.RelCollation
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+import org.apache.flink.table.api.{StreamQueryConfig, 
StreamTableEnvironment, TableException}
+import org.apache.flink.table.plan.schema.RowSchema
+
+/**
+  * Flink RelNode which matches along with Sort Rule.
+  *
+ */
+class DataStreamSort(
+  sortCollation: RelCollation,
+  sortOffset: RexNode,
+  sortFetch: RexNode,
+  cluster: RelOptCluster,
+  traitSet: RelTraitSet,
+  inputNode: RelNode,
+  schema: RowSchema,
+  inputSchema: RowSchema,
+  description: String)
+extends SingleRel(cluster, traitSet, inputNode) with DataStreamRel {
+
+  override def deriveRowType(): RelDataType = schema.logicalType
+
+  override def copy(traitSet: RelTraitSet, inputs: 
java.util.List[RelNode]): RelNode = {
+new DataStreamSort(
+  sortCollation,
+  sortOffset,
+  sortFetch,
+  cluster,
+  traitSet,
+  inputs.get(0),
+  schema,
+  inputSchema,
+  description)
+  }
+
+  override def toString: String = {
+s"Sort(by: ($SortUtil.getSortFieldToString(sortCollation, 
rowRelDataType))," +
+  " offset: $SortUtil.getOffsetToString(sortOffset)," +
   

[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116540974
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SortTest.scala
 ---
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.api.scala.stream.sql
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.TableException
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.utils.TableTestUtil._
+import org.apache.flink.table.utils.{StreamTableTestUtil, TableTestBase}
+import org.junit.Test
+
+class SortTest extends TableTestBase {
+  private val streamUtil: StreamTableTestUtil = streamTestUtil()
+  streamUtil.addTable[(Int, String, Long)]("MyTable", 'a, 'b, 'c,
+  'proctime.proctime, 'rowtime.rowtime)
+  
+  @Test
+  def testSortProcessingTime() = {
+
+val sqlQuery = "SELECT a FROM MyTable ORDER BY proctime, c"
+  
+val expected =
+  unaryNode(
+"DataStreamCalc",
+unaryNode("DataStreamSort",
+  streamTableNode(0),
+  term("orderBy", "proctime ASC, c ASC], offset=[null], 
fetch=[unlimited")),
+term("select", "a", "TIME_MATERIALIZATION(proctime) AS proctime", 
"c"))
+
+streamUtil.verifySql(sqlQuery, expected)
+  }
+  
+  
+  @Test
+  def testSortRowTime() = {
+
+val sqlQuery = "SELECT a FROM MyTable ORDER BY rowtime, c"
+  
+val expected =
+  unaryNode(
+"DataStreamCalc",
+unaryNode("DataStreamSort",
+  streamTableNode(0),
+  term("orderBy", "rowtime ASC, c ASC], offset=[null], 
fetch=[unlimited")),
+term("select", "a", "TIME_MATERIALIZATION(rowtime) AS rowtime", 
"c"))
+   
+streamUtil.verifySql(sqlQuery, expected)
+  }
+  
+   @Test
+  def testSortProcessingTimeDesc() = {
+
+val sqlQuery = "SELECT a FROM MyTable ORDER BY proctime DESC, c"
+//fail if no error is thrown
+try{
+  streamUtil.verifySql(sqlQuery, "")
+} catch {
+  case rt : Throwable => assert(true)
+}
+  }
+   
+@Test
--- End diff --

as above


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


[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116475658
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSort.scala
 ---
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.plan.{ RelOptCluster, RelTraitSet }
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.{ RelNode, RelWriter, SingleRel }
+import org.apache.flink.api.java.tuple.Tuple
+import org.apache.flink.streaming.api.datastream.{ AllWindowedStream, 
DataStream, KeyedStream, WindowedStream }
+import org.apache.flink.streaming.api.windowing.assigners._
+import org.apache.flink.streaming.api.windowing.time.Time
+import org.apache.flink.streaming.api.windowing.windows.{ Window => 
DataStreamWindow }
+import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.expressions._
+import org.apache.flink.table.plan.logical._
+import org.apache.flink.table.plan.nodes.CommonAggregate
+import org.apache.flink.table.runtime.aggregate.AggregateUtil._
+import org.apache.flink.table.runtime.aggregate._
+import org.apache.flink.table.typeutils.TypeCheckUtils.isTimeInterval
+import org.apache.flink.table.typeutils.{ RowIntervalTypeInfo, 
TimeIntervalTypeInfo }
+import org.apache.flink.types.Row
+import org.apache.calcite.sql.SqlAggFunction
+import org.apache.flink.table.plan.nodes.datastream.DataStreamRel
+import org.apache.calcite.sql.fun.SqlSingleValueAggFunction
+import org.apache.flink.api.common.functions.RichMapFunction
+import org.apache.flink.api.common.functions.RichFlatMapFunction
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.util.Collector
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.calcite.rel.core.Sort
+import org.apache.flink.api.java.functions.NullByteKeySelector
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import org.apache.flink.table.runtime.aggregate.SortUtil._
+import org.apache.calcite.rel.RelCollation
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+import org.apache.flink.table.api.{StreamQueryConfig, 
StreamTableEnvironment, TableException}
+import org.apache.flink.table.plan.schema.RowSchema
+
+/**
+  * Flink RelNode which matches along with Sort Rule.
+  *
+ */
+class DataStreamSort(
+  sortCollation: RelCollation,
+  sortOffset: RexNode,
+  sortFetch: RexNode,
+  cluster: RelOptCluster,
+  traitSet: RelTraitSet,
+  inputNode: RelNode,
+  schema: RowSchema,
+  inputSchema: RowSchema,
+  description: String)
+extends SingleRel(cluster, traitSet, inputNode) with DataStreamRel {
+
+  override def deriveRowType(): RelDataType = schema.logicalType
+
+  override def copy(traitSet: RelTraitSet, inputs: 
java.util.List[RelNode]): RelNode = {
+new DataStreamSort(
+  sortCollation,
+  sortOffset,
+  sortFetch,
+  cluster,
+  traitSet,
+  inputs.get(0),
+  schema,
+  inputSchema,
+  description)
+  }
+
+  override def toString: String = {
+s"Sort(by: ($SortUtil.getSortFieldToString(sortCollation, 
rowRelDataType))," +
+  " offset: $SortUtil.getOffsetToString(sortOffset)," +
   

[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116477715
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSort.scala
 ---
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.plan.{ RelOptCluster, RelTraitSet }
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.{ RelNode, RelWriter, SingleRel }
+import org.apache.flink.api.java.tuple.Tuple
+import org.apache.flink.streaming.api.datastream.{ AllWindowedStream, 
DataStream, KeyedStream, WindowedStream }
+import org.apache.flink.streaming.api.windowing.assigners._
+import org.apache.flink.streaming.api.windowing.time.Time
+import org.apache.flink.streaming.api.windowing.windows.{ Window => 
DataStreamWindow }
+import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.expressions._
+import org.apache.flink.table.plan.logical._
+import org.apache.flink.table.plan.nodes.CommonAggregate
+import org.apache.flink.table.runtime.aggregate.AggregateUtil._
+import org.apache.flink.table.runtime.aggregate._
+import org.apache.flink.table.typeutils.TypeCheckUtils.isTimeInterval
+import org.apache.flink.table.typeutils.{ RowIntervalTypeInfo, 
TimeIntervalTypeInfo }
+import org.apache.flink.types.Row
+import org.apache.calcite.sql.SqlAggFunction
+import org.apache.flink.table.plan.nodes.datastream.DataStreamRel
+import org.apache.calcite.sql.fun.SqlSingleValueAggFunction
+import org.apache.flink.api.common.functions.RichMapFunction
+import org.apache.flink.api.common.functions.RichFlatMapFunction
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.util.Collector
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.calcite.rel.core.Sort
+import org.apache.flink.api.java.functions.NullByteKeySelector
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import org.apache.flink.table.runtime.aggregate.SortUtil._
+import org.apache.calcite.rel.RelCollation
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+import org.apache.flink.table.api.{StreamQueryConfig, 
StreamTableEnvironment, TableException}
+import org.apache.flink.table.plan.schema.RowSchema
+
+/**
+  * Flink RelNode which matches along with Sort Rule.
+  *
+ */
+class DataStreamSort(
+  sortCollation: RelCollation,
+  sortOffset: RexNode,
+  sortFetch: RexNode,
+  cluster: RelOptCluster,
+  traitSet: RelTraitSet,
+  inputNode: RelNode,
+  schema: RowSchema,
+  inputSchema: RowSchema,
+  description: String)
+extends SingleRel(cluster, traitSet, inputNode) with DataStreamRel {
+
+  override def deriveRowType(): RelDataType = schema.logicalType
+
+  override def copy(traitSet: RelTraitSet, inputs: 
java.util.List[RelNode]): RelNode = {
+new DataStreamSort(
+  sortCollation,
+  sortOffset,
+  sortFetch,
+  cluster,
+  traitSet,
+  inputs.get(0),
+  schema,
+  inputSchema,
+  description)
+  }
+
+  override def toString: String = {
+s"Sort(by: ($SortUtil.getSortFieldToString(sortCollation, 
rowRelDataType))," +
+  " offset: $SortUtil.getOffsetToString(sortOffset)," +
   

[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116519208
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala
 ---
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.types.Row
+import org.apache.calcite.rel.`type`._
+import org.apache.calcite.rel.RelCollation
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.table.functions.AggregateFunction
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.flink.table.api.TableException
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.calcite.sql.`type`.SqlTypeName._
+import java.util.{ List => JList, ArrayList }
+import org.apache.flink.api.common.typeinfo.{ SqlTimeTypeInfo, 
TypeInformation }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import java.sql.Timestamp
+import org.apache.calcite.rel.RelFieldCollation
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import java.util.Comparator
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import 
java.lang.{Byte=>JByte,Integer=>JInt,Long=>JLong,Double=>JDouble,Short=>JShort,String=>JString,Float=>JFloat}
+import java.math.{BigDecimal=>JBigDecimal}
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.api.common.operators.Order
+import org.apache.calcite.rex.{RexLiteral, RexNode}
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.api.common.typeinfo.AtomicType
+import org.apache.flink.api.java.typeutils.runtime.RowComparator
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+import scala.collection.JavaConverters._
+
+/**
+ * Class represents a collection of helper methods to build the sort logic.
+ * It encapsulates as well the implementation for ordering and generic 
interfaces
+ */
+
+object SortUtil {
+
+  
+  /**
+   * Function creates 
[org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
+   * elements based on rowtime and potentially other fields
+   * @param collationSort The Sort collation list
+   * @param inputType input row type
+   * @param execCfg table environment execution configuration
+   * @return org.apache.flink.streaming.api.functions.ProcessFunction
+   */
+  private[flink] def createRowTimeSortFunction(
+collationSort: RelCollation,
+inputType: RelDataType,
+inputTypeInfo: TypeInformation[Row],
+execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
+
+val keySortFields = getSortFieldIndexList(collationSort)
+val keySortDirections = getSortFieldDirectionList(collationSort)
+
+   //drop time from comparison as we sort on time in the states and 
result emission
+val keyIndexesNoTime = keySortFields.slice(1, keySortFields.size)
+val keyDirectionsNoTime = keySortDirections.slice(1, 
keySortDirections.size)
+val booleanOrderings = getSortFieldDirectionBooleanList(collationSort)
+val booleanDirectionsNoTime = booleanOrderings.slice(1, 
booleanOrderings.size)
+
+val fieldComps = createFieldComparators(inputType, 
+keyIndexesNoTime, keyDirectionsNoTime, execCfg)
+val fieldCompsRefs = 
fieldComps.asInstanceOf[Array[TypeComparator[AnyRef]]]
+
+val rowComp = createRowComparator(inputType,
+   

[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116526119
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala
 ---
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.types.Row
+import org.apache.calcite.rel.`type`._
+import org.apache.calcite.rel.RelCollation
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.table.functions.AggregateFunction
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.flink.table.api.TableException
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.calcite.sql.`type`.SqlTypeName._
+import java.util.{ List => JList, ArrayList }
+import org.apache.flink.api.common.typeinfo.{ SqlTimeTypeInfo, 
TypeInformation }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import java.sql.Timestamp
+import org.apache.calcite.rel.RelFieldCollation
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import java.util.Comparator
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import 
java.lang.{Byte=>JByte,Integer=>JInt,Long=>JLong,Double=>JDouble,Short=>JShort,String=>JString,Float=>JFloat}
+import java.math.{BigDecimal=>JBigDecimal}
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.api.common.operators.Order
+import org.apache.calcite.rex.{RexLiteral, RexNode}
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.api.common.typeinfo.AtomicType
+import org.apache.flink.api.java.typeutils.runtime.RowComparator
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+import scala.collection.JavaConverters._
+
+/**
+ * Class represents a collection of helper methods to build the sort logic.
+ * It encapsulates as well the implementation for ordering and generic 
interfaces
+ */
+
+object SortUtil {
+
+  
+  /**
+   * Function creates 
[org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
+   * elements based on rowtime and potentially other fields
+   * @param collationSort The Sort collation list
+   * @param inputType input row type
+   * @param execCfg table environment execution configuration
+   * @return org.apache.flink.streaming.api.functions.ProcessFunction
+   */
+  private[flink] def createRowTimeSortFunction(
+collationSort: RelCollation,
+inputType: RelDataType,
+inputTypeInfo: TypeInformation[Row],
+execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
+
+val keySortFields = getSortFieldIndexList(collationSort)
+val keySortDirections = getSortFieldDirectionList(collationSort)
+
+   //drop time from comparison as we sort on time in the states and 
result emission
+val keyIndexesNoTime = keySortFields.slice(1, keySortFields.size)
+val keyDirectionsNoTime = keySortDirections.slice(1, 
keySortDirections.size)
+val booleanOrderings = getSortFieldDirectionBooleanList(collationSort)
+val booleanDirectionsNoTime = booleanOrderings.slice(1, 
booleanOrderings.size)
+
+val fieldComps = createFieldComparators(inputType, 
+keyIndexesNoTime, keyDirectionsNoTime, execCfg)
+val fieldCompsRefs = 
fieldComps.asInstanceOf[Array[TypeComparator[AnyRef]]]
+
+val rowComp = createRowComparator(inputType,
+keyIndexesNoTime, fieldCompsRefs, booleanDirectionsNoTime)
+val collectionRowComparator = new CollectionRowComparator(rowComp)
+
+val inputCRowType = CRowTypeInfo(inputTypeInfo)
+ 
+new 

[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116540074
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SortTest.scala
 ---
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.api.scala.stream.sql
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.TableException
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.utils.TableTestUtil._
+import org.apache.flink.table.utils.{StreamTableTestUtil, TableTestBase}
+import org.junit.Test
+
+class SortTest extends TableTestBase {
+  private val streamUtil: StreamTableTestUtil = streamTestUtil()
+  streamUtil.addTable[(Int, String, Long)]("MyTable", 'a, 'b, 'c,
+  'proctime.proctime, 'rowtime.rowtime)
+  
+  @Test
+  def testSortProcessingTime() = {
+
+val sqlQuery = "SELECT a FROM MyTable ORDER BY proctime, c"
+  
+val expected =
+  unaryNode(
+"DataStreamCalc",
+unaryNode("DataStreamSort",
+  streamTableNode(0),
+  term("orderBy", "proctime ASC, c ASC], offset=[null], 
fetch=[unlimited")),
--- End diff --

The string seems to be broken. First there is a closing `]` and later an 
opening `[`


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


[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116521769
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala
 ---
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.types.Row
+import org.apache.calcite.rel.`type`._
+import org.apache.calcite.rel.RelCollation
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.table.functions.AggregateFunction
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.flink.table.api.TableException
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.calcite.sql.`type`.SqlTypeName._
+import java.util.{ List => JList, ArrayList }
+import org.apache.flink.api.common.typeinfo.{ SqlTimeTypeInfo, 
TypeInformation }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import java.sql.Timestamp
+import org.apache.calcite.rel.RelFieldCollation
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import java.util.Comparator
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import 
java.lang.{Byte=>JByte,Integer=>JInt,Long=>JLong,Double=>JDouble,Short=>JShort,String=>JString,Float=>JFloat}
+import java.math.{BigDecimal=>JBigDecimal}
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.api.common.operators.Order
+import org.apache.calcite.rex.{RexLiteral, RexNode}
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.api.common.typeinfo.AtomicType
+import org.apache.flink.api.java.typeutils.runtime.RowComparator
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+import scala.collection.JavaConverters._
+
+/**
+ * Class represents a collection of helper methods to build the sort logic.
+ * It encapsulates as well the implementation for ordering and generic 
interfaces
+ */
+
+object SortUtil {
+
+  
+  /**
+   * Function creates 
[org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
+   * elements based on rowtime and potentially other fields
+   * @param collationSort The Sort collation list
+   * @param inputType input row type
+   * @param execCfg table environment execution configuration
+   * @return org.apache.flink.streaming.api.functions.ProcessFunction
+   */
+  private[flink] def createRowTimeSortFunction(
+collationSort: RelCollation,
+inputType: RelDataType,
+inputTypeInfo: TypeInformation[Row],
+execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
+
+val keySortFields = getSortFieldIndexList(collationSort)
+val keySortDirections = getSortFieldDirectionList(collationSort)
+
+   //drop time from comparison as we sort on time in the states and 
result emission
+val keyIndexesNoTime = keySortFields.slice(1, keySortFields.size)
+val keyDirectionsNoTime = keySortDirections.slice(1, 
keySortDirections.size)
+val booleanOrderings = getSortFieldDirectionBooleanList(collationSort)
+val booleanDirectionsNoTime = booleanOrderings.slice(1, 
booleanOrderings.size)
+
+val fieldComps = createFieldComparators(inputType, 
--- End diff --

Please wrap method arguments as described above.


> Support Limit/Top(Sort) for Stream SQL
> --
>
> Key: FLINK-6075
> 

[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116541055
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SortTest.scala
 ---
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.api.scala.stream.sql
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.TableException
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.utils.TableTestUtil._
+import org.apache.flink.table.utils.{StreamTableTestUtil, TableTestBase}
+import org.junit.Test
+
+class SortTest extends TableTestBase {
+  private val streamUtil: StreamTableTestUtil = streamTestUtil()
+  streamUtil.addTable[(Int, String, Long)]("MyTable", 'a, 'b, 'c,
+  'proctime.proctime, 'rowtime.rowtime)
+  
+  @Test
+  def testSortProcessingTime() = {
+
+val sqlQuery = "SELECT a FROM MyTable ORDER BY proctime, c"
+  
+val expected =
+  unaryNode(
+"DataStreamCalc",
+unaryNode("DataStreamSort",
+  streamTableNode(0),
+  term("orderBy", "proctime ASC, c ASC], offset=[null], 
fetch=[unlimited")),
+term("select", "a", "TIME_MATERIALIZATION(proctime) AS proctime", 
"c"))
+
+streamUtil.verifySql(sqlQuery, expected)
+  }
+  
+  
+  @Test
+  def testSortRowTime() = {
+
+val sqlQuery = "SELECT a FROM MyTable ORDER BY rowtime, c"
+  
+val expected =
+  unaryNode(
+"DataStreamCalc",
+unaryNode("DataStreamSort",
+  streamTableNode(0),
+  term("orderBy", "rowtime ASC, c ASC], offset=[null], 
fetch=[unlimited")),
+term("select", "a", "TIME_MATERIALIZATION(rowtime) AS rowtime", 
"c"))
+   
+streamUtil.verifySql(sqlQuery, expected)
+  }
+  
+   @Test
+  def testSortProcessingTimeDesc() = {
+
+val sqlQuery = "SELECT a FROM MyTable ORDER BY proctime DESC, c"
+//fail if no error is thrown
+try{
+  streamUtil.verifySql(sqlQuery, "")
+} catch {
+  case rt : Throwable => assert(true)
+}
+  }
+   
+@Test
+   def testSortProcessingTimeSecondaryField() = {
+
+val sqlQuery = "SELECT a FROM MyTable ORDER BY c, proctime"
--- End diff --

Please add a comment why the tests are expected to fail


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


[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116477715
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSort.scala
 ---
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.plan.{ RelOptCluster, RelTraitSet }
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.{ RelNode, RelWriter, SingleRel }
+import org.apache.flink.api.java.tuple.Tuple
+import org.apache.flink.streaming.api.datastream.{ AllWindowedStream, 
DataStream, KeyedStream, WindowedStream }
+import org.apache.flink.streaming.api.windowing.assigners._
+import org.apache.flink.streaming.api.windowing.time.Time
+import org.apache.flink.streaming.api.windowing.windows.{ Window => 
DataStreamWindow }
+import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.expressions._
+import org.apache.flink.table.plan.logical._
+import org.apache.flink.table.plan.nodes.CommonAggregate
+import org.apache.flink.table.runtime.aggregate.AggregateUtil._
+import org.apache.flink.table.runtime.aggregate._
+import org.apache.flink.table.typeutils.TypeCheckUtils.isTimeInterval
+import org.apache.flink.table.typeutils.{ RowIntervalTypeInfo, 
TimeIntervalTypeInfo }
+import org.apache.flink.types.Row
+import org.apache.calcite.sql.SqlAggFunction
+import org.apache.flink.table.plan.nodes.datastream.DataStreamRel
+import org.apache.calcite.sql.fun.SqlSingleValueAggFunction
+import org.apache.flink.api.common.functions.RichMapFunction
+import org.apache.flink.api.common.functions.RichFlatMapFunction
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.util.Collector
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.calcite.rel.core.Sort
+import org.apache.flink.api.java.functions.NullByteKeySelector
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import org.apache.flink.table.runtime.aggregate.SortUtil._
+import org.apache.calcite.rel.RelCollation
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+import org.apache.flink.table.api.{StreamQueryConfig, 
StreamTableEnvironment, TableException}
+import org.apache.flink.table.plan.schema.RowSchema
+
+/**
+  * Flink RelNode which matches along with Sort Rule.
+  *
+ */
+class DataStreamSort(
+  sortCollation: RelCollation,
+  sortOffset: RexNode,
+  sortFetch: RexNode,
+  cluster: RelOptCluster,
+  traitSet: RelTraitSet,
+  inputNode: RelNode,
+  schema: RowSchema,
+  inputSchema: RowSchema,
+  description: String)
+extends SingleRel(cluster, traitSet, inputNode) with DataStreamRel {
+
+  override def deriveRowType(): RelDataType = schema.logicalType
+
+  override def copy(traitSet: RelTraitSet, inputs: 
java.util.List[RelNode]): RelNode = {
+new DataStreamSort(
+  sortCollation,
+  sortOffset,
+  sortFetch,
+  cluster,
+  traitSet,
+  inputs.get(0),
+  schema,
+  inputSchema,
+  description)
+  }
+
+  override def toString: String = {
+s"Sort(by: ($SortUtil.getSortFieldToString(sortCollation, 
rowRelDataType))," +
+  " offset: $SortUtil.getOffsetToString(sortOffset)," +
+  " fetch: $SortUtil.getFetchToString(sortFetch, sortOffset))"
+  }
+  
+  override def explainTerms(pw: RelWriter) : RelWriter = {
+
+//need to identify time between others order fields. Time needs to be 

[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116470198
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSort.scala
 ---
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.plan.{ RelOptCluster, RelTraitSet }
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.{ RelNode, RelWriter, SingleRel }
+import org.apache.flink.api.java.tuple.Tuple
+import org.apache.flink.streaming.api.datastream.{ AllWindowedStream, 
DataStream, KeyedStream, WindowedStream }
+import org.apache.flink.streaming.api.windowing.assigners._
+import org.apache.flink.streaming.api.windowing.time.Time
+import org.apache.flink.streaming.api.windowing.windows.{ Window => 
DataStreamWindow }
+import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.expressions._
+import org.apache.flink.table.plan.logical._
+import org.apache.flink.table.plan.nodes.CommonAggregate
+import org.apache.flink.table.runtime.aggregate.AggregateUtil._
+import org.apache.flink.table.runtime.aggregate._
+import org.apache.flink.table.typeutils.TypeCheckUtils.isTimeInterval
+import org.apache.flink.table.typeutils.{ RowIntervalTypeInfo, 
TimeIntervalTypeInfo }
+import org.apache.flink.types.Row
+import org.apache.calcite.sql.SqlAggFunction
+import org.apache.flink.table.plan.nodes.datastream.DataStreamRel
+import org.apache.calcite.sql.fun.SqlSingleValueAggFunction
+import org.apache.flink.api.common.functions.RichMapFunction
+import org.apache.flink.api.common.functions.RichFlatMapFunction
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.util.Collector
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.calcite.rel.core.Sort
+import org.apache.flink.api.java.functions.NullByteKeySelector
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import org.apache.flink.table.runtime.aggregate.SortUtil._
+import org.apache.calcite.rel.RelCollation
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+import org.apache.flink.table.api.{StreamQueryConfig, 
StreamTableEnvironment, TableException}
+import org.apache.flink.table.plan.schema.RowSchema
+
+/**
+  * Flink RelNode which matches along with Sort Rule.
+  *
+ */
+class DataStreamSort(
+  sortCollation: RelCollation,
+  sortOffset: RexNode,
+  sortFetch: RexNode,
+  cluster: RelOptCluster,
+  traitSet: RelTraitSet,
+  inputNode: RelNode,
+  schema: RowSchema,
+  inputSchema: RowSchema,
+  description: String)
+extends SingleRel(cluster, traitSet, inputNode) with DataStreamRel {
+
+  override def deriveRowType(): RelDataType = schema.logicalType
+
+  override def copy(traitSet: RelTraitSet, inputs: 
java.util.List[RelNode]): RelNode = {
+new DataStreamSort(
+  sortCollation,
+  sortOffset,
+  sortFetch,
+  cluster,
+  traitSet,
+  inputs.get(0),
+  schema,
+  inputSchema,
+  description)
+  }
+
+  override def toString: String = {
+s"Sort(by: ($SortUtil.getSortFieldToString(sortCollation, 
rowRelDataType))," +
+  " offset: $SortUtil.getOffsetToString(sortOffset)," +
+  " fetch: $SortUtil.getFetchToString(sortFetch, sortOffset))"
+  }
+  
+  override def explainTerms(pw: RelWriter) : RelWriter = {
+
+//need to identify time between others order fields. Time needs to be 

[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116527674
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala
 ---
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.types.Row
+import org.apache.calcite.rel.`type`._
+import org.apache.calcite.rel.RelCollation
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.table.functions.AggregateFunction
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.flink.table.api.TableException
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.calcite.sql.`type`.SqlTypeName._
+import java.util.{ List => JList, ArrayList }
+import org.apache.flink.api.common.typeinfo.{ SqlTimeTypeInfo, 
TypeInformation }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import java.sql.Timestamp
+import org.apache.calcite.rel.RelFieldCollation
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import java.util.Comparator
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import 
java.lang.{Byte=>JByte,Integer=>JInt,Long=>JLong,Double=>JDouble,Short=>JShort,String=>JString,Float=>JFloat}
+import java.math.{BigDecimal=>JBigDecimal}
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.api.common.operators.Order
+import org.apache.calcite.rex.{RexLiteral, RexNode}
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.api.common.typeinfo.AtomicType
+import org.apache.flink.api.java.typeutils.runtime.RowComparator
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+import scala.collection.JavaConverters._
+
+/**
+ * Class represents a collection of helper methods to build the sort logic.
+ * It encapsulates as well the implementation for ordering and generic 
interfaces
+ */
+
+object SortUtil {
+
+  
+  /**
+   * Function creates 
[org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
+   * elements based on rowtime and potentially other fields
+   * @param collationSort The Sort collation list
+   * @param inputType input row type
+   * @param execCfg table environment execution configuration
+   * @return org.apache.flink.streaming.api.functions.ProcessFunction
+   */
+  private[flink] def createRowTimeSortFunction(
+collationSort: RelCollation,
+inputType: RelDataType,
+inputTypeInfo: TypeInformation[Row],
+execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
+
+val keySortFields = getSortFieldIndexList(collationSort)
+val keySortDirections = getSortFieldDirectionList(collationSort)
+
+   //drop time from comparison as we sort on time in the states and 
result emission
+val keyIndexesNoTime = keySortFields.slice(1, keySortFields.size)
+val keyDirectionsNoTime = keySortDirections.slice(1, 
keySortDirections.size)
+val booleanOrderings = getSortFieldDirectionBooleanList(collationSort)
+val booleanDirectionsNoTime = booleanOrderings.slice(1, 
booleanOrderings.size)
+
+val fieldComps = createFieldComparators(inputType, 
+keyIndexesNoTime, keyDirectionsNoTime, execCfg)
+val fieldCompsRefs = 
fieldComps.asInstanceOf[Array[TypeComparator[AnyRef]]]
+
+val rowComp = createRowComparator(inputType,
+   

[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116528048
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala
 ---
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.types.Row
+import org.apache.calcite.rel.`type`._
+import org.apache.calcite.rel.RelCollation
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.table.functions.AggregateFunction
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.flink.table.api.TableException
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.calcite.sql.`type`.SqlTypeName._
+import java.util.{ List => JList, ArrayList }
+import org.apache.flink.api.common.typeinfo.{ SqlTimeTypeInfo, 
TypeInformation }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import java.sql.Timestamp
+import org.apache.calcite.rel.RelFieldCollation
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import java.util.Comparator
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import 
java.lang.{Byte=>JByte,Integer=>JInt,Long=>JLong,Double=>JDouble,Short=>JShort,String=>JString,Float=>JFloat}
+import java.math.{BigDecimal=>JBigDecimal}
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.api.common.operators.Order
+import org.apache.calcite.rex.{RexLiteral, RexNode}
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.api.common.typeinfo.AtomicType
+import org.apache.flink.api.java.typeutils.runtime.RowComparator
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+import scala.collection.JavaConverters._
+
+/**
+ * Class represents a collection of helper methods to build the sort logic.
+ * It encapsulates as well the implementation for ordering and generic 
interfaces
+ */
+
+object SortUtil {
+
+  
+  /**
+   * Function creates 
[org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
+   * elements based on rowtime and potentially other fields
+   * @param collationSort The Sort collation list
+   * @param inputType input row type
+   * @param execCfg table environment execution configuration
+   * @return org.apache.flink.streaming.api.functions.ProcessFunction
+   */
+  private[flink] def createRowTimeSortFunction(
+collationSort: RelCollation,
+inputType: RelDataType,
+inputTypeInfo: TypeInformation[Row],
+execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
+
+val keySortFields = getSortFieldIndexList(collationSort)
+val keySortDirections = getSortFieldDirectionList(collationSort)
+
+   //drop time from comparison as we sort on time in the states and 
result emission
+val keyIndexesNoTime = keySortFields.slice(1, keySortFields.size)
+val keyDirectionsNoTime = keySortDirections.slice(1, 
keySortDirections.size)
+val booleanOrderings = getSortFieldDirectionBooleanList(collationSort)
+val booleanDirectionsNoTime = booleanOrderings.slice(1, 
booleanOrderings.size)
+
+val fieldComps = createFieldComparators(inputType, 
+keyIndexesNoTime, keyDirectionsNoTime, execCfg)
+val fieldCompsRefs = 
fieldComps.asInstanceOf[Array[TypeComparator[AnyRef]]]
+
+val rowComp = createRowComparator(inputType,
+keyIndexesNoTime, fieldCompsRefs, booleanDirectionsNoTime)
+val collectionRowComparator = new CollectionRowComparator(rowComp)
+
+val inputCRowType = CRowTypeInfo(inputTypeInfo)
+ 
+new 

[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116528048
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala
 ---
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.types.Row
+import org.apache.calcite.rel.`type`._
+import org.apache.calcite.rel.RelCollation
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.table.functions.AggregateFunction
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.flink.table.api.TableException
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.calcite.sql.`type`.SqlTypeName._
+import java.util.{ List => JList, ArrayList }
+import org.apache.flink.api.common.typeinfo.{ SqlTimeTypeInfo, 
TypeInformation }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import java.sql.Timestamp
+import org.apache.calcite.rel.RelFieldCollation
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import java.util.Comparator
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import 
java.lang.{Byte=>JByte,Integer=>JInt,Long=>JLong,Double=>JDouble,Short=>JShort,String=>JString,Float=>JFloat}
+import java.math.{BigDecimal=>JBigDecimal}
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.api.common.operators.Order
+import org.apache.calcite.rex.{RexLiteral, RexNode}
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.api.common.typeinfo.AtomicType
+import org.apache.flink.api.java.typeutils.runtime.RowComparator
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+import scala.collection.JavaConverters._
+
+/**
+ * Class represents a collection of helper methods to build the sort logic.
+ * It encapsulates as well the implementation for ordering and generic 
interfaces
+ */
+
+object SortUtil {
+
+  
+  /**
+   * Function creates 
[org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
+   * elements based on rowtime and potentially other fields
+   * @param collationSort The Sort collation list
+   * @param inputType input row type
+   * @param execCfg table environment execution configuration
+   * @return org.apache.flink.streaming.api.functions.ProcessFunction
+   */
+  private[flink] def createRowTimeSortFunction(
+collationSort: RelCollation,
+inputType: RelDataType,
+inputTypeInfo: TypeInformation[Row],
+execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
+
+val keySortFields = getSortFieldIndexList(collationSort)
+val keySortDirections = getSortFieldDirectionList(collationSort)
+
+   //drop time from comparison as we sort on time in the states and 
result emission
+val keyIndexesNoTime = keySortFields.slice(1, keySortFields.size)
+val keyDirectionsNoTime = keySortDirections.slice(1, 
keySortDirections.size)
+val booleanOrderings = getSortFieldDirectionBooleanList(collationSort)
+val booleanDirectionsNoTime = booleanOrderings.slice(1, 
booleanOrderings.size)
+
+val fieldComps = createFieldComparators(inputType, 
+keyIndexesNoTime, keyDirectionsNoTime, execCfg)
+val fieldCompsRefs = 
fieldComps.asInstanceOf[Array[TypeComparator[AnyRef]]]
+
+val rowComp = createRowComparator(inputType,
+   

[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116482324
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeSortProcessFunction.scala
 ---
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.api.common.state.{ ListState, ListStateDescriptor }
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.{RowTypeInfo, ListTypeInfo}
+import org.apache.flink.runtime.state.{ FunctionInitializationContext, 
FunctionSnapshotContext }
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.types.Row
+import org.apache.flink.util.{ Collector, Preconditions }
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import scala.util.control.Breaks._
+import org.apache.flink.api.java.tuple.{ Tuple2 => JTuple2 }
+import org.apache.flink.api.common.state.MapState
+import org.apache.flink.api.common.state.MapStateDescriptor
+import org.apache.flink.configuration.Configuration
+import java.util.Comparator
+import java.util.ArrayList
+import java.util.Collections
+import org.apache.flink.api.common.typeutils.TypeComparator
+import java.util.{List => JList, ArrayList => JArrayList}
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+/**
+ * Process Function used for the aggregate in bounded rowtime sort without 
offset/fetch
+ * [[org.apache.flink.streaming.api.datastream.DataStream]]
+ *
+ * @param fieldCount Is used to indicate fields in the current element to 
forward
+ * @param inputType It is used to mark the type of the incoming data
+ * @param rowComparator the [[java.util.Comparator]] is used for this sort 
aggregation
+ */
+class RowTimeSortProcessFunction(
+  private val fieldCount: Int,
+  private val inputRowType: CRowTypeInfo,
+  private val rowComparator: CollectionRowComparator)
+extends ProcessFunction[CRow, CRow] {
+
+  Preconditions.checkNotNull(rowComparator)
+
+  private val sortArray: ArrayList[Row] = new ArrayList[Row]
+  
+  // the state which keeps all the events that are not expired.
+  // Each timestamp will contain an associated list with the events 
+  // received at that timestamp
+  private var dataState: MapState[Long, JList[Row]] = _
+
+// the state which keeps the last triggering timestamp to filter late 
events
+  private var lastTriggeringTsState: ValueState[Long] = _
+  
+  private var outputC: CRow = _
+  
+  
--- End diff --

remove newline


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


[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116539089
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SortITCase.scala
 ---
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.stream.sql
+
+import org.apache.flink.api.scala._
+import 
org.apache.flink.table.api.scala.stream.sql.SortITCase.{EventTimeSourceFunction,StringRowSelectorSink}
+import org.apache.flink.streaming.api.functions.source.SourceFunction
+import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
+import org.apache.flink.table.api.TableEnvironment
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.api.scala.stream.utils.{StreamITCase, 
StreamTestData, StreamingWithStateTestBase}
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.types.Row
+import org.junit.Assert._
+import org.junit._
+import org.apache.flink.streaming.api.TimeCharacteristic
+import 
org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext
+import org.apache.flink.streaming.api.watermark.Watermark
+import scala.collection.mutable
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction
+
+class SortITCase extends StreamingWithStateTestBase {
+
+  @Test
+  def testEventTimeOrderBy(): Unit = {
+val data = Seq(
--- End diff --

at max there are two records with the same timestamp. Can you use a dataset 
with more records on the same time?


> Support Limit/Top(Sort) for Stream SQL
> --
>
> Key: FLINK-6075
> URL: https://issues.apache.org/jira/browse/FLINK-6075
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: radu
>  Labels: features
> Attachments: sort.png
>
>
> These will be split in 3 separated JIRA issues. However, the design is the 
> same only the processing function differs in terms of the output. Hence, the 
> design is the same for all of them.
> Time target: Proc Time
> **SQL targeted query examples:**
> *Sort example*
> Q1)` SELECT a FROM stream1 GROUP BY HOP(proctime, INTERVAL '1' HOUR, INTERVAL 
> '3' HOUR) ORDER BY b` 
> Comment: window is defined using GROUP BY
> Comment: ASC or DESC keywords can be placed to mark the ordering type
> *Limit example*
> Q2) `SELECT a FROM stream1 WHERE rowtime BETWEEN current_timestamp - INTERVAL 
> '1' HOUR AND current_timestamp ORDER BY b LIMIT 10`
> Comment: window is defined using time ranges in the WHERE clause
> Comment: window is row triggered
> *Top example*
> Q3) `SELECT sum(a) OVER (ORDER BY proctime RANGE INTERVAL '1' HOUR PRECEDING 
> LIMIT 10) FROM stream1`  
> Comment: limit over the contents of the sliding window
> General Comments:
> -All these SQL clauses are supported only over windows (bounded collections 
> of data). 
> -Each of the 3 operators will be supported with each of the types of 
> expressing the windows. 
> **Description**
> The 3 operations (limit, top and sort) are similar in behavior as they all 
> require a sorted collection of the data on which the logic will be applied 
> (i.e., select a subset of the items or the entire sorted set). These 
> functions would make sense in the streaming context only in the context of a 
> window. Without defining a window the functions could never emit as the sort 
> operation would never trigger. If an SQL query will be provided without 
> limits an error will be thrown (`SELECT a FROM stream1 TOP 10` -> 

[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116527214
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala
 ---
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.types.Row
+import org.apache.calcite.rel.`type`._
+import org.apache.calcite.rel.RelCollation
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.table.functions.AggregateFunction
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.flink.table.api.TableException
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.calcite.sql.`type`.SqlTypeName._
+import java.util.{ List => JList, ArrayList }
+import org.apache.flink.api.common.typeinfo.{ SqlTimeTypeInfo, 
TypeInformation }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import java.sql.Timestamp
+import org.apache.calcite.rel.RelFieldCollation
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import java.util.Comparator
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import 
java.lang.{Byte=>JByte,Integer=>JInt,Long=>JLong,Double=>JDouble,Short=>JShort,String=>JString,Float=>JFloat}
+import java.math.{BigDecimal=>JBigDecimal}
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.api.common.operators.Order
+import org.apache.calcite.rex.{RexLiteral, RexNode}
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.api.common.typeinfo.AtomicType
+import org.apache.flink.api.java.typeutils.runtime.RowComparator
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+import scala.collection.JavaConverters._
+
+/**
+ * Class represents a collection of helper methods to build the sort logic.
+ * It encapsulates as well the implementation for ordering and generic 
interfaces
+ */
+
+object SortUtil {
+
+  
+  /**
+   * Function creates 
[org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
+   * elements based on rowtime and potentially other fields
+   * @param collationSort The Sort collation list
+   * @param inputType input row type
+   * @param execCfg table environment execution configuration
+   * @return org.apache.flink.streaming.api.functions.ProcessFunction
+   */
+  private[flink] def createRowTimeSortFunction(
+collationSort: RelCollation,
+inputType: RelDataType,
+inputTypeInfo: TypeInformation[Row],
+execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
+
+val keySortFields = getSortFieldIndexList(collationSort)
+val keySortDirections = getSortFieldDirectionList(collationSort)
+
+   //drop time from comparison as we sort on time in the states and 
result emission
+val keyIndexesNoTime = keySortFields.slice(1, keySortFields.size)
+val keyDirectionsNoTime = keySortDirections.slice(1, 
keySortDirections.size)
+val booleanOrderings = getSortFieldDirectionBooleanList(collationSort)
+val booleanDirectionsNoTime = booleanOrderings.slice(1, 
booleanOrderings.size)
+
+val fieldComps = createFieldComparators(inputType, 
+keyIndexesNoTime, keyDirectionsNoTime, execCfg)
+val fieldCompsRefs = 
fieldComps.asInstanceOf[Array[TypeComparator[AnyRef]]]
+
+val rowComp = createRowComparator(inputType,
+   

[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116537188
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SortITCase.scala
 ---
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.stream.sql
+
+import org.apache.flink.api.scala._
+import 
org.apache.flink.table.api.scala.stream.sql.SortITCase.{EventTimeSourceFunction,StringRowSelectorSink}
+import org.apache.flink.streaming.api.functions.source.SourceFunction
+import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
+import org.apache.flink.table.api.TableEnvironment
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.api.scala.stream.utils.{StreamITCase, 
StreamTestData, StreamingWithStateTestBase}
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.types.Row
+import org.junit.Assert._
+import org.junit._
+import org.apache.flink.streaming.api.TimeCharacteristic
+import 
org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext
+import org.apache.flink.streaming.api.watermark.Watermark
+import scala.collection.mutable
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction
+
+class SortITCase extends StreamingWithStateTestBase {
+
+  @Test
+  def testEventTimeOrderBy(): Unit = {
+val data = Seq(
--- End diff --

The dataset is more complex than necessary for this test


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


[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116496352
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/ProcTimeSortProcessFunctionTest.scala
 ---
@@ -0,0 +1,165 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import java.util.Comparator
+import java.util.concurrent.ConcurrentLinkedQueue
+import java.lang.{Integer => JInt, Long => JLong}
+
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.functions.KeySelector
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.streaming.api.operators.KeyedProcessOperator
+import org.apache.flink.streaming.api.watermark.Watermark
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord
+import 
org.apache.flink.streaming.util.{KeyedOneInputStreamOperatorTestHarness, 
TestHarnessUtil}
+import org.apache.flink.types.Row
+import org.junit.Test
+import org.apache.flink.table.runtime.aggregate.ProcTimeSortProcessFunction
+import 
org.apache.flink.table.runtime.aggregate.ProcTimeSortProcessFunctionTest._
+import org.apache.flink.api.java.typeutils.runtime.RowComparator
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+class ProcTimeSortProcessFunctionTest{
+
+  
+  @Test
+  def testSortProcTimeHarnessPartitioned(): Unit = {
--- End diff --

would be good to have a Harness test for `RowTimeSortProcessFunction` as 
well.


> Support Limit/Top(Sort) for Stream SQL
> --
>
> Key: FLINK-6075
> URL: https://issues.apache.org/jira/browse/FLINK-6075
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: radu
>  Labels: features
> Attachments: sort.png
>
>
> These will be split in 3 separated JIRA issues. However, the design is the 
> same only the processing function differs in terms of the output. Hence, the 
> design is the same for all of them.
> Time target: Proc Time
> **SQL targeted query examples:**
> *Sort example*
> Q1)` SELECT a FROM stream1 GROUP BY HOP(proctime, INTERVAL '1' HOUR, INTERVAL 
> '3' HOUR) ORDER BY b` 
> Comment: window is defined using GROUP BY
> Comment: ASC or DESC keywords can be placed to mark the ordering type
> *Limit example*
> Q2) `SELECT a FROM stream1 WHERE rowtime BETWEEN current_timestamp - INTERVAL 
> '1' HOUR AND current_timestamp ORDER BY b LIMIT 10`
> Comment: window is defined using time ranges in the WHERE clause
> Comment: window is row triggered
> *Top example*
> Q3) `SELECT sum(a) OVER (ORDER BY proctime RANGE INTERVAL '1' HOUR PRECEDING 
> LIMIT 10) FROM stream1`  
> Comment: limit over the contents of the sliding window
> General Comments:
> -All these SQL clauses are supported only over windows (bounded collections 
> of data). 
> -Each of the 3 operators will be supported with each of the types of 
> expressing the windows. 
> **Description**
> The 3 operations (limit, top and sort) are similar in behavior as they all 
> require a sorted collection of the data on which the logic will be applied 
> (i.e., select a subset of the items or the entire sorted set). These 
> functions would make sense in the streaming context only in the context of a 
> window. Without defining a window the functions could never emit as the sort 
> operation would never trigger. If an SQL query will be provided without 
> limits an error 

[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116496352
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/ProcTimeSortProcessFunctionTest.scala
 ---
@@ -0,0 +1,165 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import java.util.Comparator
+import java.util.concurrent.ConcurrentLinkedQueue
+import java.lang.{Integer => JInt, Long => JLong}
+
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.functions.KeySelector
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.streaming.api.operators.KeyedProcessOperator
+import org.apache.flink.streaming.api.watermark.Watermark
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord
+import 
org.apache.flink.streaming.util.{KeyedOneInputStreamOperatorTestHarness, 
TestHarnessUtil}
+import org.apache.flink.types.Row
+import org.junit.Test
+import org.apache.flink.table.runtime.aggregate.ProcTimeSortProcessFunction
+import 
org.apache.flink.table.runtime.aggregate.ProcTimeSortProcessFunctionTest._
+import org.apache.flink.api.java.typeutils.runtime.RowComparator
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+class ProcTimeSortProcessFunctionTest{
+
+  
+  @Test
+  def testSortProcTimeHarnessPartitioned(): Unit = {
--- End diff --

would be good to have a Harness test for `RowTimeSortProcessFunction` as 
well.


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


[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116480310
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeSortProcessFunction.scala
 ---
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.api.common.state.{ ListState, ListStateDescriptor }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.runtime.state.{ FunctionInitializationContext, 
FunctionSnapshotContext }
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.types.Row
+import org.apache.flink.util.{ Collector, Preconditions }
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import scala.util.control.Breaks._
+import org.apache.flink.api.java.tuple.{ Tuple2 => JTuple2 }
+import org.apache.flink.api.common.state.MapState
+import org.apache.flink.api.common.state.MapStateDescriptor
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import java.util.Comparator
+import java.util.ArrayList
+import java.util.Collections
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+/**
+ * Process Function used for the aggregate in bounded proctime sort 
without offset/fetch
+ * [[org.apache.flink.streaming.api.datastream.DataStream]]
+ *
+ * @param fieldCount Is used to indicate fields in the current element to 
forward
+ * @param inputType It is used to mark the type of the incoming data
+ * @param rowComparator the [[java.util.Comparator]] is used for this sort 
aggregation
+ */
+class ProcTimeSortProcessFunction(
+  private val fieldCount: Int,
+  private val inputRowType: CRowTypeInfo,
+  private val rowComparator: CollectionRowComparator)
+extends ProcessFunction[CRow, CRow] {
+
+  Preconditions.checkNotNull(rowComparator)
+
+  private var stateEventsBuffer: ListState[Row] = _
+  private val sortArray: ArrayList[Row] = new ArrayList[Row]
+  
+  private var outputC: CRow = _
+  
+  override def open(config: Configuration) {
+val sortDescriptor = new ListStateDescriptor[Row]("sortState",
+inputRowType.asInstanceOf[CRowTypeInfo].rowType)
+stateEventsBuffer = getRuntimeContext.getListState(sortDescriptor)
+
+  }
+
+  override def processElement(
+inputC: CRow,
+ctx: ProcessFunction[CRow, CRow]#Context,
+out: Collector[CRow]): Unit = {
+
+val input = inputC.row
+
+if( outputC == null) {
+  outputC = new CRow(input, true)
+}
+
+val currentTime = ctx.timerService.currentProcessingTime
+//buffer the event incoming event
+  
+//we accumulate the events as they arrive within the given proctime
--- End diff --

remove this comment?


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


[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116495740
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeSortProcessFunction.scala
 ---
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.api.common.state.{ ListState, ListStateDescriptor }
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.{RowTypeInfo, ListTypeInfo}
+import org.apache.flink.runtime.state.{ FunctionInitializationContext, 
FunctionSnapshotContext }
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.types.Row
+import org.apache.flink.util.{ Collector, Preconditions }
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import scala.util.control.Breaks._
+import org.apache.flink.api.java.tuple.{ Tuple2 => JTuple2 }
+import org.apache.flink.api.common.state.MapState
+import org.apache.flink.api.common.state.MapStateDescriptor
+import org.apache.flink.configuration.Configuration
+import java.util.Comparator
+import java.util.ArrayList
+import java.util.Collections
+import org.apache.flink.api.common.typeutils.TypeComparator
+import java.util.{List => JList, ArrayList => JArrayList}
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+/**
+ * Process Function used for the aggregate in bounded rowtime sort without 
offset/fetch
+ * [[org.apache.flink.streaming.api.datastream.DataStream]]
+ *
+ * @param fieldCount Is used to indicate fields in the current element to 
forward
+ * @param inputType It is used to mark the type of the incoming data
+ * @param rowComparator the [[java.util.Comparator]] is used for this sort 
aggregation
+ */
+class RowTimeSortProcessFunction(
+  private val fieldCount: Int,
+  private val inputRowType: CRowTypeInfo,
+  private val rowComparator: CollectionRowComparator)
+extends ProcessFunction[CRow, CRow] {
+
+  Preconditions.checkNotNull(rowComparator)
+
+  private val sortArray: ArrayList[Row] = new ArrayList[Row]
+  
+  // the state which keeps all the events that are not expired.
+  // Each timestamp will contain an associated list with the events 
+  // received at that timestamp
+  private var dataState: MapState[Long, JList[Row]] = _
+
+// the state which keeps the last triggering timestamp to filter late 
events
+  private var lastTriggeringTsState: ValueState[Long] = _
+  
+  private var outputC: CRow = _
+  
+  
+  override def open(config: Configuration) {
+ 
+val keyTypeInformation: TypeInformation[Long] =
+  BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]]
+val valueTypeInformation: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](
+inputRowType.asInstanceOf[CRowTypeInfo].rowType)
+
+val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]](
+"dataState",
+keyTypeInformation,
+valueTypeInformation)
+
+dataState = getRuntimeContext.getMapState(mapStateDescriptor)
+
+val lastTriggeringTsDescriptor: ValueStateDescriptor[Long] =
+  new ValueStateDescriptor[Long]("lastTriggeringTsState", 
classOf[Long])
+lastTriggeringTsState = 
getRuntimeContext.getState(lastTriggeringTsDescriptor)
+  }
+
+  
+  override def processElement(
+inputC: CRow,
+ctx: ProcessFunction[CRow, CRow]#Context,
+out: Collector[CRow]): Unit = {
+
+ val input = inputC.row
+
+ if( outputC == null) {
+  outputC = new CRow(input, true)
+}
+
+// triggering timestamp for trigger 

[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116495976
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala
 ---
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.types.Row
+import org.apache.calcite.rel.`type`._
+import org.apache.calcite.rel.RelCollation
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.table.functions.AggregateFunction
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.flink.table.api.TableException
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.calcite.sql.`type`.SqlTypeName._
+import java.util.{ List => JList, ArrayList }
+import org.apache.flink.api.common.typeinfo.{ SqlTimeTypeInfo, 
TypeInformation }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import java.sql.Timestamp
+import org.apache.calcite.rel.RelFieldCollation
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import java.util.Comparator
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import 
java.lang.{Byte=>JByte,Integer=>JInt,Long=>JLong,Double=>JDouble,Short=>JShort,String=>JString,Float=>JFloat}
+import java.math.{BigDecimal=>JBigDecimal}
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.api.common.operators.Order
+import org.apache.calcite.rex.{RexLiteral, RexNode}
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.api.common.typeinfo.AtomicType
+import org.apache.flink.api.java.typeutils.runtime.RowComparator
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+import scala.collection.JavaConverters._
+
+/**
+ * Class represents a collection of helper methods to build the sort logic.
+ * It encapsulates as well the implementation for ordering and generic 
interfaces
+ */
+
--- End diff --

rm newline


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


[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116482718
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeSortProcessFunction.scala
 ---
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.api.common.state.{ ListState, ListStateDescriptor }
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.{RowTypeInfo, ListTypeInfo}
+import org.apache.flink.runtime.state.{ FunctionInitializationContext, 
FunctionSnapshotContext }
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.types.Row
+import org.apache.flink.util.{ Collector, Preconditions }
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import scala.util.control.Breaks._
+import org.apache.flink.api.java.tuple.{ Tuple2 => JTuple2 }
+import org.apache.flink.api.common.state.MapState
+import org.apache.flink.api.common.state.MapStateDescriptor
+import org.apache.flink.configuration.Configuration
+import java.util.Comparator
+import java.util.ArrayList
+import java.util.Collections
+import org.apache.flink.api.common.typeutils.TypeComparator
+import java.util.{List => JList, ArrayList => JArrayList}
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+/**
+ * Process Function used for the aggregate in bounded rowtime sort without 
offset/fetch
+ * [[org.apache.flink.streaming.api.datastream.DataStream]]
+ *
+ * @param fieldCount Is used to indicate fields in the current element to 
forward
+ * @param inputType It is used to mark the type of the incoming data
+ * @param rowComparator the [[java.util.Comparator]] is used for this sort 
aggregation
+ */
+class RowTimeSortProcessFunction(
+  private val fieldCount: Int,
+  private val inputRowType: CRowTypeInfo,
+  private val rowComparator: CollectionRowComparator)
+extends ProcessFunction[CRow, CRow] {
+
+  Preconditions.checkNotNull(rowComparator)
+
+  private val sortArray: ArrayList[Row] = new ArrayList[Row]
+  
+  // the state which keeps all the events that are not expired.
+  // Each timestamp will contain an associated list with the events 
+  // received at that timestamp
+  private var dataState: MapState[Long, JList[Row]] = _
+
+// the state which keeps the last triggering timestamp to filter late 
events
+  private var lastTriggeringTsState: ValueState[Long] = _
+  
+  private var outputC: CRow = _
+  
+  
+  override def open(config: Configuration) {
+ 
+val keyTypeInformation: TypeInformation[Long] =
+  BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]]
+val valueTypeInformation: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](
+inputRowType.asInstanceOf[CRowTypeInfo].rowType)
+
+val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]](
+"dataState",
+keyTypeInformation,
+valueTypeInformation)
+
+dataState = getRuntimeContext.getMapState(mapStateDescriptor)
+
+val lastTriggeringTsDescriptor: ValueStateDescriptor[Long] =
+  new ValueStateDescriptor[Long]("lastTriggeringTsState", 
classOf[Long])
+lastTriggeringTsState = 
getRuntimeContext.getState(lastTriggeringTsDescriptor)
+  }
+
+  
+  override def processElement(
+inputC: CRow,
+ctx: ProcessFunction[CRow, CRow]#Context,
+out: Collector[CRow]): Unit = {
+
+ val input = inputC.row
--- End diff --

indention by multiples of 2


---
If your project is set up for it, you can reply to this email and have your
reply appear on 

[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116521149
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala
 ---
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.types.Row
+import org.apache.calcite.rel.`type`._
+import org.apache.calcite.rel.RelCollation
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.table.functions.AggregateFunction
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.flink.table.api.TableException
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.calcite.sql.`type`.SqlTypeName._
+import java.util.{ List => JList, ArrayList }
+import org.apache.flink.api.common.typeinfo.{ SqlTimeTypeInfo, 
TypeInformation }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import java.sql.Timestamp
+import org.apache.calcite.rel.RelFieldCollation
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import java.util.Comparator
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import 
java.lang.{Byte=>JByte,Integer=>JInt,Long=>JLong,Double=>JDouble,Short=>JShort,String=>JString,Float=>JFloat}
+import java.math.{BigDecimal=>JBigDecimal}
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.api.common.operators.Order
+import org.apache.calcite.rex.{RexLiteral, RexNode}
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.api.common.typeinfo.AtomicType
+import org.apache.flink.api.java.typeutils.runtime.RowComparator
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+import scala.collection.JavaConverters._
+
+/**
+ * Class represents a collection of helper methods to build the sort logic.
+ * It encapsulates as well the implementation for ordering and generic 
interfaces
+ */
+
+object SortUtil {
+
+  
+  /**
+   * Function creates 
[org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
+   * elements based on rowtime and potentially other fields
+   * @param collationSort The Sort collation list
+   * @param inputType input row type
+   * @param execCfg table environment execution configuration
+   * @return org.apache.flink.streaming.api.functions.ProcessFunction
+   */
+  private[flink] def createRowTimeSortFunction(
+collationSort: RelCollation,
+inputType: RelDataType,
+inputTypeInfo: TypeInformation[Row],
+execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
+
+val keySortFields = getSortFieldIndexList(collationSort)
+val keySortDirections = getSortFieldDirectionList(collationSort)
+
+   //drop time from comparison as we sort on time in the states and 
result emission
+val keyIndexesNoTime = keySortFields.slice(1, keySortFields.size)
+val keyDirectionsNoTime = keySortDirections.slice(1, 
keySortDirections.size)
+val booleanOrderings = getSortFieldDirectionBooleanList(collationSort)
+val booleanDirectionsNoTime = booleanOrderings.slice(1, 
booleanOrderings.size)
+
+val fieldComps = createFieldComparators(inputType, 
+keyIndexesNoTime, keyDirectionsNoTime, execCfg)
+val fieldCompsRefs = 
fieldComps.asInstanceOf[Array[TypeComparator[AnyRef]]]
+
+val rowComp = createRowComparator(inputType,
+keyIndexesNoTime, fieldCompsRefs, booleanDirectionsNoTime)
+val collectionRowComparator = new CollectionRowComparator(rowComp)
+
+val inputCRowType = CRowTypeInfo(inputTypeInfo)
+ 
+new 

[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116483111
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeSortProcessFunction.scala
 ---
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.api.common.state.{ ListState, ListStateDescriptor }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.runtime.state.{ FunctionInitializationContext, 
FunctionSnapshotContext }
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.types.Row
+import org.apache.flink.util.{ Collector, Preconditions }
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import scala.util.control.Breaks._
+import org.apache.flink.api.java.tuple.{ Tuple2 => JTuple2 }
+import org.apache.flink.api.common.state.MapState
+import org.apache.flink.api.common.state.MapStateDescriptor
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import java.util.Comparator
+import java.util.ArrayList
+import java.util.Collections
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+/**
+ * Process Function used for the aggregate in bounded proctime sort 
without offset/fetch
+ * [[org.apache.flink.streaming.api.datastream.DataStream]]
+ *
+ * @param fieldCount Is used to indicate fields in the current element to 
forward
+ * @param inputType It is used to mark the type of the incoming data
+ * @param rowComparator the [[java.util.Comparator]] is used for this sort 
aggregation
+ */
+class ProcTimeSortProcessFunction(
+  private val fieldCount: Int,
+  private val inputRowType: CRowTypeInfo,
+  private val rowComparator: CollectionRowComparator)
+extends ProcessFunction[CRow, CRow] {
+
+  Preconditions.checkNotNull(rowComparator)
+
+  private var stateEventsBuffer: ListState[Row] = _
+  private val sortArray: ArrayList[Row] = new ArrayList[Row]
+  
+  private var outputC: CRow = _
+  
+  override def open(config: Configuration) {
+val sortDescriptor = new ListStateDescriptor[Row]("sortState",
+inputRowType.asInstanceOf[CRowTypeInfo].rowType)
+stateEventsBuffer = getRuntimeContext.getListState(sortDescriptor)
+
+  }
+
+  override def processElement(
+inputC: CRow,
+ctx: ProcessFunction[CRow, CRow]#Context,
+out: Collector[CRow]): Unit = {
+
+val input = inputC.row
+
+if( outputC == null) {
+  outputC = new CRow(input, true)
--- End diff --

instantiate in `open()`


> Support Limit/Top(Sort) for Stream SQL
> --
>
> Key: FLINK-6075
> URL: https://issues.apache.org/jira/browse/FLINK-6075
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: radu
>  Labels: features
> Attachments: sort.png
>
>
> These will be split in 3 separated JIRA issues. However, the design is the 
> same only the processing function differs in terms of the output. Hence, the 
> design is the same for all of them.
> Time target: Proc Time
> **SQL targeted query examples:**
> *Sort example*
> Q1)` SELECT a FROM stream1 GROUP BY HOP(proctime, INTERVAL '1' HOUR, INTERVAL 
> '3' HOUR) ORDER BY b` 
> Comment: window is defined using 

[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116478698
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeSortProcessFunction.scala
 ---
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.api.common.state.{ ListState, ListStateDescriptor }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.runtime.state.{ FunctionInitializationContext, 
FunctionSnapshotContext }
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.types.Row
+import org.apache.flink.util.{ Collector, Preconditions }
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import scala.util.control.Breaks._
+import org.apache.flink.api.java.tuple.{ Tuple2 => JTuple2 }
+import org.apache.flink.api.common.state.MapState
+import org.apache.flink.api.common.state.MapStateDescriptor
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import java.util.Comparator
+import java.util.ArrayList
+import java.util.Collections
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+/**
+ * Process Function used for the aggregate in bounded proctime sort 
without offset/fetch
+ * [[org.apache.flink.streaming.api.datastream.DataStream]]
+ *
+ * @param fieldCount Is used to indicate fields in the current element to 
forward
+ * @param inputType It is used to mark the type of the incoming data
+ * @param rowComparator the [[java.util.Comparator]] is used for this sort 
aggregation
+ */
+class ProcTimeSortProcessFunction(
+  private val fieldCount: Int,
--- End diff --

`fieldCount` is not used and can be removed


> Support Limit/Top(Sort) for Stream SQL
> --
>
> Key: FLINK-6075
> URL: https://issues.apache.org/jira/browse/FLINK-6075
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: radu
>  Labels: features
> Attachments: sort.png
>
>
> These will be split in 3 separated JIRA issues. However, the design is the 
> same only the processing function differs in terms of the output. Hence, the 
> design is the same for all of them.
> Time target: Proc Time
> **SQL targeted query examples:**
> *Sort example*
> Q1)` SELECT a FROM stream1 GROUP BY HOP(proctime, INTERVAL '1' HOUR, INTERVAL 
> '3' HOUR) ORDER BY b` 
> Comment: window is defined using GROUP BY
> Comment: ASC or DESC keywords can be placed to mark the ordering type
> *Limit example*
> Q2) `SELECT a FROM stream1 WHERE rowtime BETWEEN current_timestamp - INTERVAL 
> '1' HOUR AND current_timestamp ORDER BY b LIMIT 10`
> Comment: window is defined using time ranges in the WHERE clause
> Comment: window is row triggered
> *Top example*
> Q3) `SELECT sum(a) OVER (ORDER BY proctime RANGE INTERVAL '1' HOUR PRECEDING 
> LIMIT 10) FROM stream1`  
> Comment: limit over the contents of the sliding window
> General Comments:
> -All these SQL clauses are supported only over windows (bounded collections 
> of data). 
> -Each of the 3 operators will be supported with each of the types of 
> expressing the windows. 
> **Description**
> The 3 operations (limit, top and sort) are similar in behavior as they all 
> require a sorted collection of the data on which the logic will be applied 
> (i.e., select a subset of 

[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116496006
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala
 ---
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.types.Row
+import org.apache.calcite.rel.`type`._
+import org.apache.calcite.rel.RelCollation
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.table.functions.AggregateFunction
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.flink.table.api.TableException
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.calcite.sql.`type`.SqlTypeName._
+import java.util.{ List => JList, ArrayList }
+import org.apache.flink.api.common.typeinfo.{ SqlTimeTypeInfo, 
TypeInformation }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import java.sql.Timestamp
+import org.apache.calcite.rel.RelFieldCollation
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import java.util.Comparator
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import 
java.lang.{Byte=>JByte,Integer=>JInt,Long=>JLong,Double=>JDouble,Short=>JShort,String=>JString,Float=>JFloat}
+import java.math.{BigDecimal=>JBigDecimal}
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.api.common.operators.Order
+import org.apache.calcite.rex.{RexLiteral, RexNode}
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.api.common.typeinfo.AtomicType
+import org.apache.flink.api.java.typeutils.runtime.RowComparator
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+import scala.collection.JavaConverters._
+
+/**
+ * Class represents a collection of helper methods to build the sort logic.
+ * It encapsulates as well the implementation for ordering and generic 
interfaces
+ */
+
+object SortUtil {
+
+  
--- End diff --

rm newline


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


[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116540074
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SortTest.scala
 ---
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.api.scala.stream.sql
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.TableException
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.utils.TableTestUtil._
+import org.apache.flink.table.utils.{StreamTableTestUtil, TableTestBase}
+import org.junit.Test
+
+class SortTest extends TableTestBase {
+  private val streamUtil: StreamTableTestUtil = streamTestUtil()
+  streamUtil.addTable[(Int, String, Long)]("MyTable", 'a, 'b, 'c,
+  'proctime.proctime, 'rowtime.rowtime)
+  
+  @Test
+  def testSortProcessingTime() = {
+
+val sqlQuery = "SELECT a FROM MyTable ORDER BY proctime, c"
+  
+val expected =
+  unaryNode(
+"DataStreamCalc",
+unaryNode("DataStreamSort",
+  streamTableNode(0),
+  term("orderBy", "proctime ASC, c ASC], offset=[null], 
fetch=[unlimited")),
--- End diff --

The string seems to be broken. First there is a closing `]` and later an 
opening `[`


> Support Limit/Top(Sort) for Stream SQL
> --
>
> Key: FLINK-6075
> URL: https://issues.apache.org/jira/browse/FLINK-6075
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: radu
>  Labels: features
> Attachments: sort.png
>
>
> These will be split in 3 separated JIRA issues. However, the design is the 
> same only the processing function differs in terms of the output. Hence, the 
> design is the same for all of them.
> Time target: Proc Time
> **SQL targeted query examples:**
> *Sort example*
> Q1)` SELECT a FROM stream1 GROUP BY HOP(proctime, INTERVAL '1' HOUR, INTERVAL 
> '3' HOUR) ORDER BY b` 
> Comment: window is defined using GROUP BY
> Comment: ASC or DESC keywords can be placed to mark the ordering type
> *Limit example*
> Q2) `SELECT a FROM stream1 WHERE rowtime BETWEEN current_timestamp - INTERVAL 
> '1' HOUR AND current_timestamp ORDER BY b LIMIT 10`
> Comment: window is defined using time ranges in the WHERE clause
> Comment: window is row triggered
> *Top example*
> Q3) `SELECT sum(a) OVER (ORDER BY proctime RANGE INTERVAL '1' HOUR PRECEDING 
> LIMIT 10) FROM stream1`  
> Comment: limit over the contents of the sliding window
> General Comments:
> -All these SQL clauses are supported only over windows (bounded collections 
> of data). 
> -Each of the 3 operators will be supported with each of the types of 
> expressing the windows. 
> **Description**
> The 3 operations (limit, top and sort) are similar in behavior as they all 
> require a sorted collection of the data on which the logic will be applied 
> (i.e., select a subset of the items or the entire sorted set). These 
> functions would make sense in the streaming context only in the context of a 
> window. Without defining a window the functions could never emit as the sort 
> operation would never trigger. If an SQL query will be provided without 
> limits an error will be thrown (`SELECT a FROM stream1 TOP 10` -> ERROR). 
> Although not targeted by this JIRA, in the case of working based on event 
> time order, the retraction mechanisms of windows and the lateness mechanisms 
> can be used to deal with out of order events and retraction/updates of 
> results.
> **Functionality example**
> We exemplify with the query below for all the 3 types of operators (sorting, 
> limit and top). Rowtime indicates 

[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116480769
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeSortProcessFunction.scala
 ---
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.api.common.state.{ ListState, ListStateDescriptor }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.runtime.state.{ FunctionInitializationContext, 
FunctionSnapshotContext }
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.types.Row
+import org.apache.flink.util.{ Collector, Preconditions }
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import scala.util.control.Breaks._
+import org.apache.flink.api.java.tuple.{ Tuple2 => JTuple2 }
+import org.apache.flink.api.common.state.MapState
+import org.apache.flink.api.common.state.MapStateDescriptor
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import java.util.Comparator
+import java.util.ArrayList
+import java.util.Collections
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+/**
+ * Process Function used for the aggregate in bounded proctime sort 
without offset/fetch
+ * [[org.apache.flink.streaming.api.datastream.DataStream]]
+ *
+ * @param fieldCount Is used to indicate fields in the current element to 
forward
+ * @param inputType It is used to mark the type of the incoming data
+ * @param rowComparator the [[java.util.Comparator]] is used for this sort 
aggregation
+ */
+class ProcTimeSortProcessFunction(
+  private val fieldCount: Int,
+  private val inputRowType: CRowTypeInfo,
+  private val rowComparator: CollectionRowComparator)
+extends ProcessFunction[CRow, CRow] {
+
+  Preconditions.checkNotNull(rowComparator)
+
+  private var stateEventsBuffer: ListState[Row] = _
+  private val sortArray: ArrayList[Row] = new ArrayList[Row]
+  
+  private var outputC: CRow = _
+  
+  override def open(config: Configuration) {
+val sortDescriptor = new ListStateDescriptor[Row]("sortState",
+inputRowType.asInstanceOf[CRowTypeInfo].rowType)
+stateEventsBuffer = getRuntimeContext.getListState(sortDescriptor)
+
+  }
+
+  override def processElement(
+inputC: CRow,
+ctx: ProcessFunction[CRow, CRow]#Context,
+out: Collector[CRow]): Unit = {
+
+val input = inputC.row
+
+if( outputC == null) {
+  outputC = new CRow(input, true)
+}
+
+val currentTime = ctx.timerService.currentProcessingTime
+//buffer the event incoming event
+  
+//we accumulate the events as they arrive within the given proctime
+stateEventsBuffer.add(input)
+
+//deduplication of multiple registered timers is done automatically
+ctx.timerService.registerProcessingTimeTimer(currentTime + 1)  
+
+  }
+  
+  override def onTimer(
+timestamp: Long,
+ctx: ProcessFunction[CRow, CRow]#OnTimerContext,
+out: Collector[CRow]): Unit = {
+
+var i = 0
+val iter =  stateEventsBuffer.get.iterator()
+
+
+sortArray.clear()
+while(iter.hasNext()) {
+  sortArray.add(iter.next())
+}
+
+//if we do not rely on java 

[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116538474
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SortITCase.scala
 ---
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.stream.sql
+
+import org.apache.flink.api.scala._
+import 
org.apache.flink.table.api.scala.stream.sql.SortITCase.{EventTimeSourceFunction,StringRowSelectorSink}
+import org.apache.flink.streaming.api.functions.source.SourceFunction
+import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
+import org.apache.flink.table.api.TableEnvironment
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.api.scala.stream.utils.{StreamITCase, 
StreamTestData, StreamingWithStateTestBase}
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.types.Row
+import org.junit.Assert._
+import org.junit._
+import org.apache.flink.streaming.api.TimeCharacteristic
+import 
org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext
+import org.apache.flink.streaming.api.watermark.Watermark
+import scala.collection.mutable
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction
+
+class SortITCase extends StreamingWithStateTestBase {
+
+  @Test
+  def testEventTimeOrderBy(): Unit = {
+val data = Seq(
+  Left((1500L, (1L, 15, "Hello"))),
+  Left((1600L, (1L, 16, "Hello"))),
+  Left((1000L, (1L, 1, "Hello"))),
+  Left((2000L, (2L, 2, "Hello"))),
+  Right(1000L),
+  Left((2000L, (2L, 2, "Hello"))),
+  Left((2000L, (2L, 3, "Hello"))),
+  Left((3000L, (3L, 3, "Hello"))),
+  Right(2000L),
+  Left((4000L, (4L, 4, "Hello"))),
+  Right(3000L),
+  Left((5000L, (5L, 5, "Hello"))),
+  Right(5000L),
+  Left((6000L, (6L, 65, "Hello"))),
+  Left((6000L, (6L, 6, "Hello"))),
+  Right(7000L),
+  Left((9000L, (6L, 9, "Hello"))),
+  Left((8500L, (6L, 18, "Hello"))),
+  Left((9000L, (6L, 7, "Hello"))),
+  Right(1L),
+  Left((1L, (7L, 7, "Hello World"))),
+  Left((11000L, (7L, 77, "Hello World"))),
+  Left((11000L, (7L, 17, "Hello World"))),
+  Right(12000L),
+  Left((14000L, (7L, 18, "Hello World"))),
+  Right(14000L),
+  Left((15000L, (8L, 8, "Hello World"))),
+  Right(17000L),
+  Left((2L, (20L, 20, "Hello World"))), 
+  Right(19000L))
+
+val env = StreamExecutionEnvironment.getExecutionEnvironment
+env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+val tEnv = TableEnvironment.getTableEnvironment(env)
+StreamITCase.clear
+
+val t1 = env.addSource(new EventTimeSourceFunction[(Long, Int, 
String)](data))
+  .toTable(tEnv, 'a, 'b, 'c, 'rowtime.rowtime)
+  
+tEnv.registerTable("T1", t1)
+
+val  sqlQuery = "SELECT b FROM T1 " +
+  "ORDER BY rowtime, b ASC ";
+  
+  
+val result = tEnv.sql(sqlQuery).toDataStream[Row]
+result.addSink(new StringRowSelectorSink(0)).setParallelism(1)
+env.execute()
+
+val expected = mutable.MutableList(
+  "1", "15", "16",
+  "2", "2", "3",
+  "3",
+  "4",
+  "5",
+  "6", "65",
+  "18", "7", "9",
+  "7", "17", "77", 
+  "18",
+  "8",
+  "20")
+assertEquals(expected, SortITCase.testResults)
+  }
+}
+
+object SortITCase {
+
+  class EventTimeSourceFunction[T](
--- End diff --

Make the `EventTimeSourceFunction` in `OverWindowITCase` a util class and 
reuse it here instead of 

[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116470767
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSort.scala
 ---
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.plan.{ RelOptCluster, RelTraitSet }
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.{ RelNode, RelWriter, SingleRel }
+import org.apache.flink.api.java.tuple.Tuple
+import org.apache.flink.streaming.api.datastream.{ AllWindowedStream, 
DataStream, KeyedStream, WindowedStream }
+import org.apache.flink.streaming.api.windowing.assigners._
+import org.apache.flink.streaming.api.windowing.time.Time
+import org.apache.flink.streaming.api.windowing.windows.{ Window => 
DataStreamWindow }
+import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.expressions._
+import org.apache.flink.table.plan.logical._
+import org.apache.flink.table.plan.nodes.CommonAggregate
+import org.apache.flink.table.runtime.aggregate.AggregateUtil._
+import org.apache.flink.table.runtime.aggregate._
+import org.apache.flink.table.typeutils.TypeCheckUtils.isTimeInterval
+import org.apache.flink.table.typeutils.{ RowIntervalTypeInfo, 
TimeIntervalTypeInfo }
+import org.apache.flink.types.Row
+import org.apache.calcite.sql.SqlAggFunction
+import org.apache.flink.table.plan.nodes.datastream.DataStreamRel
+import org.apache.calcite.sql.fun.SqlSingleValueAggFunction
+import org.apache.flink.api.common.functions.RichMapFunction
+import org.apache.flink.api.common.functions.RichFlatMapFunction
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.util.Collector
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.calcite.rel.core.Sort
+import org.apache.flink.api.java.functions.NullByteKeySelector
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import org.apache.flink.table.runtime.aggregate.SortUtil._
+import org.apache.calcite.rel.RelCollation
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+import org.apache.flink.table.api.{StreamQueryConfig, 
StreamTableEnvironment, TableException}
+import org.apache.flink.table.plan.schema.RowSchema
+
+/**
+  * Flink RelNode which matches along with Sort Rule.
+  *
+ */
+class DataStreamSort(
+  sortCollation: RelCollation,
+  sortOffset: RexNode,
+  sortFetch: RexNode,
+  cluster: RelOptCluster,
+  traitSet: RelTraitSet,
+  inputNode: RelNode,
+  schema: RowSchema,
+  inputSchema: RowSchema,
+  description: String)
+extends SingleRel(cluster, traitSet, inputNode) with DataStreamRel {
+
+  override def deriveRowType(): RelDataType = schema.logicalType
+
+  override def copy(traitSet: RelTraitSet, inputs: 
java.util.List[RelNode]): RelNode = {
+new DataStreamSort(
+  sortCollation,
+  sortOffset,
+  sortFetch,
+  cluster,
+  traitSet,
+  inputs.get(0),
+  schema,
+  inputSchema,
+  description)
+  }
+
+  override def toString: String = {
+s"Sort(by: ($SortUtil.getSortFieldToString(sortCollation, 
rowRelDataType))," +
+  " offset: $SortUtil.getOffsetToString(sortOffset)," +
+  " fetch: $SortUtil.getFetchToString(sortFetch, sortOffset))"
+  }
+  
+  override def explainTerms(pw: RelWriter) : RelWriter = {
+
+//need to identify time between others order fields. Time needs to be 

[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116482152
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeSortProcessFunction.scala
 ---
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.api.common.state.{ ListState, ListStateDescriptor }
--- End diff --

remove unused imports


> Support Limit/Top(Sort) for Stream SQL
> --
>
> Key: FLINK-6075
> URL: https://issues.apache.org/jira/browse/FLINK-6075
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: radu
>  Labels: features
> Attachments: sort.png
>
>
> These will be split in 3 separated JIRA issues. However, the design is the 
> same only the processing function differs in terms of the output. Hence, the 
> design is the same for all of them.
> Time target: Proc Time
> **SQL targeted query examples:**
> *Sort example*
> Q1)` SELECT a FROM stream1 GROUP BY HOP(proctime, INTERVAL '1' HOUR, INTERVAL 
> '3' HOUR) ORDER BY b` 
> Comment: window is defined using GROUP BY
> Comment: ASC or DESC keywords can be placed to mark the ordering type
> *Limit example*
> Q2) `SELECT a FROM stream1 WHERE rowtime BETWEEN current_timestamp - INTERVAL 
> '1' HOUR AND current_timestamp ORDER BY b LIMIT 10`
> Comment: window is defined using time ranges in the WHERE clause
> Comment: window is row triggered
> *Top example*
> Q3) `SELECT sum(a) OVER (ORDER BY proctime RANGE INTERVAL '1' HOUR PRECEDING 
> LIMIT 10) FROM stream1`  
> Comment: limit over the contents of the sliding window
> General Comments:
> -All these SQL clauses are supported only over windows (bounded collections 
> of data). 
> -Each of the 3 operators will be supported with each of the types of 
> expressing the windows. 
> **Description**
> The 3 operations (limit, top and sort) are similar in behavior as they all 
> require a sorted collection of the data on which the logic will be applied 
> (i.e., select a subset of the items or the entire sorted set). These 
> functions would make sense in the streaming context only in the context of a 
> window. Without defining a window the functions could never emit as the sort 
> operation would never trigger. If an SQL query will be provided without 
> limits an error will be thrown (`SELECT a FROM stream1 TOP 10` -> ERROR). 
> Although not targeted by this JIRA, in the case of working based on event 
> time order, the retraction mechanisms of windows and the lateness mechanisms 
> can be used to deal with out of order events and retraction/updates of 
> results.
> **Functionality example**
> We exemplify with the query below for all the 3 types of operators (sorting, 
> limit and top). Rowtime indicates when the HOP window will trigger – which 
> can be observed in the fact that outputs are generated only at those moments. 
> The HOP windows will trigger at every hour (fixed hour) and each event will 
> contribute/ be duplicated for 2 consecutive hour intervals. Proctime 
> indicates the processing time when a new event arrives in the system. Events 
> are of the type (a,b) with the ordering being applied on the b field.
> `SELECT a FROM stream1 HOP(proctime, INTERVAL '1' HOUR, INTERVAL '2' HOUR) 
> ORDER BY b (LIMIT 2/ TOP 2 / [ASC/DESC] `)
> ||Rowtime||   Proctime||  Stream1||   Limit 2||   Top 2|| Sort 
> [ASC]||
> | |10:00:00  |(aaa, 11)   |   | | 
>|
> | |10:05:00|(aab, 7)  |   | ||
> |10-11  |11:00:00  |  |   aab,aaa |aab,aaa  | aab,aaa 
>|
> | 

[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116523262
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala
 ---
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.types.Row
+import org.apache.calcite.rel.`type`._
+import org.apache.calcite.rel.RelCollation
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.table.functions.AggregateFunction
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.flink.table.api.TableException
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.calcite.sql.`type`.SqlTypeName._
+import java.util.{ List => JList, ArrayList }
+import org.apache.flink.api.common.typeinfo.{ SqlTimeTypeInfo, 
TypeInformation }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import java.sql.Timestamp
+import org.apache.calcite.rel.RelFieldCollation
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import java.util.Comparator
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import 
java.lang.{Byte=>JByte,Integer=>JInt,Long=>JLong,Double=>JDouble,Short=>JShort,String=>JString,Float=>JFloat}
+import java.math.{BigDecimal=>JBigDecimal}
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.api.common.operators.Order
+import org.apache.calcite.rex.{RexLiteral, RexNode}
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.api.common.typeinfo.AtomicType
+import org.apache.flink.api.java.typeutils.runtime.RowComparator
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+import scala.collection.JavaConverters._
+
+/**
+ * Class represents a collection of helper methods to build the sort logic.
+ * It encapsulates as well the implementation for ordering and generic 
interfaces
+ */
+
+object SortUtil {
+
+  
+  /**
+   * Function creates 
[org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
+   * elements based on rowtime and potentially other fields
+   * @param collationSort The Sort collation list
+   * @param inputType input row type
+   * @param execCfg table environment execution configuration
+   * @return org.apache.flink.streaming.api.functions.ProcessFunction
+   */
+  private[flink] def createRowTimeSortFunction(
+collationSort: RelCollation,
+inputType: RelDataType,
+inputTypeInfo: TypeInformation[Row],
+execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
+
+val keySortFields = getSortFieldIndexList(collationSort)
+val keySortDirections = getSortFieldDirectionList(collationSort)
+
+   //drop time from comparison as we sort on time in the states and 
result emission
+val keyIndexesNoTime = keySortFields.slice(1, keySortFields.size)
+val keyDirectionsNoTime = keySortDirections.slice(1, 
keySortDirections.size)
+val booleanOrderings = getSortFieldDirectionBooleanList(collationSort)
+val booleanDirectionsNoTime = booleanOrderings.slice(1, 
booleanOrderings.size)
+
+val fieldComps = createFieldComparators(inputType, 
+keyIndexesNoTime, keyDirectionsNoTime, execCfg)
+val fieldCompsRefs = 
fieldComps.asInstanceOf[Array[TypeComparator[AnyRef]]]
+
+val rowComp = createRowComparator(inputType,
+   

[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116482838
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeSortProcessFunction.scala
 ---
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.api.common.state.{ ListState, ListStateDescriptor }
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.{RowTypeInfo, ListTypeInfo}
+import org.apache.flink.runtime.state.{ FunctionInitializationContext, 
FunctionSnapshotContext }
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.types.Row
+import org.apache.flink.util.{ Collector, Preconditions }
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import scala.util.control.Breaks._
+import org.apache.flink.api.java.tuple.{ Tuple2 => JTuple2 }
+import org.apache.flink.api.common.state.MapState
+import org.apache.flink.api.common.state.MapStateDescriptor
+import org.apache.flink.configuration.Configuration
+import java.util.Comparator
+import java.util.ArrayList
+import java.util.Collections
+import org.apache.flink.api.common.typeutils.TypeComparator
+import java.util.{List => JList, ArrayList => JArrayList}
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+/**
+ * Process Function used for the aggregate in bounded rowtime sort without 
offset/fetch
+ * [[org.apache.flink.streaming.api.datastream.DataStream]]
+ *
+ * @param fieldCount Is used to indicate fields in the current element to 
forward
+ * @param inputType It is used to mark the type of the incoming data
+ * @param rowComparator the [[java.util.Comparator]] is used for this sort 
aggregation
+ */
+class RowTimeSortProcessFunction(
+  private val fieldCount: Int,
+  private val inputRowType: CRowTypeInfo,
+  private val rowComparator: CollectionRowComparator)
+extends ProcessFunction[CRow, CRow] {
+
+  Preconditions.checkNotNull(rowComparator)
+
+  private val sortArray: ArrayList[Row] = new ArrayList[Row]
+  
+  // the state which keeps all the events that are not expired.
+  // Each timestamp will contain an associated list with the events 
+  // received at that timestamp
+  private var dataState: MapState[Long, JList[Row]] = _
+
+// the state which keeps the last triggering timestamp to filter late 
events
+  private var lastTriggeringTsState: ValueState[Long] = _
+  
+  private var outputC: CRow = _
+  
+  
+  override def open(config: Configuration) {
+ 
+val keyTypeInformation: TypeInformation[Long] =
+  BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]]
+val valueTypeInformation: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](
+inputRowType.asInstanceOf[CRowTypeInfo].rowType)
+
+val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]](
+"dataState",
+keyTypeInformation,
+valueTypeInformation)
+
+dataState = getRuntimeContext.getMapState(mapStateDescriptor)
+
+val lastTriggeringTsDescriptor: ValueStateDescriptor[Long] =
+  new ValueStateDescriptor[Long]("lastTriggeringTsState", 
classOf[Long])
+lastTriggeringTsState = 
getRuntimeContext.getState(lastTriggeringTsDescriptor)
+  }
+
+  
+  override def processElement(
+inputC: CRow,
+ctx: ProcessFunction[CRow, CRow]#Context,

[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116480397
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeSortProcessFunction.scala
 ---
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.api.common.state.{ ListState, ListStateDescriptor }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.runtime.state.{ FunctionInitializationContext, 
FunctionSnapshotContext }
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.types.Row
+import org.apache.flink.util.{ Collector, Preconditions }
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import scala.util.control.Breaks._
+import org.apache.flink.api.java.tuple.{ Tuple2 => JTuple2 }
+import org.apache.flink.api.common.state.MapState
+import org.apache.flink.api.common.state.MapStateDescriptor
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import java.util.Comparator
+import java.util.ArrayList
+import java.util.Collections
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+/**
+ * Process Function used for the aggregate in bounded proctime sort 
without offset/fetch
+ * [[org.apache.flink.streaming.api.datastream.DataStream]]
+ *
+ * @param fieldCount Is used to indicate fields in the current element to 
forward
+ * @param inputType It is used to mark the type of the incoming data
+ * @param rowComparator the [[java.util.Comparator]] is used for this sort 
aggregation
+ */
+class ProcTimeSortProcessFunction(
+  private val fieldCount: Int,
+  private val inputRowType: CRowTypeInfo,
+  private val rowComparator: CollectionRowComparator)
+extends ProcessFunction[CRow, CRow] {
+
+  Preconditions.checkNotNull(rowComparator)
+
+  private var stateEventsBuffer: ListState[Row] = _
+  private val sortArray: ArrayList[Row] = new ArrayList[Row]
+  
+  private var outputC: CRow = _
+  
+  override def open(config: Configuration) {
+val sortDescriptor = new ListStateDescriptor[Row]("sortState",
+inputRowType.asInstanceOf[CRowTypeInfo].rowType)
+stateEventsBuffer = getRuntimeContext.getListState(sortDescriptor)
+
+  }
+
+  override def processElement(
+inputC: CRow,
+ctx: ProcessFunction[CRow, CRow]#Context,
+out: Collector[CRow]): Unit = {
+
+val input = inputC.row
+
+if( outputC == null) {
+  outputC = new CRow(input, true)
+}
+
+val currentTime = ctx.timerService.currentProcessingTime
+//buffer the event incoming event
+  
+//we accumulate the events as they arrive within the given proctime
+stateEventsBuffer.add(input)
+
+//deduplication of multiple registered timers is done automatically
+ctx.timerService.registerProcessingTimeTimer(currentTime + 1)  
+
+  }
+  
+  override def onTimer(
+timestamp: Long,
+ctx: ProcessFunction[CRow, CRow]#OnTimerContext,
+out: Collector[CRow]): Unit = {
+
+var i = 0
+val iter =  stateEventsBuffer.get.iterator()
+
+
--- End diff --

remove second newline


> Support Limit/Top(Sort) for Stream SQL
> --
>
> Key: 

[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116481878
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeSortProcessFunction.scala
 ---
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.api.common.state.{ ListState, ListStateDescriptor }
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.{RowTypeInfo, ListTypeInfo}
+import org.apache.flink.runtime.state.{ FunctionInitializationContext, 
FunctionSnapshotContext }
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.types.Row
+import org.apache.flink.util.{ Collector, Preconditions }
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import scala.util.control.Breaks._
+import org.apache.flink.api.java.tuple.{ Tuple2 => JTuple2 }
+import org.apache.flink.api.common.state.MapState
+import org.apache.flink.api.common.state.MapStateDescriptor
+import org.apache.flink.configuration.Configuration
+import java.util.Comparator
+import java.util.ArrayList
+import java.util.Collections
+import org.apache.flink.api.common.typeutils.TypeComparator
+import java.util.{List => JList, ArrayList => JArrayList}
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+/**
+ * Process Function used for the aggregate in bounded rowtime sort without 
offset/fetch
+ * [[org.apache.flink.streaming.api.datastream.DataStream]]
+ *
+ * @param fieldCount Is used to indicate fields in the current element to 
forward
+ * @param inputType It is used to mark the type of the incoming data
+ * @param rowComparator the [[java.util.Comparator]] is used for this sort 
aggregation
+ */
+class RowTimeSortProcessFunction(
+  private val fieldCount: Int,
--- End diff --

`fieldCount` is not used and can be removed


> Support Limit/Top(Sort) for Stream SQL
> --
>
> Key: FLINK-6075
> URL: https://issues.apache.org/jira/browse/FLINK-6075
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: radu
>  Labels: features
> Attachments: sort.png
>
>
> These will be split in 3 separated JIRA issues. However, the design is the 
> same only the processing function differs in terms of the output. Hence, the 
> design is the same for all of them.
> Time target: Proc Time
> **SQL targeted query examples:**
> *Sort example*
> Q1)` SELECT a FROM stream1 GROUP BY HOP(proctime, INTERVAL '1' HOUR, INTERVAL 
> '3' HOUR) ORDER BY b` 
> Comment: window is defined using GROUP BY
> Comment: ASC or DESC keywords can be placed to mark the ordering type
> *Limit example*
> Q2) `SELECT a FROM stream1 WHERE rowtime BETWEEN current_timestamp - INTERVAL 
> '1' HOUR AND current_timestamp ORDER BY b LIMIT 10`
> Comment: window is defined using time ranges in the WHERE clause
> Comment: window is row triggered
> *Top example*
> Q3) `SELECT sum(a) OVER (ORDER BY proctime RANGE INTERVAL '1' HOUR PRECEDING 
> LIMIT 10) FROM stream1`  
> Comment: limit over the contents of the sliding window
> General Comments:
> -All these SQL clauses are supported only over windows (bounded collections 
> of data). 
> -Each of the 3 operators will be supported with each of the types of 
> expressing the windows. 
> **Description**
> The 3 operations (limit, top and sort) are similar in behavior as they all 
> require a sorted collection of the data on which the logic will be 

[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116455942
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSort.scala
 ---
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.plan.{ RelOptCluster, RelTraitSet }
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.{ RelNode, RelWriter, SingleRel }
+import org.apache.flink.api.java.tuple.Tuple
+import org.apache.flink.streaming.api.datastream.{ AllWindowedStream, 
DataStream, KeyedStream, WindowedStream }
+import org.apache.flink.streaming.api.windowing.assigners._
+import org.apache.flink.streaming.api.windowing.time.Time
+import org.apache.flink.streaming.api.windowing.windows.{ Window => 
DataStreamWindow }
+import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.expressions._
+import org.apache.flink.table.plan.logical._
+import org.apache.flink.table.plan.nodes.CommonAggregate
+import org.apache.flink.table.runtime.aggregate.AggregateUtil._
+import org.apache.flink.table.runtime.aggregate._
+import org.apache.flink.table.typeutils.TypeCheckUtils.isTimeInterval
+import org.apache.flink.table.typeutils.{ RowIntervalTypeInfo, 
TimeIntervalTypeInfo }
+import org.apache.flink.types.Row
+import org.apache.calcite.sql.SqlAggFunction
+import org.apache.flink.table.plan.nodes.datastream.DataStreamRel
+import org.apache.calcite.sql.fun.SqlSingleValueAggFunction
+import org.apache.flink.api.common.functions.RichMapFunction
+import org.apache.flink.api.common.functions.RichFlatMapFunction
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.util.Collector
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.calcite.rel.core.Sort
+import org.apache.flink.api.java.functions.NullByteKeySelector
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import org.apache.flink.table.runtime.aggregate.SortUtil._
+import org.apache.calcite.rel.RelCollation
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+import org.apache.flink.table.api.{StreamQueryConfig, 
StreamTableEnvironment, TableException}
+import org.apache.flink.table.plan.schema.RowSchema
+
+/**
+  * Flink RelNode which matches along with Sort Rule.
+  *
+ */
+class DataStreamSort(
+  sortCollation: RelCollation,
+  sortOffset: RexNode,
+  sortFetch: RexNode,
+  cluster: RelOptCluster,
+  traitSet: RelTraitSet,
+  inputNode: RelNode,
+  schema: RowSchema,
+  inputSchema: RowSchema,
+  description: String)
+extends SingleRel(cluster, traitSet, inputNode) with DataStreamRel {
+
+  override def deriveRowType(): RelDataType = schema.logicalType
+
+  override def copy(traitSet: RelTraitSet, inputs: 
java.util.List[RelNode]): RelNode = {
+new DataStreamSort(
+  sortCollation,
+  sortOffset,
+  sortFetch,
+  cluster,
+  traitSet,
+  inputs.get(0),
+  schema,
+  inputSchema,
+  description)
+  }
+
+  override def toString: String = {
+s"Sort(by: ($SortUtil.getSortFieldToString(sortCollation, 
rowRelDataType))," +
--- End diff --

Move the methods which are common 

[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116495740
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeSortProcessFunction.scala
 ---
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.api.common.state.{ ListState, ListStateDescriptor }
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.{RowTypeInfo, ListTypeInfo}
+import org.apache.flink.runtime.state.{ FunctionInitializationContext, 
FunctionSnapshotContext }
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.types.Row
+import org.apache.flink.util.{ Collector, Preconditions }
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import scala.util.control.Breaks._
+import org.apache.flink.api.java.tuple.{ Tuple2 => JTuple2 }
+import org.apache.flink.api.common.state.MapState
+import org.apache.flink.api.common.state.MapStateDescriptor
+import org.apache.flink.configuration.Configuration
+import java.util.Comparator
+import java.util.ArrayList
+import java.util.Collections
+import org.apache.flink.api.common.typeutils.TypeComparator
+import java.util.{List => JList, ArrayList => JArrayList}
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+/**
+ * Process Function used for the aggregate in bounded rowtime sort without 
offset/fetch
+ * [[org.apache.flink.streaming.api.datastream.DataStream]]
+ *
+ * @param fieldCount Is used to indicate fields in the current element to 
forward
+ * @param inputType It is used to mark the type of the incoming data
+ * @param rowComparator the [[java.util.Comparator]] is used for this sort 
aggregation
+ */
+class RowTimeSortProcessFunction(
+  private val fieldCount: Int,
+  private val inputRowType: CRowTypeInfo,
+  private val rowComparator: CollectionRowComparator)
+extends ProcessFunction[CRow, CRow] {
+
+  Preconditions.checkNotNull(rowComparator)
+
+  private val sortArray: ArrayList[Row] = new ArrayList[Row]
+  
+  // the state which keeps all the events that are not expired.
+  // Each timestamp will contain an associated list with the events 
+  // received at that timestamp
+  private var dataState: MapState[Long, JList[Row]] = _
+
+// the state which keeps the last triggering timestamp to filter late 
events
+  private var lastTriggeringTsState: ValueState[Long] = _
+  
+  private var outputC: CRow = _
+  
+  
+  override def open(config: Configuration) {
+ 
+val keyTypeInformation: TypeInformation[Long] =
+  BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]]
+val valueTypeInformation: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](
+inputRowType.asInstanceOf[CRowTypeInfo].rowType)
+
+val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]](
+"dataState",
+keyTypeInformation,
+valueTypeInformation)
+
+dataState = getRuntimeContext.getMapState(mapStateDescriptor)
+
+val lastTriggeringTsDescriptor: ValueStateDescriptor[Long] =
+  new ValueStateDescriptor[Long]("lastTriggeringTsState", 
classOf[Long])
+lastTriggeringTsState = 
getRuntimeContext.getState(lastTriggeringTsDescriptor)
+  }
+
+  
+  override def processElement(
+inputC: CRow,
+ctx: ProcessFunction[CRow, CRow]#Context,

[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116518779
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala
 ---
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.table.calcite.FlinkTypeFactory
--- End diff --

remove unused imports


> Support Limit/Top(Sort) for Stream SQL
> --
>
> Key: FLINK-6075
> URL: https://issues.apache.org/jira/browse/FLINK-6075
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: radu
>  Labels: features
> Attachments: sort.png
>
>
> These will be split in 3 separated JIRA issues. However, the design is the 
> same only the processing function differs in terms of the output. Hence, the 
> design is the same for all of them.
> Time target: Proc Time
> **SQL targeted query examples:**
> *Sort example*
> Q1)` SELECT a FROM stream1 GROUP BY HOP(proctime, INTERVAL '1' HOUR, INTERVAL 
> '3' HOUR) ORDER BY b` 
> Comment: window is defined using GROUP BY
> Comment: ASC or DESC keywords can be placed to mark the ordering type
> *Limit example*
> Q2) `SELECT a FROM stream1 WHERE rowtime BETWEEN current_timestamp - INTERVAL 
> '1' HOUR AND current_timestamp ORDER BY b LIMIT 10`
> Comment: window is defined using time ranges in the WHERE clause
> Comment: window is row triggered
> *Top example*
> Q3) `SELECT sum(a) OVER (ORDER BY proctime RANGE INTERVAL '1' HOUR PRECEDING 
> LIMIT 10) FROM stream1`  
> Comment: limit over the contents of the sliding window
> General Comments:
> -All these SQL clauses are supported only over windows (bounded collections 
> of data). 
> -Each of the 3 operators will be supported with each of the types of 
> expressing the windows. 
> **Description**
> The 3 operations (limit, top and sort) are similar in behavior as they all 
> require a sorted collection of the data on which the logic will be applied 
> (i.e., select a subset of the items or the entire sorted set). These 
> functions would make sense in the streaming context only in the context of a 
> window. Without defining a window the functions could never emit as the sort 
> operation would never trigger. If an SQL query will be provided without 
> limits an error will be thrown (`SELECT a FROM stream1 TOP 10` -> ERROR). 
> Although not targeted by this JIRA, in the case of working based on event 
> time order, the retraction mechanisms of windows and the lateness mechanisms 
> can be used to deal with out of order events and retraction/updates of 
> results.
> **Functionality example**
> We exemplify with the query below for all the 3 types of operators (sorting, 
> limit and top). Rowtime indicates when the HOP window will trigger – which 
> can be observed in the fact that outputs are generated only at those moments. 
> The HOP windows will trigger at every hour (fixed hour) and each event will 
> contribute/ be duplicated for 2 consecutive hour intervals. Proctime 
> indicates the processing time when a new event arrives in the system. Events 
> are of the type (a,b) with the ordering being applied on the b field.
> `SELECT a FROM stream1 HOP(proctime, INTERVAL '1' HOUR, INTERVAL '2' HOUR) 
> ORDER BY b (LIMIT 2/ TOP 2 / [ASC/DESC] `)
> ||Rowtime||   Proctime||  Stream1||   Limit 2||   Top 2|| Sort 
> [ASC]||
> | |10:00:00  |(aaa, 11)   |   | | 
>|
> | |10:05:00|(aab, 7)  |   | ||
> |10-11  |11:00:00  |  |   aab,aaa |aab,aaa  | aab,aaa 
>|
> | |11:03:00  |(aac,21)  |   | 

[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116483111
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeSortProcessFunction.scala
 ---
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.api.common.state.{ ListState, ListStateDescriptor }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.runtime.state.{ FunctionInitializationContext, 
FunctionSnapshotContext }
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.types.Row
+import org.apache.flink.util.{ Collector, Preconditions }
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import scala.util.control.Breaks._
+import org.apache.flink.api.java.tuple.{ Tuple2 => JTuple2 }
+import org.apache.flink.api.common.state.MapState
+import org.apache.flink.api.common.state.MapStateDescriptor
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import java.util.Comparator
+import java.util.ArrayList
+import java.util.Collections
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+/**
+ * Process Function used for the aggregate in bounded proctime sort 
without offset/fetch
+ * [[org.apache.flink.streaming.api.datastream.DataStream]]
+ *
+ * @param fieldCount Is used to indicate fields in the current element to 
forward
+ * @param inputType It is used to mark the type of the incoming data
+ * @param rowComparator the [[java.util.Comparator]] is used for this sort 
aggregation
+ */
+class ProcTimeSortProcessFunction(
+  private val fieldCount: Int,
+  private val inputRowType: CRowTypeInfo,
+  private val rowComparator: CollectionRowComparator)
+extends ProcessFunction[CRow, CRow] {
+
+  Preconditions.checkNotNull(rowComparator)
+
+  private var stateEventsBuffer: ListState[Row] = _
+  private val sortArray: ArrayList[Row] = new ArrayList[Row]
+  
+  private var outputC: CRow = _
+  
+  override def open(config: Configuration) {
+val sortDescriptor = new ListStateDescriptor[Row]("sortState",
+inputRowType.asInstanceOf[CRowTypeInfo].rowType)
+stateEventsBuffer = getRuntimeContext.getListState(sortDescriptor)
+
+  }
+
+  override def processElement(
+inputC: CRow,
+ctx: ProcessFunction[CRow, CRow]#Context,
+out: Collector[CRow]): Unit = {
+
+val input = inputC.row
+
+if( outputC == null) {
+  outputC = new CRow(input, true)
--- End diff --

instantiate in `open()`


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


[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116490564
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeSortProcessFunction.scala
 ---
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.api.common.state.{ ListState, ListStateDescriptor }
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.{RowTypeInfo, ListTypeInfo}
+import org.apache.flink.runtime.state.{ FunctionInitializationContext, 
FunctionSnapshotContext }
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.types.Row
+import org.apache.flink.util.{ Collector, Preconditions }
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import scala.util.control.Breaks._
+import org.apache.flink.api.java.tuple.{ Tuple2 => JTuple2 }
+import org.apache.flink.api.common.state.MapState
+import org.apache.flink.api.common.state.MapStateDescriptor
+import org.apache.flink.configuration.Configuration
+import java.util.Comparator
+import java.util.ArrayList
+import java.util.Collections
+import org.apache.flink.api.common.typeutils.TypeComparator
+import java.util.{List => JList, ArrayList => JArrayList}
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+/**
+ * Process Function used for the aggregate in bounded rowtime sort without 
offset/fetch
+ * [[org.apache.flink.streaming.api.datastream.DataStream]]
+ *
+ * @param fieldCount Is used to indicate fields in the current element to 
forward
+ * @param inputType It is used to mark the type of the incoming data
+ * @param rowComparator the [[java.util.Comparator]] is used for this sort 
aggregation
+ */
+class RowTimeSortProcessFunction(
+  private val fieldCount: Int,
+  private val inputRowType: CRowTypeInfo,
+  private val rowComparator: CollectionRowComparator)
+extends ProcessFunction[CRow, CRow] {
+
+  Preconditions.checkNotNull(rowComparator)
+
+  private val sortArray: ArrayList[Row] = new ArrayList[Row]
+  
+  // the state which keeps all the events that are not expired.
+  // Each timestamp will contain an associated list with the events 
+  // received at that timestamp
+  private var dataState: MapState[Long, JList[Row]] = _
+
+// the state which keeps the last triggering timestamp to filter late 
events
+  private var lastTriggeringTsState: ValueState[Long] = _
+  
+  private var outputC: CRow = _
+  
+  
+  override def open(config: Configuration) {
+ 
+val keyTypeInformation: TypeInformation[Long] =
+  BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]]
+val valueTypeInformation: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](
+inputRowType.asInstanceOf[CRowTypeInfo].rowType)
+
+val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]](
+"dataState",
+keyTypeInformation,
+valueTypeInformation)
+
+dataState = getRuntimeContext.getMapState(mapStateDescriptor)
+
+val lastTriggeringTsDescriptor: ValueStateDescriptor[Long] =
+  new ValueStateDescriptor[Long]("lastTriggeringTsState", 
classOf[Long])
+lastTriggeringTsState = 
getRuntimeContext.getState(lastTriggeringTsDescriptor)
+  }
+
+  
+  override def processElement(
+inputC: CRow,
+ctx: ProcessFunction[CRow, CRow]#Context,
+out: Collector[CRow]): Unit = {
+
+ val input = inputC.row
+
+ if( outputC == null) {
+  outputC = new CRow(input, true)
+}
+
+// triggering timestamp for trigger 

[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116490564
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeSortProcessFunction.scala
 ---
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.api.common.state.{ ListState, ListStateDescriptor }
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.{RowTypeInfo, ListTypeInfo}
+import org.apache.flink.runtime.state.{ FunctionInitializationContext, 
FunctionSnapshotContext }
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.types.Row
+import org.apache.flink.util.{ Collector, Preconditions }
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import scala.util.control.Breaks._
+import org.apache.flink.api.java.tuple.{ Tuple2 => JTuple2 }
+import org.apache.flink.api.common.state.MapState
+import org.apache.flink.api.common.state.MapStateDescriptor
+import org.apache.flink.configuration.Configuration
+import java.util.Comparator
+import java.util.ArrayList
+import java.util.Collections
+import org.apache.flink.api.common.typeutils.TypeComparator
+import java.util.{List => JList, ArrayList => JArrayList}
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+/**
+ * Process Function used for the aggregate in bounded rowtime sort without 
offset/fetch
+ * [[org.apache.flink.streaming.api.datastream.DataStream]]
+ *
+ * @param fieldCount Is used to indicate fields in the current element to 
forward
+ * @param inputType It is used to mark the type of the incoming data
+ * @param rowComparator the [[java.util.Comparator]] is used for this sort 
aggregation
+ */
+class RowTimeSortProcessFunction(
+  private val fieldCount: Int,
+  private val inputRowType: CRowTypeInfo,
+  private val rowComparator: CollectionRowComparator)
+extends ProcessFunction[CRow, CRow] {
+
+  Preconditions.checkNotNull(rowComparator)
+
+  private val sortArray: ArrayList[Row] = new ArrayList[Row]
+  
+  // the state which keeps all the events that are not expired.
+  // Each timestamp will contain an associated list with the events 
+  // received at that timestamp
+  private var dataState: MapState[Long, JList[Row]] = _
+
+// the state which keeps the last triggering timestamp to filter late 
events
+  private var lastTriggeringTsState: ValueState[Long] = _
+  
+  private var outputC: CRow = _
+  
+  
+  override def open(config: Configuration) {
+ 
+val keyTypeInformation: TypeInformation[Long] =
+  BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]]
+val valueTypeInformation: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](
+inputRowType.asInstanceOf[CRowTypeInfo].rowType)
+
+val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]](
+"dataState",
+keyTypeInformation,
+valueTypeInformation)
+
+dataState = getRuntimeContext.getMapState(mapStateDescriptor)
+
+val lastTriggeringTsDescriptor: ValueStateDescriptor[Long] =
+  new ValueStateDescriptor[Long]("lastTriggeringTsState", 
classOf[Long])
+lastTriggeringTsState = 
getRuntimeContext.getState(lastTriggeringTsDescriptor)
+  }
+
+  
+  override def processElement(
+inputC: CRow,
+ctx: ProcessFunction[CRow, CRow]#Context,

[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116468573
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala
 ---
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.types.Row
+import org.apache.calcite.rel.`type`._
+import org.apache.calcite.rel.RelCollation
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.table.functions.AggregateFunction
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.flink.table.api.TableException
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.calcite.sql.`type`.SqlTypeName._
+import java.util.{ List => JList, ArrayList }
+import org.apache.flink.api.common.typeinfo.{ SqlTimeTypeInfo, 
TypeInformation }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import java.sql.Timestamp
+import org.apache.calcite.rel.RelFieldCollation
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import java.util.Comparator
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import 
java.lang.{Byte=>JByte,Integer=>JInt,Long=>JLong,Double=>JDouble,Short=>JShort,String=>JString,Float=>JFloat}
+import java.math.{BigDecimal=>JBigDecimal}
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.api.common.operators.Order
+import org.apache.calcite.rex.{RexLiteral, RexNode}
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.api.common.typeinfo.AtomicType
+import org.apache.flink.api.java.typeutils.runtime.RowComparator
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+import scala.collection.JavaConverters._
+
+/**
+ * Class represents a collection of helper methods to build the sort logic.
+ * It encapsulates as well the implementation for ordering and generic 
interfaces
+ */
+
+object SortUtil {
+
+  
+  /**
+   * Function creates 
[org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
+   * elements based on rowtime and potentially other fields
+   * @param collationSort The Sort collation list
+   * @param inputType input row type
+   * @param execCfg table environment execution configuration
+   * @return org.apache.flink.streaming.api.functions.ProcessFunction
+   */
+  private[flink] def createRowTimeSortFunction(
+collationSort: RelCollation,
+inputType: RelDataType,
+inputTypeInfo: TypeInformation[Row],
+execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
+
+val keySortFields = getSortFieldIndexList(collationSort)
+val keySortDirections = getSortFieldDirectionList(collationSort)
+
+   //drop time from comparison as we sort on time in the states and 
result emission
+val keyIndexesNoTime = keySortFields.slice(1, keySortFields.size)
+val keyDirectionsNoTime = keySortDirections.slice(1, 
keySortDirections.size)
+val booleanOrderings = getSortFieldDirectionBooleanList(collationSort)
+val booleanDirectionsNoTime = booleanOrderings.slice(1, 
booleanOrderings.size)
+
+val fieldComps = createFieldComparators(inputType, 
+keyIndexesNoTime, keyDirectionsNoTime, execCfg)
+val fieldCompsRefs = 
fieldComps.asInstanceOf[Array[TypeComparator[AnyRef]]]
+
+val rowComp = createRowComparator(inputType,
+   

[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116524015
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala
 ---
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.types.Row
+import org.apache.calcite.rel.`type`._
+import org.apache.calcite.rel.RelCollation
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.table.functions.AggregateFunction
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.flink.table.api.TableException
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.calcite.sql.`type`.SqlTypeName._
+import java.util.{ List => JList, ArrayList }
+import org.apache.flink.api.common.typeinfo.{ SqlTimeTypeInfo, 
TypeInformation }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import java.sql.Timestamp
+import org.apache.calcite.rel.RelFieldCollation
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import java.util.Comparator
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import 
java.lang.{Byte=>JByte,Integer=>JInt,Long=>JLong,Double=>JDouble,Short=>JShort,String=>JString,Float=>JFloat}
+import java.math.{BigDecimal=>JBigDecimal}
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.api.common.operators.Order
+import org.apache.calcite.rex.{RexLiteral, RexNode}
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.api.common.typeinfo.AtomicType
+import org.apache.flink.api.java.typeutils.runtime.RowComparator
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+import scala.collection.JavaConverters._
+
+/**
+ * Class represents a collection of helper methods to build the sort logic.
+ * It encapsulates as well the implementation for ordering and generic 
interfaces
+ */
+
+object SortUtil {
+
+  
+  /**
+   * Function creates 
[org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
+   * elements based on rowtime and potentially other fields
+   * @param collationSort The Sort collation list
+   * @param inputType input row type
+   * @param execCfg table environment execution configuration
+   * @return org.apache.flink.streaming.api.functions.ProcessFunction
+   */
+  private[flink] def createRowTimeSortFunction(
+collationSort: RelCollation,
+inputType: RelDataType,
+inputTypeInfo: TypeInformation[Row],
+execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
+
+val keySortFields = getSortFieldIndexList(collationSort)
+val keySortDirections = getSortFieldDirectionList(collationSort)
+
+   //drop time from comparison as we sort on time in the states and 
result emission
+val keyIndexesNoTime = keySortFields.slice(1, keySortFields.size)
+val keyDirectionsNoTime = keySortDirections.slice(1, 
keySortDirections.size)
+val booleanOrderings = getSortFieldDirectionBooleanList(collationSort)
+val booleanDirectionsNoTime = booleanOrderings.slice(1, 
booleanOrderings.size)
+
+val fieldComps = createFieldComparators(inputType, 
+keyIndexesNoTime, keyDirectionsNoTime, execCfg)
+val fieldCompsRefs = 
fieldComps.asInstanceOf[Array[TypeComparator[AnyRef]]]
+
+val rowComp = createRowComparator(inputType,
+keyIndexesNoTime, fieldCompsRefs, booleanDirectionsNoTime)
+val collectionRowComparator = new CollectionRowComparator(rowComp)
+
+val inputCRowType = CRowTypeInfo(inputTypeInfo)
+ 
+new 

[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116526737
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala
 ---
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.types.Row
+import org.apache.calcite.rel.`type`._
+import org.apache.calcite.rel.RelCollation
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.table.functions.AggregateFunction
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.flink.table.api.TableException
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.calcite.sql.`type`.SqlTypeName._
+import java.util.{ List => JList, ArrayList }
+import org.apache.flink.api.common.typeinfo.{ SqlTimeTypeInfo, 
TypeInformation }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import java.sql.Timestamp
+import org.apache.calcite.rel.RelFieldCollation
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import java.util.Comparator
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import 
java.lang.{Byte=>JByte,Integer=>JInt,Long=>JLong,Double=>JDouble,Short=>JShort,String=>JString,Float=>JFloat}
+import java.math.{BigDecimal=>JBigDecimal}
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.api.common.operators.Order
+import org.apache.calcite.rex.{RexLiteral, RexNode}
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.api.common.typeinfo.AtomicType
+import org.apache.flink.api.java.typeutils.runtime.RowComparator
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+import scala.collection.JavaConverters._
+
+/**
+ * Class represents a collection of helper methods to build the sort logic.
+ * It encapsulates as well the implementation for ordering and generic 
interfaces
+ */
+
+object SortUtil {
+
+  
+  /**
+   * Function creates 
[org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
+   * elements based on rowtime and potentially other fields
+   * @param collationSort The Sort collation list
+   * @param inputType input row type
+   * @param execCfg table environment execution configuration
+   * @return org.apache.flink.streaming.api.functions.ProcessFunction
+   */
+  private[flink] def createRowTimeSortFunction(
+collationSort: RelCollation,
+inputType: RelDataType,
+inputTypeInfo: TypeInformation[Row],
+execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
+
+val keySortFields = getSortFieldIndexList(collationSort)
+val keySortDirections = getSortFieldDirectionList(collationSort)
+
+   //drop time from comparison as we sort on time in the states and 
result emission
+val keyIndexesNoTime = keySortFields.slice(1, keySortFields.size)
+val keyDirectionsNoTime = keySortDirections.slice(1, 
keySortDirections.size)
+val booleanOrderings = getSortFieldDirectionBooleanList(collationSort)
+val booleanDirectionsNoTime = booleanOrderings.slice(1, 
booleanOrderings.size)
+
+val fieldComps = createFieldComparators(inputType, 
+keyIndexesNoTime, keyDirectionsNoTime, execCfg)
+val fieldCompsRefs = 
fieldComps.asInstanceOf[Array[TypeComparator[AnyRef]]]
+
+val rowComp = createRowComparator(inputType,
+keyIndexesNoTime, fieldCompsRefs, booleanDirectionsNoTime)
+val collectionRowComparator = new CollectionRowComparator(rowComp)
+
+val inputCRowType = CRowTypeInfo(inputTypeInfo)
+ 
+new 

[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116482838
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeSortProcessFunction.scala
 ---
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.api.common.state.{ ListState, ListStateDescriptor }
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.{RowTypeInfo, ListTypeInfo}
+import org.apache.flink.runtime.state.{ FunctionInitializationContext, 
FunctionSnapshotContext }
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.types.Row
+import org.apache.flink.util.{ Collector, Preconditions }
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import scala.util.control.Breaks._
+import org.apache.flink.api.java.tuple.{ Tuple2 => JTuple2 }
+import org.apache.flink.api.common.state.MapState
+import org.apache.flink.api.common.state.MapStateDescriptor
+import org.apache.flink.configuration.Configuration
+import java.util.Comparator
+import java.util.ArrayList
+import java.util.Collections
+import org.apache.flink.api.common.typeutils.TypeComparator
+import java.util.{List => JList, ArrayList => JArrayList}
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+/**
+ * Process Function used for the aggregate in bounded rowtime sort without 
offset/fetch
+ * [[org.apache.flink.streaming.api.datastream.DataStream]]
+ *
+ * @param fieldCount Is used to indicate fields in the current element to 
forward
+ * @param inputType It is used to mark the type of the incoming data
+ * @param rowComparator the [[java.util.Comparator]] is used for this sort 
aggregation
+ */
+class RowTimeSortProcessFunction(
+  private val fieldCount: Int,
+  private val inputRowType: CRowTypeInfo,
+  private val rowComparator: CollectionRowComparator)
+extends ProcessFunction[CRow, CRow] {
+
+  Preconditions.checkNotNull(rowComparator)
+
+  private val sortArray: ArrayList[Row] = new ArrayList[Row]
+  
+  // the state which keeps all the events that are not expired.
+  // Each timestamp will contain an associated list with the events 
+  // received at that timestamp
+  private var dataState: MapState[Long, JList[Row]] = _
+
+// the state which keeps the last triggering timestamp to filter late 
events
+  private var lastTriggeringTsState: ValueState[Long] = _
+  
+  private var outputC: CRow = _
+  
+  
+  override def open(config: Configuration) {
+ 
+val keyTypeInformation: TypeInformation[Long] =
+  BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]]
+val valueTypeInformation: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](
+inputRowType.asInstanceOf[CRowTypeInfo].rowType)
+
+val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]](
+"dataState",
+keyTypeInformation,
+valueTypeInformation)
+
+dataState = getRuntimeContext.getMapState(mapStateDescriptor)
+
+val lastTriggeringTsDescriptor: ValueStateDescriptor[Long] =
+  new ValueStateDescriptor[Long]("lastTriggeringTsState", 
classOf[Long])
+lastTriggeringTsState = 
getRuntimeContext.getState(lastTriggeringTsDescriptor)
+  }
+
+  
+  override def processElement(
+inputC: CRow,
+ctx: ProcessFunction[CRow, CRow]#Context,
+out: Collector[CRow]): Unit = {
+
+ val input = inputC.row
+
+ if( outputC == null) {
--- End diff --

spaces: `if (outputC == null) {`


---
If your project is set up for it, you can reply 

[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116518779
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala
 ---
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.table.calcite.FlinkTypeFactory
--- End diff --

remove unused imports


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


[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116482324
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeSortProcessFunction.scala
 ---
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.api.common.state.{ ListState, ListStateDescriptor }
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.{RowTypeInfo, ListTypeInfo}
+import org.apache.flink.runtime.state.{ FunctionInitializationContext, 
FunctionSnapshotContext }
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.types.Row
+import org.apache.flink.util.{ Collector, Preconditions }
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import scala.util.control.Breaks._
+import org.apache.flink.api.java.tuple.{ Tuple2 => JTuple2 }
+import org.apache.flink.api.common.state.MapState
+import org.apache.flink.api.common.state.MapStateDescriptor
+import org.apache.flink.configuration.Configuration
+import java.util.Comparator
+import java.util.ArrayList
+import java.util.Collections
+import org.apache.flink.api.common.typeutils.TypeComparator
+import java.util.{List => JList, ArrayList => JArrayList}
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+/**
+ * Process Function used for the aggregate in bounded rowtime sort without 
offset/fetch
+ * [[org.apache.flink.streaming.api.datastream.DataStream]]
+ *
+ * @param fieldCount Is used to indicate fields in the current element to 
forward
+ * @param inputType It is used to mark the type of the incoming data
+ * @param rowComparator the [[java.util.Comparator]] is used for this sort 
aggregation
+ */
+class RowTimeSortProcessFunction(
+  private val fieldCount: Int,
+  private val inputRowType: CRowTypeInfo,
+  private val rowComparator: CollectionRowComparator)
+extends ProcessFunction[CRow, CRow] {
+
+  Preconditions.checkNotNull(rowComparator)
+
+  private val sortArray: ArrayList[Row] = new ArrayList[Row]
+  
+  // the state which keeps all the events that are not expired.
+  // Each timestamp will contain an associated list with the events 
+  // received at that timestamp
+  private var dataState: MapState[Long, JList[Row]] = _
+
+// the state which keeps the last triggering timestamp to filter late 
events
+  private var lastTriggeringTsState: ValueState[Long] = _
+  
+  private var outputC: CRow = _
+  
+  
--- End diff --

remove newline


> Support Limit/Top(Sort) for Stream SQL
> --
>
> Key: FLINK-6075
> URL: https://issues.apache.org/jira/browse/FLINK-6075
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: radu
>  Labels: features
> Attachments: sort.png
>
>
> These will be split in 3 separated JIRA issues. However, the design is the 
> same only the processing function differs in terms of the output. Hence, the 
> design is the same for all of them.
> Time target: Proc Time
> **SQL targeted query examples:**
> *Sort example*
> Q1)` SELECT a FROM stream1 GROUP BY HOP(proctime, INTERVAL '1' HOUR, INTERVAL 
> '3' HOUR) ORDER BY b` 
> Comment: window is defined using GROUP BY
> Comment: ASC or DESC keywords can be placed to mark the ordering type
> *Limit example*
> Q2) `SELECT a FROM stream1 WHERE rowtime BETWEEN current_timestamp - INTERVAL 
> 

[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116475658
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSort.scala
 ---
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.plan.{ RelOptCluster, RelTraitSet }
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.{ RelNode, RelWriter, SingleRel }
+import org.apache.flink.api.java.tuple.Tuple
+import org.apache.flink.streaming.api.datastream.{ AllWindowedStream, 
DataStream, KeyedStream, WindowedStream }
+import org.apache.flink.streaming.api.windowing.assigners._
+import org.apache.flink.streaming.api.windowing.time.Time
+import org.apache.flink.streaming.api.windowing.windows.{ Window => 
DataStreamWindow }
+import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.expressions._
+import org.apache.flink.table.plan.logical._
+import org.apache.flink.table.plan.nodes.CommonAggregate
+import org.apache.flink.table.runtime.aggregate.AggregateUtil._
+import org.apache.flink.table.runtime.aggregate._
+import org.apache.flink.table.typeutils.TypeCheckUtils.isTimeInterval
+import org.apache.flink.table.typeutils.{ RowIntervalTypeInfo, 
TimeIntervalTypeInfo }
+import org.apache.flink.types.Row
+import org.apache.calcite.sql.SqlAggFunction
+import org.apache.flink.table.plan.nodes.datastream.DataStreamRel
+import org.apache.calcite.sql.fun.SqlSingleValueAggFunction
+import org.apache.flink.api.common.functions.RichMapFunction
+import org.apache.flink.api.common.functions.RichFlatMapFunction
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.util.Collector
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.calcite.rel.core.Sort
+import org.apache.flink.api.java.functions.NullByteKeySelector
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import org.apache.flink.table.runtime.aggregate.SortUtil._
+import org.apache.calcite.rel.RelCollation
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+import org.apache.flink.table.api.{StreamQueryConfig, 
StreamTableEnvironment, TableException}
+import org.apache.flink.table.plan.schema.RowSchema
+
+/**
+  * Flink RelNode which matches along with Sort Rule.
+  *
+ */
+class DataStreamSort(
+  sortCollation: RelCollation,
+  sortOffset: RexNode,
+  sortFetch: RexNode,
+  cluster: RelOptCluster,
+  traitSet: RelTraitSet,
+  inputNode: RelNode,
+  schema: RowSchema,
+  inputSchema: RowSchema,
+  description: String)
+extends SingleRel(cluster, traitSet, inputNode) with DataStreamRel {
+
+  override def deriveRowType(): RelDataType = schema.logicalType
+
+  override def copy(traitSet: RelTraitSet, inputs: 
java.util.List[RelNode]): RelNode = {
+new DataStreamSort(
+  sortCollation,
+  sortOffset,
+  sortFetch,
+  cluster,
+  traitSet,
+  inputs.get(0),
+  schema,
+  inputSchema,
+  description)
+  }
+
+  override def toString: String = {
+s"Sort(by: ($SortUtil.getSortFieldToString(sortCollation, 
rowRelDataType))," +
+  " offset: $SortUtil.getOffsetToString(sortOffset)," +
+  " fetch: $SortUtil.getFetchToString(sortFetch, sortOffset))"
+  }
+  
+  override def explainTerms(pw: RelWriter) : RelWriter = {
+
+//need to identify time between others order fields. Time needs to be 

[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116526119
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala
 ---
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.types.Row
+import org.apache.calcite.rel.`type`._
+import org.apache.calcite.rel.RelCollation
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.table.functions.AggregateFunction
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.flink.table.api.TableException
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.calcite.sql.`type`.SqlTypeName._
+import java.util.{ List => JList, ArrayList }
+import org.apache.flink.api.common.typeinfo.{ SqlTimeTypeInfo, 
TypeInformation }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import java.sql.Timestamp
+import org.apache.calcite.rel.RelFieldCollation
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import java.util.Comparator
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import 
java.lang.{Byte=>JByte,Integer=>JInt,Long=>JLong,Double=>JDouble,Short=>JShort,String=>JString,Float=>JFloat}
+import java.math.{BigDecimal=>JBigDecimal}
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.api.common.operators.Order
+import org.apache.calcite.rex.{RexLiteral, RexNode}
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.api.common.typeinfo.AtomicType
+import org.apache.flink.api.java.typeutils.runtime.RowComparator
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+import scala.collection.JavaConverters._
+
+/**
+ * Class represents a collection of helper methods to build the sort logic.
+ * It encapsulates as well the implementation for ordering and generic 
interfaces
+ */
+
+object SortUtil {
+
+  
+  /**
+   * Function creates 
[org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
+   * elements based on rowtime and potentially other fields
+   * @param collationSort The Sort collation list
+   * @param inputType input row type
+   * @param execCfg table environment execution configuration
+   * @return org.apache.flink.streaming.api.functions.ProcessFunction
+   */
+  private[flink] def createRowTimeSortFunction(
+collationSort: RelCollation,
+inputType: RelDataType,
+inputTypeInfo: TypeInformation[Row],
+execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
+
+val keySortFields = getSortFieldIndexList(collationSort)
+val keySortDirections = getSortFieldDirectionList(collationSort)
+
+   //drop time from comparison as we sort on time in the states and 
result emission
+val keyIndexesNoTime = keySortFields.slice(1, keySortFields.size)
+val keyDirectionsNoTime = keySortDirections.slice(1, 
keySortDirections.size)
+val booleanOrderings = getSortFieldDirectionBooleanList(collationSort)
+val booleanDirectionsNoTime = booleanOrderings.slice(1, 
booleanOrderings.size)
+
+val fieldComps = createFieldComparators(inputType, 
+keyIndexesNoTime, keyDirectionsNoTime, execCfg)
+val fieldCompsRefs = 
fieldComps.asInstanceOf[Array[TypeComparator[AnyRef]]]
+
+val rowComp = createRowComparator(inputType,
+   

[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116520149
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala
 ---
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.types.Row
+import org.apache.calcite.rel.`type`._
+import org.apache.calcite.rel.RelCollation
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.table.functions.AggregateFunction
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.flink.table.api.TableException
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.calcite.sql.`type`.SqlTypeName._
+import java.util.{ List => JList, ArrayList }
+import org.apache.flink.api.common.typeinfo.{ SqlTimeTypeInfo, 
TypeInformation }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import java.sql.Timestamp
+import org.apache.calcite.rel.RelFieldCollation
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import java.util.Comparator
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import 
java.lang.{Byte=>JByte,Integer=>JInt,Long=>JLong,Double=>JDouble,Short=>JShort,String=>JString,Float=>JFloat}
+import java.math.{BigDecimal=>JBigDecimal}
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.api.common.operators.Order
+import org.apache.calcite.rex.{RexLiteral, RexNode}
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.api.common.typeinfo.AtomicType
+import org.apache.flink.api.java.typeutils.runtime.RowComparator
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+import scala.collection.JavaConverters._
+
+/**
+ * Class represents a collection of helper methods to build the sort logic.
+ * It encapsulates as well the implementation for ordering and generic 
interfaces
+ */
+
+object SortUtil {
+
+  
+  /**
+   * Function creates 
[org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
+   * elements based on rowtime and potentially other fields
+   * @param collationSort The Sort collation list
+   * @param inputType input row type
+   * @param execCfg table environment execution configuration
+   * @return org.apache.flink.streaming.api.functions.ProcessFunction
+   */
+  private[flink] def createRowTimeSortFunction(
+collationSort: RelCollation,
+inputType: RelDataType,
+inputTypeInfo: TypeInformation[Row],
+execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
+
+val keySortFields = getSortFieldIndexList(collationSort)
+val keySortDirections = getSortFieldDirectionList(collationSort)
--- End diff --

is not required. We can use `booleanOrdering` instead.


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


[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116521769
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala
 ---
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.types.Row
+import org.apache.calcite.rel.`type`._
+import org.apache.calcite.rel.RelCollation
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.table.functions.AggregateFunction
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.flink.table.api.TableException
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.calcite.sql.`type`.SqlTypeName._
+import java.util.{ List => JList, ArrayList }
+import org.apache.flink.api.common.typeinfo.{ SqlTimeTypeInfo, 
TypeInformation }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import java.sql.Timestamp
+import org.apache.calcite.rel.RelFieldCollation
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import java.util.Comparator
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import 
java.lang.{Byte=>JByte,Integer=>JInt,Long=>JLong,Double=>JDouble,Short=>JShort,String=>JString,Float=>JFloat}
+import java.math.{BigDecimal=>JBigDecimal}
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.api.common.operators.Order
+import org.apache.calcite.rex.{RexLiteral, RexNode}
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.api.common.typeinfo.AtomicType
+import org.apache.flink.api.java.typeutils.runtime.RowComparator
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+import scala.collection.JavaConverters._
+
+/**
+ * Class represents a collection of helper methods to build the sort logic.
+ * It encapsulates as well the implementation for ordering and generic 
interfaces
+ */
+
+object SortUtil {
+
+  
+  /**
+   * Function creates 
[org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
+   * elements based on rowtime and potentially other fields
+   * @param collationSort The Sort collation list
+   * @param inputType input row type
+   * @param execCfg table environment execution configuration
+   * @return org.apache.flink.streaming.api.functions.ProcessFunction
+   */
+  private[flink] def createRowTimeSortFunction(
+collationSort: RelCollation,
+inputType: RelDataType,
+inputTypeInfo: TypeInformation[Row],
+execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
+
+val keySortFields = getSortFieldIndexList(collationSort)
+val keySortDirections = getSortFieldDirectionList(collationSort)
+
+   //drop time from comparison as we sort on time in the states and 
result emission
+val keyIndexesNoTime = keySortFields.slice(1, keySortFields.size)
+val keyDirectionsNoTime = keySortDirections.slice(1, 
keySortDirections.size)
+val booleanOrderings = getSortFieldDirectionBooleanList(collationSort)
+val booleanDirectionsNoTime = booleanOrderings.slice(1, 
booleanOrderings.size)
+
+val fieldComps = createFieldComparators(inputType, 
--- End diff --

Please wrap method arguments as described above.


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


[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116482918
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeSortProcessFunction.scala
 ---
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.api.common.state.{ ListState, ListStateDescriptor }
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.{RowTypeInfo, ListTypeInfo}
+import org.apache.flink.runtime.state.{ FunctionInitializationContext, 
FunctionSnapshotContext }
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.types.Row
+import org.apache.flink.util.{ Collector, Preconditions }
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import scala.util.control.Breaks._
+import org.apache.flink.api.java.tuple.{ Tuple2 => JTuple2 }
+import org.apache.flink.api.common.state.MapState
+import org.apache.flink.api.common.state.MapStateDescriptor
+import org.apache.flink.configuration.Configuration
+import java.util.Comparator
+import java.util.ArrayList
+import java.util.Collections
+import org.apache.flink.api.common.typeutils.TypeComparator
+import java.util.{List => JList, ArrayList => JArrayList}
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+/**
+ * Process Function used for the aggregate in bounded rowtime sort without 
offset/fetch
+ * [[org.apache.flink.streaming.api.datastream.DataStream]]
+ *
+ * @param fieldCount Is used to indicate fields in the current element to 
forward
+ * @param inputType It is used to mark the type of the incoming data
+ * @param rowComparator the [[java.util.Comparator]] is used for this sort 
aggregation
+ */
+class RowTimeSortProcessFunction(
+  private val fieldCount: Int,
+  private val inputRowType: CRowTypeInfo,
+  private val rowComparator: CollectionRowComparator)
+extends ProcessFunction[CRow, CRow] {
+
+  Preconditions.checkNotNull(rowComparator)
+
+  private val sortArray: ArrayList[Row] = new ArrayList[Row]
+  
+  // the state which keeps all the events that are not expired.
+  // Each timestamp will contain an associated list with the events 
+  // received at that timestamp
+  private var dataState: MapState[Long, JList[Row]] = _
+
+// the state which keeps the last triggering timestamp to filter late 
events
+  private var lastTriggeringTsState: ValueState[Long] = _
+  
+  private var outputC: CRow = _
+  
+  
+  override def open(config: Configuration) {
+ 
+val keyTypeInformation: TypeInformation[Long] =
+  BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]]
+val valueTypeInformation: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](
+inputRowType.asInstanceOf[CRowTypeInfo].rowType)
+
+val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]](
+"dataState",
+keyTypeInformation,
+valueTypeInformation)
+
+dataState = getRuntimeContext.getMapState(mapStateDescriptor)
+
+val lastTriggeringTsDescriptor: ValueStateDescriptor[Long] =
+  new ValueStateDescriptor[Long]("lastTriggeringTsState", 
classOf[Long])
+lastTriggeringTsState = 
getRuntimeContext.getState(lastTriggeringTsDescriptor)
+  }
+
+  
+  override def processElement(
+inputC: CRow,
+ctx: ProcessFunction[CRow, CRow]#Context,

[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116524015
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala
 ---
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.types.Row
+import org.apache.calcite.rel.`type`._
+import org.apache.calcite.rel.RelCollation
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.table.functions.AggregateFunction
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.flink.table.api.TableException
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.calcite.sql.`type`.SqlTypeName._
+import java.util.{ List => JList, ArrayList }
+import org.apache.flink.api.common.typeinfo.{ SqlTimeTypeInfo, 
TypeInformation }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import java.sql.Timestamp
+import org.apache.calcite.rel.RelFieldCollation
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import java.util.Comparator
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import 
java.lang.{Byte=>JByte,Integer=>JInt,Long=>JLong,Double=>JDouble,Short=>JShort,String=>JString,Float=>JFloat}
+import java.math.{BigDecimal=>JBigDecimal}
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.api.common.operators.Order
+import org.apache.calcite.rex.{RexLiteral, RexNode}
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.api.common.typeinfo.AtomicType
+import org.apache.flink.api.java.typeutils.runtime.RowComparator
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+import scala.collection.JavaConverters._
+
+/**
+ * Class represents a collection of helper methods to build the sort logic.
+ * It encapsulates as well the implementation for ordering and generic 
interfaces
+ */
+
+object SortUtil {
+
+  
+  /**
+   * Function creates 
[org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
+   * elements based on rowtime and potentially other fields
+   * @param collationSort The Sort collation list
+   * @param inputType input row type
+   * @param execCfg table environment execution configuration
+   * @return org.apache.flink.streaming.api.functions.ProcessFunction
+   */
+  private[flink] def createRowTimeSortFunction(
+collationSort: RelCollation,
+inputType: RelDataType,
+inputTypeInfo: TypeInformation[Row],
+execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
+
+val keySortFields = getSortFieldIndexList(collationSort)
+val keySortDirections = getSortFieldDirectionList(collationSort)
+
+   //drop time from comparison as we sort on time in the states and 
result emission
+val keyIndexesNoTime = keySortFields.slice(1, keySortFields.size)
+val keyDirectionsNoTime = keySortDirections.slice(1, 
keySortDirections.size)
+val booleanOrderings = getSortFieldDirectionBooleanList(collationSort)
+val booleanDirectionsNoTime = booleanOrderings.slice(1, 
booleanOrderings.size)
+
+val fieldComps = createFieldComparators(inputType, 
+keyIndexesNoTime, keyDirectionsNoTime, execCfg)
+val fieldCompsRefs = 
fieldComps.asInstanceOf[Array[TypeComparator[AnyRef]]]
+
+val rowComp = createRowComparator(inputType,
+   

[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116455372
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSort.scala
 ---
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.plan.{ RelOptCluster, RelTraitSet }
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.{ RelNode, RelWriter, SingleRel }
+import org.apache.flink.api.java.tuple.Tuple
+import org.apache.flink.streaming.api.datastream.{ AllWindowedStream, 
DataStream, KeyedStream, WindowedStream }
+import org.apache.flink.streaming.api.windowing.assigners._
+import org.apache.flink.streaming.api.windowing.time.Time
+import org.apache.flink.streaming.api.windowing.windows.{ Window => 
DataStreamWindow }
+import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.expressions._
+import org.apache.flink.table.plan.logical._
+import org.apache.flink.table.plan.nodes.CommonAggregate
+import org.apache.flink.table.runtime.aggregate.AggregateUtil._
+import org.apache.flink.table.runtime.aggregate._
+import org.apache.flink.table.typeutils.TypeCheckUtils.isTimeInterval
+import org.apache.flink.table.typeutils.{ RowIntervalTypeInfo, 
TimeIntervalTypeInfo }
+import org.apache.flink.types.Row
+import org.apache.calcite.sql.SqlAggFunction
+import org.apache.flink.table.plan.nodes.datastream.DataStreamRel
+import org.apache.calcite.sql.fun.SqlSingleValueAggFunction
+import org.apache.flink.api.common.functions.RichMapFunction
+import org.apache.flink.api.common.functions.RichFlatMapFunction
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.util.Collector
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.calcite.rel.core.Sort
+import org.apache.flink.api.java.functions.NullByteKeySelector
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import org.apache.flink.table.runtime.aggregate.SortUtil._
+import org.apache.calcite.rel.RelCollation
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+import org.apache.flink.table.api.{StreamQueryConfig, 
StreamTableEnvironment, TableException}
+import org.apache.flink.table.plan.schema.RowSchema
+
+/**
+  * Flink RelNode which matches along with Sort Rule.
+  *
+ */
+class DataStreamSort(
+  sortCollation: RelCollation,
+  sortOffset: RexNode,
+  sortFetch: RexNode,
+  cluster: RelOptCluster,
--- End diff --

Please keep the order of parameters consistent to other classes (for 
instance `DataStreamCalc`), i.e., cluster, traitset, input, inputschema, 
schema, sortCollation, offset, fetch, description.


> Support Limit/Top(Sort) for Stream SQL
> --
>
> Key: FLINK-6075
> URL: https://issues.apache.org/jira/browse/FLINK-6075
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: radu
>  Labels: features
> Attachments: sort.png
>
>
> These will be split in 3 separated JIRA issues. However, the design is the 
> same only the processing function differs in terms of the output. Hence, the 
> design is the same for all of them.
> Time target: Proc Time
> **SQL 

[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116537830
  
--- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SortITCase.scala
 ---
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.stream.sql
+
+import org.apache.flink.api.scala._
+import 
org.apache.flink.table.api.scala.stream.sql.SortITCase.{EventTimeSourceFunction,StringRowSelectorSink}
+import org.apache.flink.streaming.api.functions.source.SourceFunction
+import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
+import org.apache.flink.table.api.TableEnvironment
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.api.scala.stream.utils.{StreamITCase, 
StreamTestData, StreamingWithStateTestBase}
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.types.Row
+import org.junit.Assert._
+import org.junit._
+import org.apache.flink.streaming.api.TimeCharacteristic
+import 
org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext
+import org.apache.flink.streaming.api.watermark.Watermark
+import scala.collection.mutable
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction
+
+class SortITCase extends StreamingWithStateTestBase {
+
+  @Test
+  def testEventTimeOrderBy(): Unit = {
+val data = Seq(
+  Left((1500L, (1L, 15, "Hello"))),
+  Left((1600L, (1L, 16, "Hello"))),
+  Left((1000L, (1L, 1, "Hello"))),
+  Left((2000L, (2L, 2, "Hello"))),
+  Right(1000L),
+  Left((2000L, (2L, 2, "Hello"))),
+  Left((2000L, (2L, 3, "Hello"))),
+  Left((3000L, (3L, 3, "Hello"))),
+  Right(2000L),
+  Left((4000L, (4L, 4, "Hello"))),
+  Right(3000L),
+  Left((5000L, (5L, 5, "Hello"))),
+  Right(5000L),
+  Left((6000L, (6L, 65, "Hello"))),
+  Left((6000L, (6L, 6, "Hello"))),
+  Right(7000L),
+  Left((9000L, (6L, 9, "Hello"))),
+  Left((8500L, (6L, 18, "Hello"))),
+  Left((9000L, (6L, 7, "Hello"))),
+  Right(1L),
+  Left((1L, (7L, 7, "Hello World"))),
+  Left((11000L, (7L, 77, "Hello World"))),
+  Left((11000L, (7L, 17, "Hello World"))),
+  Right(12000L),
+  Left((14000L, (7L, 18, "Hello World"))),
+  Right(14000L),
+  Left((15000L, (8L, 8, "Hello World"))),
+  Right(17000L),
+  Left((2L, (20L, 20, "Hello World"))), 
+  Right(19000L))
+
+val env = StreamExecutionEnvironment.getExecutionEnvironment
--- End diff --

set the state backend to ensure correct serialization


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


[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116523262
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala
 ---
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.types.Row
+import org.apache.calcite.rel.`type`._
+import org.apache.calcite.rel.RelCollation
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.table.functions.AggregateFunction
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.flink.table.api.TableException
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.calcite.sql.`type`.SqlTypeName._
+import java.util.{ List => JList, ArrayList }
+import org.apache.flink.api.common.typeinfo.{ SqlTimeTypeInfo, 
TypeInformation }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import java.sql.Timestamp
+import org.apache.calcite.rel.RelFieldCollation
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import java.util.Comparator
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import 
java.lang.{Byte=>JByte,Integer=>JInt,Long=>JLong,Double=>JDouble,Short=>JShort,String=>JString,Float=>JFloat}
+import java.math.{BigDecimal=>JBigDecimal}
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.api.common.operators.Order
+import org.apache.calcite.rex.{RexLiteral, RexNode}
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.api.common.typeinfo.AtomicType
+import org.apache.flink.api.java.typeutils.runtime.RowComparator
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+import scala.collection.JavaConverters._
+
+/**
+ * Class represents a collection of helper methods to build the sort logic.
+ * It encapsulates as well the implementation for ordering and generic 
interfaces
+ */
+
+object SortUtil {
+
+  
+  /**
+   * Function creates 
[org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
+   * elements based on rowtime and potentially other fields
+   * @param collationSort The Sort collation list
+   * @param inputType input row type
+   * @param execCfg table environment execution configuration
+   * @return org.apache.flink.streaming.api.functions.ProcessFunction
+   */
+  private[flink] def createRowTimeSortFunction(
+collationSort: RelCollation,
+inputType: RelDataType,
+inputTypeInfo: TypeInformation[Row],
+execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
+
+val keySortFields = getSortFieldIndexList(collationSort)
+val keySortDirections = getSortFieldDirectionList(collationSort)
+
+   //drop time from comparison as we sort on time in the states and 
result emission
+val keyIndexesNoTime = keySortFields.slice(1, keySortFields.size)
+val keyDirectionsNoTime = keySortDirections.slice(1, 
keySortDirections.size)
+val booleanOrderings = getSortFieldDirectionBooleanList(collationSort)
+val booleanDirectionsNoTime = booleanOrderings.slice(1, 
booleanOrderings.size)
+
+val fieldComps = createFieldComparators(inputType, 
+keyIndexesNoTime, keyDirectionsNoTime, execCfg)
+val fieldCompsRefs = 
fieldComps.asInstanceOf[Array[TypeComparator[AnyRef]]]
+
+val rowComp = createRowComparator(inputType,
+keyIndexesNoTime, fieldCompsRefs, booleanDirectionsNoTime)
+val collectionRowComparator = new CollectionRowComparator(rowComp)
+
+val inputCRowType = CRowTypeInfo(inputTypeInfo)
+ 
+new 

[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116494872
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeSortProcessFunction.scala
 ---
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.api.common.state.{ ListState, ListStateDescriptor }
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.{RowTypeInfo, ListTypeInfo}
+import org.apache.flink.runtime.state.{ FunctionInitializationContext, 
FunctionSnapshotContext }
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.types.Row
+import org.apache.flink.util.{ Collector, Preconditions }
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import scala.util.control.Breaks._
+import org.apache.flink.api.java.tuple.{ Tuple2 => JTuple2 }
+import org.apache.flink.api.common.state.MapState
+import org.apache.flink.api.common.state.MapStateDescriptor
+import org.apache.flink.configuration.Configuration
+import java.util.Comparator
+import java.util.ArrayList
+import java.util.Collections
+import org.apache.flink.api.common.typeutils.TypeComparator
+import java.util.{List => JList, ArrayList => JArrayList}
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+/**
+ * Process Function used for the aggregate in bounded rowtime sort without 
offset/fetch
+ * [[org.apache.flink.streaming.api.datastream.DataStream]]
+ *
+ * @param fieldCount Is used to indicate fields in the current element to 
forward
+ * @param inputType It is used to mark the type of the incoming data
+ * @param rowComparator the [[java.util.Comparator]] is used for this sort 
aggregation
+ */
+class RowTimeSortProcessFunction(
+  private val fieldCount: Int,
+  private val inputRowType: CRowTypeInfo,
+  private val rowComparator: CollectionRowComparator)
+extends ProcessFunction[CRow, CRow] {
+
+  Preconditions.checkNotNull(rowComparator)
+
+  private val sortArray: ArrayList[Row] = new ArrayList[Row]
+  
+  // the state which keeps all the events that are not expired.
+  // Each timestamp will contain an associated list with the events 
+  // received at that timestamp
+  private var dataState: MapState[Long, JList[Row]] = _
+
+// the state which keeps the last triggering timestamp to filter late 
events
+  private var lastTriggeringTsState: ValueState[Long] = _
+  
+  private var outputC: CRow = _
+  
+  
+  override def open(config: Configuration) {
+ 
+val keyTypeInformation: TypeInformation[Long] =
+  BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]]
+val valueTypeInformation: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](
+inputRowType.asInstanceOf[CRowTypeInfo].rowType)
+
+val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]](
+"dataState",
+keyTypeInformation,
+valueTypeInformation)
+
+dataState = getRuntimeContext.getMapState(mapStateDescriptor)
+
+val lastTriggeringTsDescriptor: ValueStateDescriptor[Long] =
+  new ValueStateDescriptor[Long]("lastTriggeringTsState", 
classOf[Long])
+lastTriggeringTsState = 
getRuntimeContext.getState(lastTriggeringTsDescriptor)
+  }
+
+  
+  override def processElement(
+inputC: CRow,
+ctx: ProcessFunction[CRow, CRow]#Context,

[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116520149
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala
 ---
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.types.Row
+import org.apache.calcite.rel.`type`._
+import org.apache.calcite.rel.RelCollation
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.table.functions.AggregateFunction
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.flink.table.api.TableException
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.calcite.sql.`type`.SqlTypeName._
+import java.util.{ List => JList, ArrayList }
+import org.apache.flink.api.common.typeinfo.{ SqlTimeTypeInfo, 
TypeInformation }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import java.sql.Timestamp
+import org.apache.calcite.rel.RelFieldCollation
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import java.util.Comparator
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import 
java.lang.{Byte=>JByte,Integer=>JInt,Long=>JLong,Double=>JDouble,Short=>JShort,String=>JString,Float=>JFloat}
+import java.math.{BigDecimal=>JBigDecimal}
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.api.common.operators.Order
+import org.apache.calcite.rex.{RexLiteral, RexNode}
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.api.common.typeinfo.AtomicType
+import org.apache.flink.api.java.typeutils.runtime.RowComparator
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+import scala.collection.JavaConverters._
+
+/**
+ * Class represents a collection of helper methods to build the sort logic.
+ * It encapsulates as well the implementation for ordering and generic 
interfaces
+ */
+
+object SortUtil {
+
+  
+  /**
+   * Function creates 
[org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
+   * elements based on rowtime and potentially other fields
+   * @param collationSort The Sort collation list
+   * @param inputType input row type
+   * @param execCfg table environment execution configuration
+   * @return org.apache.flink.streaming.api.functions.ProcessFunction
+   */
+  private[flink] def createRowTimeSortFunction(
+collationSort: RelCollation,
+inputType: RelDataType,
+inputTypeInfo: TypeInformation[Row],
+execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
+
+val keySortFields = getSortFieldIndexList(collationSort)
+val keySortDirections = getSortFieldDirectionList(collationSort)
--- End diff --

is not required. We can use `booleanOrdering` instead.


> Support Limit/Top(Sort) for Stream SQL
> --
>
> Key: FLINK-6075
> URL: https://issues.apache.org/jira/browse/FLINK-6075
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: radu
>  Labels: features
> Attachments: sort.png
>
>
> These will be split in 3 separated JIRA issues. However, the design is the 
> same only the processing function differs in terms of the output. Hence, the 
> design is the same for all of them.
> Time target: Proc Time
> **SQL targeted query 

[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116494872
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeSortProcessFunction.scala
 ---
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.api.common.state.{ ListState, ListStateDescriptor }
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.{RowTypeInfo, ListTypeInfo}
+import org.apache.flink.runtime.state.{ FunctionInitializationContext, 
FunctionSnapshotContext }
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.types.Row
+import org.apache.flink.util.{ Collector, Preconditions }
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import scala.util.control.Breaks._
+import org.apache.flink.api.java.tuple.{ Tuple2 => JTuple2 }
+import org.apache.flink.api.common.state.MapState
+import org.apache.flink.api.common.state.MapStateDescriptor
+import org.apache.flink.configuration.Configuration
+import java.util.Comparator
+import java.util.ArrayList
+import java.util.Collections
+import org.apache.flink.api.common.typeutils.TypeComparator
+import java.util.{List => JList, ArrayList => JArrayList}
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+/**
+ * Process Function used for the aggregate in bounded rowtime sort without 
offset/fetch
+ * [[org.apache.flink.streaming.api.datastream.DataStream]]
+ *
+ * @param fieldCount Is used to indicate fields in the current element to 
forward
+ * @param inputType It is used to mark the type of the incoming data
+ * @param rowComparator the [[java.util.Comparator]] is used for this sort 
aggregation
+ */
+class RowTimeSortProcessFunction(
+  private val fieldCount: Int,
+  private val inputRowType: CRowTypeInfo,
+  private val rowComparator: CollectionRowComparator)
+extends ProcessFunction[CRow, CRow] {
+
+  Preconditions.checkNotNull(rowComparator)
+
+  private val sortArray: ArrayList[Row] = new ArrayList[Row]
+  
+  // the state which keeps all the events that are not expired.
+  // Each timestamp will contain an associated list with the events 
+  // received at that timestamp
+  private var dataState: MapState[Long, JList[Row]] = _
+
+// the state which keeps the last triggering timestamp to filter late 
events
+  private var lastTriggeringTsState: ValueState[Long] = _
+  
+  private var outputC: CRow = _
+  
+  
+  override def open(config: Configuration) {
+ 
+val keyTypeInformation: TypeInformation[Long] =
+  BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]]
+val valueTypeInformation: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](
+inputRowType.asInstanceOf[CRowTypeInfo].rowType)
+
+val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]](
+"dataState",
+keyTypeInformation,
+valueTypeInformation)
+
+dataState = getRuntimeContext.getMapState(mapStateDescriptor)
+
+val lastTriggeringTsDescriptor: ValueStateDescriptor[Long] =
+  new ValueStateDescriptor[Long]("lastTriggeringTsState", 
classOf[Long])
+lastTriggeringTsState = 
getRuntimeContext.getState(lastTriggeringTsDescriptor)
+  }
+
+  
+  override def processElement(
+inputC: CRow,
+ctx: ProcessFunction[CRow, CRow]#Context,
+out: Collector[CRow]): Unit = {
+
+ val input = inputC.row
+
+ if( outputC == null) {
+  outputC = new CRow(input, true)
+}
+
+// triggering timestamp for trigger 

[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116480335
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeSortProcessFunction.scala
 ---
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.api.common.state.{ ListState, ListStateDescriptor }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.runtime.state.{ FunctionInitializationContext, 
FunctionSnapshotContext }
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.types.Row
+import org.apache.flink.util.{ Collector, Preconditions }
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import scala.util.control.Breaks._
+import org.apache.flink.api.java.tuple.{ Tuple2 => JTuple2 }
+import org.apache.flink.api.common.state.MapState
+import org.apache.flink.api.common.state.MapStateDescriptor
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import java.util.Comparator
+import java.util.ArrayList
+import java.util.Collections
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+/**
+ * Process Function used for the aggregate in bounded proctime sort 
without offset/fetch
+ * [[org.apache.flink.streaming.api.datastream.DataStream]]
+ *
+ * @param fieldCount Is used to indicate fields in the current element to 
forward
+ * @param inputType It is used to mark the type of the incoming data
+ * @param rowComparator the [[java.util.Comparator]] is used for this sort 
aggregation
+ */
+class ProcTimeSortProcessFunction(
+  private val fieldCount: Int,
+  private val inputRowType: CRowTypeInfo,
+  private val rowComparator: CollectionRowComparator)
+extends ProcessFunction[CRow, CRow] {
+
+  Preconditions.checkNotNull(rowComparator)
+
+  private var stateEventsBuffer: ListState[Row] = _
+  private val sortArray: ArrayList[Row] = new ArrayList[Row]
+  
+  private var outputC: CRow = _
+  
+  override def open(config: Configuration) {
+val sortDescriptor = new ListStateDescriptor[Row]("sortState",
+inputRowType.asInstanceOf[CRowTypeInfo].rowType)
+stateEventsBuffer = getRuntimeContext.getListState(sortDescriptor)
+
+  }
+
+  override def processElement(
+inputC: CRow,
+ctx: ProcessFunction[CRow, CRow]#Context,
+out: Collector[CRow]): Unit = {
+
+val input = inputC.row
+
+if( outputC == null) {
+  outputC = new CRow(input, true)
+}
+
+val currentTime = ctx.timerService.currentProcessingTime
+//buffer the event incoming event
+  
+//we accumulate the events as they arrive within the given proctime
+stateEventsBuffer.add(input)
+
+//deduplication of multiple registered timers is done automatically
+ctx.timerService.registerProcessingTimeTimer(currentTime + 1)  
+
+  }
+  
+  override def onTimer(
+timestamp: Long,
+ctx: ProcessFunction[CRow, CRow]#OnTimerContext,
+out: Collector[CRow]): Unit = {
+
+var i = 0
--- End diff --

`i` is not used


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


[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116496006
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala
 ---
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.types.Row
+import org.apache.calcite.rel.`type`._
+import org.apache.calcite.rel.RelCollation
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.table.functions.AggregateFunction
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.flink.table.api.TableException
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.calcite.sql.`type`.SqlTypeName._
+import java.util.{ List => JList, ArrayList }
+import org.apache.flink.api.common.typeinfo.{ SqlTimeTypeInfo, 
TypeInformation }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import java.sql.Timestamp
+import org.apache.calcite.rel.RelFieldCollation
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import java.util.Comparator
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import 
java.lang.{Byte=>JByte,Integer=>JInt,Long=>JLong,Double=>JDouble,Short=>JShort,String=>JString,Float=>JFloat}
+import java.math.{BigDecimal=>JBigDecimal}
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.api.common.operators.Order
+import org.apache.calcite.rex.{RexLiteral, RexNode}
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.api.common.typeinfo.AtomicType
+import org.apache.flink.api.java.typeutils.runtime.RowComparator
+import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+import scala.collection.JavaConverters._
+
+/**
+ * Class represents a collection of helper methods to build the sort logic.
+ * It encapsulates as well the implementation for ordering and generic 
interfaces
+ */
+
+object SortUtil {
+
+  
--- End diff --

rm newline


> Support Limit/Top(Sort) for Stream SQL
> --
>
> Key: FLINK-6075
> URL: https://issues.apache.org/jira/browse/FLINK-6075
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: radu
>  Labels: features
> Attachments: sort.png
>
>
> These will be split in 3 separated JIRA issues. However, the design is the 
> same only the processing function differs in terms of the output. Hence, the 
> design is the same for all of them.
> Time target: Proc Time
> **SQL targeted query examples:**
> *Sort example*
> Q1)` SELECT a FROM stream1 GROUP BY HOP(proctime, INTERVAL '1' HOUR, INTERVAL 
> '3' HOUR) ORDER BY b` 
> Comment: window is defined using GROUP BY
> Comment: ASC or DESC keywords can be placed to mark the ordering type
> *Limit example*
> Q2) `SELECT a FROM stream1 WHERE rowtime BETWEEN current_timestamp - INTERVAL 
> '1' HOUR AND current_timestamp ORDER BY b LIMIT 10`
> Comment: window is defined using time ranges in the WHERE clause
> Comment: window is row triggered
> *Top example*
> Q3) `SELECT sum(a) OVER (ORDER BY proctime RANGE INTERVAL '1' HOUR PRECEDING 
> LIMIT 10) FROM stream1`  
> Comment: limit over the contents of the sliding window
> General Comments:
> -All these SQL clauses are supported only over windows (bounded collections 
> of data). 
> -Each of the 3 operators will be supported with each 

[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116480769
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeSortProcessFunction.scala
 ---
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.api.common.state.{ ListState, ListStateDescriptor }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.runtime.state.{ FunctionInitializationContext, 
FunctionSnapshotContext }
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.types.Row
+import org.apache.flink.util.{ Collector, Preconditions }
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import scala.util.control.Breaks._
+import org.apache.flink.api.java.tuple.{ Tuple2 => JTuple2 }
+import org.apache.flink.api.common.state.MapState
+import org.apache.flink.api.common.state.MapStateDescriptor
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import java.util.Comparator
+import java.util.ArrayList
+import java.util.Collections
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+/**
+ * Process Function used for the aggregate in bounded proctime sort 
without offset/fetch
+ * [[org.apache.flink.streaming.api.datastream.DataStream]]
+ *
+ * @param fieldCount Is used to indicate fields in the current element to 
forward
+ * @param inputType It is used to mark the type of the incoming data
+ * @param rowComparator the [[java.util.Comparator]] is used for this sort 
aggregation
+ */
+class ProcTimeSortProcessFunction(
+  private val fieldCount: Int,
+  private val inputRowType: CRowTypeInfo,
+  private val rowComparator: CollectionRowComparator)
+extends ProcessFunction[CRow, CRow] {
+
+  Preconditions.checkNotNull(rowComparator)
+
+  private var stateEventsBuffer: ListState[Row] = _
+  private val sortArray: ArrayList[Row] = new ArrayList[Row]
+  
+  private var outputC: CRow = _
+  
+  override def open(config: Configuration) {
+val sortDescriptor = new ListStateDescriptor[Row]("sortState",
+inputRowType.asInstanceOf[CRowTypeInfo].rowType)
+stateEventsBuffer = getRuntimeContext.getListState(sortDescriptor)
+
+  }
+
+  override def processElement(
+inputC: CRow,
+ctx: ProcessFunction[CRow, CRow]#Context,
+out: Collector[CRow]): Unit = {
+
+val input = inputC.row
+
+if( outputC == null) {
+  outputC = new CRow(input, true)
+}
+
+val currentTime = ctx.timerService.currentProcessingTime
+//buffer the event incoming event
+  
+//we accumulate the events as they arrive within the given proctime
+stateEventsBuffer.add(input)
+
+//deduplication of multiple registered timers is done automatically
+ctx.timerService.registerProcessingTimeTimer(currentTime + 1)  
+
+  }
+  
+  override def onTimer(
+timestamp: Long,
+ctx: ProcessFunction[CRow, CRow]#OnTimerContext,
+out: Collector[CRow]): Unit = {
+
+var i = 0
+val iter =  stateEventsBuffer.get.iterator()
+
+
+sortArray.clear()
+while(iter.hasNext()) {
+  sortArray.add(iter.next())
+}
+
+//if we do not rely on java collections to do the sort we could 
implement 
+//an insertion sort as we get the elements  from the state
+Collections.sort(sortArray, rowComparator)
+
+//no retraction now
--- End diff --

remove 

[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116470198
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSort.scala
 ---
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.plan.{ RelOptCluster, RelTraitSet }
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.{ RelNode, RelWriter, SingleRel }
+import org.apache.flink.api.java.tuple.Tuple
+import org.apache.flink.streaming.api.datastream.{ AllWindowedStream, 
DataStream, KeyedStream, WindowedStream }
+import org.apache.flink.streaming.api.windowing.assigners._
+import org.apache.flink.streaming.api.windowing.time.Time
+import org.apache.flink.streaming.api.windowing.windows.{ Window => 
DataStreamWindow }
+import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.expressions._
+import org.apache.flink.table.plan.logical._
+import org.apache.flink.table.plan.nodes.CommonAggregate
+import org.apache.flink.table.runtime.aggregate.AggregateUtil._
+import org.apache.flink.table.runtime.aggregate._
+import org.apache.flink.table.typeutils.TypeCheckUtils.isTimeInterval
+import org.apache.flink.table.typeutils.{ RowIntervalTypeInfo, 
TimeIntervalTypeInfo }
+import org.apache.flink.types.Row
+import org.apache.calcite.sql.SqlAggFunction
+import org.apache.flink.table.plan.nodes.datastream.DataStreamRel
+import org.apache.calcite.sql.fun.SqlSingleValueAggFunction
+import org.apache.flink.api.common.functions.RichMapFunction
+import org.apache.flink.api.common.functions.RichFlatMapFunction
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.util.Collector
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.calcite.rel.core.Sort
+import org.apache.flink.api.java.functions.NullByteKeySelector
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import org.apache.flink.table.runtime.aggregate.SortUtil._
+import org.apache.calcite.rel.RelCollation
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+import org.apache.flink.table.api.{StreamQueryConfig, 
StreamTableEnvironment, TableException}
+import org.apache.flink.table.plan.schema.RowSchema
+
+/**
+  * Flink RelNode which matches along with Sort Rule.
+  *
+ */
+class DataStreamSort(
+  sortCollation: RelCollation,
+  sortOffset: RexNode,
+  sortFetch: RexNode,
+  cluster: RelOptCluster,
+  traitSet: RelTraitSet,
+  inputNode: RelNode,
+  schema: RowSchema,
+  inputSchema: RowSchema,
+  description: String)
+extends SingleRel(cluster, traitSet, inputNode) with DataStreamRel {
+
+  override def deriveRowType(): RelDataType = schema.logicalType
+
+  override def copy(traitSet: RelTraitSet, inputs: 
java.util.List[RelNode]): RelNode = {
+new DataStreamSort(
+  sortCollation,
+  sortOffset,
+  sortFetch,
+  cluster,
+  traitSet,
+  inputs.get(0),
+  schema,
+  inputSchema,
+  description)
+  }
+
+  override def toString: String = {
+s"Sort(by: ($SortUtil.getSortFieldToString(sortCollation, 
rowRelDataType))," +
+  " offset: $SortUtil.getOffsetToString(sortOffset)," +
   

[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116480335
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeSortProcessFunction.scala
 ---
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.api.common.state.{ ListState, ListStateDescriptor }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.runtime.state.{ FunctionInitializationContext, 
FunctionSnapshotContext }
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.types.Row
+import org.apache.flink.util.{ Collector, Preconditions }
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import scala.util.control.Breaks._
+import org.apache.flink.api.java.tuple.{ Tuple2 => JTuple2 }
+import org.apache.flink.api.common.state.MapState
+import org.apache.flink.api.common.state.MapStateDescriptor
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import java.util.Comparator
+import java.util.ArrayList
+import java.util.Collections
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+/**
+ * Process Function used for the aggregate in bounded proctime sort 
without offset/fetch
+ * [[org.apache.flink.streaming.api.datastream.DataStream]]
+ *
+ * @param fieldCount Is used to indicate fields in the current element to 
forward
+ * @param inputType It is used to mark the type of the incoming data
+ * @param rowComparator the [[java.util.Comparator]] is used for this sort 
aggregation
+ */
+class ProcTimeSortProcessFunction(
+  private val fieldCount: Int,
+  private val inputRowType: CRowTypeInfo,
+  private val rowComparator: CollectionRowComparator)
+extends ProcessFunction[CRow, CRow] {
+
+  Preconditions.checkNotNull(rowComparator)
+
+  private var stateEventsBuffer: ListState[Row] = _
+  private val sortArray: ArrayList[Row] = new ArrayList[Row]
+  
+  private var outputC: CRow = _
+  
+  override def open(config: Configuration) {
+val sortDescriptor = new ListStateDescriptor[Row]("sortState",
+inputRowType.asInstanceOf[CRowTypeInfo].rowType)
+stateEventsBuffer = getRuntimeContext.getListState(sortDescriptor)
+
+  }
+
+  override def processElement(
+inputC: CRow,
+ctx: ProcessFunction[CRow, CRow]#Context,
+out: Collector[CRow]): Unit = {
+
+val input = inputC.row
+
+if( outputC == null) {
+  outputC = new CRow(input, true)
+}
+
+val currentTime = ctx.timerService.currentProcessingTime
+//buffer the event incoming event
+  
+//we accumulate the events as they arrive within the given proctime
+stateEventsBuffer.add(input)
+
+//deduplication of multiple registered timers is done automatically
+ctx.timerService.registerProcessingTimeTimer(currentTime + 1)  
+
+  }
+  
+  override def onTimer(
+timestamp: Long,
+ctx: ProcessFunction[CRow, CRow]#OnTimerContext,
+out: Collector[CRow]): Unit = {
+
+var i = 0
--- End diff --

`i` is not used


> Support Limit/Top(Sort) for Stream SQL
> --
>
> Key: FLINK-6075
> URL: https://issues.apache.org/jira/browse/FLINK-6075
> 

[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116475931
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamSortRule.scala
 ---
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.plan.rules.datastream
+
+import org.apache.calcite.plan.volcano.RelSubset
+import org.apache.calcite.plan.{ Convention, RelOptRule, RelOptRuleCall, 
RelTraitSet }
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.calcite.rel.logical.{ LogicalFilter, LogicalCorrelate, 
LogicalTableFunctionScan }
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.table.plan.nodes.datastream.DataStreamCorrelate
+import org.apache.calcite.rel.logical.LogicalSort
+import org.apache.flink.table.plan.nodes.datastream.DataStreamSort
+import org.apache.calcite.rel.RelCollation
+import org.apache.flink.table.plan.nodes.FlinkConventions
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.table.plan.nodes.logical.FlinkLogicalSort
+
+/**
+ * Rule to convert a LogicalSort into a DataStreamSort.
+ */
+class DataStreamSortRule
+extends ConverterRule(
+  classOf[FlinkLogicalSort],
+  FlinkConventions.LOGICAL,
+  FlinkConventions.DATASTREAM,
+  "DataStreamSortRule") {
+
+  override def matches(call: RelOptRuleCall): Boolean = {
+super.matches(call)
--- End diff --

We should check here that we can execute the sort (type and order of first 
collation).


> Support Limit/Top(Sort) for Stream SQL
> --
>
> Key: FLINK-6075
> URL: https://issues.apache.org/jira/browse/FLINK-6075
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: radu
>  Labels: features
> Attachments: sort.png
>
>
> These will be split in 3 separated JIRA issues. However, the design is the 
> same only the processing function differs in terms of the output. Hence, the 
> design is the same for all of them.
> Time target: Proc Time
> **SQL targeted query examples:**
> *Sort example*
> Q1)` SELECT a FROM stream1 GROUP BY HOP(proctime, INTERVAL '1' HOUR, INTERVAL 
> '3' HOUR) ORDER BY b` 
> Comment: window is defined using GROUP BY
> Comment: ASC or DESC keywords can be placed to mark the ordering type
> *Limit example*
> Q2) `SELECT a FROM stream1 WHERE rowtime BETWEEN current_timestamp - INTERVAL 
> '1' HOUR AND current_timestamp ORDER BY b LIMIT 10`
> Comment: window is defined using time ranges in the WHERE clause
> Comment: window is row triggered
> *Top example*
> Q3) `SELECT sum(a) OVER (ORDER BY proctime RANGE INTERVAL '1' HOUR PRECEDING 
> LIMIT 10) FROM stream1`  
> Comment: limit over the contents of the sliding window
> General Comments:
> -All these SQL clauses are supported only over windows (bounded collections 
> of data). 
> -Each of the 3 operators will be supported with each of the types of 
> expressing the windows. 
> **Description**
> The 3 operations (limit, top and sort) are similar in behavior as they all 
> require a sorted collection of the data on which the logic will be applied 
> (i.e., select a subset of the items or the entire sorted set). These 
> functions would make sense in the streaming context only in the context of a 
> window. Without defining a window the functions could never emit as the sort 
> operation would never trigger. If an SQL query will be provided without 
> limits an error will be thrown (`SELECT a FROM stream1 TOP 10` -> ERROR). 
> Although not targeted by this JIRA, in the case of working based on event 

[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116475426
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSort.scala
 ---
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.plan.{ RelOptCluster, RelTraitSet }
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.{ RelNode, RelWriter, SingleRel }
+import org.apache.flink.api.java.tuple.Tuple
+import org.apache.flink.streaming.api.datastream.{ AllWindowedStream, 
DataStream, KeyedStream, WindowedStream }
+import org.apache.flink.streaming.api.windowing.assigners._
+import org.apache.flink.streaming.api.windowing.time.Time
+import org.apache.flink.streaming.api.windowing.windows.{ Window => 
DataStreamWindow }
+import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.expressions._
+import org.apache.flink.table.plan.logical._
+import org.apache.flink.table.plan.nodes.CommonAggregate
+import org.apache.flink.table.runtime.aggregate.AggregateUtil._
+import org.apache.flink.table.runtime.aggregate._
+import org.apache.flink.table.typeutils.TypeCheckUtils.isTimeInterval
+import org.apache.flink.table.typeutils.{ RowIntervalTypeInfo, 
TimeIntervalTypeInfo }
+import org.apache.flink.types.Row
+import org.apache.calcite.sql.SqlAggFunction
+import org.apache.flink.table.plan.nodes.datastream.DataStreamRel
+import org.apache.calcite.sql.fun.SqlSingleValueAggFunction
+import org.apache.flink.api.common.functions.RichMapFunction
+import org.apache.flink.api.common.functions.RichFlatMapFunction
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.util.Collector
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.calcite.rel.core.Sort
+import org.apache.flink.api.java.functions.NullByteKeySelector
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import org.apache.flink.table.runtime.aggregate.SortUtil._
+import org.apache.calcite.rel.RelCollation
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+import org.apache.flink.table.api.{StreamQueryConfig, 
StreamTableEnvironment, TableException}
+import org.apache.flink.table.plan.schema.RowSchema
+
+/**
+  * Flink RelNode which matches along with Sort Rule.
+  *
+ */
+class DataStreamSort(
+  sortCollation: RelCollation,
+  sortOffset: RexNode,
+  sortFetch: RexNode,
+  cluster: RelOptCluster,
+  traitSet: RelTraitSet,
+  inputNode: RelNode,
+  schema: RowSchema,
+  inputSchema: RowSchema,
+  description: String)
+extends SingleRel(cluster, traitSet, inputNode) with DataStreamRel {
+
+  override def deriveRowType(): RelDataType = schema.logicalType
+
+  override def copy(traitSet: RelTraitSet, inputs: 
java.util.List[RelNode]): RelNode = {
+new DataStreamSort(
+  sortCollation,
+  sortOffset,
+  sortFetch,
+  cluster,
+  traitSet,
+  inputs.get(0),
+  schema,
+  inputSchema,
+  description)
+  }
+
+  override def toString: String = {
+s"Sort(by: ($SortUtil.getSortFieldToString(sortCollation, 
rowRelDataType))," +
+  " offset: $SortUtil.getOffsetToString(sortOffset)," +
+  " fetch: $SortUtil.getFetchToString(sortFetch, sortOffset))"
+  }
+  
+  override def explainTerms(pw: RelWriter) : RelWriter = {
+
+//need to identify time between others order fields. Time needs to be 

[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116480397
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeSortProcessFunction.scala
 ---
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.api.common.state.{ ListState, ListStateDescriptor }
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.runtime.state.{ FunctionInitializationContext, 
FunctionSnapshotContext }
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.types.Row
+import org.apache.flink.util.{ Collector, Preconditions }
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import scala.util.control.Breaks._
+import org.apache.flink.api.java.tuple.{ Tuple2 => JTuple2 }
+import org.apache.flink.api.common.state.MapState
+import org.apache.flink.api.common.state.MapStateDescriptor
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import java.util.Comparator
+import java.util.ArrayList
+import java.util.Collections
+import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+/**
+ * Process Function used for the aggregate in bounded proctime sort 
without offset/fetch
+ * [[org.apache.flink.streaming.api.datastream.DataStream]]
+ *
+ * @param fieldCount Is used to indicate fields in the current element to 
forward
+ * @param inputType It is used to mark the type of the incoming data
+ * @param rowComparator the [[java.util.Comparator]] is used for this sort 
aggregation
+ */
+class ProcTimeSortProcessFunction(
+  private val fieldCount: Int,
+  private val inputRowType: CRowTypeInfo,
+  private val rowComparator: CollectionRowComparator)
+extends ProcessFunction[CRow, CRow] {
+
+  Preconditions.checkNotNull(rowComparator)
+
+  private var stateEventsBuffer: ListState[Row] = _
+  private val sortArray: ArrayList[Row] = new ArrayList[Row]
+  
+  private var outputC: CRow = _
+  
+  override def open(config: Configuration) {
+val sortDescriptor = new ListStateDescriptor[Row]("sortState",
+inputRowType.asInstanceOf[CRowTypeInfo].rowType)
+stateEventsBuffer = getRuntimeContext.getListState(sortDescriptor)
+
+  }
+
+  override def processElement(
+inputC: CRow,
+ctx: ProcessFunction[CRow, CRow]#Context,
+out: Collector[CRow]): Unit = {
+
+val input = inputC.row
+
+if( outputC == null) {
+  outputC = new CRow(input, true)
+}
+
+val currentTime = ctx.timerService.currentProcessingTime
+//buffer the event incoming event
+  
+//we accumulate the events as they arrive within the given proctime
+stateEventsBuffer.add(input)
+
+//deduplication of multiple registered timers is done automatically
+ctx.timerService.registerProcessingTimeTimer(currentTime + 1)  
+
+  }
+  
+  override def onTimer(
+timestamp: Long,
+ctx: ProcessFunction[CRow, CRow]#OnTimerContext,
+out: Collector[CRow]): Unit = {
+
+var i = 0
+val iter =  stateEventsBuffer.get.iterator()
+
+
--- End diff --

remove second newline


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


[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116470767
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSort.scala
 ---
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.plan.{ RelOptCluster, RelTraitSet }
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.{ RelNode, RelWriter, SingleRel }
+import org.apache.flink.api.java.tuple.Tuple
+import org.apache.flink.streaming.api.datastream.{ AllWindowedStream, 
DataStream, KeyedStream, WindowedStream }
+import org.apache.flink.streaming.api.windowing.assigners._
+import org.apache.flink.streaming.api.windowing.time.Time
+import org.apache.flink.streaming.api.windowing.windows.{ Window => 
DataStreamWindow }
+import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.expressions._
+import org.apache.flink.table.plan.logical._
+import org.apache.flink.table.plan.nodes.CommonAggregate
+import org.apache.flink.table.runtime.aggregate.AggregateUtil._
+import org.apache.flink.table.runtime.aggregate._
+import org.apache.flink.table.typeutils.TypeCheckUtils.isTimeInterval
+import org.apache.flink.table.typeutils.{ RowIntervalTypeInfo, 
TimeIntervalTypeInfo }
+import org.apache.flink.types.Row
+import org.apache.calcite.sql.SqlAggFunction
+import org.apache.flink.table.plan.nodes.datastream.DataStreamRel
+import org.apache.calcite.sql.fun.SqlSingleValueAggFunction
+import org.apache.flink.api.common.functions.RichMapFunction
+import org.apache.flink.api.common.functions.RichFlatMapFunction
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.util.Collector
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.calcite.rel.core.Sort
+import org.apache.flink.api.java.functions.NullByteKeySelector
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import org.apache.flink.table.runtime.aggregate.SortUtil._
+import org.apache.calcite.rel.RelCollation
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+import org.apache.flink.table.api.{StreamQueryConfig, 
StreamTableEnvironment, TableException}
+import org.apache.flink.table.plan.schema.RowSchema
+
+/**
+  * Flink RelNode which matches along with Sort Rule.
+  *
+ */
+class DataStreamSort(
+  sortCollation: RelCollation,
+  sortOffset: RexNode,
+  sortFetch: RexNode,
+  cluster: RelOptCluster,
+  traitSet: RelTraitSet,
+  inputNode: RelNode,
+  schema: RowSchema,
+  inputSchema: RowSchema,
+  description: String)
+extends SingleRel(cluster, traitSet, inputNode) with DataStreamRel {
+
+  override def deriveRowType(): RelDataType = schema.logicalType
+
+  override def copy(traitSet: RelTraitSet, inputs: 
java.util.List[RelNode]): RelNode = {
+new DataStreamSort(
+  sortCollation,
+  sortOffset,
+  sortFetch,
+  cluster,
+  traitSet,
+  inputs.get(0),
+  schema,
+  inputSchema,
+  description)
+  }
+
+  override def toString: String = {
+s"Sort(by: ($SortUtil.getSortFieldToString(sortCollation, 
rowRelDataType))," +
+  " offset: $SortUtil.getOffsetToString(sortOffset)," +
   

[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116455372
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSort.scala
 ---
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.plan.{ RelOptCluster, RelTraitSet }
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.{ RelNode, RelWriter, SingleRel }
+import org.apache.flink.api.java.tuple.Tuple
+import org.apache.flink.streaming.api.datastream.{ AllWindowedStream, 
DataStream, KeyedStream, WindowedStream }
+import org.apache.flink.streaming.api.windowing.assigners._
+import org.apache.flink.streaming.api.windowing.time.Time
+import org.apache.flink.streaming.api.windowing.windows.{ Window => 
DataStreamWindow }
+import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.expressions._
+import org.apache.flink.table.plan.logical._
+import org.apache.flink.table.plan.nodes.CommonAggregate
+import org.apache.flink.table.runtime.aggregate.AggregateUtil._
+import org.apache.flink.table.runtime.aggregate._
+import org.apache.flink.table.typeutils.TypeCheckUtils.isTimeInterval
+import org.apache.flink.table.typeutils.{ RowIntervalTypeInfo, 
TimeIntervalTypeInfo }
+import org.apache.flink.types.Row
+import org.apache.calcite.sql.SqlAggFunction
+import org.apache.flink.table.plan.nodes.datastream.DataStreamRel
+import org.apache.calcite.sql.fun.SqlSingleValueAggFunction
+import org.apache.flink.api.common.functions.RichMapFunction
+import org.apache.flink.api.common.functions.RichFlatMapFunction
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.util.Collector
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.calcite.rel.core.Sort
+import org.apache.flink.api.java.functions.NullByteKeySelector
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import org.apache.flink.table.runtime.aggregate.SortUtil._
+import org.apache.calcite.rel.RelCollation
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+import org.apache.flink.table.api.{StreamQueryConfig, 
StreamTableEnvironment, TableException}
+import org.apache.flink.table.plan.schema.RowSchema
+
+/**
+  * Flink RelNode which matches along with Sort Rule.
+  *
+ */
+class DataStreamSort(
+  sortCollation: RelCollation,
+  sortOffset: RexNode,
+  sortFetch: RexNode,
+  cluster: RelOptCluster,
--- End diff --

Please keep the order of parameters consistent to other classes (for 
instance `DataStreamCalc`), i.e., cluster, traitset, input, inputschema, 
schema, sortCollation, offset, fetch, description.


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


[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116482918
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeSortProcessFunction.scala
 ---
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.api.common.state.{ ListState, ListStateDescriptor }
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.{RowTypeInfo, ListTypeInfo}
+import org.apache.flink.runtime.state.{ FunctionInitializationContext, 
FunctionSnapshotContext }
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.types.Row
+import org.apache.flink.util.{ Collector, Preconditions }
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import scala.util.control.Breaks._
+import org.apache.flink.api.java.tuple.{ Tuple2 => JTuple2 }
+import org.apache.flink.api.common.state.MapState
+import org.apache.flink.api.common.state.MapStateDescriptor
+import org.apache.flink.configuration.Configuration
+import java.util.Comparator
+import java.util.ArrayList
+import java.util.Collections
+import org.apache.flink.api.common.typeutils.TypeComparator
+import java.util.{List => JList, ArrayList => JArrayList}
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+/**
+ * Process Function used for the aggregate in bounded rowtime sort without 
offset/fetch
+ * [[org.apache.flink.streaming.api.datastream.DataStream]]
+ *
+ * @param fieldCount Is used to indicate fields in the current element to 
forward
+ * @param inputType It is used to mark the type of the incoming data
+ * @param rowComparator the [[java.util.Comparator]] is used for this sort 
aggregation
+ */
+class RowTimeSortProcessFunction(
+  private val fieldCount: Int,
+  private val inputRowType: CRowTypeInfo,
+  private val rowComparator: CollectionRowComparator)
+extends ProcessFunction[CRow, CRow] {
+
+  Preconditions.checkNotNull(rowComparator)
+
+  private val sortArray: ArrayList[Row] = new ArrayList[Row]
+  
+  // the state which keeps all the events that are not expired.
+  // Each timestamp will contain an associated list with the events 
+  // received at that timestamp
+  private var dataState: MapState[Long, JList[Row]] = _
+
+// the state which keeps the last triggering timestamp to filter late 
events
+  private var lastTriggeringTsState: ValueState[Long] = _
+  
+  private var outputC: CRow = _
+  
+  
+  override def open(config: Configuration) {
+ 
+val keyTypeInformation: TypeInformation[Long] =
+  BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]]
+val valueTypeInformation: TypeInformation[JList[Row]] = new 
ListTypeInfo[Row](
+inputRowType.asInstanceOf[CRowTypeInfo].rowType)
+
+val mapStateDescriptor: MapStateDescriptor[Long, JList[Row]] =
+  new MapStateDescriptor[Long, JList[Row]](
+"dataState",
+keyTypeInformation,
+valueTypeInformation)
+
+dataState = getRuntimeContext.getMapState(mapStateDescriptor)
+
+val lastTriggeringTsDescriptor: ValueStateDescriptor[Long] =
+  new ValueStateDescriptor[Long]("lastTriggeringTsState", 
classOf[Long])
+lastTriggeringTsState = 
getRuntimeContext.getState(lastTriggeringTsDescriptor)
+  }
+
+  
+  override def processElement(
+inputC: CRow,
+ctx: ProcessFunction[CRow, CRow]#Context,
+out: Collector[CRow]): Unit = {
+
+ val input = inputC.row
+
+ if( outputC == null) {
+  outputC = new CRow(input, true)
--- End diff --

can be instantiated in `open()` to avoid 

[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116470110
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSort.scala
 ---
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.plan.{ RelOptCluster, RelTraitSet }
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.{ RelNode, RelWriter, SingleRel }
+import org.apache.flink.api.java.tuple.Tuple
+import org.apache.flink.streaming.api.datastream.{ AllWindowedStream, 
DataStream, KeyedStream, WindowedStream }
+import org.apache.flink.streaming.api.windowing.assigners._
+import org.apache.flink.streaming.api.windowing.time.Time
+import org.apache.flink.streaming.api.windowing.windows.{ Window => 
DataStreamWindow }
+import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.expressions._
+import org.apache.flink.table.plan.logical._
+import org.apache.flink.table.plan.nodes.CommonAggregate
+import org.apache.flink.table.runtime.aggregate.AggregateUtil._
+import org.apache.flink.table.runtime.aggregate._
+import org.apache.flink.table.typeutils.TypeCheckUtils.isTimeInterval
+import org.apache.flink.table.typeutils.{ RowIntervalTypeInfo, 
TimeIntervalTypeInfo }
+import org.apache.flink.types.Row
+import org.apache.calcite.sql.SqlAggFunction
+import org.apache.flink.table.plan.nodes.datastream.DataStreamRel
+import org.apache.calcite.sql.fun.SqlSingleValueAggFunction
+import org.apache.flink.api.common.functions.RichMapFunction
+import org.apache.flink.api.common.functions.RichFlatMapFunction
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.util.Collector
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.calcite.rel.core.Sort
+import org.apache.flink.api.java.functions.NullByteKeySelector
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import org.apache.flink.table.runtime.aggregate.SortUtil._
+import org.apache.calcite.rel.RelCollation
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+import org.apache.flink.table.api.{StreamQueryConfig, 
StreamTableEnvironment, TableException}
+import org.apache.flink.table.plan.schema.RowSchema
+
+/**
+  * Flink RelNode which matches along with Sort Rule.
+  *
+ */
+class DataStreamSort(
+  sortCollation: RelCollation,
+  sortOffset: RexNode,
+  sortFetch: RexNode,
+  cluster: RelOptCluster,
+  traitSet: RelTraitSet,
+  inputNode: RelNode,
+  schema: RowSchema,
+  inputSchema: RowSchema,
+  description: String)
+extends SingleRel(cluster, traitSet, inputNode) with DataStreamRel {
+
+  override def deriveRowType(): RelDataType = schema.logicalType
+
+  override def copy(traitSet: RelTraitSet, inputs: 
java.util.List[RelNode]): RelNode = {
+new DataStreamSort(
+  sortCollation,
+  sortOffset,
+  sortFetch,
+  cluster,
+  traitSet,
+  inputs.get(0),
+  schema,
+  inputSchema,
+  description)
+  }
+
+  override def toString: String = {
+s"Sort(by: ($SortUtil.getSortFieldToString(sortCollation, 
rowRelDataType))," +
+  " offset: $SortUtil.getOffsetToString(sortOffset)," +
+  " fetch: $SortUtil.getFetchToString(sortFetch, sortOffset))"
+  }
+  
+  override def explainTerms(pw: RelWriter) : RelWriter = {
+
+//need to identify time between others order fields. Time needs to be 

[jira] [Commented] (FLINK-6075) Support Limit/Top(Sort) for Stream SQL

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

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

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

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

https://github.com/apache/flink/pull/3889#discussion_r116471426
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSort.scala
 ---
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.plan.{ RelOptCluster, RelTraitSet }
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.{ RelNode, RelWriter, SingleRel }
+import org.apache.flink.api.java.tuple.Tuple
+import org.apache.flink.streaming.api.datastream.{ AllWindowedStream, 
DataStream, KeyedStream, WindowedStream }
+import org.apache.flink.streaming.api.windowing.assigners._
+import org.apache.flink.streaming.api.windowing.time.Time
+import org.apache.flink.streaming.api.windowing.windows.{ Window => 
DataStreamWindow }
+import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.expressions._
+import org.apache.flink.table.plan.logical._
+import org.apache.flink.table.plan.nodes.CommonAggregate
+import org.apache.flink.table.runtime.aggregate.AggregateUtil._
+import org.apache.flink.table.runtime.aggregate._
+import org.apache.flink.table.typeutils.TypeCheckUtils.isTimeInterval
+import org.apache.flink.table.typeutils.{ RowIntervalTypeInfo, 
TimeIntervalTypeInfo }
+import org.apache.flink.types.Row
+import org.apache.calcite.sql.SqlAggFunction
+import org.apache.flink.table.plan.nodes.datastream.DataStreamRel
+import org.apache.calcite.sql.fun.SqlSingleValueAggFunction
+import org.apache.flink.api.common.functions.RichMapFunction
+import org.apache.flink.api.common.functions.RichFlatMapFunction
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.util.Collector
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.calcite.rel.core.Sort
+import org.apache.flink.api.java.functions.NullByteKeySelector
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import org.apache.flink.table.runtime.aggregate.SortUtil._
+import org.apache.calcite.rel.RelCollation
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+import org.apache.flink.table.api.{StreamQueryConfig, 
StreamTableEnvironment, TableException}
+import org.apache.flink.table.plan.schema.RowSchema
+
+/**
+  * Flink RelNode which matches along with Sort Rule.
+  *
+ */
+class DataStreamSort(
+  sortCollation: RelCollation,
+  sortOffset: RexNode,
+  sortFetch: RexNode,
+  cluster: RelOptCluster,
+  traitSet: RelTraitSet,
+  inputNode: RelNode,
+  schema: RowSchema,
+  inputSchema: RowSchema,
+  description: String)
+extends SingleRel(cluster, traitSet, inputNode) with DataStreamRel {
+
+  override def deriveRowType(): RelDataType = schema.logicalType
+
+  override def copy(traitSet: RelTraitSet, inputs: 
java.util.List[RelNode]): RelNode = {
+new DataStreamSort(
+  sortCollation,
+  sortOffset,
+  sortFetch,
+  cluster,
+  traitSet,
+  inputs.get(0),
+  schema,
+  inputSchema,
+  description)
+  }
+
+  override def toString: String = {
+s"Sort(by: ($SortUtil.getSortFieldToString(sortCollation, 
rowRelDataType))," +
+  " offset: $SortUtil.getOffsetToString(sortOffset)," +
   

[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116481843
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeSortProcessFunction.scala
 ---
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.aggregate
+
+import org.apache.flink.api.common.state.{ ListState, ListStateDescriptor }
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
+import org.apache.flink.api.java.typeutils.{RowTypeInfo, ListTypeInfo}
+import org.apache.flink.runtime.state.{ FunctionInitializationContext, 
FunctionSnapshotContext }
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.types.Row
+import org.apache.flink.util.{ Collector, Preconditions }
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import scala.util.control.Breaks._
+import org.apache.flink.api.java.tuple.{ Tuple2 => JTuple2 }
+import org.apache.flink.api.common.state.MapState
+import org.apache.flink.api.common.state.MapStateDescriptor
+import org.apache.flink.configuration.Configuration
+import java.util.Comparator
+import java.util.ArrayList
+import java.util.Collections
+import org.apache.flink.api.common.typeutils.TypeComparator
+import java.util.{List => JList, ArrayList => JArrayList}
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+
+/**
+ * Process Function used for the aggregate in bounded rowtime sort without 
offset/fetch
+ * [[org.apache.flink.streaming.api.datastream.DataStream]]
+ *
+ * @param fieldCount Is used to indicate fields in the current element to 
forward
+ * @param inputType It is used to mark the type of the incoming data
--- End diff --

`inputRowType`


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


[GitHub] flink pull request #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream ...

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

https://github.com/apache/flink/pull/3889#discussion_r116455942
  
--- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSort.scala
 ---
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.plan.{ RelOptCluster, RelTraitSet }
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.{ RelNode, RelWriter, SingleRel }
+import org.apache.flink.api.java.tuple.Tuple
+import org.apache.flink.streaming.api.datastream.{ AllWindowedStream, 
DataStream, KeyedStream, WindowedStream }
+import org.apache.flink.streaming.api.windowing.assigners._
+import org.apache.flink.streaming.api.windowing.time.Time
+import org.apache.flink.streaming.api.windowing.windows.{ Window => 
DataStreamWindow }
+import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.expressions._
+import org.apache.flink.table.plan.logical._
+import org.apache.flink.table.plan.nodes.CommonAggregate
+import org.apache.flink.table.runtime.aggregate.AggregateUtil._
+import org.apache.flink.table.runtime.aggregate._
+import org.apache.flink.table.typeutils.TypeCheckUtils.isTimeInterval
+import org.apache.flink.table.typeutils.{ RowIntervalTypeInfo, 
TimeIntervalTypeInfo }
+import org.apache.flink.types.Row
+import org.apache.calcite.sql.SqlAggFunction
+import org.apache.flink.table.plan.nodes.datastream.DataStreamRel
+import org.apache.calcite.sql.fun.SqlSingleValueAggFunction
+import org.apache.flink.api.common.functions.RichMapFunction
+import org.apache.flink.api.common.functions.RichFlatMapFunction
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.util.Collector
+import org.apache.flink.api.common.state.ValueState
+import org.apache.flink.api.common.state.ValueStateDescriptor
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.calcite.rel.core.Sort
+import org.apache.flink.api.java.functions.NullByteKeySelector
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import org.apache.flink.table.runtime.aggregate.SortUtil._
+import org.apache.calcite.rel.RelCollation
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+import org.apache.flink.table.api.{StreamQueryConfig, 
StreamTableEnvironment, TableException}
+import org.apache.flink.table.plan.schema.RowSchema
+
+/**
+  * Flink RelNode which matches along with Sort Rule.
+  *
+ */
+class DataStreamSort(
+  sortCollation: RelCollation,
+  sortOffset: RexNode,
+  sortFetch: RexNode,
+  cluster: RelOptCluster,
+  traitSet: RelTraitSet,
+  inputNode: RelNode,
+  schema: RowSchema,
+  inputSchema: RowSchema,
+  description: String)
+extends SingleRel(cluster, traitSet, inputNode) with DataStreamRel {
+
+  override def deriveRowType(): RelDataType = schema.logicalType
+
+  override def copy(traitSet: RelTraitSet, inputs: 
java.util.List[RelNode]): RelNode = {
+new DataStreamSort(
+  sortCollation,
+  sortOffset,
+  sortFetch,
+  cluster,
+  traitSet,
+  inputs.get(0),
+  schema,
+  inputSchema,
+  description)
+  }
+
+  override def toString: String = {
+s"Sort(by: ($SortUtil.getSortFieldToString(sortCollation, 
rowRelDataType))," +
--- End diff --

Move the methods which are common among `DataSetSort` and `DataStreamSort` 
(e.g. methods to create strings) into a new class `CommonSort`, similar to 
`CommonCalc` or `CommonCorrelate`.


---
If your project is set up for it, you can reply to this email and have your
reply 

[jira] [Created] (FLINK-6593) Fix Bug in ProctimeAttribute or RowtimeAttribute with join UDTF

2017-05-15 Thread Ruidong Li (JIRA)
Ruidong Li created FLINK-6593:
-

 Summary: Fix Bug in ProctimeAttribute or RowtimeAttribute with 
join UDTF
 Key: FLINK-6593
 URL: https://issues.apache.org/jira/browse/FLINK-6593
 Project: Flink
  Issue Type: Bug
Reporter: Ruidong Li
Assignee: Ruidong Li


when a Table has ProctimeAttribute or RowtimeAttribute and join with A UDTF 
function, the `resultNames` parameter of `generateResultExpression` method in 
class `CodeGenerator` did not filter the time indicators, causing a 
CodeGenException("Arity of result field names does not match number of  
expressions.")




--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Created] (FLINK-6592) RawLocalFileSystem created even if fs.hdfs.hadoopconf specified

2017-05-15 Thread Andrey (JIRA)
Andrey created FLINK-6592:
-

 Summary: RawLocalFileSystem created even if fs.hdfs.hadoopconf 
specified
 Key: FLINK-6592
 URL: https://issues.apache.org/jira/browse/FLINK-6592
 Project: Flink
  Issue Type: Bug
Affects Versions: 1.2.0
Reporter: Andrey


This issue affects only BucketingSink.

Steps to reproduce:
* configure "fs.hdfs.hadoopconf" to point to the existing directory with 
core-site.xml and hdfs-site.xml:
** core-site.xml contains valid "fs.defaultFS" entry which points to 
"hdfs://"
* configure bucketing sink: 
{code}
new BucketingSink("/some/path/within/defaultFS")
{code}
* Run the job
* In the logs:
{code}
Mkdirs failed to create /some/path/within/defaultFS
{code}
* hdfs filesystem implementation determined based on "basePath" and its schema: 
{code}
new Path(basePath).getFileSystem(hadoopConf);
{code}

Expected:
* BucketingSink detect filesystem type based on 
"org.apache.hadoop.conf.Configuration" rather than "basePath".




--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Closed] (FLINK-2593) Revisit Flink's shading policy

2017-05-15 Thread Robert Metzger (JIRA)

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

Robert Metzger closed FLINK-2593.
-
Resolution: Duplicate

Subsumed by FLINK-6529

> Revisit Flink's shading policy
> --
>
> Key: FLINK-2593
> URL: https://issues.apache.org/jira/browse/FLINK-2593
> Project: Flink
>  Issue Type: Improvement
>  Components: Build System
>Reporter: Till Rohrmann
>Priority: Minor
>
> At the moment Flink does not follow a consistent shading policy which 
> dictates which dependencies should be shaded away and which to keep. This 
> becomes especially important for utility libraries which are likely to be 
> used by the users as well. By not shading these dependencies, the user is 
> constrained to use exactly the same version in his code. This might cause 
> problems for the user.
> Some of the dependencies which Flink should probably shade away are {{Akka}}, 
> {{Protobuf}} and {{Curator}}, for example.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Closed] (FLINK-2816) Memory allocation module as a separate maven artifact

2017-05-15 Thread Robert Metzger (JIRA)

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

Robert Metzger closed FLINK-2816.
-
Resolution: Won't Fix

> Memory allocation module as a separate maven artifact
> -
>
> Key: FLINK-2816
> URL: https://issues.apache.org/jira/browse/FLINK-2816
> Project: Flink
>  Issue Type: Task
>  Components: Build System
>Reporter: Suminda Dharmasena
>Priority: Minor
>
> Can you package memory allocation module as a separate maven artifact



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Closed] (FLINK-5661) flink-mesos should shade its httpclient dependency away

2017-05-15 Thread Robert Metzger (JIRA)

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

Robert Metzger closed FLINK-5661.
-
Resolution: Invalid

Addressed in 63bbb4f533a0f11376834d584308e90b6b8b5f10

> flink-mesos should shade its httpclient dependency away
> ---
>
> Key: FLINK-5661
> URL: https://issues.apache.org/jira/browse/FLINK-5661
> Project: Flink
>  Issue Type: Bug
>  Components: Build System, Mesos
>Affects Versions: 1.3.0
>Reporter: Robert Metzger
>Assignee: Tzu-Li (Gordon) Tai
>
> This issue has been reported by a user: 
> http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Flink-dependencies-shading-td11304.html
> The httpclient library tends to clash with various other dependencies 
> (elasticsearch, awssdk, ...).
> I think we can just shade it in the flink-mesos module.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[GitHub] flink issue #3908: [FLINK-5340] Document job uptime/downtime/restartTime/ful...

2017-05-15 Thread alpinegizmo
Github user alpinegizmo commented on the issue:

https://github.com/apache/flink/pull/3908
  
"The time that the job has not been running in a failing/recovering 
situation." is confusing. How about this:

The time the job has spent in a failing/recovering situation.



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


[jira] [Commented] (FLINK-5340) Add a metric exposing jobs uptimes

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

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

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

Github user alpinegizmo commented on the issue:

https://github.com/apache/flink/pull/3908
  
"The time that the job has not been running in a failing/recovering 
situation." is confusing. How about this:

The time the job has spent in a failing/recovering situation.



> Add a metric exposing jobs uptimes
> --
>
> Key: FLINK-5340
> URL: https://issues.apache.org/jira/browse/FLINK-5340
> Project: Flink
>  Issue Type: New Feature
>  Components: Core
>Reporter: Dan Bress
>Assignee: Stephan Ewen
>Priority: Minor
> Fix For: 1.3.0
>
>
> I would like the job manager to expose a metric indicating how long each job 
> has been up.  This way I can grab this number and measure the health of my 
> job.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Updated] (FLINK-6591) Extend functionality of final ConversionMapper

2017-05-15 Thread Timo Walther (JIRA)

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

Timo Walther updated FLINK-6591:

Affects Version/s: 1.3.0

> Extend functionality of final ConversionMapper
> --
>
> Key: FLINK-6591
> URL: https://issues.apache.org/jira/browse/FLINK-6591
> Project: Flink
>  Issue Type: Improvement
>  Components: Table API & SQL
>Affects Versions: 1.3.0
>Reporter: Timo Walther
>
> The functionality of the ConversionMapper generated in 
> {{TableEnvironment#generateRowConverterFunction}} is very limited right now:
> - It does not support conversion of nested Row types, e.g. a nested 
> Avro-record can be read with the KafkaAvroTableSource into a nested row 
> structure, but this structure can not be converted back into a Pojo or Avro 
> structure. The code generator needs to be extended for this.
> - The Table API supports BasicTypeInfo (e.g. {{Integer[]}}) as an input 
> field, but since it works with ObjectTypeInfo internally, it cannot output 
> the array. I disabled the test 
> {{TableEnvironmentITCase#testAsFromTupleToPojo}} for now.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Created] (FLINK-6591) Extend functionality of final ConversionMapper

2017-05-15 Thread Timo Walther (JIRA)
Timo Walther created FLINK-6591:
---

 Summary: Extend functionality of final ConversionMapper
 Key: FLINK-6591
 URL: https://issues.apache.org/jira/browse/FLINK-6591
 Project: Flink
  Issue Type: Improvement
  Components: Table API & SQL
Reporter: Timo Walther


The functionality of the ConversionMapper generated in 
{{TableEnvironment#generateRowConverterFunction}} is very limited right now:


- It does not support conversion of nested Row types, e.g. a nested Avro-record 
can be read with the KafkaAvroTableSource into a nested row structure, but this 
structure can not be converted back into a Pojo or Avro structure. The code 
generator needs to be extended for this.

- The Table API supports BasicTypeInfo (e.g. {{Integer[]}}) as an input field, 
but since it works with ObjectTypeInfo internally, it cannot output the array. 
I disabled the test {{TableEnvironmentITCase#testAsFromTupleToPojo}} for now.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (FLINK-6440) Noisy logs from metric fetcher

2017-05-15 Thread Stephan Ewen (JIRA)

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

Stephan Ewen commented on FLINK-6440:
-

Just logging on {{debug}} might be a reasonable first fix.
Ideally there is one {{info}} level log event at the first failed poll, and 
then more debug level logs for succeeding failed pools, but that requires state 
(and cleanup/expiry of that), which is not a road we should go down, I think...

> Noisy logs from metric fetcher
> --
>
> Key: FLINK-6440
> URL: https://issues.apache.org/jira/browse/FLINK-6440
> Project: Flink
>  Issue Type: Bug
>  Components: Webfrontend
>Affects Versions: 1.3.0
>Reporter: Stephan Ewen
>Priority: Critical
> Fix For: 1.3.0
>
>
> In cases where TaskManagers fail, the web frontend in the Job Manager starts 
> logging the exception below every few seconds.
> I labeled this as critical, because it actually makes debugging in such a 
> situation complicated through a log that is flooded with noise.
> {code}
> 2017-05-03 19:37:07,823 WARN  
> org.apache.flink.runtime.webmonitor.metrics.MetricFetcher - Fetching 
> metrics failed.
> akka.pattern.AskTimeoutException: Ask timed out on 
> [Actor[akka.tcp://flink@herman:52175/user/MetricQueryService_136f717a6b91e248282cb2937d22088c]]
>  after [1 ms]
> at 
> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:334)
> at akka.actor.Scheduler$$anon$7.run(Scheduler.scala:117)
> at 
> scala.concurrent.Future$InternalCallbackExecutor$.scala$concurrent$Future$InternalCallbackExecutor$$unbatchedExecute(Future.scala:694)
> at 
> scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:691)
> at 
> akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(Scheduler.scala:474)
> at 
> akka.actor.LightArrayRevolverScheduler$$anon$8.executeBucket$1(Scheduler.scala:425)
> at 
> akka.actor.LightArrayRevolverScheduler$$anon$8.nextTick(Scheduler.scala:429)
> at 
> akka.actor.LightArrayRevolverScheduler$$anon$8.run(Scheduler.scala:381)
> at java.lang.Thread.run(Thread.java:745)
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (FLINK-6442) Extend TableAPI Support Sink Table Registration and ‘insert into’ Clause in SQL

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

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

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

Github user lincoln-lil commented on the issue:

https://github.com/apache/flink/pull/3829
  
rebased on master branch


> Extend TableAPI Support Sink Table Registration and ‘insert into’ Clause in 
> SQL
> ---
>
> Key: FLINK-6442
> URL: https://issues.apache.org/jira/browse/FLINK-6442
> Project: Flink
>  Issue Type: New Feature
>  Components: Table API & SQL
>Reporter: lincoln.lee
>Assignee: lincoln.lee
>Priority: Minor
>
> Currently in TableAPI  there’s only registration method for source table,  
> when we use SQL writing a streaming job, we should add additional part for 
> the sink, like TableAPI does:
> {code}
> val sqlQuery = "SELECT * FROM MyTable WHERE _1 = 3"
> val t = StreamTestData.getSmall3TupleDataStream(env)
> tEnv.registerDataStream("MyTable", t)
> // one way: invoke tableAPI’s writeToSink method directly
> val result = tEnv.sql(sqlQuery)
> result.writeToSink(new YourStreamSink)
> // another way: convert to datastream first and then invoke addSink 
> val result = tEnv.sql(sqlQuery).toDataStream[Row]
> result.addSink(new StreamITCase.StringSink)
> {code}
> From the api we can see the sink table always be a derived table because its 
> 'schema' is inferred from the result type of upstream query.
> Compare to traditional RDBMS which support DML syntax, a query with a target 
> output could be written like this:
> {code}
> insert into table target_table_name
> [(column_name [ ,...n ])]
> query
> {code}
> The equivalent form of the example above is as follows:
> {code}
> tEnv.registerTableSink("targetTable", new YourSink)
> val sql = "INSERT INTO targetTable SELECT a, b, c FROM sourceTable"
> val result = tEnv.sql(sql)
> {code}
> It is supported by Calcite’s grammar: 
> {code}
>  insert:( INSERT | UPSERT ) INTO tablePrimary
>  [ '(' column [, column ]* ')' ]
>  query
> {code}
> I'd like to extend Flink TableAPI to support such feature.  see design doc: 
> https://goo.gl/n3phK5



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Created] (FLINK-6590) Integrate generated tables into documentation

2017-05-15 Thread Chesnay Schepler (JIRA)
Chesnay Schepler created FLINK-6590:
---

 Summary: Integrate generated tables into documentation
 Key: FLINK-6590
 URL: https://issues.apache.org/jira/browse/FLINK-6590
 Project: Flink
  Issue Type: Sub-task
  Components: Documentation
Reporter: Chesnay Schepler
Assignee: Chesnay Schepler






--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[GitHub] flink issue #3829: [FLINK-6442] [table] Extend TableAPI Support Sink Table R...

2017-05-15 Thread lincoln-lil
Github user lincoln-lil commented on the issue:

https://github.com/apache/flink/pull/3829
  
rebased on master branch


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


[jira] [Commented] (FLINK-6370) FileAlreadyExistsException on startup

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

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

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

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

https://github.com/apache/flink/pull/3896#discussion_r116500253
  
--- Diff: 
flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java
 ---
@@ -234,8 +237,17 @@ private void respondAsLeader(ChannelHandlerContext 
ctx, HttpRequest request, Str
if 
(!rootURI.relativize(requestedURI).equals(requestedURI)) {

logger.debug("Loading missing file from classloader: {}", requestPath);
// ensure that 
directory to file exists.
-   
file.getParentFile().mkdirs();
-   
Files.copy(resourceStream, file.toPath());
+   if 
(!file.getParentFile().mkdirs()) {
+   throw 
new IOException("Could not create directories for file " + file);
+   }
+   synchronized 
(COPY_LOCK) {
--- End diff --

There is nothing stopping us from changing to eager loading later on, so I 
opted for the fastest, yet still reasonable, fix for the problem.


> FileAlreadyExistsException on startup
> -
>
> Key: FLINK-6370
> URL: https://issues.apache.org/jira/browse/FLINK-6370
> Project: Flink
>  Issue Type: Bug
>  Components: Webfrontend
>Affects Versions: 1.2.0
>Reporter: Andrey
>Assignee: Chesnay Schepler
>
> Currently static web resources are lazily cached onto disk during first 
> request. However if 2 concurrent requests will be executed, then 
> FileAlreadyExistsException will be in logs.
> {code}
> 2017-04-24 14:00:58,075 ERROR 
> org.apache.flink.runtime.webmonitor.files.StaticFileServerHandler  - error 
> while responding [nioEventLoopGroup-3-2]
> java.nio.file.FileAlreadyExistsException: 
> /flink/web/flink-web-528f8cb8-dd60-433c-8f6c-df49ad0b79e0/index.html
>   at 
> sun.nio.fs.UnixException.translateToIOException(UnixException.java:88)
>   at sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:102)
>   at sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:107)
>   at 
> sun.nio.fs.UnixFileSystemProvider.newByteChannel(UnixFileSystemProvider.java:214)
>   at 
> java.nio.file.spi.FileSystemProvider.newOutputStream(FileSystemProvider.java:434)
>   at java.nio.file.Files.newOutputStream(Files.java:216)
>   at java.nio.file.Files.copy(Files.java:3016)
>   at 
> org.apache.flink.runtime.webmonitor.files.StaticFileServerHandler.respondAsLeader(StaticFileServerHandler.java:238)
>   at 
> org.apache.flink.runtime.webmonitor.files.StaticFileServerHandler.channelRead0(StaticFileServerHandler.java:197)
>   at 
> org.apache.flink.runtime.webmonitor.files.StaticFileServerHandler.channelRead0(StaticFileServerHandler.java:99)
>   at 
> io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105)
>   at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:339)
>   at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:324)
>   at io.netty.handler.codec.http.router.Handler.routed(Handler.java:62)
> {code}
> Expect: 
> * extract all static resources on startup in main thread and before opening 
> http port.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[GitHub] flink pull request #3896: [FLINK-6370] [webUI] Handle races for single file ...

2017-05-15 Thread zentol
Github user zentol commented on a diff in the pull request:

https://github.com/apache/flink/pull/3896#discussion_r116500253
  
--- Diff: 
flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java
 ---
@@ -234,8 +237,17 @@ private void respondAsLeader(ChannelHandlerContext 
ctx, HttpRequest request, Str
if 
(!rootURI.relativize(requestedURI).equals(requestedURI)) {

logger.debug("Loading missing file from classloader: {}", requestPath);
// ensure that 
directory to file exists.
-   
file.getParentFile().mkdirs();
-   
Files.copy(resourceStream, file.toPath());
+   if 
(!file.getParentFile().mkdirs()) {
+   throw 
new IOException("Could not create directories for file " + file);
+   }
+   synchronized 
(COPY_LOCK) {
--- End diff --

There is nothing stopping us from changing to eager loading later on, so I 
opted for the fastest, yet still reasonable, fix for the problem.


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


[jira] [Commented] (FLINK-6370) FileAlreadyExistsException on startup

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

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

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

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

https://github.com/apache/flink/pull/3896#discussion_r116500027
  
--- Diff: 
flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/files/StaticFileServerHandler.java
 ---
@@ -234,8 +237,17 @@ private void respondAsLeader(ChannelHandlerContext 
ctx, HttpRequest request, Str
if 
(!rootURI.relativize(requestedURI).equals(requestedURI)) {

logger.debug("Loading missing file from classloader: {}", requestPath);
// ensure that 
directory to file exists.
-   
file.getParentFile().mkdirs();
-   
Files.copy(resourceStream, file.toPath());
+   if 
(!file.getParentFile().mkdirs()) {
+   throw 
new IOException("Could not create directories for file " + file);
+   }
+   synchronized 
(COPY_LOCK) {
--- End diff --

Because this was easy to write, is easy to review, does not change any 
behavior and isn't particularly intrusive.

I agree that we should have a discussion as to whether we should load the 
files lazily or not, but not now when the next release is coming up and 
everyone is scrambling to fix the most issues in as little time as possible


> FileAlreadyExistsException on startup
> -
>
> Key: FLINK-6370
> URL: https://issues.apache.org/jira/browse/FLINK-6370
> Project: Flink
>  Issue Type: Bug
>  Components: Webfrontend
>Affects Versions: 1.2.0
>Reporter: Andrey
>Assignee: Chesnay Schepler
>
> Currently static web resources are lazily cached onto disk during first 
> request. However if 2 concurrent requests will be executed, then 
> FileAlreadyExistsException will be in logs.
> {code}
> 2017-04-24 14:00:58,075 ERROR 
> org.apache.flink.runtime.webmonitor.files.StaticFileServerHandler  - error 
> while responding [nioEventLoopGroup-3-2]
> java.nio.file.FileAlreadyExistsException: 
> /flink/web/flink-web-528f8cb8-dd60-433c-8f6c-df49ad0b79e0/index.html
>   at 
> sun.nio.fs.UnixException.translateToIOException(UnixException.java:88)
>   at sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:102)
>   at sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:107)
>   at 
> sun.nio.fs.UnixFileSystemProvider.newByteChannel(UnixFileSystemProvider.java:214)
>   at 
> java.nio.file.spi.FileSystemProvider.newOutputStream(FileSystemProvider.java:434)
>   at java.nio.file.Files.newOutputStream(Files.java:216)
>   at java.nio.file.Files.copy(Files.java:3016)
>   at 
> org.apache.flink.runtime.webmonitor.files.StaticFileServerHandler.respondAsLeader(StaticFileServerHandler.java:238)
>   at 
> org.apache.flink.runtime.webmonitor.files.StaticFileServerHandler.channelRead0(StaticFileServerHandler.java:197)
>   at 
> org.apache.flink.runtime.webmonitor.files.StaticFileServerHandler.channelRead0(StaticFileServerHandler.java:99)
>   at 
> io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105)
>   at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:339)
>   at 
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:324)
>   at io.netty.handler.codec.http.router.Handler.routed(Handler.java:62)
> {code}
> Expect: 
> * extract all static resources on startup in main thread and before opening 
> http port.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


<    1   2   3   4   5   >