You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by rtudoran <gi...@git.apache.org> on 2017/05/12 18:45:39 UTC

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

GitHub user rtudoran opened a pull request:

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

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

    Order by time and optionally other fields is supported
    
    Thanks for contributing to Apache Flink. Before you open your pull request, please take the following check list into consideration.
    If your changes take all of the items into account, feel free to open your pull request. For more information and/or questions please refer to the [How To Contribute guide](http://flink.apache.org/how-to-contribute.html).
    In addition to going through the list, please provide a meaningful description of your changes.
    
    - [x ] General
      - The pull request references the related JIRA issue ("[FLINK-XXX] Jira title text")
      - The pull request addresses only one issue
      - Each commit in the PR has a meaningful commit message (including the JIRA id)
    
    - [x ] Documentation
      - Documentation has been added for new functionality
      - Old documentation affected by the pull request has been updated
      - JavaDoc for public methods has been added
    
    - [x ] Tests & Build
      - Functionality added by the pull request is covered by tests
      - `mvn clean verify` has been executed successfully locally or a Travis build has passed


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

    $ git pull https://github.com/huawei-flink/flink FLINK-6075Re3R

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

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

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

    This closes #3889
    
----
commit 233577a7029f671e4aac24dc05d367d018d978a0
Author: rtudoran <tu...@ymail.com>
Date:   2017-05-12T18:41:30Z

    Sort support moved to 1.3 interface
    Order by time and optionally other fields is supported

----


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123712536
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetSort.scala ---
    @@ -136,36 +130,21 @@ class DataSetSort(
         }
       }
     
    -  private def directionToOrder(direction: Direction) = {
    -    direction match {
    -      case Direction.ASCENDING | Direction.STRICTLY_ASCENDING => Order.ASCENDING
    -      case Direction.DESCENDING | Direction.STRICTLY_DESCENDING => Order.DESCENDING
    -      case _ => throw new IllegalArgumentException("Unsupported direction.")
    -    }
    -
    -  }
    -
       private val fieldCollations = collations.getFieldCollations.asScala
         .map(c => (c.getFieldIndex, directionToOrder(c.getDirection)))
     
    -  private val sortFieldsToString = fieldCollations
    -    .map(col => s"${getRowType.getFieldNames.get(col._1)} ${col._2.getShortName}" ).mkString(", ")
    -
    -  private val offsetToString = s"$offset"
    -
    -  private val fetchToString = if (limitEnd == Long.MaxValue) {
    -    "unlimited"
    -  } else {
    -    s"$limitEnd"
    +    
    +  override def toString: String = {
    +    s"Sort(by: ($$sortFieldsToString(collations, getRowType))," +
    +      " offset: $offsetToString(offset)," +
    +      " fetch: $fetchToString(fetch, offset))"
       }
    -
    -  override def toString: String =
    -    s"Sort(by: ($sortFieldsToString), offset: $offsetToString, fetch: $fetchToString)"
    -
    +    
    +    
       override def explainTerms(pw: RelWriter) : RelWriter = {
         super.explainTerms(pw)
    -      .item("orderBy", sortFieldsToString)
    -      .item("offset", offsetToString)
    -      .item("fetch", fetchToString)
    +      .item("orderBy", sortFieldsToString(collations, getRowType))
    --- End diff --
    
    Move the implementation to `CommonSort`


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

[GitHub] flink issue #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream SQL

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

    https://github.com/apache/flink/pull/3889
  
    @fhueske I updated the code to work with 1.3 (CRow, FlinkLogicalSort....) and all the implied changes.
    We can close #3714 after we merge this i guess
    ..that one is the same logical code only on the old interface
    



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

[GitHub] flink issue #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream SQL

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

    https://github.com/apache/flink/pull/3889
  
    @fhueske @stefanobortoli @shijinkui @hongyuhong 
    I have updated the PR. In principle all remarks were addressed. This should be ready to merge (i hope)


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

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

Posted by fhueske <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

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

    https://github.com/apache/flink/pull/3889#discussion_r124574652
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/TimeSortProcessFunctionTest.scala ---
    @@ -0,0 +1,256 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.RowTimeSortProcessFunction
    +import org.apache.flink.table.runtime.aggregate.TimeSortProcessFunctionTest._
    +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}
    +import org.apache.flink.streaming.api.TimeCharacteristic
    +
    +class TimeSortProcessFunctionTest{
    --- End diff --
    
    I do not understand - this test was added (see below testSortRowTimeHarnessPartitioned) ...perhaps this is an old 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

Posted by fhueske <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

Posted by fhueske <gi...@git.apache.org>.
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


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123759708
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/TimeSortProcessFunctionTest.scala ---
    @@ -0,0 +1,256 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.RowTimeSortProcessFunction
    +import org.apache.flink.table.runtime.aggregate.TimeSortProcessFunctionTest._
    +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}
    +import org.apache.flink.streaming.api.TimeCharacteristic
    +
    +class TimeSortProcessFunctionTest{
    +
    +  
    +  @Test
    +  def testSortProcTimeHarnessPartitioned(): Unit = {
    +    
    +    val rT =  new RowTypeInfo(Array[TypeInformation[_]](
    +      INT_TYPE_INFO,
    +      LONG_TYPE_INFO,
    +      INT_TYPE_INFO,
    +      STRING_TYPE_INFO,
    +      LONG_TYPE_INFO),
    +      Array("a","b","c","d","e"))
    +    
    +    val rTA =  new RowTypeInfo(Array[TypeInformation[_]](
    +     LONG_TYPE_INFO), Array("count"))
    +    val indexes = Array(1,2)
    --- End diff --
    
    space


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r116526993
  
    --- 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 RowTimeSortProcessFunction(
    +      inputType.getFieldCount,
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +  
    +  
    +  /**
    +   * Function creates [org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
    +   * elements based on proctime 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 createProcTimeSortFunction(
    +    collationSort: RelCollation,
    +    inputType: RelDataType,
    +    inputTypeInfo: TypeInformation[Row],
    +    execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
    +
    +    val keySortFields = getSortFieldIndexList(collationSort)
    +    val keySortDirections = getSortFieldDirectionList(collationSort)
    +
    +    
    +       //drop time from comparison
    +    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 ProcTimeSortProcessFunction(
    +      inputType.getFieldCount,
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +
    +  
    +   /**
    +   * Function creates comparison objects based on the field types
    +   * @param inputType input row type
    +   * @param keyIndex the indexes of the fields on which the sorting is done. 
    +   * @param orderDirection the directions of each sort field. 
    +   * @param execConfig the configuration environment
    +   * @return Array of TypeComparator objects
    +   */
    +  def createFieldComparators(
    +      inputType: RelDataType,
    +      keyIndex: Array[Int],
    +      orderDirection: Array[Direction],
    +      execConfig: ExecutionConfig): Array[TypeComparator[_]] = {
    +    
    +    var iOrder = 0
    +    for (i <- keyIndex) yield {
    +
    +      val order = if (orderDirection(iOrder) == Direction.ASCENDING) true else false
    +      iOrder += 1
    +      val fieldTypeInfo = FlinkTypeFactory.toTypeInfo(inputType.getFieldList.get(i).getType)
    +      fieldTypeInfo match {
    +        case a: AtomicType[_] => a.createComparator(order, execConfig)
    +        case _ => throw new TableException(s"Unsupported field type $fieldTypeInfo to sort on.")
    +      }
    +    }
    +  }
    +  
    +   /**
    +   * Function creates a RowComparator based on the typed comparators
    +   * @param inputRowType input row type
    +   * @param fieldIdxs the indexes of the fields on which the sorting is done. 
    +   * @param fieldComps the array of typed comparators
    +   * @param fieldOrders the directions of each sort field (true = ASC). 
    +   * @return A Row TypeComparator object 
    +   */
    +  def createRowComparator(
    +    inputRowType: RelDataType,
    +    fieldIdxs: Array[Int],
    +    fieldComps: Array[TypeComparator[AnyRef]],
    +    fieldOrders: Array[Boolean]): TypeComparator[Row] = {
    +
    +  val rowComp = new RowComparator(
    +    inputRowType.getFieldCount,
    +    fieldIdxs,
    +    fieldComps,
    +    new Array[TypeSerializer[AnyRef]](0), //used only for serialized comparisons
    +    fieldOrders)
    +
    +  rowComp
    +}
    +  
    + 
    +  /**
    +   * Function returns the array of indexes for the fields on which the sort is done
    +   * @param collationSort The Sort collation list
    +   * @return [Array[Int]]
    +   */
    +  def getSortFieldIndexList(collationSort: RelCollation): Array[Int] = {
    +    val keyFields = collationSort.getFieldCollations.toArray()
    +    for (f <- keyFields) yield f.asInstanceOf[RelFieldCollation].getFieldIndex
    +  }
    +  
    +   /**
    +   * Function returns the array of sort direction for each of the sort fields 
    +   * @param collationSort The Sort collation list
    +   * @return [Array[Direction]]
    +   */
    +  def getSortFieldDirectionList(collationSort: RelCollation): Array[Direction] = {
    +    val keyFields = collationSort.getFieldCollations.toArray()
    +    for(f <- keyFields) yield f.asInstanceOf[RelFieldCollation].getDirection
    +  }
    +
    +   /**
    +   * Function returns the array of sort direction for each of the sort fields 
    +   * @param collationSort The Sort collation list
    +   * @return [Array[Direction]]
    +   */
    +  def getSortFieldDirectionBooleanList(collationSort: RelCollation): Array[Boolean] = {
    +    val keyFields = collationSort.getFieldCollations
    +    var i = 0
    +    val keySortDirection = new Array[Boolean](keyFields.size())
    +    while (i < keyFields.size()) {
    +      keySortDirection(i) = 
    +        if (getDirection(collationSort,i) == Direction.ASCENDING) true else false 
    --- End diff --
    
    +space: `getDirection(collationSort, i)`


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r124563186
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 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}
    +
    +/**
    + * 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)
    +    //drop time from comparison as we sort on time in the states and result emission
    +    val keyIndexesNoTime = keySortFields.slice(1, keySortFields.size)
    +    val booleanOrderings = getSortFieldDirectionBooleanList(collationSort)
    +    val booleanDirectionsNoTime = booleanOrderings.slice(1, booleanOrderings.size)
    +    
    +    val fieldComps = createFieldComparators(
    +      inputType, 
    +      keyIndexesNoTime, 
    +      booleanDirectionsNoTime,
    +      execCfg)
    +    val fieldCompsRefs = fieldComps.asInstanceOf[Array[TypeComparator[AnyRef]]]
    +    
    +    val rowComp = new RowComparator(
    +       inputType.getFieldCount,
    +       keyIndexesNoTime,
    +       fieldCompsRefs,
    +       new Array[TypeSerializer[AnyRef]](0), //used only for object comparisons
    +       booleanDirectionsNoTime)
    +      
    +    val collectionRowComparator = new CollectionRowComparator(rowComp)
    +    
    +    val inputCRowType = CRowTypeInfo(inputTypeInfo)
    + 
    +    new RowTimeSortProcessFunction(
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +  
    +  
    +  /**
    +   * Function creates [org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
    +   * elements based on proctime 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 createProcTimeSortFunction(
    +    collationSort: RelCollation,
    +    inputType: RelDataType,
    +    inputTypeInfo: TypeInformation[Row],
    +    execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
    +
    +    val keySortFields = getSortFieldIndexList(collationSort)
    --- End diff --
    
    I am not sure I understand what you mean. But I think the current setup to create the processfunction based on rowtime/ proctime are needed and is useful also for adding later the cases when we have offset/fetch. I would not like to modify this as it does not bring value (but it might be that I misunderstood what you mean)


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123744145
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 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}
    +
    +/**
    + * 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)
    +    //drop time from comparison as we sort on time in the states and result emission
    +    val keyIndexesNoTime = keySortFields.slice(1, keySortFields.size)
    +    val booleanOrderings = getSortFieldDirectionBooleanList(collationSort)
    +    val booleanDirectionsNoTime = booleanOrderings.slice(1, booleanOrderings.size)
    +    
    +    val fieldComps = createFieldComparators(
    +      inputType, 
    +      keyIndexesNoTime, 
    +      booleanDirectionsNoTime,
    +      execCfg)
    +    val fieldCompsRefs = fieldComps.asInstanceOf[Array[TypeComparator[AnyRef]]]
    --- End diff --
    
    You can change `createFieldComparators` to return an `Array[TypeComparator[AnyRef]]` and avoid the additional cast.


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123718029
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamSortRule.scala ---
    @@ -0,0 +1,111 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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
    +import org.apache.flink.table.calcite.FlinkTypeFactory
    +import org.apache.flink.table.api.TableException
    +import org.apache.calcite.rel.RelFieldCollation.Direction
    +import org.apache.flink.table.runtime.aggregate.SortUtil
    +
    +/**
    + * 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 = {
    +
    +    val result = super.matches(call)
    +    
    +    //need to identify time between others order fields. Time needs to be first sort element
    +    // we can safely convert the object if the match rule succeeded 
    +    if(result) {
    +      val calcSort: FlinkLogicalSort = call.rel(0).asInstanceOf[FlinkLogicalSort]
    +      checkTimeOrder(calcSort)
    +    }
    +    
    +    result
    +  }
    +
    +  override def convert(rel: RelNode): RelNode = {
    +    val calcSort: FlinkLogicalSort = rel.asInstanceOf[FlinkLogicalSort]
    +    val traitSet: RelTraitSet = rel.getTraitSet.replace(FlinkConventions.DATASTREAM)
    +    val convInput: RelNode = RelOptRule.convert(calcSort.getInput(0), FlinkConventions.DATASTREAM)
    +    
    +    val inputRowType = convInput.asInstanceOf[RelSubset].getOriginal.getRowType
    +
    +    new DataStreamSort(
    +      rel.getCluster,
    +      traitSet,
    +      convInput,
    +      new RowSchema(inputRowType),
    +      new RowSchema(rel.getRowType),
    +      calcSort.collation,
    +      calcSort.offset,
    +      calcSort.fetch,
    +      description)
    +    
    +  }
    +  
    +   
    +  /**
    +   * Function is used to check at verification time if the SQL syntax is supported
    +   */
    +  
    +  def checkTimeOrder(calcSort: FlinkLogicalSort) = {
    +    
    +    val rowType = calcSort.getRowType
    +    val sortCollation = calcSort.collation 
    +     //need to identify time between others order fields. Time needs to be first sort element
    --- End diff --
    
    indention


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r124491949
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonSort.scala ---
    @@ -0,0 +1,91 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.table.plan.nodes
    +
    +import org.apache.calcite.rex.{RexLiteral, RexNode}
    +import org.apache.calcite.rel.RelFieldCollation
    +import org.apache.calcite.rel.RelCollation
    +import org.apache.calcite.rel.RelFieldCollation.Direction
    +import org.apache.calcite.rel.`type`._
    +import scala.collection.JavaConverters._
    +import org.apache.flink.api.common.operators.Order
    +
    +
    +/**
    + * Trait represents a collection of sort methods to manipulate the parameters
    + */
    +
    +trait CommonSort {
    +  
    +  private[flink] def offsetToString(offset: RexNode): String = {
    +    val offsetToString = s"$offset"
    +    offsetToString
    +  }
    +  
    +  
    +  private[flink] def sortFieldsToString(
    --- End diff --
    
    it cannot be moved because explainTerms is a method from the super class SingleRel and only the DataStream/SetSort extends it not the CommonSort. Or is it possible in scala to have a trait that overrides a method that is inherited by another class?


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

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

Posted by fhueske <gi...@git.apache.org>.
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 RowTimeSortProcessFunction(
    +      inputType.getFieldCount,
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +  
    +  
    +  /**
    +   * Function creates [org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
    +   * elements based on proctime 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 createProcTimeSortFunction(
    +    collationSort: RelCollation,
    +    inputType: RelDataType,
    +    inputTypeInfo: TypeInformation[Row],
    +    execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
    +
    +    val keySortFields = getSortFieldIndexList(collationSort)
    +    val keySortDirections = getSortFieldDirectionList(collationSort)
    +
    +    
    +       //drop time from comparison
    +    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 ProcTimeSortProcessFunction(
    +      inputType.getFieldCount,
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +
    +  
    +   /**
    +   * Function creates comparison objects based on the field types
    +   * @param inputType input row type
    +   * @param keyIndex the indexes of the fields on which the sorting is done. 
    +   * @param orderDirection the directions of each sort field. 
    +   * @param execConfig the configuration environment
    +   * @return Array of TypeComparator objects
    +   */
    +  def createFieldComparators(
    +      inputType: RelDataType,
    +      keyIndex: Array[Int],
    +      orderDirection: Array[Direction],
    +      execConfig: ExecutionConfig): Array[TypeComparator[_]] = {
    +    
    +    var iOrder = 0
    +    for (i <- keyIndex) yield {
    +
    +      val order = if (orderDirection(iOrder) == Direction.ASCENDING) true else false
    --- End diff --
    
    can be done as
    
    ```
    for ((k, o) <- keyIndex.zip(orderDirection)) yield {
      FlinkTypeFactory.toTypeInfo(inputType.getFieldList.get(k).getType) match {
        case a: AtomicType[_] => a.createComparator(o, execConfig)
        case x: TypeInformation[_] => 
          throw new TableException(s"Unsupported field type $x to sort on.")
      }
    }
    ```


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123709906
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SortITCase.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.api.scala.stream.sql
    +
    +import org.apache.flink.api.scala._
    +import org.apache.flink.table.api.scala.stream.sql.SortITCase.StringRowSelectorSink
    +import org.apache.flink.table.api.scala.stream.sql.TimeTestUtil.EventTimeSourceFunction
    +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"))),
    +      Left((2000L, (3L, 1, "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"))),
    +      Left((6000L, (6L, 67, "Hello"))),
    +      Left((6000L, (6L, -1, "Hello"))),
    +      Left((6000L, (6L, 6, "Hello"))),
    +      Right(7000L),
    +      Left((9000L, (6L, 9, "Hello"))),
    +      Left((8500L, (6L, 18, "Hello"))),
    +      Left((9000L, (6L, 7, "Hello"))),
    +      Right(10000L),
    +      Left((10000L, (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((20000L, (20L, 20, "Hello World"))), 
    +      Right(19000L))
    +
    +    val env = StreamExecutionEnvironment.getExecutionEnvironment
    +    env.setStateBackend(getStateBackend)
    +    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]
    --- End diff --
    
    `toDataStream` is deprecated. Please use `toAppendStream` instead.


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

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

Posted by fhueske <gi...@git.apache.org>.
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


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

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

Posted by fhueske <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

Posted by fhueske <gi...@git.apache.org>.
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,
    +        keyIndexesNoTime, fieldCompsRefs, booleanDirectionsNoTime)
    +    val collectionRowComparator = new CollectionRowComparator(rowComp)
    +    
    +    val inputCRowType = CRowTypeInfo(inputTypeInfo)
    + 
    +    new RowTimeSortProcessFunction(
    +      inputType.getFieldCount,
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +  
    +  
    +  /**
    +   * Function creates [org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
    +   * elements based on proctime 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 createProcTimeSortFunction(
    +    collationSort: RelCollation,
    +    inputType: RelDataType,
    +    inputTypeInfo: TypeInformation[Row],
    +    execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
    +
    +    val keySortFields = getSortFieldIndexList(collationSort)
    +    val keySortDirections = getSortFieldDirectionList(collationSort)
    +
    +    
    +       //drop time from comparison
    +    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 ProcTimeSortProcessFunction(
    +      inputType.getFieldCount,
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +
    +  
    +   /**
    +   * Function creates comparison objects based on the field types
    +   * @param inputType input row type
    +   * @param keyIndex the indexes of the fields on which the sorting is done. 
    +   * @param orderDirection the directions of each sort field. 
    +   * @param execConfig the configuration environment
    +   * @return Array of TypeComparator objects
    +   */
    +  def createFieldComparators(
    +      inputType: RelDataType,
    +      keyIndex: Array[Int],
    +      orderDirection: Array[Direction],
    +      execConfig: ExecutionConfig): Array[TypeComparator[_]] = {
    +    
    +    var iOrder = 0
    +    for (i <- keyIndex) yield {
    +
    +      val order = if (orderDirection(iOrder) == Direction.ASCENDING) true else false
    +      iOrder += 1
    +      val fieldTypeInfo = FlinkTypeFactory.toTypeInfo(inputType.getFieldList.get(i).getType)
    +      fieldTypeInfo match {
    +        case a: AtomicType[_] => a.createComparator(order, execConfig)
    +        case _ => throw new TableException(s"Unsupported field type $fieldTypeInfo to sort on.")
    +      }
    +    }
    +  }
    +  
    +   /**
    +   * Function creates a RowComparator based on the typed comparators
    +   * @param inputRowType input row type
    +   * @param fieldIdxs the indexes of the fields on which the sorting is done. 
    +   * @param fieldComps the array of typed comparators
    +   * @param fieldOrders the directions of each sort field (true = ASC). 
    +   * @return A Row TypeComparator object 
    +   */
    +  def createRowComparator(
    +    inputRowType: RelDataType,
    +    fieldIdxs: Array[Int],
    +    fieldComps: Array[TypeComparator[AnyRef]],
    +    fieldOrders: Array[Boolean]): TypeComparator[Row] = {
    +
    +  val rowComp = new RowComparator(
    +    inputRowType.getFieldCount,
    +    fieldIdxs,
    +    fieldComps,
    +    new Array[TypeSerializer[AnyRef]](0), //used only for serialized comparisons
    +    fieldOrders)
    +
    +  rowComp
    +}
    +  
    + 
    +  /**
    +   * Function returns the array of indexes for the fields on which the sort is done
    +   * @param collationSort The Sort collation list
    +   * @return [Array[Int]]
    +   */
    +  def getSortFieldIndexList(collationSort: RelCollation): Array[Int] = {
    +    val keyFields = collationSort.getFieldCollations.toArray()
    +    for (f <- keyFields) yield f.asInstanceOf[RelFieldCollation].getFieldIndex
    +  }
    +  
    +   /**
    +   * Function returns the array of sort direction for each of the sort fields 
    +   * @param collationSort The Sort collation list
    +   * @return [Array[Direction]]
    +   */
    +  def getSortFieldDirectionList(collationSort: RelCollation): Array[Direction] = {
    +    val keyFields = collationSort.getFieldCollations.toArray()
    +    for(f <- keyFields) yield f.asInstanceOf[RelFieldCollation].getDirection
    +  }
    +
    +   /**
    +   * Function returns the array of sort direction for each of the sort fields 
    +   * @param collationSort The Sort collation list
    +   * @return [Array[Direction]]
    +   */
    +  def getSortFieldDirectionBooleanList(collationSort: RelCollation): Array[Boolean] = {
    +    val keyFields = collationSort.getFieldCollations
    +    var i = 0
    +    val keySortDirection = new Array[Boolean](keyFields.size())
    +    while (i < keyFields.size()) {
    +      keySortDirection(i) = 
    +        if (getDirection(collationSort,i) == Direction.ASCENDING) true else false 
    +      i += 1
    +    }
    +    keySortDirection
    +  }
    +  
    +  
    +   /**
    +   * Function returns the direction type of the time in order clause. 
    +   * @param collationSort The Sort collation list of objects
    +   * @return [org.apache.calcite.rel.RelFieldCollation.Direction]
    +   */
    +  def getTimeDirection(collationSort: RelCollation): Direction = {
    +    collationSort.getFieldCollations.get(0).direction
    +  }
    +  
    +   /**
    +   * Function returns the time type in order clause. 
    +   * Expectation is that it is the primary sort field
    +   * @param collationSort The Sort collation list
    +   * @param rowType The data type of the input
    +   * @return [org.apache.calcite.rel.type.RelDataType]
    +   */
    +  def getTimeType(collationSort: RelCollation, rowType: RelDataType): RelDataType = {
    +
    +    //need to identify time between others ordering fields
    +    val ind = collationSort.getFieldCollations.get(0).getFieldIndex
    +    rowType.getFieldList.get(ind).getValue
    +  }
    +
    +   /**
    +   * Function returns the direction type of a field in order clause. 
    +   * @param collationSort The Sort collation list
    +   * @return [org.apache.calcite.rel.RelFieldCollation.Direction]
    +   */
    +  def getDirection(collationSort: RelCollation, sortField:Int): Direction = {
    --- End diff --
    
    method is only used once. Can be inlined 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123745157
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 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}
    +
    +/**
    + * 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)
    +    //drop time from comparison as we sort on time in the states and result emission
    +    val keyIndexesNoTime = keySortFields.slice(1, keySortFields.size)
    +    val booleanOrderings = getSortFieldDirectionBooleanList(collationSort)
    +    val booleanDirectionsNoTime = booleanOrderings.slice(1, booleanOrderings.size)
    +    
    +    val fieldComps = createFieldComparators(
    +      inputType, 
    +      keyIndexesNoTime, 
    +      booleanDirectionsNoTime,
    +      execCfg)
    +    val fieldCompsRefs = fieldComps.asInstanceOf[Array[TypeComparator[AnyRef]]]
    +    
    +    val rowComp = new RowComparator(
    +       inputType.getFieldCount,
    --- End diff --
    
    Indention is off 3 instead of 2


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

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

Posted by fhueske <gi...@git.apache.org>.
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


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

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

Posted by fhueske <gi...@git.apache.org>.
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(10000L),
    +      Left((10000L, (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((20000L, (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 duplicating the code.


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

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

Posted by fhueske <gi...@git.apache.org>.
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 calculation
    +    val triggeringTs = ctx.timestamp
    +
    +    val lastTriggeringTs = lastTriggeringTsState.value
    +
    +    // check if the data is expired, if not, save the data and register event time timer
    +    if (triggeringTs > lastTriggeringTs) {
    +      val data = dataState.get(triggeringTs)
    +      if (null != data) {
    +        data.add(input)
    +        dataState.put(triggeringTs, data)
    +      } else {
    +        val data = new JArrayList[Row]
    +        data.add(input)
    +        dataState.put(triggeringTs, data)
    +        // register event time timer
    +        ctx.timerService.registerEventTimeTimer(triggeringTs)
    +      }
    +    }
    +  }
    +  
    +  
    +  override def onTimer(
    +    timestamp: Long,
    +    ctx: ProcessFunction[CRow, CRow]#OnTimerContext,
    +    out: Collector[CRow]): Unit = {
    +    
    +    // gets all window data from state for the calculation
    +    val inputs: JList[Row] = dataState.get(timestamp)
    +
    +    if (null != inputs) {
    +      
    +      var dataListIndex = 0
    +
    +      // no retraction needed for time order sort
    --- End diff --
    
    remove "todo" comments


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

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

Posted by fhueske <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

Posted by fhueske <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123754095
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeSortProcessFunction.scala ---
    @@ -0,0 +1,120 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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}
    +import java.lang.{Integer=>JInt}
    +
    +/**
    + * Process Function used for the aggregate in bounded proctime sort without offset/fetch
    + * [[org.apache.flink.streaming.api.datastream.DataStream]]
    + *
    + * @param inputRowType 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 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)
    +    
    +    if (outputC == null) {
    +      val arity:Integer = inputRowType.getArity
    +      outputC = new CRow(Row.of(arity), true)
    +    }
    +    
    +  }
    +
    +  override def processElement(
    +    inputC: CRow,
    +    ctx: ProcessFunction[CRow, CRow]#Context,
    +    out: Collector[CRow]): Unit = {
    +
    +    val input = inputC.row
    +    
    +    val currentTime = ctx.timerService.currentProcessingTime
    +    //buffer the event incoming event
    +  
    +    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 = {
    +    
    +    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)
    +    
    +    //we need to build the output and emit the events in order
    +    var iElemenets = 0
    +    while (iElemenets < sortArray.size) {
    +      // do we need to recreate the object no to mess references in previous results?
    --- End diff --
    
    No, the implementation is fine. You can remove the 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

Posted by fhueske <gi...@git.apache.org>.
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 first sort element
    +    checkTimeOrder()
    +    
    +    super.explainTerms(pw)
    +      .item("orderBy", SortUtil.getSortFieldToString(sortCollation, schema.logicalType))
    +      .item("offset", SortUtil.getOffsetToString(sortOffset))
    +      .item("fetch", SortUtil.getFetchToString(sortFetch, sortOffset))
    +  }
    +
    +  override def translateToPlan(
    +      tableEnv: StreamTableEnvironment,
    +      queryConfig: StreamQueryConfig): DataStream[CRow] = {
    +    
    +    val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
    +    
    +    //need to identify time between others order fields. Time needs to be first sort element
    +    val timeType = SortUtil.getTimeType(sortCollation, schema.logicalType)
    +    
    +    //time ordering needs to be ascending
    +    if (SortUtil.getTimeDirection(sortCollation) != Direction.ASCENDING) {
    +      throw new TableException("SQL/Table supports only ascending time ordering")
    +    }
    +    
    +    val execCfg = tableEnv.execEnv.getConfig
    +      
    +    //enable to extend for other types of aggregates that will not be implemented in a window
    +    timeType match {
    +        case _ if FlinkTypeFactory.isProctimeIndicatorType(timeType)  =>
    +            (sortOffset,sortFetch) match {
    +              case (o: Any, f: Any)  => // offset and fetch needs retraction
    +                throw new TableException("SQL/Table does not support sort with offset and fetch") 
    +              case (_, f: Any) => // offset needs retraction
    +                throw new TableException("SQL/Table does not support sort with fetch")
    +              case (o: Any, _) =>  // fetch needs retraction
    +                throw new TableException("SQL/Table does not support sort with offset")
    +              case _ => createSortProcTime(inputDS, execCfg)  //sort can be done without retraction
    +            }
    +        case _ if FlinkTypeFactory.isRowtimeIndicatorType(timeType) =>
    +            (sortOffset,sortFetch) match {
    +              case (o: Any, f: Any)  => // offset and fetch needs retraction
    +                throw new TableException("SQL/Table does not support sort with offset and fetch") 
    +              case (_, f: Any) => // offset needs retraction
    +                throw new TableException("SQL/Table does not support sort with fetch")
    +              case (o: Any, _) =>  // fetch needs retraction
    +                throw new TableException("SQL/Table does not support sort with offset")
    +              case _ => createSortRowTime(inputDS, execCfg)  //sort can be done without retraction
    +            }
    +        case _ =>
    +          throw new TableException("SQL/Table needs to have sort on time as first sort element")
    +    }
    +    
    +  }
    +
    +  /**
    +   * Create Sort logic based on processing time
    +   */
    +  def createSortProcTime(
    +    inputDS: DataStream[CRow],
    +    execCfg: ExecutionConfig): DataStream[CRow] = {
    +
    +   val returnTypeInfo = CRowTypeInfo(schema.physicalTypeInfo)
    +    
    +    //if the order has secondary sorting fields in addition to the proctime
    +    if( SortUtil.getSortFieldIndexList(sortCollation).size > 1) {
    +    
    +      val processFunction = SortUtil.createProcTimeSortFunction(sortCollation,
    +           inputSchema.logicalType, inputSchema.physicalTypeInfo, execCfg)
    +      
    +      inputDS
    +            .keyBy(new NullByteKeySelector[CRow])
    +            .process(processFunction).setParallelism(1).setMaxParallelism(1)
    +            .returns(returnTypeInfo)
    +            .asInstanceOf[DataStream[CRow]]
    +    } else {
    +      //if the order is done only on proctime we only need to forward the elements
    +        inputDS.keyBy(new NullByteKeySelector[CRow])
    --- End diff --
    
    -2 spaces


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

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

Posted by fhueske <gi...@git.apache.org>.
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 the repeated check in `processElement()`


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r117182703
  
    --- 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 calculation
    +    val triggeringTs = ctx.timestamp
    +
    +    val lastTriggeringTs = lastTriggeringTsState.value
    +
    +    // check if the data is expired, if not, save the data and register event time timer
    +    if (triggeringTs > lastTriggeringTs) {
    +      val data = dataState.get(triggeringTs)
    +      if (null != data) {
    +        data.add(input)
    +        dataState.put(triggeringTs, data)
    +      } else {
    +        val data = new JArrayList[Row]
    +        data.add(input)
    +        dataState.put(triggeringTs, data)
    +        // register event time timer
    +        ctx.timerService.registerEventTimeTimer(triggeringTs)
    +      }
    +    }
    +  }
    +  
    +  
    +  override def onTimer(
    +    timestamp: Long,
    +    ctx: ProcessFunction[CRow, CRow]#OnTimerContext,
    +    out: Collector[CRow]): Unit = {
    +    
    +    // gets all window data from state for the calculation
    +    val inputs: JList[Row] = dataState.get(timestamp)
    +
    +    if (null != inputs) {
    +      
    +      var dataListIndex = 0
    +
    +      // no retraction needed for time order sort
    +      
    +      //no selection of offset/fetch
    +      
    +      dataListIndex = 0
    +      sortArray.clear()
    --- End diff --
    
    `inputs` is not a `ListState` but an actual `ArrayList` that was returned from the `dataState: MapState[JList[Row]]`. So we are copying the elements from one `ArrayList` into another.
    
    In `ProctimeSortProcessFunction` the `ListState[Row]` is much better than `ValueState[JList[Row]]` because adding to the `ListState` is basically free, while `ValueState` would need to deserialized the `List` every time we read or write.


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123718523
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamSortRule.scala ---
    @@ -0,0 +1,111 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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
    +import org.apache.flink.table.calcite.FlinkTypeFactory
    +import org.apache.flink.table.api.TableException
    +import org.apache.calcite.rel.RelFieldCollation.Direction
    +import org.apache.flink.table.runtime.aggregate.SortUtil
    +
    +/**
    + * 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 = {
    +
    +    val result = super.matches(call)
    +    
    +    //need to identify time between others order fields. Time needs to be first sort element
    +    // we can safely convert the object if the match rule succeeded 
    +    if(result) {
    +      val calcSort: FlinkLogicalSort = call.rel(0).asInstanceOf[FlinkLogicalSort]
    +      checkTimeOrder(calcSort)
    +    }
    +    
    +    result
    +  }
    +
    +  override def convert(rel: RelNode): RelNode = {
    +    val calcSort: FlinkLogicalSort = rel.asInstanceOf[FlinkLogicalSort]
    +    val traitSet: RelTraitSet = rel.getTraitSet.replace(FlinkConventions.DATASTREAM)
    +    val convInput: RelNode = RelOptRule.convert(calcSort.getInput(0), FlinkConventions.DATASTREAM)
    +    
    +    val inputRowType = convInput.asInstanceOf[RelSubset].getOriginal.getRowType
    +
    +    new DataStreamSort(
    +      rel.getCluster,
    +      traitSet,
    +      convInput,
    +      new RowSchema(inputRowType),
    +      new RowSchema(rel.getRowType),
    +      calcSort.collation,
    +      calcSort.offset,
    +      calcSort.fetch,
    +      description)
    +    
    +  }
    +  
    +   
    +  /**
    +   * Function is used to check at verification time if the SQL syntax is supported
    +   */
    +  
    +  def checkTimeOrder(calcSort: FlinkLogicalSort) = {
    +    
    +    val rowType = calcSort.getRowType
    +    val sortCollation = calcSort.collation 
    +     //need to identify time between others order fields. Time needs to be first sort element
    +    val timeType = SortUtil.getTimeType(sortCollation, rowType)
    +    //time ordering needs to be ascending
    +    if (SortUtil.getTimeDirection(sortCollation) != Direction.ASCENDING) {
    +      throw new TableException("SQL/Table supports only ascending time ordering")
    +    }
    +    //enable to extend for other types of aggregates that will not be implemented in a window
    +    timeType match {
    +        case _ if FlinkTypeFactory.isProctimeIndicatorType(timeType) =>
    --- End diff --
    
    `FlinkTypeFactory.isTimeIndicatorType` covers processing and event time


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123758098
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSort.scala ---
    @@ -0,0 +1,206 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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
    +import org.apache.flink.table.plan.nodes.CommonSort
    +
    +/**
    +  * Flink RelNode which matches along with Sort Rule.
    +  *
    + */
    +class DataStreamSort(
    +    cluster: RelOptCluster,
    +    traitSet: RelTraitSet,
    +    inputNode: RelNode,
    +    inputSchema: RowSchema,
    +    schema: RowSchema,
    +    sortCollation: RelCollation,
    +    sortOffset: RexNode,
    +    sortFetch: RexNode,
    +    description: String)
    +  extends SingleRel(cluster, traitSet, inputNode)
    +  with CommonSort
    +  with DataStreamRel {
    +
    +  override def deriveRowType(): RelDataType = schema.logicalType
    +
    +  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
    +    new DataStreamSort(
    +      cluster,
    +      traitSet,
    +      inputs.get(0),
    +      inputSchema,
    +      schema,
    +      sortCollation,
    +      sortOffset,
    +      sortFetch,
    +      description)
    +  }
    +
    +  override def toString: String = {
    +    s"Sort(by: ($$sortFieldsToString(sortCollation, schema.logicalType))," +
    +      " offset: $offsetToString(sortOffset)," +
    +      " fetch: $fetchToString(sortFetch, sortOffset))"
    +  }
    +  
    +  override def explainTerms(pw: RelWriter) : RelWriter = {
    +    
    +    super.explainTerms(pw)
    +      .item("orderBy", sortFieldsToString(sortCollation, schema.logicalType))
    --- End diff --
    
    use `itemIf` and check if `offset` and `fetch` are defined.


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123755279
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeSortProcessFunction.scala ---
    @@ -0,0 +1,146 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.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 inputRowType 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 inputRowType: CRowTypeInfo,
    +  private val rowComparator: CollectionRowComparator)
    +    extends ProcessFunction[CRow, CRow] {
    +
    +  Preconditions.checkNotNull(rowComparator)
    +
    +  // 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)
    +    
    +    if (outputC == null) {
    --- End diff --
    
    Can be done as `outputC = new CRow()`


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123711129
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonSort.scala ---
    @@ -0,0 +1,91 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.table.plan.nodes
    +
    +import org.apache.calcite.rex.{RexLiteral, RexNode}
    +import org.apache.calcite.rel.RelFieldCollation
    +import org.apache.calcite.rel.RelCollation
    +import org.apache.calcite.rel.RelFieldCollation.Direction
    +import org.apache.calcite.rel.`type`._
    +import scala.collection.JavaConverters._
    +import org.apache.flink.api.common.operators.Order
    +
    +
    +/**
    + * Trait represents a collection of sort methods to manipulate the parameters
    + */
    +
    +trait CommonSort {
    +  
    +  private[flink] def offsetToString(offset: RexNode): String = {
    --- End diff --
    
    IMO, this method can be removed. It simply calls `offset.toString()`


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123713677
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonSort.scala ---
    @@ -0,0 +1,91 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.table.plan.nodes
    +
    +import org.apache.calcite.rex.{RexLiteral, RexNode}
    +import org.apache.calcite.rel.RelFieldCollation
    +import org.apache.calcite.rel.RelCollation
    +import org.apache.calcite.rel.RelFieldCollation.Direction
    +import org.apache.calcite.rel.`type`._
    +import scala.collection.JavaConverters._
    +import org.apache.flink.api.common.operators.Order
    +
    +
    +/**
    + * Trait represents a collection of sort methods to manipulate the parameters
    + */
    +
    +trait CommonSort {
    +  
    +  private[flink] def offsetToString(offset: RexNode): String = {
    +    val offsetToString = s"$offset"
    +    offsetToString
    +  }
    +  
    +  
    +  private[flink] def sortFieldsToString(
    +      collationSort: RelCollation, 
    +      rowRelDataType: RelDataType): String = {
    +    val fieldCollations = collationSort.getFieldCollations.asScala  
    +    .map(c => (c.getFieldIndex, directionToOrder(c.getDirection)))
    +
    +    val sortFieldsToString = fieldCollations
    +      .map(col => s"${
    +        rowRelDataType.getFieldNames.get(col._1)} ${col._2.getShortName}" ).mkString(", ")
    +    
    +    sortFieldsToString
    +  }
    +  
    +  private[flink] def directionToOrder(direction: Direction) = {
    +    direction match {
    +      case Direction.ASCENDING | Direction.STRICTLY_ASCENDING => Order.ASCENDING
    +      case Direction.DESCENDING | Direction.STRICTLY_DESCENDING => Order.DESCENDING
    +      case _ => throw new IllegalArgumentException("Unsupported direction.")
    +    }
    +  }
    +  
    +  private[flink] def fetchToString(fetch: RexNode, offset: RexNode): String = {
    +    val limitEnd = getFetchLimitEnd(fetch, offset)
    +    val fetchToString = if (limitEnd == Long.MaxValue) {
    --- End diff --
    
    same for the return values of `getFetchLimitEnd` and `getFetchLimitStart`


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

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

Posted by fhueske <gi...@git.apache.org>.
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 first sort element
    +    checkTimeOrder()
    +    
    +    super.explainTerms(pw)
    +      .item("orderBy", SortUtil.getSortFieldToString(sortCollation, schema.logicalType))
    +      .item("offset", SortUtil.getOffsetToString(sortOffset))
    +      .item("fetch", SortUtil.getFetchToString(sortFetch, sortOffset))
    +  }
    +
    +  override def translateToPlan(
    +      tableEnv: StreamTableEnvironment,
    +      queryConfig: StreamQueryConfig): DataStream[CRow] = {
    +    
    +    val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
    +    
    +    //need to identify time between others order fields. Time needs to be first sort element
    +    val timeType = SortUtil.getTimeType(sortCollation, schema.logicalType)
    +    
    +    //time ordering needs to be ascending
    +    if (SortUtil.getTimeDirection(sortCollation) != Direction.ASCENDING) {
    +      throw new TableException("SQL/Table supports only ascending time ordering")
    +    }
    +    
    +    val execCfg = tableEnv.execEnv.getConfig
    +      
    +    //enable to extend for other types of aggregates that will not be implemented in a window
    +    timeType match {
    +        case _ if FlinkTypeFactory.isProctimeIndicatorType(timeType)  =>
    +            (sortOffset,sortFetch) match {
    +              case (o: Any, f: Any)  => // offset and fetch needs retraction
    --- End diff --
    
    Why not use the more concrete type `RexNode` instead of `Any`?
    if `o` and `f` are not used, you can write `case (_: RexNode, _: RexNode) =>`


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

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

Posted by fhueske <gi...@git.apache.org>.
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 appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

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

    https://github.com/apache/flink/pull/3889#discussion_r116470594
  
    --- 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 first sort element
    +    checkTimeOrder()
    +    
    +    super.explainTerms(pw)
    +      .item("orderBy", SortUtil.getSortFieldToString(sortCollation, schema.logicalType))
    +      .item("offset", SortUtil.getOffsetToString(sortOffset))
    +      .item("fetch", SortUtil.getFetchToString(sortFetch, sortOffset))
    +  }
    +
    +  override def translateToPlan(
    +      tableEnv: StreamTableEnvironment,
    +      queryConfig: StreamQueryConfig): DataStream[CRow] = {
    +    
    +    val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
    +    
    +    //need to identify time between others order fields. Time needs to be first sort element
    +    val timeType = SortUtil.getTimeType(sortCollation, schema.logicalType)
    +    
    +    //time ordering needs to be ascending
    +    if (SortUtil.getTimeDirection(sortCollation) != Direction.ASCENDING) {
    +      throw new TableException("SQL/Table supports only ascending time ordering")
    +    }
    +    
    +    val execCfg = tableEnv.execEnv.getConfig
    +      
    +    //enable to extend for other types of aggregates that will not be implemented in a window
    +    timeType match {
    +        case _ if FlinkTypeFactory.isProctimeIndicatorType(timeType)  =>
    +            (sortOffset,sortFetch) match {
    +              case (o: Any, f: Any)  => // offset and fetch needs retraction
    +                throw new TableException("SQL/Table does not support sort with offset and fetch") 
    +              case (_, f: Any) => // offset needs retraction
    +                throw new TableException("SQL/Table does not support sort with fetch")
    +              case (o: Any, _) =>  // fetch needs retraction
    +                throw new TableException("SQL/Table does not support sort with offset")
    +              case _ => createSortProcTime(inputDS, execCfg)  //sort can be done without retraction
    +            }
    +        case _ if FlinkTypeFactory.isRowtimeIndicatorType(timeType) =>
    +            (sortOffset,sortFetch) match {
    +              case (o: Any, f: Any)  => // offset and fetch needs retraction
    +                throw new TableException("SQL/Table does not support sort with offset and fetch") 
    +              case (_, f: Any) => // offset needs retraction
    +                throw new TableException("SQL/Table does not support sort with fetch")
    +              case (o: Any, _) =>  // fetch needs retraction
    +                throw new TableException("SQL/Table does not support sort with offset")
    +              case _ => createSortRowTime(inputDS, execCfg)  //sort can be done without retraction
    +            }
    +        case _ =>
    +          throw new TableException("SQL/Table needs to have sort on time as first sort element")
    +    }
    +    
    +  }
    +
    +  /**
    +   * Create Sort logic based on processing time
    +   */
    +  def createSortProcTime(
    +    inputDS: DataStream[CRow],
    +    execCfg: ExecutionConfig): DataStream[CRow] = {
    +
    +   val returnTypeInfo = CRowTypeInfo(schema.physicalTypeInfo)
    +    
    +    //if the order has secondary sorting fields in addition to the proctime
    +    if( SortUtil.getSortFieldIndexList(sortCollation).size > 1) {
    +    
    +      val processFunction = SortUtil.createProcTimeSortFunction(sortCollation,
    --- End diff --
    
    we wrap method parameters as follows:
    ```
    val result = method(
      arg1,
      arg2,
      arg3)
    ```
    
    Each parameter (incl. the first) goes to a new line indented by two spaces.


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r118423723
  
    --- 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 RowTimeSortProcessFunction(
    +      inputType.getFieldCount,
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +  
    +  
    +  /**
    +   * Function creates [org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
    +   * elements based on proctime 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 createProcTimeSortFunction(
    +    collationSort: RelCollation,
    +    inputType: RelDataType,
    +    inputTypeInfo: TypeInformation[Row],
    +    execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
    +
    +    val keySortFields = getSortFieldIndexList(collationSort)
    +    val keySortDirections = getSortFieldDirectionList(collationSort)
    +
    +    
    +       //drop time from comparison
    +    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 ProcTimeSortProcessFunction(
    +      inputType.getFieldCount,
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +
    +  
    +   /**
    +   * Function creates comparison objects based on the field types
    +   * @param inputType input row type
    +   * @param keyIndex the indexes of the fields on which the sorting is done. 
    +   * @param orderDirection the directions of each sort field. 
    +   * @param execConfig the configuration environment
    +   * @return Array of TypeComparator objects
    +   */
    +  def createFieldComparators(
    +      inputType: RelDataType,
    +      keyIndex: Array[Int],
    +      orderDirection: Array[Direction],
    +      execConfig: ExecutionConfig): Array[TypeComparator[_]] = {
    +    
    +    var iOrder = 0
    +    for (i <- keyIndex) yield {
    +
    +      val order = if (orderDirection(iOrder) == Direction.ASCENDING) true else false
    +      iOrder += 1
    +      val fieldTypeInfo = FlinkTypeFactory.toTypeInfo(inputType.getFieldList.get(i).getType)
    +      fieldTypeInfo match {
    +        case a: AtomicType[_] => a.createComparator(order, execConfig)
    +        case _ => throw new TableException(s"Unsupported field type $fieldTypeInfo to sort on.")
    +      }
    +    }
    +  }
    +  
    +   /**
    +   * Function creates a RowComparator based on the typed comparators
    +   * @param inputRowType input row type
    +   * @param fieldIdxs the indexes of the fields on which the sorting is done. 
    +   * @param fieldComps the array of typed comparators
    +   * @param fieldOrders the directions of each sort field (true = ASC). 
    +   * @return A Row TypeComparator object 
    +   */
    +  def createRowComparator(
    +    inputRowType: RelDataType,
    +    fieldIdxs: Array[Int],
    +    fieldComps: Array[TypeComparator[AnyRef]],
    +    fieldOrders: Array[Boolean]): TypeComparator[Row] = {
    +
    +  val rowComp = new RowComparator(
    +    inputRowType.getFieldCount,
    +    fieldIdxs,
    +    fieldComps,
    +    new Array[TypeSerializer[AnyRef]](0), //used only for serialized comparisons
    +    fieldOrders)
    +
    +  rowComp
    +}
    +  
    + 
    +  /**
    +   * Function returns the array of indexes for the fields on which the sort is done
    +   * @param collationSort The Sort collation list
    +   * @return [Array[Int]]
    +   */
    +  def getSortFieldIndexList(collationSort: RelCollation): Array[Int] = {
    +    val keyFields = collationSort.getFieldCollations.toArray()
    +    for (f <- keyFields) yield f.asInstanceOf[RelFieldCollation].getFieldIndex
    +  }
    +  
    +   /**
    +   * Function returns the array of sort direction for each of the sort fields 
    +   * @param collationSort The Sort collation list
    +   * @return [Array[Direction]]
    +   */
    +  def getSortFieldDirectionList(collationSort: RelCollation): Array[Direction] = {
    +    val keyFields = collationSort.getFieldCollations.toArray()
    +    for(f <- keyFields) yield f.asInstanceOf[RelFieldCollation].getDirection
    +  }
    +
    +   /**
    +   * Function returns the array of sort direction for each of the sort fields 
    +   * @param collationSort The Sort collation list
    +   * @return [Array[Direction]]
    +   */
    +  def getSortFieldDirectionBooleanList(collationSort: RelCollation): Array[Boolean] = {
    +    val keyFields = collationSort.getFieldCollations
    +    var i = 0
    +    val keySortDirection = new Array[Boolean](keyFields.size())
    +    while (i < keyFields.size()) {
    +      keySortDirection(i) = 
    +        if (getDirection(collationSort,i) == Direction.ASCENDING) true else false 
    +      i += 1
    +    }
    +    keySortDirection
    +  }
    +  
    +  
    +   /**
    +   * Function returns the direction type of the time in order clause. 
    +   * @param collationSort The Sort collation list of objects
    +   * @return [org.apache.calcite.rel.RelFieldCollation.Direction]
    +   */
    +  def getTimeDirection(collationSort: RelCollation): Direction = {
    +    collationSort.getFieldCollations.get(0).direction
    --- End diff --
    
    indeed, but i used the model also from the orderwindow - to have a method to hide the path to the time pointer in the logical objects. Although for the other similar methods (e.g., creating row comparison) i agree to delete the method - i would propose to keep this one. It looks more elegant in the code where this is used and there is no performance cost 


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

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

Posted by fhueske <gi...@git.apache.org>.
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 RowTimeSortProcessFunction(
    +      inputType.getFieldCount,
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +  
    +  
    +  /**
    +   * Function creates [org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
    +   * elements based on proctime 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 createProcTimeSortFunction(
    +    collationSort: RelCollation,
    +    inputType: RelDataType,
    +    inputTypeInfo: TypeInformation[Row],
    +    execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
    +
    +    val keySortFields = getSortFieldIndexList(collationSort)
    +    val keySortDirections = getSortFieldDirectionList(collationSort)
    +
    +    
    +       //drop time from comparison
    +    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 ProcTimeSortProcessFunction(
    +      inputType.getFieldCount,
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +
    +  
    +   /**
    +   * Function creates comparison objects based on the field types
    +   * @param inputType input row type
    +   * @param keyIndex the indexes of the fields on which the sorting is done. 
    +   * @param orderDirection the directions of each sort field. 
    +   * @param execConfig the configuration environment
    +   * @return Array of TypeComparator objects
    +   */
    +  def createFieldComparators(
    +      inputType: RelDataType,
    +      keyIndex: Array[Int],
    +      orderDirection: Array[Direction],
    +      execConfig: ExecutionConfig): Array[TypeComparator[_]] = {
    +    
    +    var iOrder = 0
    +    for (i <- keyIndex) yield {
    +
    +      val order = if (orderDirection(iOrder) == Direction.ASCENDING) true else false
    +      iOrder += 1
    +      val fieldTypeInfo = FlinkTypeFactory.toTypeInfo(inputType.getFieldList.get(i).getType)
    +      fieldTypeInfo match {
    +        case a: AtomicType[_] => a.createComparator(order, execConfig)
    +        case _ => throw new TableException(s"Unsupported field type $fieldTypeInfo to sort on.")
    +      }
    +    }
    +  }
    +  
    +   /**
    +   * Function creates a RowComparator based on the typed comparators
    +   * @param inputRowType input row type
    +   * @param fieldIdxs the indexes of the fields on which the sorting is done. 
    +   * @param fieldComps the array of typed comparators
    +   * @param fieldOrders the directions of each sort field (true = ASC). 
    +   * @return A Row TypeComparator object 
    +   */
    +  def createRowComparator(
    +    inputRowType: RelDataType,
    +    fieldIdxs: Array[Int],
    +    fieldComps: Array[TypeComparator[AnyRef]],
    +    fieldOrders: Array[Boolean]): TypeComparator[Row] = {
    +
    +  val rowComp = new RowComparator(
    +    inputRowType.getFieldCount,
    +    fieldIdxs,
    +    fieldComps,
    +    new Array[TypeSerializer[AnyRef]](0), //used only for serialized comparisons
    +    fieldOrders)
    +
    +  rowComp
    +}
    +  
    + 
    +  /**
    +   * Function returns the array of indexes for the fields on which the sort is done
    +   * @param collationSort The Sort collation list
    +   * @return [Array[Int]]
    +   */
    +  def getSortFieldIndexList(collationSort: RelCollation): Array[Int] = {
    +    val keyFields = collationSort.getFieldCollations.toArray()
    +    for (f <- keyFields) yield f.asInstanceOf[RelFieldCollation].getFieldIndex
    +  }
    +  
    +   /**
    +   * Function returns the array of sort direction for each of the sort fields 
    +   * @param collationSort The Sort collation list
    +   * @return [Array[Direction]]
    +   */
    +  def getSortFieldDirectionList(collationSort: RelCollation): Array[Direction] = {
    --- End diff --
    
    Is not needed. We can use `getSortFieldDirectionBooleanList` instead


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123742381
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 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}
    +
    +/**
    + * 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)
    +    //drop time from comparison as we sort on time in the states and result emission
    +    val keyIndexesNoTime = keySortFields.slice(1, keySortFields.size)
    +    val booleanOrderings = getSortFieldDirectionBooleanList(collationSort)
    +    val booleanDirectionsNoTime = booleanOrderings.slice(1, booleanOrderings.size)
    +    
    +    val fieldComps = createFieldComparators(
    +      inputType, 
    +      keyIndexesNoTime, 
    +      booleanDirectionsNoTime,
    +      execCfg)
    +    val fieldCompsRefs = fieldComps.asInstanceOf[Array[TypeComparator[AnyRef]]]
    +    
    +    val rowComp = new RowComparator(
    +       inputType.getFieldCount,
    +       keyIndexesNoTime,
    +       fieldCompsRefs,
    +       new Array[TypeSerializer[AnyRef]](0), //used only for object comparisons
    +       booleanDirectionsNoTime)
    +      
    +    val collectionRowComparator = new CollectionRowComparator(rowComp)
    +    
    +    val inputCRowType = CRowTypeInfo(inputTypeInfo)
    + 
    +    new RowTimeSortProcessFunction(
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +  
    +  
    +  /**
    +   * Function creates [org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
    +   * elements based on proctime 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 createProcTimeSortFunction(
    +    collationSort: RelCollation,
    +    inputType: RelDataType,
    +    inputTypeInfo: TypeInformation[Row],
    +    execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
    +
    +    val keySortFields = getSortFieldIndexList(collationSort)
    +    //drop time from comparison
    +    val keyIndexesNoTime = keySortFields.slice(1, keySortFields.size)
    +    val booleanOrderings = getSortFieldDirectionBooleanList(collationSort)
    +    val booleanDirectionsNoTime = booleanOrderings.slice(1, booleanOrderings.size)
    +    
    +    val fieldComps = createFieldComparators(
    +      inputType, 
    +      keyIndexesNoTime, 
    +      booleanDirectionsNoTime,
    +      execCfg)
    +    val fieldCompsRefs = fieldComps.asInstanceOf[Array[TypeComparator[AnyRef]]]
    +    
    +    val rowComp = new RowComparator(
    +       inputType.getFieldCount,
    +       keyIndexesNoTime,
    +       fieldCompsRefs,
    +       new Array[TypeSerializer[AnyRef]](0), //used only for object comparisons
    +       booleanDirectionsNoTime)
    +      
    +    val collectionRowComparator = new CollectionRowComparator(rowComp)
    +    
    +    val inputCRowType = CRowTypeInfo(inputTypeInfo)
    +    
    +    new ProcTimeSortProcessFunction(
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +
    +  
    +   /**
    +   * Function creates comparison objects based on the field types
    +   * @param inputType input row type
    +   * @param keyIndex the indexes of the fields on which the sorting is done. 
    +   * @param orderDirection the directions of each sort field. 
    --- End diff --
    
    parameter has been renamed


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123717968
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamSortRule.scala ---
    @@ -0,0 +1,111 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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
    +import org.apache.flink.table.calcite.FlinkTypeFactory
    +import org.apache.flink.table.api.TableException
    +import org.apache.calcite.rel.RelFieldCollation.Direction
    +import org.apache.flink.table.runtime.aggregate.SortUtil
    +
    +/**
    + * 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 = {
    +
    +    val result = super.matches(call)
    +    
    +    //need to identify time between others order fields. Time needs to be first sort element
    +    // we can safely convert the object if the match rule succeeded 
    +    if(result) {
    +      val calcSort: FlinkLogicalSort = call.rel(0).asInstanceOf[FlinkLogicalSort]
    +      checkTimeOrder(calcSort)
    +    }
    +    
    +    result
    +  }
    +
    +  override def convert(rel: RelNode): RelNode = {
    +    val calcSort: FlinkLogicalSort = rel.asInstanceOf[FlinkLogicalSort]
    +    val traitSet: RelTraitSet = rel.getTraitSet.replace(FlinkConventions.DATASTREAM)
    +    val convInput: RelNode = RelOptRule.convert(calcSort.getInput(0), FlinkConventions.DATASTREAM)
    +    
    +    val inputRowType = convInput.asInstanceOf[RelSubset].getOriginal.getRowType
    +
    +    new DataStreamSort(
    +      rel.getCluster,
    +      traitSet,
    +      convInput,
    +      new RowSchema(inputRowType),
    +      new RowSchema(rel.getRowType),
    +      calcSort.collation,
    +      calcSort.offset,
    +      calcSort.fetch,
    +      description)
    +    
    +  }
    +  
    +   
    +  /**
    +   * Function is used to check at verification time if the SQL syntax is supported
    +   */
    +  
    +  def checkTimeOrder(calcSort: FlinkLogicalSort) = {
    --- End diff --
    
    `calcSort` -> `sort`


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

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

Posted by fhueske <gi...@git.apache.org>.
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,
    +        keyIndexesNoTime, fieldCompsRefs, booleanDirectionsNoTime)
    +    val collectionRowComparator = new CollectionRowComparator(rowComp)
    +    
    +    val inputCRowType = CRowTypeInfo(inputTypeInfo)
    + 
    +    new RowTimeSortProcessFunction(
    +      inputType.getFieldCount,
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +  
    +  
    +  /**
    +   * Function creates [org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
    +   * elements based on proctime 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 createProcTimeSortFunction(
    +    collationSort: RelCollation,
    +    inputType: RelDataType,
    +    inputTypeInfo: TypeInformation[Row],
    +    execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
    +
    +    val keySortFields = getSortFieldIndexList(collationSort)
    +    val keySortDirections = getSortFieldDirectionList(collationSort)
    +
    +    
    +       //drop time from comparison
    +    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 ProcTimeSortProcessFunction(
    +      inputType.getFieldCount,
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +
    +  
    +   /**
    +   * Function creates comparison objects based on the field types
    +   * @param inputType input row type
    +   * @param keyIndex the indexes of the fields on which the sorting is done. 
    +   * @param orderDirection the directions of each sort field. 
    +   * @param execConfig the configuration environment
    +   * @return Array of TypeComparator objects
    +   */
    +  def createFieldComparators(
    +      inputType: RelDataType,
    +      keyIndex: Array[Int],
    +      orderDirection: Array[Direction],
    +      execConfig: ExecutionConfig): Array[TypeComparator[_]] = {
    +    
    +    var iOrder = 0
    +    for (i <- keyIndex) yield {
    +
    +      val order = if (orderDirection(iOrder) == Direction.ASCENDING) true else false
    +      iOrder += 1
    +      val fieldTypeInfo = FlinkTypeFactory.toTypeInfo(inputType.getFieldList.get(i).getType)
    +      fieldTypeInfo match {
    +        case a: AtomicType[_] => a.createComparator(order, execConfig)
    +        case _ => throw new TableException(s"Unsupported field type $fieldTypeInfo to sort on.")
    +      }
    +    }
    +  }
    +  
    +   /**
    +   * Function creates a RowComparator based on the typed comparators
    +   * @param inputRowType input row type
    +   * @param fieldIdxs the indexes of the fields on which the sorting is done. 
    +   * @param fieldComps the array of typed comparators
    +   * @param fieldOrders the directions of each sort field (true = ASC). 
    +   * @return A Row TypeComparator object 
    +   */
    +  def createRowComparator(
    +    inputRowType: RelDataType,
    +    fieldIdxs: Array[Int],
    +    fieldComps: Array[TypeComparator[AnyRef]],
    +    fieldOrders: Array[Boolean]): TypeComparator[Row] = {
    +
    +  val rowComp = new RowComparator(
    +    inputRowType.getFieldCount,
    +    fieldIdxs,
    +    fieldComps,
    +    new Array[TypeSerializer[AnyRef]](0), //used only for serialized comparisons
    +    fieldOrders)
    +
    +  rowComp
    +}
    +  
    + 
    +  /**
    +   * Function returns the array of indexes for the fields on which the sort is done
    +   * @param collationSort The Sort collation list
    +   * @return [Array[Int]]
    +   */
    +  def getSortFieldIndexList(collationSort: RelCollation): Array[Int] = {
    +    val keyFields = collationSort.getFieldCollations.toArray()
    +    for (f <- keyFields) yield f.asInstanceOf[RelFieldCollation].getFieldIndex
    +  }
    +  
    +   /**
    +   * Function returns the array of sort direction for each of the sort fields 
    +   * @param collationSort The Sort collation list
    +   * @return [Array[Direction]]
    +   */
    +  def getSortFieldDirectionList(collationSort: RelCollation): Array[Direction] = {
    +    val keyFields = collationSort.getFieldCollations.toArray()
    +    for(f <- keyFields) yield f.asInstanceOf[RelFieldCollation].getDirection
    +  }
    +
    +   /**
    +   * Function returns the array of sort direction for each of the sort fields 
    +   * @param collationSort The Sort collation list
    +   * @return [Array[Direction]]
    +   */
    +  def getSortFieldDirectionBooleanList(collationSort: RelCollation): Array[Boolean] = {
    +    val keyFields = collationSort.getFieldCollations
    +    var i = 0
    +    val keySortDirection = new Array[Boolean](keyFields.size())
    +    while (i < keyFields.size()) {
    +      keySortDirection(i) = 
    +        if (getDirection(collationSort,i) == Direction.ASCENDING) true else false 
    +      i += 1
    +    }
    +    keySortDirection
    +  }
    +  
    +  
    +   /**
    +   * Function returns the direction type of the time in order clause. 
    +   * @param collationSort The Sort collation list of objects
    +   * @return [org.apache.calcite.rel.RelFieldCollation.Direction]
    +   */
    +  def getTimeDirection(collationSort: RelCollation): Direction = {
    +    collationSort.getFieldCollations.get(0).direction
    +  }
    +  
    +   /**
    +   * Function returns the time type in order clause. 
    +   * Expectation is that it is the primary sort field
    +   * @param collationSort The Sort collation list
    +   * @param rowType The data type of the input
    +   * @return [org.apache.calcite.rel.type.RelDataType]
    +   */
    +  def getTimeType(collationSort: RelCollation, rowType: RelDataType): RelDataType = {
    +
    +    //need to identify time between others ordering fields
    +    val ind = collationSort.getFieldCollations.get(0).getFieldIndex
    +    rowType.getFieldList.get(ind).getValue
    --- End diff --
    
    use `getType` instead of `getValue`


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

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

Posted by fhueske <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

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

    https://github.com/apache/flink/pull/3889#discussion_r118241119
  
    --- 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 --
    
    I am only using it to extract the names


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

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

Posted by fhueske <gi...@git.apache.org>.
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 "todo" comments


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123742995
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 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}
    +
    +/**
    + * 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)
    +    //drop time from comparison as we sort on time in the states and result emission
    +    val keyIndexesNoTime = keySortFields.slice(1, keySortFields.size)
    --- End diff --
    
    for Strings and Arrays, `length` should be used instead of `size`.


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123713179
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonSort.scala ---
    @@ -0,0 +1,91 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.table.plan.nodes
    +
    +import org.apache.calcite.rex.{RexLiteral, RexNode}
    +import org.apache.calcite.rel.RelFieldCollation
    +import org.apache.calcite.rel.RelCollation
    +import org.apache.calcite.rel.RelFieldCollation.Direction
    +import org.apache.calcite.rel.`type`._
    +import scala.collection.JavaConverters._
    +import org.apache.flink.api.common.operators.Order
    +
    +
    +/**
    + * Trait represents a collection of sort methods to manipulate the parameters
    + */
    +
    +trait CommonSort {
    +  
    +  private[flink] def offsetToString(offset: RexNode): String = {
    +    val offsetToString = s"$offset"
    +    offsetToString
    +  }
    +  
    +  
    +  private[flink] def sortFieldsToString(
    +      collationSort: RelCollation, 
    +      rowRelDataType: RelDataType): String = {
    +    val fieldCollations = collationSort.getFieldCollations.asScala  
    +    .map(c => (c.getFieldIndex, directionToOrder(c.getDirection)))
    +
    +    val sortFieldsToString = fieldCollations
    --- End diff --
    
    `sortFieldsToString` is only used as return value. In that case we can simply return the result of the expression without defining a `val`:
    ```
    fieldCollations
      .map(col => s"${rowRelDataType.getFieldNames.get(col._1)} ${col._2.getShortName}" )
      .mkString(", ")
    ```


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

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

Posted by fhueske <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream SQL

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

    https://github.com/apache/flink/pull/3889
  
    @fhueske Thanks for the review and for handling the the fixes (sorry for still kipping some :( ). I will create this week the PR for the rest of the functions.


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

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

Posted by fhueske <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

Posted by fhueske <gi...@git.apache.org>.
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 first sort element
    +    checkTimeOrder()
    +    
    +    super.explainTerms(pw)
    +      .item("orderBy", SortUtil.getSortFieldToString(sortCollation, schema.logicalType))
    +      .item("offset", SortUtil.getOffsetToString(sortOffset))
    +      .item("fetch", SortUtil.getFetchToString(sortFetch, sortOffset))
    +  }
    +
    +  override def translateToPlan(
    +      tableEnv: StreamTableEnvironment,
    +      queryConfig: StreamQueryConfig): DataStream[CRow] = {
    +    
    +    val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
    +    
    +    //need to identify time between others order fields. Time needs to be first sort element
    +    val timeType = SortUtil.getTimeType(sortCollation, schema.logicalType)
    +    
    +    //time ordering needs to be ascending
    +    if (SortUtil.getTimeDirection(sortCollation) != Direction.ASCENDING) {
    +      throw new TableException("SQL/Table supports only ascending time ordering")
    +    }
    +    
    +    val execCfg = tableEnv.execEnv.getConfig
    +      
    +    //enable to extend for other types of aggregates that will not be implemented in a window
    +    timeType match {
    +        case _ if FlinkTypeFactory.isProctimeIndicatorType(timeType)  =>
    +            (sortOffset,sortFetch) match {
    +              case (o: Any, f: Any)  => // offset and fetch needs retraction
    +                throw new TableException("SQL/Table does not support sort with offset and fetch") 
    +              case (_, f: Any) => // offset needs retraction
    +                throw new TableException("SQL/Table does not support sort with fetch")
    +              case (o: Any, _) =>  // fetch needs retraction
    +                throw new TableException("SQL/Table does not support sort with offset")
    +              case _ => createSortProcTime(inputDS, execCfg)  //sort can be done without retraction
    +            }
    +        case _ if FlinkTypeFactory.isRowtimeIndicatorType(timeType) =>
    +            (sortOffset,sortFetch) match {
    +              case (o: Any, f: Any)  => // offset and fetch needs retraction
    +                throw new TableException("SQL/Table does not support sort with offset and fetch") 
    +              case (_, f: Any) => // offset needs retraction
    +                throw new TableException("SQL/Table does not support sort with fetch")
    +              case (o: Any, _) =>  // fetch needs retraction
    +                throw new TableException("SQL/Table does not support sort with offset")
    +              case _ => createSortRowTime(inputDS, execCfg)  //sort can be done without retraction
    +            }
    +        case _ =>
    +          throw new TableException("SQL/Table needs to have sort on time as first sort element")
    +    }
    +    
    +  }
    +
    +  /**
    +   * Create Sort logic based on processing time
    +   */
    +  def createSortProcTime(
    +    inputDS: DataStream[CRow],
    +    execCfg: ExecutionConfig): DataStream[CRow] = {
    +
    +   val returnTypeInfo = CRowTypeInfo(schema.physicalTypeInfo)
    +    
    +    //if the order has secondary sorting fields in addition to the proctime
    +    if( SortUtil.getSortFieldIndexList(sortCollation).size > 1) {
    +    
    +      val processFunction = SortUtil.createProcTimeSortFunction(sortCollation,
    +           inputSchema.logicalType, inputSchema.physicalTypeInfo, execCfg)
    +      
    +      inputDS
    +            .keyBy(new NullByteKeySelector[CRow])
    +            .process(processFunction).setParallelism(1).setMaxParallelism(1)
    +            .returns(returnTypeInfo)
    +            .asInstanceOf[DataStream[CRow]]
    +    } else {
    +      //if the order is done only on proctime we only need to forward the elements
    +        inputDS.keyBy(new NullByteKeySelector[CRow])
    +          .map(new IdentityRowMap())
    --- End diff --
    
    rename to `IdentityCRowMap`


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123736450
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSort.scala ---
    @@ -0,0 +1,206 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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
    +import org.apache.flink.table.plan.nodes.CommonSort
    +
    +/**
    +  * Flink RelNode which matches along with Sort Rule.
    +  *
    + */
    +class DataStreamSort(
    +    cluster: RelOptCluster,
    +    traitSet: RelTraitSet,
    +    inputNode: RelNode,
    +    inputSchema: RowSchema,
    +    schema: RowSchema,
    +    sortCollation: RelCollation,
    +    sortOffset: RexNode,
    +    sortFetch: RexNode,
    +    description: String)
    +  extends SingleRel(cluster, traitSet, inputNode)
    +  with CommonSort
    +  with DataStreamRel {
    +
    +  override def deriveRowType(): RelDataType = schema.logicalType
    +
    +  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
    +    new DataStreamSort(
    +      cluster,
    +      traitSet,
    +      inputs.get(0),
    +      inputSchema,
    +      schema,
    +      sortCollation,
    +      sortOffset,
    +      sortFetch,
    +      description)
    +  }
    +
    +  override def toString: String = {
    +    s"Sort(by: ($$sortFieldsToString(sortCollation, schema.logicalType))," +
    +      " offset: $offsetToString(sortOffset)," +
    +      " fetch: $fetchToString(sortFetch, sortOffset))"
    +  }
    +  
    +  override def explainTerms(pw: RelWriter) : RelWriter = {
    +    
    +    super.explainTerms(pw)
    +      .item("orderBy", sortFieldsToString(sortCollation, schema.logicalType))
    +      .item("offset", offsetToString(sortOffset))
    +      .item("fetch", fetchToString(sortFetch, sortOffset))
    +  }
    +
    +  override def translateToPlan(
    +      tableEnv: StreamTableEnvironment,
    +      queryConfig: StreamQueryConfig): DataStream[CRow] = {
    +    
    +    val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
    +    
    +    //need to identify time between others order fields. Time needs to be first sort element
    +    val timeType = SortUtil.getTimeType(sortCollation, schema.logicalType)
    +    
    +    //time ordering needs to be ascending
    +    if (SortUtil.getTimeDirection(sortCollation) != Direction.ASCENDING) {
    +      throw new TableException("SQL/Table supports only ascending time ordering")
    +    }
    +    
    +    val execCfg = tableEnv.execEnv.getConfig
    +      
    +    //enable to extend for other types of aggregates that will not be implemented in a window
    +    timeType match {
    +        case _ if FlinkTypeFactory.isProctimeIndicatorType(timeType)  =>
    +            (sortOffset, sortFetch) match {
    +              case (_: RexNode, _: RexNode)  => // offset and fetch needs retraction
    +                throw new TableException("SQL/Table does not support sort with offset and fetch") 
    +              case (_, _: RexNode) => // offset needs retraction
    +                throw new TableException("SQL/Table does not support sort with fetch")
    +              case (_: RexNode, _) =>  // fetch needs retraction
    +                throw new TableException("SQL/Table does not support sort with offset")
    +              case _ => createSortProcTime(inputDS, execCfg)  //sort can be done without retraction
    +            }
    +        case _ if FlinkTypeFactory.isRowtimeIndicatorType(timeType) =>
    +            (sortOffset, sortFetch) match {
    +              case (_: RexNode, _: RexNode)  => // offset and fetch needs retraction
    +                throw new TableException("SQL/Table does not support sort with offset and fetch") 
    +              case (_, _: RexNode) => // offset needs retraction
    +                throw new TableException("SQL/Table does not support sort with fetch")
    +              case (_: RexNode, _) =>  // fetch needs retraction
    +                throw new TableException("SQL/Table does not support sort with offset")
    +              case _ => createSortRowTime(inputDS, execCfg)  //sort can be done without retraction
    +            }
    +        case _ =>
    +          throw new TableException("SQL/Table needs to have sort on time as first sort element")
    +    }
    +    
    +  }
    +
    +  /**
    +   * Create Sort logic based on processing time
    +   */
    +  def createSortProcTime(
    +    inputDS: DataStream[CRow],
    +    execCfg: ExecutionConfig): DataStream[CRow] = {
    +
    +   val returnTypeInfo = CRowTypeInfo(schema.physicalTypeInfo)
    +    
    +    //if the order has secondary sorting fields in addition to the proctime
    +    if( SortUtil.getSortFieldIndexList(sortCollation).size > 1) {
    +    
    +      val processFunction = SortUtil.createProcTimeSortFunction(
    +        sortCollation,
    +        inputSchema.logicalType, 
    +        inputSchema.physicalTypeInfo, 
    +        execCfg)
    +      
    +      inputDS.keyBy(new NullByteKeySelector[CRow])
    +        .process(processFunction).setParallelism(1).setMaxParallelism(1)
    +        .returns(returnTypeInfo)
    +        .asInstanceOf[DataStream[CRow]]
    +    } else {
    +      //if the order is done only on proctime we only need to forward the elements
    +      inputDS.keyBy(new NullByteKeySelector[CRow])
    --- End diff --
    
    the identity mapper does not need a keyBy. It's sufficient to set the parallelism and maxParallelism to 1.


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123709798
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SortITCase.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.api.scala.stream.sql
    +
    +import org.apache.flink.api.scala._
    +import org.apache.flink.table.api.scala.stream.sql.SortITCase.StringRowSelectorSink
    +import org.apache.flink.table.api.scala.stream.sql.TimeTestUtil.EventTimeSourceFunction
    +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"))),
    +      Left((2000L, (3L, 1, "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"))),
    +      Left((6000L, (6L, 67, "Hello"))),
    +      Left((6000L, (6L, -1, "Hello"))),
    +      Left((6000L, (6L, 6, "Hello"))),
    +      Right(7000L),
    +      Left((9000L, (6L, 9, "Hello"))),
    +      Left((8500L, (6L, 18, "Hello"))),
    +      Left((9000L, (6L, 7, "Hello"))),
    +      Right(10000L),
    +      Left((10000L, (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((20000L, (20L, 20, "Hello World"))), 
    +      Right(19000L))
    +
    +    val env = StreamExecutionEnvironment.getExecutionEnvironment
    +    env.setStateBackend(getStateBackend)
    +    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 ";
    --- End diff --
    
    rm `;`


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r125006340
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala ---
    @@ -92,8 +92,19 @@ class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttle {
       override def visit(minus: LogicalMinus): RelNode =
         throw new TableException("Logical minus in a stream environment is not supported yet.")
     
    -  override def visit(sort: LogicalSort): RelNode =
    -    throw new TableException("Logical sort in a stream environment is not supported yet.")
    +  override def visit(sort: LogicalSort): RelNode = {
    +
    +    val input = sort.getInput.accept(this)
    +    
    +    val materializer = new RexTimeIndicatorMaterializer(
    --- End diff --
    
    unused. Should be removed


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123713771
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonSort.scala ---
    @@ -0,0 +1,91 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.table.plan.nodes
    +
    +import org.apache.calcite.rex.{RexLiteral, RexNode}
    +import org.apache.calcite.rel.RelFieldCollation
    +import org.apache.calcite.rel.RelCollation
    +import org.apache.calcite.rel.RelFieldCollation.Direction
    +import org.apache.calcite.rel.`type`._
    +import scala.collection.JavaConverters._
    +import org.apache.flink.api.common.operators.Order
    +
    +
    +/**
    + * Trait represents a collection of sort methods to manipulate the parameters
    + */
    +
    +trait CommonSort {
    +  
    +  private[flink] def offsetToString(offset: RexNode): String = {
    +    val offsetToString = s"$offset"
    +    offsetToString
    +  }
    +  
    +  
    +  private[flink] def sortFieldsToString(
    --- End diff --
    
    This is probably true for most other methods in this class.


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

[GitHub] flink issue #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream SQL

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

    https://github.com/apache/flink/pull/3889
  
    @hongyuhong @stefanobortoli @shijinkui i forgot to add you to the PR
    



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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123717004
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamSortRule.scala ---
    @@ -0,0 +1,111 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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
    +import org.apache.flink.table.calcite.FlinkTypeFactory
    +import org.apache.flink.table.api.TableException
    +import org.apache.calcite.rel.RelFieldCollation.Direction
    +import org.apache.flink.table.runtime.aggregate.SortUtil
    +
    +/**
    + * 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 = {
    +
    +    val result = super.matches(call)
    +    
    +    //need to identify time between others order fields. Time needs to be first sort element
    +    // we can safely convert the object if the match rule succeeded 
    +    if(result) {
    --- End diff --
    
    please add a space after `if`, `for`, `while`, etc.


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123713426
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonSort.scala ---
    @@ -0,0 +1,91 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.table.plan.nodes
    +
    +import org.apache.calcite.rex.{RexLiteral, RexNode}
    +import org.apache.calcite.rel.RelFieldCollation
    +import org.apache.calcite.rel.RelCollation
    +import org.apache.calcite.rel.RelFieldCollation.Direction
    +import org.apache.calcite.rel.`type`._
    +import scala.collection.JavaConverters._
    +import org.apache.flink.api.common.operators.Order
    +
    +
    +/**
    + * Trait represents a collection of sort methods to manipulate the parameters
    + */
    +
    +trait CommonSort {
    +  
    +  private[flink] def offsetToString(offset: RexNode): String = {
    +    val offsetToString = s"$offset"
    +    offsetToString
    +  }
    +  
    +  
    +  private[flink] def sortFieldsToString(
    +      collationSort: RelCollation, 
    +      rowRelDataType: RelDataType): String = {
    +    val fieldCollations = collationSort.getFieldCollations.asScala  
    +    .map(c => (c.getFieldIndex, directionToOrder(c.getDirection)))
    +
    +    val sortFieldsToString = fieldCollations
    +      .map(col => s"${
    +        rowRelDataType.getFieldNames.get(col._1)} ${col._2.getShortName}" ).mkString(", ")
    +    
    +    sortFieldsToString
    +  }
    +  
    +  private[flink] def directionToOrder(direction: Direction) = {
    +    direction match {
    +      case Direction.ASCENDING | Direction.STRICTLY_ASCENDING => Order.ASCENDING
    +      case Direction.DESCENDING | Direction.STRICTLY_DESCENDING => Order.DESCENDING
    +      case _ => throw new IllegalArgumentException("Unsupported direction.")
    +    }
    +  }
    +  
    +  private[flink] def fetchToString(fetch: RexNode, offset: RexNode): String = {
    +    val limitEnd = getFetchLimitEnd(fetch, offset)
    +    val fetchToString = if (limitEnd == Long.MaxValue) {
    --- End diff --
    
    No need to define `val fetchToString` if it is only returned.


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

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

Posted by fhueske <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

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

    https://github.com/apache/flink/pull/3889#discussion_r116540900
  
    --- 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
    --- End diff --
    
    -1 space. Check expected exceptions with `@Test(expected = classOf[YourExpectedException])`


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

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

Posted by fhueske <gi...@git.apache.org>.
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,
    +        keyIndexesNoTime, fieldCompsRefs, booleanDirectionsNoTime)
    +    val collectionRowComparator = new CollectionRowComparator(rowComp)
    +    
    +    val inputCRowType = CRowTypeInfo(inputTypeInfo)
    + 
    +    new RowTimeSortProcessFunction(
    +      inputType.getFieldCount,
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +  
    +  
    +  /**
    +   * Function creates [org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
    +   * elements based on proctime 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 createProcTimeSortFunction(
    +    collationSort: RelCollation,
    +    inputType: RelDataType,
    +    inputTypeInfo: TypeInformation[Row],
    +    execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
    +
    +    val keySortFields = getSortFieldIndexList(collationSort)
    +    val keySortDirections = getSortFieldDirectionList(collationSort)
    +
    +    
    +       //drop time from comparison
    +    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 ProcTimeSortProcessFunction(
    +      inputType.getFieldCount,
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +
    +  
    +   /**
    +   * Function creates comparison objects based on the field types
    +   * @param inputType input row type
    +   * @param keyIndex the indexes of the fields on which the sorting is done. 
    +   * @param orderDirection the directions of each sort field. 
    +   * @param execConfig the configuration environment
    +   * @return Array of TypeComparator objects
    +   */
    +  def createFieldComparators(
    +      inputType: RelDataType,
    +      keyIndex: Array[Int],
    +      orderDirection: Array[Direction],
    +      execConfig: ExecutionConfig): Array[TypeComparator[_]] = {
    +    
    +    var iOrder = 0
    +    for (i <- keyIndex) yield {
    +
    +      val order = if (orderDirection(iOrder) == Direction.ASCENDING) true else false
    +      iOrder += 1
    +      val fieldTypeInfo = FlinkTypeFactory.toTypeInfo(inputType.getFieldList.get(i).getType)
    +      fieldTypeInfo match {
    +        case a: AtomicType[_] => a.createComparator(order, execConfig)
    +        case _ => throw new TableException(s"Unsupported field type $fieldTypeInfo to sort on.")
    +      }
    +    }
    +  }
    +  
    +   /**
    +   * Function creates a RowComparator based on the typed comparators
    +   * @param inputRowType input row type
    +   * @param fieldIdxs the indexes of the fields on which the sorting is done. 
    +   * @param fieldComps the array of typed comparators
    +   * @param fieldOrders the directions of each sort field (true = ASC). 
    +   * @return A Row TypeComparator object 
    +   */
    +  def createRowComparator(
    +    inputRowType: RelDataType,
    --- End diff --
    
    `arity: Int` is sufficient here


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

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

Posted by fhueske <gi...@git.apache.org>.
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 RowTimeSortProcessFunction(
    +      inputType.getFieldCount,
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +  
    +  
    +  /**
    +   * Function creates [org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
    +   * elements based on proctime 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 createProcTimeSortFunction(
    +    collationSort: RelCollation,
    +    inputType: RelDataType,
    +    inputTypeInfo: TypeInformation[Row],
    +    execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
    +
    +    val keySortFields = getSortFieldIndexList(collationSort)
    +    val keySortDirections = getSortFieldDirectionList(collationSort)
    +
    +    
    +       //drop time from comparison
    +    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 ProcTimeSortProcessFunction(
    +      inputType.getFieldCount,
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +
    +  
    +   /**
    +   * Function creates comparison objects based on the field types
    +   * @param inputType input row type
    +   * @param keyIndex the indexes of the fields on which the sorting is done. 
    +   * @param orderDirection the directions of each sort field. 
    +   * @param execConfig the configuration environment
    +   * @return Array of TypeComparator objects
    +   */
    +  def createFieldComparators(
    +      inputType: RelDataType,
    +      keyIndex: Array[Int],
    +      orderDirection: Array[Direction],
    +      execConfig: ExecutionConfig): Array[TypeComparator[_]] = {
    +    
    +    var iOrder = 0
    +    for (i <- keyIndex) yield {
    +
    +      val order = if (orderDirection(iOrder) == Direction.ASCENDING) true else false
    +      iOrder += 1
    +      val fieldTypeInfo = FlinkTypeFactory.toTypeInfo(inputType.getFieldList.get(i).getType)
    +      fieldTypeInfo match {
    +        case a: AtomicType[_] => a.createComparator(order, execConfig)
    +        case _ => throw new TableException(s"Unsupported field type $fieldTypeInfo to sort on.")
    +      }
    +    }
    +  }
    +  
    +   /**
    +   * Function creates a RowComparator based on the typed comparators
    +   * @param inputRowType input row type
    +   * @param fieldIdxs the indexes of the fields on which the sorting is done. 
    +   * @param fieldComps the array of typed comparators
    +   * @param fieldOrders the directions of each sort field (true = ASC). 
    +   * @return A Row TypeComparator object 
    +   */
    +  def createRowComparator(
    +    inputRowType: RelDataType,
    +    fieldIdxs: Array[Int],
    +    fieldComps: Array[TypeComparator[AnyRef]],
    +    fieldOrders: Array[Boolean]): TypeComparator[Row] = {
    +
    +  val rowComp = new RowComparator(
    +    inputRowType.getFieldCount,
    +    fieldIdxs,
    +    fieldComps,
    +    new Array[TypeSerializer[AnyRef]](0), //used only for serialized comparisons
    --- End diff --
    
    should be "used only for object comparisons"


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123760084
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/TimeSortProcessFunctionTest.scala ---
    @@ -0,0 +1,256 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.RowTimeSortProcessFunction
    +import org.apache.flink.table.runtime.aggregate.TimeSortProcessFunctionTest._
    +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}
    +import org.apache.flink.streaming.api.TimeCharacteristic
    +
    +class TimeSortProcessFunctionTest{
    +
    +  
    +  @Test
    +  def testSortProcTimeHarnessPartitioned(): Unit = {
    +    
    +    val rT =  new RowTypeInfo(Array[TypeInformation[_]](
    +      INT_TYPE_INFO,
    +      LONG_TYPE_INFO,
    +      INT_TYPE_INFO,
    +      STRING_TYPE_INFO,
    +      LONG_TYPE_INFO),
    +      Array("a","b","c","d","e"))
    +    
    +    val rTA =  new RowTypeInfo(Array[TypeInformation[_]](
    +     LONG_TYPE_INFO), Array("count"))
    +    val indexes = Array(1,2)
    +      
    +    val fieldComps = Array[TypeComparator[AnyRef]](
    +      LONG_TYPE_INFO.createComparator(true, null).asInstanceOf[TypeComparator[AnyRef]],
    +      INT_TYPE_INFO.createComparator(false, null).asInstanceOf[TypeComparator[AnyRef]] )
    +    val booleanOrders = Array(true, false)    
    +    
    +
    +    val rowComp = new RowComparator(
    +      rT.getTotalFields,
    +      indexes,
    +      fieldComps,
    +      new Array[TypeSerializer[AnyRef]](0), //used only for serialized comparisons
    +      booleanOrders)
    +    
    +    val collectionRowComparator = new CollectionRowComparator(rowComp)
    +    
    +    val inputCRowType = CRowTypeInfo(rT)
    +    
    +    val processFunction = new KeyedProcessOperator[Integer,CRow,CRow](
    +      new ProcTimeSortProcessFunction(
    +        inputCRowType,
    +        collectionRowComparator))
    +  
    +   val testHarness = new KeyedOneInputStreamOperatorTestHarness[Integer,CRow,CRow](
    +      processFunction, 
    +      new TupleRowSelector(0), 
    +      BasicTypeInfo.INT_TYPE_INFO)
    +    
    +   testHarness.open();
    +
    +   testHarness.setProcessingTime(3)
    +
    +      // timestamp is ignored in processing time
    +    testHarness.processElement(new StreamRecord(new CRow(
    +      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 11L: JLong), true), 1001))
    +    testHarness.processElement(new StreamRecord(new CRow(
    +        Row.of(1: JInt, 12L: JLong, 1: JInt, "aaa", 11L: JLong), true), 2002))
    +    testHarness.processElement(new StreamRecord(new CRow(
    +        Row.of(1: JInt, 12L: JLong, 2: JInt, "aaa", 11L: JLong), true), 2003))
    +    testHarness.processElement(new StreamRecord(new CRow(
    +        Row.of(1: JInt, 12L: JLong, 0: JInt, "aaa", 11L: JLong), true), 2004))
    +    testHarness.processElement(new StreamRecord(new CRow(
    +        Row.of(1: JInt, 10L: JLong, 0: JInt, "aaa", 11L: JLong), true), 2006))
    +
    +    //move the timestamp to ensure the execution
    +    testHarness.setProcessingTime(1005)
    --- End diff --
    
    Please run at least two sorts to ensure that all intermediate state is correctly reset.
    Also verify the state size to ensure that the operator is not leaking.


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r124491150
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonSort.scala ---
    @@ -0,0 +1,91 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.table.plan.nodes
    +
    +import org.apache.calcite.rex.{RexLiteral, RexNode}
    +import org.apache.calcite.rel.RelFieldCollation
    +import org.apache.calcite.rel.RelCollation
    +import org.apache.calcite.rel.RelFieldCollation.Direction
    +import org.apache.calcite.rel.`type`._
    +import scala.collection.JavaConverters._
    +import org.apache.flink.api.common.operators.Order
    +
    +
    +/**
    + * Trait represents a collection of sort methods to manipulate the parameters
    + */
    +
    +trait CommonSort {
    +  
    +  private[flink] def offsetToString(offset: RexNode): String = {
    --- End diff --
    
    > It is more for consistency - to have a method for "ToString" for each feature of the sort (direction, fetch, fields...). As there is not cost for it I suggest to keep it


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

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

Posted by fhueske <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

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

    https://github.com/apache/flink/pull/3889#discussion_r116521402
  
    --- 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 RowTimeSortProcessFunction(
    +      inputType.getFieldCount,
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +  
    +  
    +  /**
    +   * Function creates [org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
    +   * elements based on proctime 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 createProcTimeSortFunction(
    +    collationSort: RelCollation,
    +    inputType: RelDataType,
    +    inputTypeInfo: TypeInformation[Row],
    +    execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
    +
    +    val keySortFields = getSortFieldIndexList(collationSort)
    +    val keySortDirections = getSortFieldDirectionList(collationSort)
    +
    +    
    +       //drop time from comparison
    +    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 ProcTimeSortProcessFunction(
    +      inputType.getFieldCount,
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +
    +  
    +   /**
    +   * Function creates comparison objects based on the field types
    +   * @param inputType input row type
    +   * @param keyIndex the indexes of the fields on which the sorting is done. 
    +   * @param orderDirection the directions of each sort field. 
    +   * @param execConfig the configuration environment
    +   * @return Array of TypeComparator objects
    +   */
    +  def createFieldComparators(
    +      inputType: RelDataType,
    +      keyIndex: Array[Int],
    +      orderDirection: Array[Direction],
    +      execConfig: ExecutionConfig): Array[TypeComparator[_]] = {
    +    
    +    var iOrder = 0
    +    for (i <- keyIndex) yield {
    +
    +      val order = if (orderDirection(iOrder) == Direction.ASCENDING) true else false
    +      iOrder += 1
    +      val fieldTypeInfo = FlinkTypeFactory.toTypeInfo(inputType.getFieldList.get(i).getType)
    +      fieldTypeInfo match {
    +        case a: AtomicType[_] => a.createComparator(order, execConfig)
    +        case _ => throw new TableException(s"Unsupported field type $fieldTypeInfo to sort on.")
    +      }
    +    }
    +  }
    +  
    +   /**
    +   * Function creates a RowComparator based on the typed comparators
    +   * @param inputRowType input row type
    +   * @param fieldIdxs the indexes of the fields on which the sorting is done. 
    +   * @param fieldComps the array of typed comparators
    +   * @param fieldOrders the directions of each sort field (true = ASC). 
    +   * @return A Row TypeComparator object 
    +   */
    +  def createRowComparator(
    +    inputRowType: RelDataType,
    +    fieldIdxs: Array[Int],
    +    fieldComps: Array[TypeComparator[AnyRef]],
    +    fieldOrders: Array[Boolean]): TypeComparator[Row] = {
    +
    +  val rowComp = new RowComparator(
    +    inputRowType.getFieldCount,
    +    fieldIdxs,
    +    fieldComps,
    +    new Array[TypeSerializer[AnyRef]](0), //used only for serialized comparisons
    +    fieldOrders)
    +
    +  rowComp
    +}
    +  
    + 
    +  /**
    +   * Function returns the array of indexes for the fields on which the sort is done
    +   * @param collationSort The Sort collation list
    +   * @return [Array[Int]]
    --- End diff --
    
    Should be `[[Array[Int]]]`. Also please describe the result of the function.


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r118466310
  
    --- 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 --
    
    I modified the data set to include up to 5 elements on the same timestamp - including having duplicate values, positive and negative, as well as having timestamps with elements that are unordered


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123717194
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamSortRule.scala ---
    @@ -0,0 +1,111 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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
    +import org.apache.flink.table.calcite.FlinkTypeFactory
    +import org.apache.flink.table.api.TableException
    +import org.apache.calcite.rel.RelFieldCollation.Direction
    +import org.apache.flink.table.runtime.aggregate.SortUtil
    +
    +/**
    + * 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 = {
    +
    +    val result = super.matches(call)
    +    
    +    //need to identify time between others order fields. Time needs to be first sort element
    +    // we can safely convert the object if the match rule succeeded 
    +    if(result) {
    +      val calcSort: FlinkLogicalSort = call.rel(0).asInstanceOf[FlinkLogicalSort]
    --- End diff --
    
    rename to `sort`. It is not a `calc`


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123743615
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 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}
    +
    +/**
    + * 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)
    +    //drop time from comparison as we sort on time in the states and result emission
    +    val keyIndexesNoTime = keySortFields.slice(1, keySortFields.size)
    +    val booleanOrderings = getSortFieldDirectionBooleanList(collationSort)
    +    val booleanDirectionsNoTime = booleanOrderings.slice(1, booleanOrderings.size)
    +    
    +    val fieldComps = createFieldComparators(
    +      inputType, 
    +      keyIndexesNoTime, 
    +      booleanDirectionsNoTime,
    +      execCfg)
    +    val fieldCompsRefs = fieldComps.asInstanceOf[Array[TypeComparator[AnyRef]]]
    +    
    +    val rowComp = new RowComparator(
    +       inputType.getFieldCount,
    +       keyIndexesNoTime,
    +       fieldCompsRefs,
    +       new Array[TypeSerializer[AnyRef]](0), //used only for object comparisons
    +       booleanDirectionsNoTime)
    +      
    +    val collectionRowComparator = new CollectionRowComparator(rowComp)
    +    
    +    val inputCRowType = CRowTypeInfo(inputTypeInfo)
    + 
    +    new RowTimeSortProcessFunction(
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +  
    +  
    +  /**
    +   * Function creates [org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
    +   * elements based on proctime 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
    --- End diff --
    
    The return type is visible from the function signature. The documentation is usually to describe the return value.


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

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

Posted by fhueske <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123710581
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonSort.scala ---
    @@ -0,0 +1,91 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.table.plan.nodes
    +
    +import org.apache.calcite.rex.{RexLiteral, RexNode}
    +import org.apache.calcite.rel.RelFieldCollation
    +import org.apache.calcite.rel.RelCollation
    +import org.apache.calcite.rel.RelFieldCollation.Direction
    +import org.apache.calcite.rel.`type`._
    +import scala.collection.JavaConverters._
    +import org.apache.flink.api.common.operators.Order
    +
    +
    +/**
    + * Trait represents a collection of sort methods to manipulate the parameters
    + */
    +
    +trait CommonSort {
    +  
    +  private[flink] def offsetToString(offset: RexNode): String = {
    +    val offsetToString = s"$offset"
    +    offsetToString
    +  }
    +  
    --- End diff --
    
    rm new line


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

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

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

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


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

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

Posted by fhueske <gi...@git.apache.org>.
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 calculation
    +    val triggeringTs = ctx.timestamp
    --- End diff --
    
    rename to `rowtime`?


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123713964
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSort.scala ---
    @@ -0,0 +1,206 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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
    +import org.apache.flink.table.plan.nodes.CommonSort
    +
    +/**
    +  * Flink RelNode which matches along with Sort Rule.
    +  *
    + */
    +class DataStreamSort(
    +    cluster: RelOptCluster,
    +    traitSet: RelTraitSet,
    +    inputNode: RelNode,
    +    inputSchema: RowSchema,
    +    schema: RowSchema,
    +    sortCollation: RelCollation,
    +    sortOffset: RexNode,
    +    sortFetch: RexNode,
    +    description: String)
    +  extends SingleRel(cluster, traitSet, inputNode)
    +  with CommonSort
    +  with DataStreamRel {
    +
    +  override def deriveRowType(): RelDataType = schema.logicalType
    +
    +  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
    +    new DataStreamSort(
    +      cluster,
    +      traitSet,
    +      inputs.get(0),
    +      inputSchema,
    +      schema,
    +      sortCollation,
    +      sortOffset,
    +      sortFetch,
    +      description)
    +  }
    +
    +  override def toString: String = {
    +    s"Sort(by: ($$sortFieldsToString(sortCollation, schema.logicalType))," +
    --- End diff --
    
    Move implementation of `toString` to `CommonSort`.


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r124575119
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/TimeSortProcessFunctionTest.scala ---
    @@ -0,0 +1,256 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.RowTimeSortProcessFunction
    +import org.apache.flink.table.runtime.aggregate.TimeSortProcessFunctionTest._
    +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}
    +import org.apache.flink.streaming.api.TimeCharacteristic
    +
    +class TimeSortProcessFunctionTest{
    +
    +  
    +  @Test
    +  def testSortProcTimeHarnessPartitioned(): Unit = {
    +    
    +    val rT =  new RowTypeInfo(Array[TypeInformation[_]](
    --- End diff --
    
    I think it is ok - we sort on second and third field - on one ascending and on the other descending 


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

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

Posted by fhueske <gi...@git.apache.org>.
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 first sort element
    +    checkTimeOrder()
    +    
    +    super.explainTerms(pw)
    +      .item("orderBy", SortUtil.getSortFieldToString(sortCollation, schema.logicalType))
    +      .item("offset", SortUtil.getOffsetToString(sortOffset))
    +      .item("fetch", SortUtil.getFetchToString(sortFetch, sortOffset))
    +  }
    +
    +  override def translateToPlan(
    +      tableEnv: StreamTableEnvironment,
    +      queryConfig: StreamQueryConfig): DataStream[CRow] = {
    +    
    +    val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
    +    
    +    //need to identify time between others order fields. Time needs to be first sort element
    +    val timeType = SortUtil.getTimeType(sortCollation, schema.logicalType)
    +    
    +    //time ordering needs to be ascending
    +    if (SortUtil.getTimeDirection(sortCollation) != Direction.ASCENDING) {
    +      throw new TableException("SQL/Table supports only ascending time ordering")
    +    }
    +    
    +    val execCfg = tableEnv.execEnv.getConfig
    +      
    +    //enable to extend for other types of aggregates that will not be implemented in a window
    +    timeType match {
    +        case _ if FlinkTypeFactory.isProctimeIndicatorType(timeType)  =>
    +            (sortOffset,sortFetch) match {
    +              case (o: Any, f: Any)  => // offset and fetch needs retraction
    +                throw new TableException("SQL/Table does not support sort with offset and fetch") 
    +              case (_, f: Any) => // offset needs retraction
    +                throw new TableException("SQL/Table does not support sort with fetch")
    +              case (o: Any, _) =>  // fetch needs retraction
    +                throw new TableException("SQL/Table does not support sort with offset")
    +              case _ => createSortProcTime(inputDS, execCfg)  //sort can be done without retraction
    +            }
    +        case _ if FlinkTypeFactory.isRowtimeIndicatorType(timeType) =>
    +            (sortOffset,sortFetch) match {
    +              case (o: Any, f: Any)  => // offset and fetch needs retraction
    +                throw new TableException("SQL/Table does not support sort with offset and fetch") 
    +              case (_, f: Any) => // offset needs retraction
    +                throw new TableException("SQL/Table does not support sort with fetch")
    +              case (o: Any, _) =>  // fetch needs retraction
    +                throw new TableException("SQL/Table does not support sort with offset")
    +              case _ => createSortRowTime(inputDS, execCfg)  //sort can be done without retraction
    +            }
    +        case _ =>
    +          throw new TableException("SQL/Table needs to have sort on time as first sort element")
    +    }
    +    
    +  }
    +
    +  /**
    +   * Create Sort logic based on processing time
    +   */
    +  def createSortProcTime(
    +    inputDS: DataStream[CRow],
    +    execCfg: ExecutionConfig): DataStream[CRow] = {
    +
    +   val returnTypeInfo = CRowTypeInfo(schema.physicalTypeInfo)
    +    
    +    //if the order has secondary sorting fields in addition to the proctime
    +    if( SortUtil.getSortFieldIndexList(sortCollation).size > 1) {
    +    
    +      val processFunction = SortUtil.createProcTimeSortFunction(sortCollation,
    +           inputSchema.logicalType, inputSchema.physicalTypeInfo, execCfg)
    +      
    +      inputDS
    +            .keyBy(new NullByteKeySelector[CRow])
    --- End diff --
    
    indent by two spaces


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

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

Posted by fhueske <gi...@git.apache.org>.
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,
    +        keyIndexesNoTime, fieldCompsRefs, booleanDirectionsNoTime)
    +    val collectionRowComparator = new CollectionRowComparator(rowComp)
    +    
    +    val inputCRowType = CRowTypeInfo(inputTypeInfo)
    + 
    +    new RowTimeSortProcessFunction(
    +      inputType.getFieldCount,
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +  
    +  
    +  /**
    +   * Function creates [org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
    +   * elements based on proctime 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 createProcTimeSortFunction(
    +    collationSort: RelCollation,
    +    inputType: RelDataType,
    +    inputTypeInfo: TypeInformation[Row],
    +    execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
    +
    +    val keySortFields = getSortFieldIndexList(collationSort)
    +    val keySortDirections = getSortFieldDirectionList(collationSort)
    +
    +    
    +       //drop time from comparison
    +    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 ProcTimeSortProcessFunction(
    +      inputType.getFieldCount,
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +
    +  
    +   /**
    +   * Function creates comparison objects based on the field types
    +   * @param inputType input row type
    +   * @param keyIndex the indexes of the fields on which the sorting is done. 
    +   * @param orderDirection the directions of each sort field. 
    +   * @param execConfig the configuration environment
    +   * @return Array of TypeComparator objects
    +   */
    +  def createFieldComparators(
    +      inputType: RelDataType,
    +      keyIndex: Array[Int],
    +      orderDirection: Array[Direction],
    +      execConfig: ExecutionConfig): Array[TypeComparator[_]] = {
    +    
    +    var iOrder = 0
    +    for (i <- keyIndex) yield {
    +
    +      val order = if (orderDirection(iOrder) == Direction.ASCENDING) true else false
    +      iOrder += 1
    +      val fieldTypeInfo = FlinkTypeFactory.toTypeInfo(inputType.getFieldList.get(i).getType)
    +      fieldTypeInfo match {
    +        case a: AtomicType[_] => a.createComparator(order, execConfig)
    +        case _ => throw new TableException(s"Unsupported field type $fieldTypeInfo to sort on.")
    +      }
    +    }
    +  }
    +  
    +   /**
    +   * Function creates a RowComparator based on the typed comparators
    +   * @param inputRowType input row type
    +   * @param fieldIdxs the indexes of the fields on which the sorting is done. 
    +   * @param fieldComps the array of typed comparators
    +   * @param fieldOrders the directions of each sort field (true = ASC). 
    +   * @return A Row TypeComparator object 
    +   */
    +  def createRowComparator(
    +    inputRowType: RelDataType,
    +    fieldIdxs: Array[Int],
    +    fieldComps: Array[TypeComparator[AnyRef]],
    +    fieldOrders: Array[Boolean]): TypeComparator[Row] = {
    +
    +  val rowComp = new RowComparator(
    +    inputRowType.getFieldCount,
    +    fieldIdxs,
    +    fieldComps,
    +    new Array[TypeSerializer[AnyRef]](0), //used only for serialized comparisons
    +    fieldOrders)
    +
    +  rowComp
    +}
    +  
    + 
    +  /**
    +   * Function returns the array of indexes for the fields on which the sort is done
    --- End diff --
    
    -"Function", i.e. "Returns the array of indexes ..."


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

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

Posted by fhueske <gi...@git.apache.org>.
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?


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123757465
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SortTest.scala ---
    @@ -0,0 +1,87 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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"),
    +          term("offset", "null"), 
    +          term("fetch","unlimited")),
    +        term("select", "a", "TIME_MATERIALIZATION(proctime) AS proctime", "c"))
    --- End diff --
    
    shouldn't this be just `a` since we do `SELECT a FROM ...` and not `SELECT a, proctime, c FROM ...`?


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r118237512
  
    --- 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 first sort element
    +    checkTimeOrder()
    +    
    +    super.explainTerms(pw)
    +      .item("orderBy", SortUtil.getSortFieldToString(sortCollation, schema.logicalType))
    +      .item("offset", SortUtil.getOffsetToString(sortOffset))
    +      .item("fetch", SortUtil.getFetchToString(sortFetch, sortOffset))
    +  }
    +
    +  override def translateToPlan(
    +      tableEnv: StreamTableEnvironment,
    +      queryConfig: StreamQueryConfig): DataStream[CRow] = {
    +    
    +    val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
    +    
    +    //need to identify time between others order fields. Time needs to be first sort element
    +    val timeType = SortUtil.getTimeType(sortCollation, schema.logicalType)
    +    
    +    //time ordering needs to be ascending
    +    if (SortUtil.getTimeDirection(sortCollation) != Direction.ASCENDING) {
    +      throw new TableException("SQL/Table supports only ascending time ordering")
    +    }
    +    
    +    val execCfg = tableEnv.execEnv.getConfig
    +      
    +    //enable to extend for other types of aggregates that will not be implemented in a window
    +    timeType match {
    +        case _ if FlinkTypeFactory.isProctimeIndicatorType(timeType)  =>
    +            (sortOffset,sortFetch) match {
    +              case (o: Any, f: Any)  => // offset and fetch needs retraction
    +                throw new TableException("SQL/Table does not support sort with offset and fetch") 
    +              case (_, f: Any) => // offset needs retraction
    +                throw new TableException("SQL/Table does not support sort with fetch")
    +              case (o: Any, _) =>  // fetch needs retraction
    +                throw new TableException("SQL/Table does not support sort with offset")
    +              case _ => createSortProcTime(inputDS, execCfg)  //sort can be done without retraction
    +            }
    +        case _ if FlinkTypeFactory.isRowtimeIndicatorType(timeType) =>
    +            (sortOffset,sortFetch) match {
    +              case (o: Any, f: Any)  => // offset and fetch needs retraction
    +                throw new TableException("SQL/Table does not support sort with offset and fetch") 
    +              case (_, f: Any) => // offset needs retraction
    +                throw new TableException("SQL/Table does not support sort with fetch")
    +              case (o: Any, _) =>  // fetch needs retraction
    +                throw new TableException("SQL/Table does not support sort with offset")
    +              case _ => createSortRowTime(inputDS, execCfg)  //sort can be done without retraction
    +            }
    +        case _ =>
    +          throw new TableException("SQL/Table needs to have sort on time as first sort element")
    +    }
    +    
    +  }
    +
    +  /**
    +   * Create Sort logic based on processing time
    +   */
    +  def createSortProcTime(
    +    inputDS: DataStream[CRow],
    +    execCfg: ExecutionConfig): DataStream[CRow] = {
    +
    +   val returnTypeInfo = CRowTypeInfo(schema.physicalTypeInfo)
    +    
    +    //if the order has secondary sorting fields in addition to the proctime
    +    if( SortUtil.getSortFieldIndexList(sortCollation).size > 1) {
    +    
    +      val processFunction = SortUtil.createProcTimeSortFunction(sortCollation,
    +           inputSchema.logicalType, inputSchema.physicalTypeInfo, execCfg)
    +      
    +      inputDS
    +            .keyBy(new NullByteKeySelector[CRow])
    +            .process(processFunction).setParallelism(1).setMaxParallelism(1)
    +            .returns(returnTypeInfo)
    +            .asInstanceOf[DataStream[CRow]]
    +    } else {
    +      //if the order is done only on proctime we only need to forward the elements
    +        inputDS.keyBy(new NullByteKeySelector[CRow])
    +          .map(new IdentityRowMap())
    +          .setParallelism(1).setMaxParallelism(1)
    +          .returns(returnTypeInfo)
    +          .asInstanceOf[DataStream[CRow]]
    +    }   
    +  }
    +  
    +   /**
    +   * Create Sort logic based on row time
    +   */
    +  def createSortRowTime(
    +    inputDS: DataStream[CRow],
    +    execCfg: ExecutionConfig): DataStream[CRow] = {
    +
    +    val returnTypeInfo = CRowTypeInfo(schema.physicalTypeInfo)
    +       
    +    val processFunction = SortUtil.createRowTimeSortFunction(sortCollation,
    +           inputSchema.logicalType, inputSchema.physicalTypeInfo, execCfg)
    +      
    +    inputDS
    +          .keyBy(new NullByteKeySelector[CRow])
    +          .process(processFunction).setParallelism(1).setMaxParallelism(1)
    +          .returns(returnTypeInfo)
    +          .asInstanceOf[DataStream[CRow]]
    +       
    +  }
    +  
    +  /**
    +   * Function is used to check at verification time if the SQL syntax is supported
    +   */
    +  
    +  def checkTimeOrder() = {
    --- End diff --
    
    agree - done


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

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

Posted by fhueske <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123716582
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamSortRule.scala ---
    @@ -0,0 +1,111 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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
    +import org.apache.flink.table.calcite.FlinkTypeFactory
    +import org.apache.flink.table.api.TableException
    +import org.apache.calcite.rel.RelFieldCollation.Direction
    +import org.apache.flink.table.runtime.aggregate.SortUtil
    +
    +/**
    + * 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 = {
    +
    +    val result = super.matches(call)
    --- End diff --
    
    This call always returns `true`. So we can skip it.


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123757132
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SortTest.scala ---
    @@ -0,0 +1,87 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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"),
    +          term("offset", "null"), 
    --- End diff --
    
    I think we should hide `offset` and `fetch` if they are not defined.


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

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

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

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


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123755789
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeSortProcessFunction.scala ---
    @@ -0,0 +1,146 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.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 inputRowType 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 inputRowType: CRowTypeInfo,
    +  private val rowComparator: CollectionRowComparator)
    +    extends ProcessFunction[CRow, CRow] {
    +
    +  Preconditions.checkNotNull(rowComparator)
    +
    +  // 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)
    +    
    +    if (outputC == null) {
    +      val arity:Integer = inputRowType.getArity
    +      outputC = new CRow(Row.of(arity), true)
    +    }
    +  }
    +
    +  
    +  override def processElement(
    +    inputC: CRow,
    +    ctx: ProcessFunction[CRow, CRow]#Context,
    +    out: Collector[CRow]): Unit = {
    +
    +    val input = inputC.row
    +    
    +    // triggering timestamp for trigger calculation
    +    val rowtime = ctx.timestamp
    +
    +    val lastTriggeringTs = lastTriggeringTsState.value
    +
    +    // check if the data is expired, if not, save the data and register event time timer
    +    if (rowtime > lastTriggeringTs) {
    +      val data = dataState.get(rowtime)
    +      if (null != data) {
    +        data.add(input)
    +        dataState.put(rowtime, data)
    +      } else {
    +        val data = new JArrayList[Row]
    +        data.add(input)
    +        dataState.put(rowtime, data)
    +        // register event time timer
    +        ctx.timerService.registerEventTimeTimer(rowtime)
    +      }
    +    }
    +  }
    +  
    +  
    +  override def onTimer(
    +    timestamp: Long,
    +    ctx: ProcessFunction[CRow, CRow]#OnTimerContext,
    +    out: Collector[CRow]): Unit = {
    +    
    +    // gets all window data from state for the calculation
    +    val inputs: JList[Row] = dataState.get(timestamp)
    +
    +    if (null != inputs) {
    +      
    +      Collections.sort(inputs,rowComparator)
    --- End diff --
    
    +space after `,`


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123753005
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeSortProcessFunction.scala ---
    @@ -0,0 +1,120 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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}
    +import java.lang.{Integer=>JInt}
    +
    +/**
    + * Process Function used for the aggregate in bounded proctime sort without offset/fetch
    + * [[org.apache.flink.streaming.api.datastream.DataStream]]
    + *
    + * @param inputRowType 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 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)
    +    
    +    if (outputC == null) {
    +      val arity:Integer = inputRowType.getArity
    +      outputC = new CRow(Row.of(arity), true)
    +    }
    +    
    +  }
    +
    +  override def processElement(
    +    inputC: CRow,
    +    ctx: ProcessFunction[CRow, CRow]#Context,
    +    out: Collector[CRow]): Unit = {
    +
    +    val input = inputC.row
    +    
    +    val currentTime = ctx.timerService.currentProcessingTime
    +    //buffer the event incoming event
    +  
    +    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 = {
    +    
    +    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 
    --- End diff --
    
    I think the comment can be removed. Collection sort should be fine here.


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123752704
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeSortProcessFunction.scala ---
    @@ -0,0 +1,120 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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}
    +import java.lang.{Integer=>JInt}
    +
    +/**
    + * Process Function used for the aggregate in bounded proctime sort without offset/fetch
    + * [[org.apache.flink.streaming.api.datastream.DataStream]]
    + *
    + * @param inputRowType 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 inputRowType: CRowTypeInfo,
    +  private val rowComparator: CollectionRowComparator)
    +    extends ProcessFunction[CRow, CRow] {
    +
    +  Preconditions.checkNotNull(rowComparator)
    +
    +  private var stateEventsBuffer: ListState[Row] = _
    --- End diff --
    
    rename to `bufferedEvents`?


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

[GitHub] flink issue #3889: [FLINK-6075] - Support Limit/Top(Sort) for Stream SQL

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

    https://github.com/apache/flink/pull/3889
  
    @fhueske  I have addressed the remarks you have done (with 1-2 minor exceptions for which you can see my comments). Please have a look. I think it could be ready to be merged and than I can start the preparation for the offset and fetch


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123749098
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 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}
    +
    +/**
    + * 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)
    +    //drop time from comparison as we sort on time in the states and result emission
    +    val keyIndexesNoTime = keySortFields.slice(1, keySortFields.size)
    +    val booleanOrderings = getSortFieldDirectionBooleanList(collationSort)
    +    val booleanDirectionsNoTime = booleanOrderings.slice(1, booleanOrderings.size)
    +    
    +    val fieldComps = createFieldComparators(
    +      inputType, 
    +      keyIndexesNoTime, 
    +      booleanDirectionsNoTime,
    +      execCfg)
    +    val fieldCompsRefs = fieldComps.asInstanceOf[Array[TypeComparator[AnyRef]]]
    +    
    +    val rowComp = new RowComparator(
    +       inputType.getFieldCount,
    +       keyIndexesNoTime,
    +       fieldCompsRefs,
    +       new Array[TypeSerializer[AnyRef]](0), //used only for object comparisons
    +       booleanDirectionsNoTime)
    +      
    +    val collectionRowComparator = new CollectionRowComparator(rowComp)
    +    
    +    val inputCRowType = CRowTypeInfo(inputTypeInfo)
    + 
    +    new RowTimeSortProcessFunction(
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +  
    +  
    +  /**
    +   * Function creates [org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
    +   * elements based on proctime 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 createProcTimeSortFunction(
    +    collationSort: RelCollation,
    +    inputType: RelDataType,
    +    inputTypeInfo: TypeInformation[Row],
    +    execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
    +
    +    val keySortFields = getSortFieldIndexList(collationSort)
    +    //drop time from comparison
    +    val keyIndexesNoTime = keySortFields.slice(1, keySortFields.size)
    +    val booleanOrderings = getSortFieldDirectionBooleanList(collationSort)
    +    val booleanDirectionsNoTime = booleanOrderings.slice(1, booleanOrderings.size)
    +    
    +    val fieldComps = createFieldComparators(
    +      inputType, 
    +      keyIndexesNoTime, 
    +      booleanDirectionsNoTime,
    +      execCfg)
    +    val fieldCompsRefs = fieldComps.asInstanceOf[Array[TypeComparator[AnyRef]]]
    +    
    +    val rowComp = new RowComparator(
    +       inputType.getFieldCount,
    +       keyIndexesNoTime,
    +       fieldCompsRefs,
    +       new Array[TypeSerializer[AnyRef]](0), //used only for object comparisons
    +       booleanDirectionsNoTime)
    +      
    +    val collectionRowComparator = new CollectionRowComparator(rowComp)
    +    
    +    val inputCRowType = CRowTypeInfo(inputTypeInfo)
    +    
    +    new ProcTimeSortProcessFunction(
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +
    +  
    +   /**
    --- End diff --
    
    -1 space indention (please check the other comment headers 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

Posted by fhueske <gi...@git.apache.org>.
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 to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

Posted by fhueske <gi...@git.apache.org>.
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,
    +        keyIndexesNoTime, fieldCompsRefs, booleanDirectionsNoTime)
    +    val collectionRowComparator = new CollectionRowComparator(rowComp)
    +    
    +    val inputCRowType = CRowTypeInfo(inputTypeInfo)
    + 
    +    new RowTimeSortProcessFunction(
    +      inputType.getFieldCount,
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +  
    +  
    +  /**
    +   * Function creates [org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
    +   * elements based on proctime 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 createProcTimeSortFunction(
    +    collationSort: RelCollation,
    +    inputType: RelDataType,
    +    inputTypeInfo: TypeInformation[Row],
    +    execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
    +
    +    val keySortFields = getSortFieldIndexList(collationSort)
    +    val keySortDirections = getSortFieldDirectionList(collationSort)
    +
    +    
    +       //drop time from comparison
    +    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 ProcTimeSortProcessFunction(
    +      inputType.getFieldCount,
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +
    +  
    +   /**
    +   * Function creates comparison objects based on the field types
    +   * @param inputType input row type
    +   * @param keyIndex the indexes of the fields on which the sorting is done. 
    +   * @param orderDirection the directions of each sort field. 
    +   * @param execConfig the configuration environment
    +   * @return Array of TypeComparator objects
    +   */
    +  def createFieldComparators(
    +      inputType: RelDataType,
    +      keyIndex: Array[Int],
    +      orderDirection: Array[Direction],
    +      execConfig: ExecutionConfig): Array[TypeComparator[_]] = {
    +    
    +    var iOrder = 0
    +    for (i <- keyIndex) yield {
    +
    +      val order = if (orderDirection(iOrder) == Direction.ASCENDING) true else false
    +      iOrder += 1
    +      val fieldTypeInfo = FlinkTypeFactory.toTypeInfo(inputType.getFieldList.get(i).getType)
    +      fieldTypeInfo match {
    +        case a: AtomicType[_] => a.createComparator(order, execConfig)
    +        case _ => throw new TableException(s"Unsupported field type $fieldTypeInfo to sort on.")
    +      }
    +    }
    +  }
    +  
    +   /**
    +   * Function creates a RowComparator based on the typed comparators
    +   * @param inputRowType input row type
    +   * @param fieldIdxs the indexes of the fields on which the sorting is done. 
    +   * @param fieldComps the array of typed comparators
    +   * @param fieldOrders the directions of each sort field (true = ASC). 
    +   * @return A Row TypeComparator object 
    +   */
    +  def createRowComparator(
    +    inputRowType: RelDataType,
    +    fieldIdxs: Array[Int],
    +    fieldComps: Array[TypeComparator[AnyRef]],
    +    fieldOrders: Array[Boolean]): TypeComparator[Row] = {
    +
    +  val rowComp = new RowComparator(
    +    inputRowType.getFieldCount,
    +    fieldIdxs,
    +    fieldComps,
    +    new Array[TypeSerializer[AnyRef]](0), //used only for serialized comparisons
    +    fieldOrders)
    +
    +  rowComp
    +}
    +  
    + 
    +  /**
    +   * Function returns the array of indexes for the fields on which the sort is done
    +   * @param collationSort The Sort collation list
    +   * @return [Array[Int]]
    +   */
    +  def getSortFieldIndexList(collationSort: RelCollation): Array[Int] = {
    +    val keyFields = collationSort.getFieldCollations.toArray()
    +    for (f <- keyFields) yield f.asInstanceOf[RelFieldCollation].getFieldIndex
    +  }
    +  
    +   /**
    +   * Function returns the array of sort direction for each of the sort fields 
    +   * @param collationSort The Sort collation list
    +   * @return [Array[Direction]]
    +   */
    +  def getSortFieldDirectionList(collationSort: RelCollation): Array[Direction] = {
    +    val keyFields = collationSort.getFieldCollations.toArray()
    +    for(f <- keyFields) yield f.asInstanceOf[RelFieldCollation].getDirection
    +  }
    +
    +   /**
    +   * Function returns the array of sort direction for each of the sort fields 
    +   * @param collationSort The Sort collation list
    +   * @return [Array[Direction]]
    +   */
    +  def getSortFieldDirectionBooleanList(collationSort: RelCollation): Array[Boolean] = {
    +    val keyFields = collationSort.getFieldCollations
    +    var i = 0
    +    val keySortDirection = new Array[Boolean](keyFields.size())
    +    while (i < keyFields.size()) {
    +      keySortDirection(i) = 
    +        if (getDirection(collationSort,i) == Direction.ASCENDING) true else false 
    +      i += 1
    +    }
    +    keySortDirection
    +  }
    +  
    +  
    +   /**
    +   * Function returns the direction type of the time in order clause. 
    +   * @param collationSort The Sort collation list of objects
    +   * @return [org.apache.calcite.rel.RelFieldCollation.Direction]
    +   */
    +  def getTimeDirection(collationSort: RelCollation): Direction = {
    +    collationSort.getFieldCollations.get(0).direction
    +  }
    +  
    +   /**
    +   * Function returns the time type in order clause. 
    +   * Expectation is that it is the primary sort field
    +   * @param collationSort The Sort collation list
    +   * @param rowType The data type of the input
    +   * @return [org.apache.calcite.rel.type.RelDataType]
    +   */
    +  def getTimeType(collationSort: RelCollation, rowType: RelDataType): RelDataType = {
    +
    +    //need to identify time between others ordering fields
    +    val ind = collationSort.getFieldCollations.get(0).getFieldIndex
    +    rowType.getFieldList.get(ind).getValue
    +  }
    +
    +   /**
    +   * Function returns the direction type of a field in order clause. 
    +   * @param collationSort The Sort collation list
    +   * @return [org.apache.calcite.rel.RelFieldCollation.Direction]
    +   */
    +  def getDirection(collationSort: RelCollation, sortField:Int): Direction = {
    +    
    +    collationSort.getFieldCollations.get(sortField).direction match {
    +      case Direction.ASCENDING => Direction.ASCENDING
    +      case Direction.DESCENDING => Direction.DESCENDING
    +      case _ =>  throw new TableException("SQL/Table does not support such sorting")
    +    }
    +    
    +  }
    +  
    +  def directionToOrder(direction: Direction) = {
    --- End diff --
    
    methods which are only used internally should be `private`


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r117146377
  
    --- 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 calculation
    +    val triggeringTs = ctx.timestamp
    +
    +    val lastTriggeringTs = lastTriggeringTsState.value
    +
    +    // check if the data is expired, if not, save the data and register event time timer
    +    if (triggeringTs > lastTriggeringTs) {
    +      val data = dataState.get(triggeringTs)
    +      if (null != data) {
    +        data.add(input)
    +        dataState.put(triggeringTs, data)
    +      } else {
    +        val data = new JArrayList[Row]
    +        data.add(input)
    +        dataState.put(triggeringTs, data)
    +        // register event time timer
    +        ctx.timerService.registerEventTimeTimer(triggeringTs)
    +      }
    +    }
    +  }
    +  
    +  
    +  override def onTimer(
    +    timestamp: Long,
    +    ctx: ProcessFunction[CRow, CRow]#OnTimerContext,
    +    out: Collector[CRow]): Unit = {
    +    
    +    // gets all window data from state for the calculation
    +    val inputs: JList[Row] = dataState.get(timestamp)
    +
    +    if (null != inputs) {
    +      
    +      var dataListIndex = 0
    +
    +      // no retraction needed for time order sort
    +      
    +      //no selection of offset/fetch
    +      
    +      dataListIndex = 0
    +      sortArray.clear()
    --- End diff --
    
    Thanks for the suggestion. I agree it is not desirable to copy the rows. However, i am not sure how we can access directly the returned list...as ListState give an iterator/iterator. Using this perhaps the best thing is to use insertion sort (i would manually implement this) as we iterate over the elements...but without significant performance gains.
    Alternatively i could use ValueState[List[Row]] which would indeed give me access to the whole list at once 


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

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

Posted by fhueske <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

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

    https://github.com/apache/flink/pull/3889#discussion_r125006376
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala ---
    @@ -92,8 +92,19 @@ class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttle {
       override def visit(minus: LogicalMinus): RelNode =
         throw new TableException("Logical minus in a stream environment is not supported yet.")
     
    -  override def visit(sort: LogicalSort): RelNode =
    -    throw new TableException("Logical sort in a stream environment is not supported yet.")
    +  override def visit(sort: LogicalSort): RelNode = {
    +
    +    val input = sort.getInput.accept(this)
    +    
    +    val materializer = new RexTimeIndicatorMaterializer(
    +      rexBuilder,
    +      input.getRowType.getFieldList.map(_.getType))
    +   
    +    //val offset = if(sort.offset != null) sort.offset.accept(materializer) else null
    --- End diff --
    
    Should be removed


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

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

Posted by fhueske <gi...@git.apache.org>.
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 calculation
    +    val triggeringTs = ctx.timestamp
    +
    +    val lastTriggeringTs = lastTriggeringTsState.value
    +
    +    // check if the data is expired, if not, save the data and register event time timer
    +    if (triggeringTs > lastTriggeringTs) {
    +      val data = dataState.get(triggeringTs)
    +      if (null != data) {
    +        data.add(input)
    +        dataState.put(triggeringTs, data)
    +      } else {
    +        val data = new JArrayList[Row]
    +        data.add(input)
    +        dataState.put(triggeringTs, data)
    +        // register event time timer
    +        ctx.timerService.registerEventTimeTimer(triggeringTs)
    +      }
    +    }
    +  }
    +  
    +  
    +  override def onTimer(
    +    timestamp: Long,
    +    ctx: ProcessFunction[CRow, CRow]#OnTimerContext,
    +    out: Collector[CRow]): Unit = {
    +    
    +    // gets all window data from state for the calculation
    +    val inputs: JList[Row] = dataState.get(timestamp)
    +
    +    if (null != inputs) {
    +      
    +      var dataListIndex = 0
    +
    +      // no retraction needed for time order sort
    +      
    +      //no selection of offset/fetch
    +      
    +      dataListIndex = 0
    +      sortArray.clear()
    --- End diff --
    
    I don't think we need to copy the rows into a sort array. We could just sort the List returned from the state.


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

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

Posted by fhueske <gi...@git.apache.org>.
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 as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123745861
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 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}
    +
    +/**
    + * 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)
    +    //drop time from comparison as we sort on time in the states and result emission
    +    val keyIndexesNoTime = keySortFields.slice(1, keySortFields.size)
    +    val booleanOrderings = getSortFieldDirectionBooleanList(collationSort)
    +    val booleanDirectionsNoTime = booleanOrderings.slice(1, booleanOrderings.size)
    +    
    +    val fieldComps = createFieldComparators(
    +      inputType, 
    +      keyIndexesNoTime, 
    +      booleanDirectionsNoTime,
    +      execCfg)
    +    val fieldCompsRefs = fieldComps.asInstanceOf[Array[TypeComparator[AnyRef]]]
    +    
    +    val rowComp = new RowComparator(
    +       inputType.getFieldCount,
    +       keyIndexesNoTime,
    +       fieldCompsRefs,
    +       new Array[TypeSerializer[AnyRef]](0), //used only for object comparisons
    +       booleanDirectionsNoTime)
    +      
    +    val collectionRowComparator = new CollectionRowComparator(rowComp)
    +    
    +    val inputCRowType = CRowTypeInfo(inputTypeInfo)
    + 
    +    new RowTimeSortProcessFunction(
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +  
    +  
    +  /**
    +   * Function creates [org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
    +   * elements based on proctime 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 createProcTimeSortFunction(
    +    collationSort: RelCollation,
    +    inputType: RelDataType,
    +    inputTypeInfo: TypeInformation[Row],
    +    execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
    +
    +    val keySortFields = getSortFieldIndexList(collationSort)
    --- End diff --
    
    Move the logic to create the `collectionRowComparator` to a method and use this to create the processing time and event time sort functions?


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123745122
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 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}
    +
    +/**
    + * 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)
    +    //drop time from comparison as we sort on time in the states and result emission
    +    val keyIndexesNoTime = keySortFields.slice(1, keySortFields.size)
    +    val booleanOrderings = getSortFieldDirectionBooleanList(collationSort)
    +    val booleanDirectionsNoTime = booleanOrderings.slice(1, booleanOrderings.size)
    +    
    +    val fieldComps = createFieldComparators(
    +      inputType, 
    +      keyIndexesNoTime, 
    +      booleanDirectionsNoTime,
    +      execCfg)
    +    val fieldCompsRefs = fieldComps.asInstanceOf[Array[TypeComparator[AnyRef]]]
    +    
    +    val rowComp = new RowComparator(
    +       inputType.getFieldCount,
    --- End diff --
    
    This gives the field count of the logical row type, but we need the number of physical fields (`new RowSchema(inputType).physicalArity`)


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r118459358
  
    --- 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 calculation
    +    val triggeringTs = ctx.timestamp
    +
    +    val lastTriggeringTs = lastTriggeringTsState.value
    +
    +    // check if the data is expired, if not, save the data and register event time timer
    +    if (triggeringTs > lastTriggeringTs) {
    +      val data = dataState.get(triggeringTs)
    +      if (null != data) {
    +        data.add(input)
    +        dataState.put(triggeringTs, data)
    +      } else {
    +        val data = new JArrayList[Row]
    +        data.add(input)
    +        dataState.put(triggeringTs, data)
    +        // register event time timer
    +        ctx.timerService.registerEventTimeTimer(triggeringTs)
    +      }
    +    }
    +  }
    +  
    +  
    +  override def onTimer(
    +    timestamp: Long,
    +    ctx: ProcessFunction[CRow, CRow]#OnTimerContext,
    +    out: Collector[CRow]): Unit = {
    +    
    +    // gets all window data from state for the calculation
    +    val inputs: JList[Row] = dataState.get(timestamp)
    +
    +    if (null != inputs) {
    +      
    +      var dataListIndex = 0
    +
    +      // no retraction needed for time order sort
    +      
    +      //no selection of offset/fetch
    +      
    +      dataListIndex = 0
    +      sortArray.clear()
    --- End diff --
    
    ok


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r125058944
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/TimeSortProcessFunctionTest.scala ---
    @@ -0,0 +1,256 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.RowTimeSortProcessFunction
    +import org.apache.flink.table.runtime.aggregate.TimeSortProcessFunctionTest._
    +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}
    +import org.apache.flink.streaming.api.TimeCharacteristic
    +
    +class TimeSortProcessFunctionTest{
    +
    +  
    +  @Test
    +  def testSortProcTimeHarnessPartitioned(): Unit = {
    +    
    +    val rT =  new RowTypeInfo(Array[TypeInformation[_]](
    +      INT_TYPE_INFO,
    +      LONG_TYPE_INFO,
    +      INT_TYPE_INFO,
    +      STRING_TYPE_INFO,
    +      LONG_TYPE_INFO),
    +      Array("a","b","c","d","e"))
    +    
    +    val rTA =  new RowTypeInfo(Array[TypeInformation[_]](
    +     LONG_TYPE_INFO), Array("count"))
    +    val indexes = Array(1,2)
    +      
    +    val fieldComps = Array[TypeComparator[AnyRef]](
    +      LONG_TYPE_INFO.createComparator(true, null).asInstanceOf[TypeComparator[AnyRef]],
    +      INT_TYPE_INFO.createComparator(false, null).asInstanceOf[TypeComparator[AnyRef]] )
    +    val booleanOrders = Array(true, false)    
    +    
    +
    +    val rowComp = new RowComparator(
    +      rT.getTotalFields,
    +      indexes,
    +      fieldComps,
    +      new Array[TypeSerializer[AnyRef]](0), //used only for serialized comparisons
    +      booleanOrders)
    +    
    +    val collectionRowComparator = new CollectionRowComparator(rowComp)
    +    
    +    val inputCRowType = CRowTypeInfo(rT)
    +    
    +    val processFunction = new KeyedProcessOperator[Integer,CRow,CRow](
    +      new ProcTimeSortProcessFunction(
    +        inputCRowType,
    +        collectionRowComparator))
    +  
    +   val testHarness = new KeyedOneInputStreamOperatorTestHarness[Integer,CRow,CRow](
    +      processFunction, 
    +      new TupleRowSelector(0), 
    +      BasicTypeInfo.INT_TYPE_INFO)
    +    
    +   testHarness.open();
    +
    +   testHarness.setProcessingTime(3)
    +
    +      // timestamp is ignored in processing time
    +    testHarness.processElement(new StreamRecord(new CRow(
    +      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 11L: JLong), true), 1001))
    +    testHarness.processElement(new StreamRecord(new CRow(
    +        Row.of(1: JInt, 12L: JLong, 1: JInt, "aaa", 11L: JLong), true), 2002))
    +    testHarness.processElement(new StreamRecord(new CRow(
    +        Row.of(1: JInt, 12L: JLong, 2: JInt, "aaa", 11L: JLong), true), 2003))
    +    testHarness.processElement(new StreamRecord(new CRow(
    +        Row.of(1: JInt, 12L: JLong, 0: JInt, "aaa", 11L: JLong), true), 2004))
    +    testHarness.processElement(new StreamRecord(new CRow(
    +        Row.of(1: JInt, 10L: JLong, 0: JInt, "aaa", 11L: JLong), true), 2006))
    +
    +    //move the timestamp to ensure the execution
    +    testHarness.setProcessingTime(1005)
    --- End diff --
    
    I wanted to trigger a sort a second time by adding more data and setting the processing time another time. Basically, just what you did. Same would have been good for the rowtime 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

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

    https://github.com/apache/flink/pull/3889#discussion_r118475890
  
    --- 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 --
    
    yeah - i was trying to make it match with respect to the number of parenthesis with the incoming value. The problem was that i was messing up with the string and no matter how many i pair i was adding it was replacing and adding some additional ones at the begin and end. The actual problem was that order by / offset and fetch needed to be standalone terms


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r124573957
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SortTest.scala ---
    @@ -0,0 +1,87 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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"),
    +          term("offset", "null"), 
    +          term("fetch","unlimited")),
    +        term("select", "a", "TIME_MATERIALIZATION(proctime) AS proctime", "c"))
    --- End diff --
    
    yes and no :)
    Perhaps you remember that in the previous PR I pointed this issue out. Calcite as it creates the execution plan now will have as the output type of the sort a tuple for each of the 3 fields (a, proctime, and c)...because one comes from the output and 2 are used internally. The sort does not do any projection so all the 3 fields needed in the input are carried in the output - hence the type of the output. I have asked you at that point if you believe we should drop the fields by ourselves - which you said no and I agree with that. This test is only the reflection of that case - basically the whole plan of calcite should be modified....


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r124489316
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SortITCase.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.api.scala.stream.sql
    +
    +import org.apache.flink.api.scala._
    +import org.apache.flink.table.api.scala.stream.sql.SortITCase.StringRowSelectorSink
    +import org.apache.flink.table.api.scala.stream.sql.TimeTestUtil.EventTimeSourceFunction
    +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"))),
    +      Left((2000L, (3L, 1, "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"))),
    +      Left((6000L, (6L, 67, "Hello"))),
    +      Left((6000L, (6L, -1, "Hello"))),
    +      Left((6000L, (6L, 6, "Hello"))),
    +      Right(7000L),
    +      Left((9000L, (6L, 9, "Hello"))),
    +      Left((8500L, (6L, 18, "Hello"))),
    +      Left((9000L, (6L, 7, "Hello"))),
    +      Right(10000L),
    +      Left((10000L, (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((20000L, (20L, 20, "Hello World"))), 
    +      Right(19000L))
    +
    +    val env = StreamExecutionEnvironment.getExecutionEnvironment
    +    env.setStateBackend(getStateBackend)
    +    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]
    --- End diff --
    
    In the flink version from where i started it it is not available. I think if i rebase now i will have to create a new PR... I would prefer to keep this and do an update after the merge.


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

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

Posted by fhueske <gi...@git.apache.org>.
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 first sort element
    +    checkTimeOrder()
    +    
    +    super.explainTerms(pw)
    +      .item("orderBy", SortUtil.getSortFieldToString(sortCollation, schema.logicalType))
    +      .item("offset", SortUtil.getOffsetToString(sortOffset))
    +      .item("fetch", SortUtil.getFetchToString(sortFetch, sortOffset))
    +  }
    +
    +  override def translateToPlan(
    +      tableEnv: StreamTableEnvironment,
    +      queryConfig: StreamQueryConfig): DataStream[CRow] = {
    +    
    +    val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
    +    
    +    //need to identify time between others order fields. Time needs to be first sort element
    +    val timeType = SortUtil.getTimeType(sortCollation, schema.logicalType)
    +    
    +    //time ordering needs to be ascending
    +    if (SortUtil.getTimeDirection(sortCollation) != Direction.ASCENDING) {
    --- End diff --
    
    The check for type and order of first field should go into `matches()` method of the `DataStreamSortRule`.


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

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

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

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


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

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

Posted by fhueske <gi...@git.apache.org>.
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 RowTimeSortProcessFunction(
    +      inputType.getFieldCount,
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +  
    +  
    +  /**
    +   * Function creates [org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
    +   * elements based on proctime 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 createProcTimeSortFunction(
    +    collationSort: RelCollation,
    +    inputType: RelDataType,
    +    inputTypeInfo: TypeInformation[Row],
    +    execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
    +
    +    val keySortFields = getSortFieldIndexList(collationSort)
    +    val keySortDirections = getSortFieldDirectionList(collationSort)
    +
    +    
    +       //drop time from comparison
    +    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 ProcTimeSortProcessFunction(
    +      inputType.getFieldCount,
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +
    +  
    +   /**
    +   * Function creates comparison objects based on the field types
    +   * @param inputType input row type
    +   * @param keyIndex the indexes of the fields on which the sorting is done. 
    +   * @param orderDirection the directions of each sort field. 
    +   * @param execConfig the configuration environment
    +   * @return Array of TypeComparator objects
    +   */
    +  def createFieldComparators(
    +      inputType: RelDataType,
    +      keyIndex: Array[Int],
    +      orderDirection: Array[Direction],
    +      execConfig: ExecutionConfig): Array[TypeComparator[_]] = {
    +    
    +    var iOrder = 0
    +    for (i <- keyIndex) yield {
    +
    +      val order = if (orderDirection(iOrder) == Direction.ASCENDING) true else false
    +      iOrder += 1
    +      val fieldTypeInfo = FlinkTypeFactory.toTypeInfo(inputType.getFieldList.get(i).getType)
    +      fieldTypeInfo match {
    +        case a: AtomicType[_] => a.createComparator(order, execConfig)
    +        case _ => throw new TableException(s"Unsupported field type $fieldTypeInfo to sort on.")
    +      }
    +    }
    +  }
    +  
    +   /**
    +   * Function creates a RowComparator based on the typed comparators
    +   * @param inputRowType input row type
    +   * @param fieldIdxs the indexes of the fields on which the sorting is done. 
    +   * @param fieldComps the array of typed comparators
    +   * @param fieldOrders the directions of each sort field (true = ASC). 
    +   * @return A Row TypeComparator object 
    +   */
    +  def createRowComparator(
    +    inputRowType: RelDataType,
    +    fieldIdxs: Array[Int],
    +    fieldComps: Array[TypeComparator[AnyRef]],
    +    fieldOrders: Array[Boolean]): TypeComparator[Row] = {
    +
    +  val rowComp = new RowComparator(
    +    inputRowType.getFieldCount,
    +    fieldIdxs,
    +    fieldComps,
    +    new Array[TypeSerializer[AnyRef]](0), //used only for serialized comparisons
    +    fieldOrders)
    +
    +  rowComp
    +}
    +  
    + 
    +  /**
    +   * Function returns the array of indexes for the fields on which the sort is done
    +   * @param collationSort The Sort collation list
    +   * @return [Array[Int]]
    +   */
    +  def getSortFieldIndexList(collationSort: RelCollation): Array[Int] = {
    +    val keyFields = collationSort.getFieldCollations.toArray()
    +    for (f <- keyFields) yield f.asInstanceOf[RelFieldCollation].getFieldIndex
    +  }
    +  
    +   /**
    +   * Function returns the array of sort direction for each of the sort fields 
    +   * @param collationSort The Sort collation list
    +   * @return [Array[Direction]]
    +   */
    +  def getSortFieldDirectionList(collationSort: RelCollation): Array[Direction] = {
    +    val keyFields = collationSort.getFieldCollations.toArray()
    +    for(f <- keyFields) yield f.asInstanceOf[RelFieldCollation].getDirection
    +  }
    +
    +   /**
    +   * Function returns the array of sort direction for each of the sort fields 
    +   * @param collationSort The Sort collation list
    +   * @return [Array[Direction]]
    +   */
    +  def getSortFieldDirectionBooleanList(collationSort: RelCollation): Array[Boolean] = {
    +    val keyFields = collationSort.getFieldCollations
    +    var i = 0
    +    val keySortDirection = new Array[Boolean](keyFields.size())
    +    while (i < keyFields.size()) {
    +      keySortDirection(i) = 
    +        if (getDirection(collationSort,i) == Direction.ASCENDING) true else false 
    +      i += 1
    +    }
    +    keySortDirection
    +  }
    +  
    +  
    +   /**
    +   * Function returns the direction type of the time in order clause. 
    +   * @param collationSort The Sort collation list of objects
    +   * @return [org.apache.calcite.rel.RelFieldCollation.Direction]
    +   */
    +  def getTimeDirection(collationSort: RelCollation): Direction = {
    +    collationSort.getFieldCollations.get(0).direction
    +  }
    +  
    +   /**
    +   * Function returns the time type in order clause. 
    +   * Expectation is that it is the primary sort field
    +   * @param collationSort The Sort collation list
    +   * @param rowType The data type of the input
    +   * @return [org.apache.calcite.rel.type.RelDataType]
    +   */
    +  def getTimeType(collationSort: RelCollation, rowType: RelDataType): RelDataType = {
    +
    +    //need to identify time between others ordering fields
    +    val ind = collationSort.getFieldCollations.get(0).getFieldIndex
    +    rowType.getFieldList.get(ind).getValue
    +  }
    +
    +   /**
    +   * Function returns the direction type of a field in order clause. 
    +   * @param collationSort The Sort collation list
    +   * @return [org.apache.calcite.rel.RelFieldCollation.Direction]
    +   */
    +  def getDirection(collationSort: RelCollation, sortField:Int): Direction = {
    +    
    +    collationSort.getFieldCollations.get(sortField).direction match {
    +      case Direction.ASCENDING => Direction.ASCENDING
    +      case Direction.DESCENDING => Direction.DESCENDING
    +      case _ =>  throw new TableException("SQL/Table does not support such sorting")
    +    }
    +    
    +  }
    +  
    +  def directionToOrder(direction: Direction) = {
    +    direction match {
    +      case Direction.ASCENDING | Direction.STRICTLY_ASCENDING => Order.ASCENDING
    +      case Direction.DESCENDING | Direction.STRICTLY_DESCENDING => Order.DESCENDING
    +      case _ => throw new IllegalArgumentException("Unsupported direction.")
    +    }
    +  }
    +  
    +  def getSortFieldToString(collationSort: RelCollation, rowRelDataType: RelDataType): String = {
    +    val fieldCollations = collationSort.getFieldCollations.asScala  
    +    .map(c => (c.getFieldIndex, directionToOrder(c.getDirection)))
    +
    +    val sortFieldsToString = fieldCollations
    +      .map(col => s"${
    +        rowRelDataType.getFieldNames.get(col._1)} ${col._2.getShortName}" ).mkString(", ")
    +    
    +    sortFieldsToString
    +  }
    +  
    +  def getOffsetToString(offset: RexNode): String = {
    +    val offsetToString = s"$offset"
    +    offsetToString
    +  }
    +  
    +  def getFetchToString(fetch: RexNode, offset: RexNode): String = {
    +    val limitEnd = getFetchLimitEnd(fetch, offset)
    +    val fetchToString = if (limitEnd == Long.MaxValue) {
    +      "unlimited"
    +    } else {
    +      s"$limitEnd"
    +    }
    +    fetchToString
    +  }
    +  
    +  def getFetchLimitEnd (fetch: RexNode, offset: RexNode): Long = {
    +    val limitEnd: Long = if (fetch != null) {
    +      RexLiteral.intValue(fetch) + getFetchLimitStart(fetch, offset)
    +    } else {
    +      Long.MaxValue
    +    }
    +    limitEnd
    +  }
    +  
    +  def getFetchLimitStart (fetch: RexNode, offset: RexNode): Long = {
    +     val limitStart: Long = if (offset != null) {
    +      RexLiteral.intValue(offset)
    +     } else {
    +       0L
    +     }
    +     limitStart
    +  }
    +  
    +}
    +
    +/**
    + * Wrapper for Row TypeComparator to a Java Comparator object
    + */
    +class CollectionRowComparator(
    +    private val rowComp: TypeComparator[Row]) extends Comparator[Row] with Serializable {
    +  
    +  override def compare(arg0:Row, arg1:Row):Int = {
    +    rowComp.compare(arg0, arg1)
    +  }
    +}
    +
    +
    +
    +/**
    + * Identity map for forwarding the fields based on their arriving times
    + */
    +private[flink] class IdentityRowMap extends MapFunction[CRow,CRow] {
    --- End diff --
    
    rename to `IdentityCRowMap`


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

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

Posted by fhueske <gi...@git.apache.org>.
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`


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

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

Posted by fhueske <gi...@git.apache.org>.
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 RowTimeSortProcessFunction(
    +      inputType.getFieldCount,
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +  
    +  
    +  /**
    +   * Function creates [org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
    +   * elements based on proctime 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 createProcTimeSortFunction(
    +    collationSort: RelCollation,
    +    inputType: RelDataType,
    +    inputTypeInfo: TypeInformation[Row],
    +    execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
    +
    +    val keySortFields = getSortFieldIndexList(collationSort)
    +    val keySortDirections = getSortFieldDirectionList(collationSort)
    +
    +    
    +       //drop time from comparison
    +    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 ProcTimeSortProcessFunction(
    +      inputType.getFieldCount,
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +
    +  
    +   /**
    +   * Function creates comparison objects based on the field types
    +   * @param inputType input row type
    +   * @param keyIndex the indexes of the fields on which the sorting is done. 
    +   * @param orderDirection the directions of each sort field. 
    +   * @param execConfig the configuration environment
    +   * @return Array of TypeComparator objects
    +   */
    +  def createFieldComparators(
    +      inputType: RelDataType,
    +      keyIndex: Array[Int],
    +      orderDirection: Array[Direction],
    +      execConfig: ExecutionConfig): Array[TypeComparator[_]] = {
    +    
    +    var iOrder = 0
    +    for (i <- keyIndex) yield {
    +
    +      val order = if (orderDirection(iOrder) == Direction.ASCENDING) true else false
    +      iOrder += 1
    +      val fieldTypeInfo = FlinkTypeFactory.toTypeInfo(inputType.getFieldList.get(i).getType)
    +      fieldTypeInfo match {
    +        case a: AtomicType[_] => a.createComparator(order, execConfig)
    +        case _ => throw new TableException(s"Unsupported field type $fieldTypeInfo to sort on.")
    +      }
    +    }
    +  }
    +  
    +   /**
    +   * Function creates a RowComparator based on the typed comparators
    +   * @param inputRowType input row type
    +   * @param fieldIdxs the indexes of the fields on which the sorting is done. 
    +   * @param fieldComps the array of typed comparators
    +   * @param fieldOrders the directions of each sort field (true = ASC). 
    +   * @return A Row TypeComparator object 
    +   */
    +  def createRowComparator(
    +    inputRowType: RelDataType,
    +    fieldIdxs: Array[Int],
    +    fieldComps: Array[TypeComparator[AnyRef]],
    +    fieldOrders: Array[Boolean]): TypeComparator[Row] = {
    +
    +  val rowComp = new RowComparator(
    +    inputRowType.getFieldCount,
    +    fieldIdxs,
    +    fieldComps,
    +    new Array[TypeSerializer[AnyRef]](0), //used only for serialized comparisons
    +    fieldOrders)
    +
    +  rowComp
    +}
    +  
    + 
    +  /**
    +   * Function returns the array of indexes for the fields on which the sort is done
    +   * @param collationSort The Sort collation list
    +   * @return [Array[Int]]
    +   */
    +  def getSortFieldIndexList(collationSort: RelCollation): Array[Int] = {
    +    val keyFields = collationSort.getFieldCollations.toArray()
    +    for (f <- keyFields) yield f.asInstanceOf[RelFieldCollation].getFieldIndex
    +  }
    +  
    +   /**
    +   * Function returns the array of sort direction for each of the sort fields 
    +   * @param collationSort The Sort collation list
    +   * @return [Array[Direction]]
    +   */
    +  def getSortFieldDirectionList(collationSort: RelCollation): Array[Direction] = {
    +    val keyFields = collationSort.getFieldCollations.toArray()
    +    for(f <- keyFields) yield f.asInstanceOf[RelFieldCollation].getDirection
    +  }
    +
    +   /**
    +   * Function returns the array of sort direction for each of the sort fields 
    +   * @param collationSort The Sort collation list
    +   * @return [Array[Direction]]
    +   */
    +  def getSortFieldDirectionBooleanList(collationSort: RelCollation): Array[Boolean] = {
    +    val keyFields = collationSort.getFieldCollations
    +    var i = 0
    +    val keySortDirection = new Array[Boolean](keyFields.size())
    +    while (i < keyFields.size()) {
    +      keySortDirection(i) = 
    +        if (getDirection(collationSort,i) == Direction.ASCENDING) true else false 
    +      i += 1
    +    }
    +    keySortDirection
    +  }
    +  
    +  
    +   /**
    +   * Function returns the direction type of the time in order clause. 
    +   * @param collationSort The Sort collation list of objects
    +   * @return [org.apache.calcite.rel.RelFieldCollation.Direction]
    +   */
    +  def getTimeDirection(collationSort: RelCollation): Direction = {
    +    collationSort.getFieldCollations.get(0).direction
    --- End diff --
    
    function calls a single method. could be inlined 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

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

    https://github.com/apache/flink/pull/3889#discussion_r125058559
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/TimeSortProcessFunctionTest.scala ---
    @@ -122,6 +130,11 @@ class TimeSortProcessFunctionTest{
           Row.of(1: JInt, 12L: JLong, 1: JInt, "aaa", 11L: JLong),true), 4))
         expectedOutput.add(new StreamRecord(new CRow(
           Row.of(1: JInt, 12L: JLong, 0: JInt, "aaa", 11L: JLong),true), 4))
    +    
    +    expectedOutput.add(new StreamRecord(new CRow(
    +      Row.of(1: JInt, 1L: JLong, 0: JInt, "aaa", 11L: JLong),true), 1006))
    +    expectedOutput.add(new StreamRecord(new CRow(
    +      Row.of(1: JInt, 2L: JLong, 0: JInt, "aaa", 11L: JLong),true), 1006))
           
         TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.",
    --- End diff --
    
    The test should check that the ProcessFunction emit the rows in the correct order. `assertOutputEqualsSorted` sorts the result and expected data before comparing them. We have to use `assertOutputEquals` instead.


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r118465274
  
    --- 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 --
    
    ok


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r124527971
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSort.scala ---
    @@ -0,0 +1,206 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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
    +import org.apache.flink.table.plan.nodes.CommonSort
    +
    +/**
    +  * Flink RelNode which matches along with Sort Rule.
    +  *
    + */
    +class DataStreamSort(
    +    cluster: RelOptCluster,
    +    traitSet: RelTraitSet,
    +    inputNode: RelNode,
    +    inputSchema: RowSchema,
    +    schema: RowSchema,
    +    sortCollation: RelCollation,
    +    sortOffset: RexNode,
    +    sortFetch: RexNode,
    +    description: String)
    +  extends SingleRel(cluster, traitSet, inputNode)
    +  with CommonSort
    +  with DataStreamRel {
    +
    +  override def deriveRowType(): RelDataType = schema.logicalType
    +
    +  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
    +    new DataStreamSort(
    +      cluster,
    +      traitSet,
    +      inputs.get(0),
    +      inputSchema,
    +      schema,
    +      sortCollation,
    +      sortOffset,
    +      sortFetch,
    +      description)
    +  }
    +
    +  override def toString: String = {
    +    s"Sort(by: ($$sortFieldsToString(sortCollation, schema.logicalType))," +
    +      " offset: $offsetToString(sortOffset)," +
    +      " fetch: $fetchToString(sortFetch, sortOffset))"
    +  }
    +  
    +  override def explainTerms(pw: RelWriter) : RelWriter = {
    +    
    +    super.explainTerms(pw)
    --- End diff --
    
    the implementation indeed can be moved - now I understand what you meant


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r124492156
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonSort.scala ---
    @@ -0,0 +1,91 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.table.plan.nodes
    +
    +import org.apache.calcite.rex.{RexLiteral, RexNode}
    +import org.apache.calcite.rel.RelFieldCollation
    +import org.apache.calcite.rel.RelCollation
    +import org.apache.calcite.rel.RelFieldCollation.Direction
    +import org.apache.calcite.rel.`type`._
    +import scala.collection.JavaConverters._
    +import org.apache.flink.api.common.operators.Order
    +
    +
    +/**
    + * Trait represents a collection of sort methods to manipulate the parameters
    + */
    +
    +trait CommonSort {
    +  
    +  private[flink] def offsetToString(offset: RexNode): String = {
    +    val offsetToString = s"$offset"
    +    offsetToString
    +  }
    +  
    +  
    +  private[flink] def sortFieldsToString(
    --- End diff --
    
    I do not see this behavior in the other classes that have a Common trait (e.g., DataStreamCalc has the explainsTerms in the class not in the CommonCalc)


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

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

Posted by fhueske <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

Posted by fhueske <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

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

    https://github.com/apache/flink/pull/3889#discussion_r118468562
  
    --- 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(10000L),
    +      Left((10000L, (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((20000L, (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 --
    
    ok


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

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

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

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


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r124576040
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/TimeSortProcessFunctionTest.scala ---
    @@ -0,0 +1,256 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.RowTimeSortProcessFunction
    +import org.apache.flink.table.runtime.aggregate.TimeSortProcessFunctionTest._
    +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}
    +import org.apache.flink.streaming.api.TimeCharacteristic
    +
    +class TimeSortProcessFunctionTest{
    +
    +  
    +  @Test
    +  def testSortProcTimeHarnessPartitioned(): Unit = {
    +    
    +    val rT =  new RowTypeInfo(Array[TypeInformation[_]](
    +      INT_TYPE_INFO,
    +      LONG_TYPE_INFO,
    +      INT_TYPE_INFO,
    +      STRING_TYPE_INFO,
    +      LONG_TYPE_INFO),
    +      Array("a","b","c","d","e"))
    +    
    +    val rTA =  new RowTypeInfo(Array[TypeInformation[_]](
    +     LONG_TYPE_INFO), Array("count"))
    +    val indexes = Array(1,2)
    +      
    +    val fieldComps = Array[TypeComparator[AnyRef]](
    +      LONG_TYPE_INFO.createComparator(true, null).asInstanceOf[TypeComparator[AnyRef]],
    +      INT_TYPE_INFO.createComparator(false, null).asInstanceOf[TypeComparator[AnyRef]] )
    +    val booleanOrders = Array(true, false)    
    +    
    +
    +    val rowComp = new RowComparator(
    +      rT.getTotalFields,
    +      indexes,
    +      fieldComps,
    +      new Array[TypeSerializer[AnyRef]](0), //used only for serialized comparisons
    +      booleanOrders)
    +    
    +    val collectionRowComparator = new CollectionRowComparator(rowComp)
    +    
    +    val inputCRowType = CRowTypeInfo(rT)
    +    
    +    val processFunction = new KeyedProcessOperator[Integer,CRow,CRow](
    +      new ProcTimeSortProcessFunction(
    +        inputCRowType,
    +        collectionRowComparator))
    +  
    +   val testHarness = new KeyedOneInputStreamOperatorTestHarness[Integer,CRow,CRow](
    +      processFunction, 
    +      new TupleRowSelector(0), 
    +      BasicTypeInfo.INT_TYPE_INFO)
    +    
    +   testHarness.open();
    +
    +   testHarness.setProcessingTime(3)
    +
    +      // timestamp is ignored in processing time
    +    testHarness.processElement(new StreamRecord(new CRow(
    +      Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 11L: JLong), true), 1001))
    +    testHarness.processElement(new StreamRecord(new CRow(
    +        Row.of(1: JInt, 12L: JLong, 1: JInt, "aaa", 11L: JLong), true), 2002))
    +    testHarness.processElement(new StreamRecord(new CRow(
    +        Row.of(1: JInt, 12L: JLong, 2: JInt, "aaa", 11L: JLong), true), 2003))
    +    testHarness.processElement(new StreamRecord(new CRow(
    +        Row.of(1: JInt, 12L: JLong, 0: JInt, "aaa", 11L: JLong), true), 2004))
    +    testHarness.processElement(new StreamRecord(new CRow(
    +        Row.of(1: JInt, 10L: JLong, 0: JInt, "aaa", 11L: JLong), true), 2006))
    +
    +    //move the timestamp to ensure the execution
    +    testHarness.setProcessingTime(1005)
    --- End diff --
    
    I am not sure I understand - you mean within the same test or in different tests?


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

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

Posted by fhueske <gi...@git.apache.org>.
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 RowTimeSortProcessFunction(
    +      inputType.getFieldCount,
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +  
    +  
    +  /**
    +   * Function creates [org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
    +   * elements based on proctime 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 createProcTimeSortFunction(
    +    collationSort: RelCollation,
    +    inputType: RelDataType,
    +    inputTypeInfo: TypeInformation[Row],
    +    execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
    +
    +    val keySortFields = getSortFieldIndexList(collationSort)
    +    val keySortDirections = getSortFieldDirectionList(collationSort)
    +
    +    
    +       //drop time from comparison
    +    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 ProcTimeSortProcessFunction(
    +      inputType.getFieldCount,
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +
    +  
    +   /**
    +   * Function creates comparison objects based on the field types
    +   * @param inputType input row type
    +   * @param keyIndex the indexes of the fields on which the sorting is done. 
    +   * @param orderDirection the directions of each sort field. 
    +   * @param execConfig the configuration environment
    +   * @return Array of TypeComparator objects
    +   */
    +  def createFieldComparators(
    +      inputType: RelDataType,
    +      keyIndex: Array[Int],
    +      orderDirection: Array[Direction],
    +      execConfig: ExecutionConfig): Array[TypeComparator[_]] = {
    +    
    +    var iOrder = 0
    +    for (i <- keyIndex) yield {
    +
    +      val order = if (orderDirection(iOrder) == Direction.ASCENDING) true else false
    +      iOrder += 1
    +      val fieldTypeInfo = FlinkTypeFactory.toTypeInfo(inputType.getFieldList.get(i).getType)
    +      fieldTypeInfo match {
    +        case a: AtomicType[_] => a.createComparator(order, execConfig)
    +        case _ => throw new TableException(s"Unsupported field type $fieldTypeInfo to sort on.")
    +      }
    +    }
    +  }
    +  
    +   /**
    +   * Function creates a RowComparator based on the typed comparators
    +   * @param inputRowType input row type
    +   * @param fieldIdxs the indexes of the fields on which the sorting is done. 
    +   * @param fieldComps the array of typed comparators
    +   * @param fieldOrders the directions of each sort field (true = ASC). 
    +   * @return A Row TypeComparator object 
    +   */
    +  def createRowComparator(
    +    inputRowType: RelDataType,
    +    fieldIdxs: Array[Int],
    +    fieldComps: Array[TypeComparator[AnyRef]],
    +    fieldOrders: Array[Boolean]): TypeComparator[Row] = {
    +
    +  val rowComp = new RowComparator(
    +    inputRowType.getFieldCount,
    +    fieldIdxs,
    +    fieldComps,
    +    new Array[TypeSerializer[AnyRef]](0), //used only for serialized comparisons
    +    fieldOrders)
    +
    +  rowComp
    +}
    +  
    + 
    +  /**
    +   * Function returns the array of indexes for the fields on which the sort is done
    +   * @param collationSort The Sort collation list
    +   * @return [Array[Int]]
    +   */
    +  def getSortFieldIndexList(collationSort: RelCollation): Array[Int] = {
    +    val keyFields = collationSort.getFieldCollations.toArray()
    +    for (f <- keyFields) yield f.asInstanceOf[RelFieldCollation].getFieldIndex
    +  }
    +  
    +   /**
    +   * Function returns the array of sort direction for each of the sort fields 
    +   * @param collationSort The Sort collation list
    +   * @return [Array[Direction]]
    +   */
    +  def getSortFieldDirectionList(collationSort: RelCollation): Array[Direction] = {
    +    val keyFields = collationSort.getFieldCollations.toArray()
    +    for(f <- keyFields) yield f.asInstanceOf[RelFieldCollation].getDirection
    +  }
    +
    +   /**
    +   * Function returns the array of sort direction for each of the sort fields 
    +   * @param collationSort The Sort collation list
    +   * @return [Array[Direction]]
    +   */
    +  def getSortFieldDirectionBooleanList(collationSort: RelCollation): Array[Boolean] = {
    +    val keyFields = collationSort.getFieldCollations
    +    var i = 0
    +    val keySortDirection = new Array[Boolean](keyFields.size())
    +    while (i < keyFields.size()) {
    +      keySortDirection(i) = 
    +        if (getDirection(collationSort,i) == Direction.ASCENDING) true else false 
    +      i += 1
    +    }
    +    keySortDirection
    +  }
    +  
    +  
    +   /**
    +   * Function returns the direction type of the time in order clause. 
    +   * @param collationSort The Sort collation list of objects
    +   * @return [org.apache.calcite.rel.RelFieldCollation.Direction]
    +   */
    +  def getTimeDirection(collationSort: RelCollation): Direction = {
    +    collationSort.getFieldCollations.get(0).direction
    +  }
    +  
    +   /**
    +   * Function returns the time type in order clause. 
    +   * Expectation is that it is the primary sort field
    +   * @param collationSort The Sort collation list
    +   * @param rowType The data type of the input
    +   * @return [org.apache.calcite.rel.type.RelDataType]
    +   */
    +  def getTimeType(collationSort: RelCollation, rowType: RelDataType): RelDataType = {
    +
    +    //need to identify time between others ordering fields
    +    val ind = collationSort.getFieldCollations.get(0).getFieldIndex
    +    rowType.getFieldList.get(ind).getValue
    +  }
    +
    +   /**
    +   * Function returns the direction type of a field in order clause. 
    +   * @param collationSort The Sort collation list
    +   * @return [org.apache.calcite.rel.RelFieldCollation.Direction]
    +   */
    +  def getDirection(collationSort: RelCollation, sortField:Int): Direction = {
    +    
    +    collationSort.getFieldCollations.get(sortField).direction match {
    +      case Direction.ASCENDING => Direction.ASCENDING
    +      case Direction.DESCENDING => Direction.DESCENDING
    +      case _ =>  throw new TableException("SQL/Table does not support such sorting")
    +    }
    +    
    +  }
    +  
    +  def directionToOrder(direction: Direction) = {
    +    direction match {
    +      case Direction.ASCENDING | Direction.STRICTLY_ASCENDING => Order.ASCENDING
    +      case Direction.DESCENDING | Direction.STRICTLY_DESCENDING => Order.DESCENDING
    +      case _ => throw new IllegalArgumentException("Unsupported direction.")
    +    }
    +  }
    +  
    +  def getSortFieldToString(collationSort: RelCollation, rowRelDataType: RelDataType): String = {
    --- End diff --
    
    The String methods of `DataSetSort` can be moved to a `CommonSort` class and be reused from there. This ensures consistent strings between batch and streaming operators


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

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

Posted by fhueske <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123742438
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 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}
    +
    +/**
    + * 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)
    +    //drop time from comparison as we sort on time in the states and result emission
    +    val keyIndexesNoTime = keySortFields.slice(1, keySortFields.size)
    +    val booleanOrderings = getSortFieldDirectionBooleanList(collationSort)
    +    val booleanDirectionsNoTime = booleanOrderings.slice(1, booleanOrderings.size)
    +    
    +    val fieldComps = createFieldComparators(
    +      inputType, 
    +      keyIndexesNoTime, 
    +      booleanDirectionsNoTime,
    +      execCfg)
    +    val fieldCompsRefs = fieldComps.asInstanceOf[Array[TypeComparator[AnyRef]]]
    +    
    +    val rowComp = new RowComparator(
    +       inputType.getFieldCount,
    +       keyIndexesNoTime,
    +       fieldCompsRefs,
    +       new Array[TypeSerializer[AnyRef]](0), //used only for object comparisons
    +       booleanDirectionsNoTime)
    +      
    +    val collectionRowComparator = new CollectionRowComparator(rowComp)
    +    
    +    val inputCRowType = CRowTypeInfo(inputTypeInfo)
    + 
    +    new RowTimeSortProcessFunction(
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +  
    +  
    +  /**
    +   * Function creates [org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
    +   * elements based on proctime 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 createProcTimeSortFunction(
    +    collationSort: RelCollation,
    +    inputType: RelDataType,
    +    inputTypeInfo: TypeInformation[Row],
    +    execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
    +
    +    val keySortFields = getSortFieldIndexList(collationSort)
    +    //drop time from comparison
    +    val keyIndexesNoTime = keySortFields.slice(1, keySortFields.size)
    +    val booleanOrderings = getSortFieldDirectionBooleanList(collationSort)
    +    val booleanDirectionsNoTime = booleanOrderings.slice(1, booleanOrderings.size)
    +    
    +    val fieldComps = createFieldComparators(
    +      inputType, 
    +      keyIndexesNoTime, 
    +      booleanDirectionsNoTime,
    +      execCfg)
    +    val fieldCompsRefs = fieldComps.asInstanceOf[Array[TypeComparator[AnyRef]]]
    +    
    +    val rowComp = new RowComparator(
    +       inputType.getFieldCount,
    +       keyIndexesNoTime,
    +       fieldCompsRefs,
    +       new Array[TypeSerializer[AnyRef]](0), //used only for object comparisons
    +       booleanDirectionsNoTime)
    +      
    +    val collectionRowComparator = new CollectionRowComparator(rowComp)
    +    
    +    val inputCRowType = CRowTypeInfo(inputTypeInfo)
    +    
    +    new ProcTimeSortProcessFunction(
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +
    +  
    +   /**
    +   * Function creates comparison objects based on the field types
    +   * @param inputType input row type
    +   * @param keyIndex the indexes of the fields on which the sorting is done. 
    +   * @param orderDirection the directions of each sort field. 
    +   * @param execConfig the configuration environment
    +   * @return Array of TypeComparator objects
    +   */
    +  def createFieldComparators(
    +      inputType: RelDataType,
    +      keyIndex: Array[Int],
    +      booleanOrdering: Array[Boolean],
    +      execConfig: ExecutionConfig): Array[TypeComparator[_]] = {
    +    
    +    for ((k, o) <- keyIndex.zip(booleanOrdering)) yield {
    +      FlinkTypeFactory.toTypeInfo(inputType.getFieldList.get(k).getType) match {
    +        case a: AtomicType[_] => a.createComparator(o, execConfig)
    +        case x: TypeInformation[_] =>  
    +          throw new TableException(s"Unsupported field type $x to sort on.")
    +      }
    +    }
    +    
    +  }
    +  
    + 
    +  /**
    +   * Returns the array of indexes for the fields on which the sort is done
    +   * @param collationSort The Sort collation list
    +   * @return [[Array[Int]]] The array containing the indexes of the comparison fields
    +   */
    +  def getSortFieldIndexList(collationSort: RelCollation): Array[Int] = {
    +    val keyFields = collationSort.getFieldCollations.toArray()
    +    for (f <- keyFields) yield f.asInstanceOf[RelFieldCollation].getFieldIndex
    +  }
    +  
    +  
    +   /**
    +   * Function returns the array of sort direction for each of the sort fields 
    +   * @param collationSort The Sort collation list
    +   * @return [Array[Direction]]
    --- End diff --
    
    please describe the return value of the function


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123743300
  
    --- 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 --
    
    Most IDEs support highlighting of unused imports and other common code style issues (invalid parameter documentation in Java/ScalaDocs, etc).


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123712745
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonSort.scala ---
    @@ -0,0 +1,91 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.table.plan.nodes
    +
    +import org.apache.calcite.rex.{RexLiteral, RexNode}
    +import org.apache.calcite.rel.RelFieldCollation
    +import org.apache.calcite.rel.RelCollation
    +import org.apache.calcite.rel.RelFieldCollation.Direction
    +import org.apache.calcite.rel.`type`._
    +import scala.collection.JavaConverters._
    +import org.apache.flink.api.common.operators.Order
    +
    +
    +/**
    + * Trait represents a collection of sort methods to manipulate the parameters
    + */
    +
    +trait CommonSort {
    +  
    +  private[flink] def offsetToString(offset: RexNode): String = {
    +    val offsetToString = s"$offset"
    +    offsetToString
    +  }
    +  
    +  
    +  private[flink] def sortFieldsToString(
    --- End diff --
    
    can be completely private if we move `explainTerms` to `CommonSort`


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r125063072
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SortITCase.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.api.scala.stream.sql
    +
    +import org.apache.flink.api.scala._
    +import org.apache.flink.table.api.scala.stream.sql.SortITCase.StringRowSelectorSink
    +import org.apache.flink.table.api.scala.stream.sql.TimeTestUtil.EventTimeSourceFunction
    +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"))),
    +      Left((2000L, (3L, 1, "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"))),
    +      Left((6000L, (6L, 67, "Hello"))),
    +      Left((6000L, (6L, -1, "Hello"))),
    +      Left((6000L, (6L, 6, "Hello"))),
    +      Right(7000L),
    +      Left((9000L, (6L, 9, "Hello"))),
    +      Left((8500L, (6L, 18, "Hello"))),
    +      Left((9000L, (6L, 7, "Hello"))),
    +      Right(10000L),
    +      Left((10000L, (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((20000L, (20L, 20, "Hello World"))), 
    +      Right(19000L))
    +
    +    val env = StreamExecutionEnvironment.getExecutionEnvironment
    +    env.setStateBackend(getStateBackend)
    +    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]
    --- End diff --
    
    OK, will do that before merging


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123716234
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSort.scala ---
    @@ -0,0 +1,206 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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
    +import org.apache.flink.table.plan.nodes.CommonSort
    +
    +/**
    +  * Flink RelNode which matches along with Sort Rule.
    +  *
    + */
    +class DataStreamSort(
    +    cluster: RelOptCluster,
    +    traitSet: RelTraitSet,
    +    inputNode: RelNode,
    +    inputSchema: RowSchema,
    +    schema: RowSchema,
    +    sortCollation: RelCollation,
    +    sortOffset: RexNode,
    +    sortFetch: RexNode,
    +    description: String)
    +  extends SingleRel(cluster, traitSet, inputNode)
    --- End diff --
    
    Could also extend `Sort` instead of `SingleRel`


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r125040180
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSort.scala ---
    @@ -0,0 +1,212 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.table.plan.nodes.datastream
    +
    +import org.apache.calcite.plan.{ RelOptCluster, RelTraitSet }
    --- End diff --
    
    About half of the imports are unused. Other classes have unused imports 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123717544
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamSortRule.scala ---
    @@ -0,0 +1,111 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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
    +import org.apache.flink.table.calcite.FlinkTypeFactory
    +import org.apache.flink.table.api.TableException
    +import org.apache.calcite.rel.RelFieldCollation.Direction
    +import org.apache.flink.table.runtime.aggregate.SortUtil
    +
    +/**
    + * 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 = {
    +
    +    val result = super.matches(call)
    +    
    +    //need to identify time between others order fields. Time needs to be first sort element
    +    // we can safely convert the object if the match rule succeeded 
    +    if(result) {
    +      val calcSort: FlinkLogicalSort = call.rel(0).asInstanceOf[FlinkLogicalSort]
    +      checkTimeOrder(calcSort)
    +    }
    +    
    +    result
    +  }
    +
    +  override def convert(rel: RelNode): RelNode = {
    +    val calcSort: FlinkLogicalSort = rel.asInstanceOf[FlinkLogicalSort]
    +    val traitSet: RelTraitSet = rel.getTraitSet.replace(FlinkConventions.DATASTREAM)
    +    val convInput: RelNode = RelOptRule.convert(calcSort.getInput(0), FlinkConventions.DATASTREAM)
    +    
    +    val inputRowType = convInput.asInstanceOf[RelSubset].getOriginal.getRowType
    +
    +    new DataStreamSort(
    +      rel.getCluster,
    +      traitSet,
    +      convInput,
    +      new RowSchema(inputRowType),
    +      new RowSchema(rel.getRowType),
    +      calcSort.collation,
    +      calcSort.offset,
    +      calcSort.fetch,
    +      description)
    +    
    +  }
    +  
    +   
    +  /**
    +   * Function is used to check at verification time if the SQL syntax is supported
    +   */
    +  
    +  def checkTimeOrder(calcSort: FlinkLogicalSort) = {
    +    
    +    val rowType = calcSort.getRowType
    +    val sortCollation = calcSort.collation 
    +     //need to identify time between others order fields. Time needs to be first sort element
    +    val timeType = SortUtil.getTimeType(sortCollation, rowType)
    +    //time ordering needs to be ascending
    +    if (SortUtil.getTimeDirection(sortCollation) != Direction.ASCENDING) {
    +      throw new TableException("SQL/Table supports only ascending time ordering")
    --- End diff --
    
    We do not exit the optimizer with exceptions because it prevents the optimizer to find an alternative plan.
    Rather return `false` in `matches()`


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

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

Posted by fhueske <gi...@git.apache.org>.
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(10000L),
    +      Left((10000L, (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((20000L, (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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123712493
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSort.scala ---
    @@ -0,0 +1,206 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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
    +import org.apache.flink.table.plan.nodes.CommonSort
    +
    +/**
    +  * Flink RelNode which matches along with Sort Rule.
    +  *
    + */
    +class DataStreamSort(
    +    cluster: RelOptCluster,
    +    traitSet: RelTraitSet,
    +    inputNode: RelNode,
    +    inputSchema: RowSchema,
    +    schema: RowSchema,
    +    sortCollation: RelCollation,
    +    sortOffset: RexNode,
    +    sortFetch: RexNode,
    +    description: String)
    +  extends SingleRel(cluster, traitSet, inputNode)
    +  with CommonSort
    +  with DataStreamRel {
    +
    +  override def deriveRowType(): RelDataType = schema.logicalType
    +
    +  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
    +    new DataStreamSort(
    +      cluster,
    +      traitSet,
    +      inputs.get(0),
    +      inputSchema,
    +      schema,
    +      sortCollation,
    +      sortOffset,
    +      sortFetch,
    +      description)
    +  }
    +
    +  override def toString: String = {
    +    s"Sort(by: ($$sortFieldsToString(sortCollation, schema.logicalType))," +
    +      " offset: $offsetToString(sortOffset)," +
    +      " fetch: $fetchToString(sortFetch, sortOffset))"
    +  }
    +  
    +  override def explainTerms(pw: RelWriter) : RelWriter = {
    +    
    +    super.explainTerms(pw)
    --- End diff --
    
    Move the implementation of `explainTerms` to `CommonSort`. It should be the same for DataSet and DataStream.


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

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

Posted by fhueske <gi...@git.apache.org>.
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 first sort element
    +    checkTimeOrder()
    +    
    +    super.explainTerms(pw)
    +      .item("orderBy", SortUtil.getSortFieldToString(sortCollation, schema.logicalType))
    +      .item("offset", SortUtil.getOffsetToString(sortOffset))
    +      .item("fetch", SortUtil.getFetchToString(sortFetch, sortOffset))
    +  }
    +
    +  override def translateToPlan(
    +      tableEnv: StreamTableEnvironment,
    +      queryConfig: StreamQueryConfig): DataStream[CRow] = {
    +    
    +    val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
    +    
    +    //need to identify time between others order fields. Time needs to be first sort element
    +    val timeType = SortUtil.getTimeType(sortCollation, schema.logicalType)
    +    
    +    //time ordering needs to be ascending
    +    if (SortUtil.getTimeDirection(sortCollation) != Direction.ASCENDING) {
    +      throw new TableException("SQL/Table supports only ascending time ordering")
    +    }
    +    
    +    val execCfg = tableEnv.execEnv.getConfig
    +      
    +    //enable to extend for other types of aggregates that will not be implemented in a window
    +    timeType match {
    +        case _ if FlinkTypeFactory.isProctimeIndicatorType(timeType)  =>
    +            (sortOffset,sortFetch) match {
    --- End diff --
    
    +space


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123759690
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/TimeSortProcessFunctionTest.scala ---
    @@ -0,0 +1,256 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.RowTimeSortProcessFunction
    +import org.apache.flink.table.runtime.aggregate.TimeSortProcessFunctionTest._
    +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}
    +import org.apache.flink.streaming.api.TimeCharacteristic
    +
    +class TimeSortProcessFunctionTest{
    +
    +  
    +  @Test
    +  def testSortProcTimeHarnessPartitioned(): Unit = {
    +    
    +    val rT =  new RowTypeInfo(Array[TypeInformation[_]](
    +      INT_TYPE_INFO,
    +      LONG_TYPE_INFO,
    +      INT_TYPE_INFO,
    +      STRING_TYPE_INFO,
    +      LONG_TYPE_INFO),
    +      Array("a","b","c","d","e"))
    --- End diff --
    
    space after comma


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123760629
  
    --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/TimeSortProcessFunctionTest.scala ---
    @@ -0,0 +1,256 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.RowTimeSortProcessFunction
    +import org.apache.flink.table.runtime.aggregate.TimeSortProcessFunctionTest._
    +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}
    +import org.apache.flink.streaming.api.TimeCharacteristic
    +
    +class TimeSortProcessFunctionTest{
    --- End diff --
    
    Please add a similar test for the event-time sorter which sorts for multiple timestamps and validates that all rows preserve their timestamp.


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

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

Posted by fhueske <gi...@git.apache.org>.
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 first sort element
    +    checkTimeOrder()
    +    
    +    super.explainTerms(pw)
    +      .item("orderBy", SortUtil.getSortFieldToString(sortCollation, schema.logicalType))
    +      .item("offset", SortUtil.getOffsetToString(sortOffset))
    +      .item("fetch", SortUtil.getFetchToString(sortFetch, sortOffset))
    +  }
    +
    +  override def translateToPlan(
    +      tableEnv: StreamTableEnvironment,
    +      queryConfig: StreamQueryConfig): DataStream[CRow] = {
    +    
    +    val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
    +    
    +    //need to identify time between others order fields. Time needs to be first sort element
    +    val timeType = SortUtil.getTimeType(sortCollation, schema.logicalType)
    +    
    +    //time ordering needs to be ascending
    +    if (SortUtil.getTimeDirection(sortCollation) != Direction.ASCENDING) {
    +      throw new TableException("SQL/Table supports only ascending time ordering")
    +    }
    +    
    +    val execCfg = tableEnv.execEnv.getConfig
    +      
    +    //enable to extend for other types of aggregates that will not be implemented in a window
    +    timeType match {
    +        case _ if FlinkTypeFactory.isProctimeIndicatorType(timeType)  =>
    +            (sortOffset,sortFetch) match {
    +              case (o: Any, f: Any)  => // offset and fetch needs retraction
    +                throw new TableException("SQL/Table does not support sort with offset and fetch") 
    +              case (_, f: Any) => // offset needs retraction
    +                throw new TableException("SQL/Table does not support sort with fetch")
    +              case (o: Any, _) =>  // fetch needs retraction
    +                throw new TableException("SQL/Table does not support sort with offset")
    +              case _ => createSortProcTime(inputDS, execCfg)  //sort can be done without retraction
    +            }
    +        case _ if FlinkTypeFactory.isRowtimeIndicatorType(timeType) =>
    +            (sortOffset,sortFetch) match {
    +              case (o: Any, f: Any)  => // offset and fetch needs retraction
    +                throw new TableException("SQL/Table does not support sort with offset and fetch") 
    +              case (_, f: Any) => // offset needs retraction
    +                throw new TableException("SQL/Table does not support sort with fetch")
    +              case (o: Any, _) =>  // fetch needs retraction
    +                throw new TableException("SQL/Table does not support sort with offset")
    +              case _ => createSortRowTime(inputDS, execCfg)  //sort can be done without retraction
    +            }
    +        case _ =>
    +          throw new TableException("SQL/Table needs to have sort on time as first sort element")
    +    }
    +    
    +  }
    +
    +  /**
    +   * Create Sort logic based on processing time
    +   */
    +  def createSortProcTime(
    +    inputDS: DataStream[CRow],
    +    execCfg: ExecutionConfig): DataStream[CRow] = {
    +
    +   val returnTypeInfo = CRowTypeInfo(schema.physicalTypeInfo)
    +    
    +    //if the order has secondary sorting fields in addition to the proctime
    +    if( SortUtil.getSortFieldIndexList(sortCollation).size > 1) {
    +    
    +      val processFunction = SortUtil.createProcTimeSortFunction(sortCollation,
    +           inputSchema.logicalType, inputSchema.physicalTypeInfo, execCfg)
    +      
    +      inputDS
    +            .keyBy(new NullByteKeySelector[CRow])
    +            .process(processFunction).setParallelism(1).setMaxParallelism(1)
    +            .returns(returnTypeInfo)
    +            .asInstanceOf[DataStream[CRow]]
    +    } else {
    +      //if the order is done only on proctime we only need to forward the elements
    +        inputDS.keyBy(new NullByteKeySelector[CRow])
    +          .map(new IdentityRowMap())
    +          .setParallelism(1).setMaxParallelism(1)
    +          .returns(returnTypeInfo)
    +          .asInstanceOf[DataStream[CRow]]
    +    }   
    +  }
    +  
    +   /**
    +   * Create Sort logic based on row time
    +   */
    +  def createSortRowTime(
    +    inputDS: DataStream[CRow],
    +    execCfg: ExecutionConfig): DataStream[CRow] = {
    +
    +    val returnTypeInfo = CRowTypeInfo(schema.physicalTypeInfo)
    +       
    +    val processFunction = SortUtil.createRowTimeSortFunction(sortCollation,
    +           inputSchema.logicalType, inputSchema.physicalTypeInfo, execCfg)
    +      
    +    inputDS
    +          .keyBy(new NullByteKeySelector[CRow])
    +          .process(processFunction).setParallelism(1).setMaxParallelism(1)
    +          .returns(returnTypeInfo)
    +          .asInstanceOf[DataStream[CRow]]
    +       
    +  }
    +  
    +  /**
    +   * Function is used to check at verification time if the SQL syntax is supported
    +   */
    +  
    +  def checkTimeOrder() = {
    --- End diff --
    
    I think we should move the check for time being the first sorting column and ASCENDING only into the `DataStreamSortRule`. When we later add sorting which is not based on time, we might want to make a distinction in the rule.


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

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

Posted by fhueske <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

Posted by fhueske <gi...@git.apache.org>.
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 RowTimeSortProcessFunction(
    +      inputType.getFieldCount,
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +  
    +  
    +  /**
    +   * Function creates [org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
    +   * elements based on proctime 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 createProcTimeSortFunction(
    +    collationSort: RelCollation,
    +    inputType: RelDataType,
    +    inputTypeInfo: TypeInformation[Row],
    +    execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
    +
    +    val keySortFields = getSortFieldIndexList(collationSort)
    +    val keySortDirections = getSortFieldDirectionList(collationSort)
    +
    +    
    +       //drop time from comparison
    +    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 ProcTimeSortProcessFunction(
    +      inputType.getFieldCount,
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +
    +  
    +   /**
    +   * Function creates comparison objects based on the field types
    +   * @param inputType input row type
    +   * @param keyIndex the indexes of the fields on which the sorting is done. 
    +   * @param orderDirection the directions of each sort field. 
    +   * @param execConfig the configuration environment
    +   * @return Array of TypeComparator objects
    +   */
    +  def createFieldComparators(
    +      inputType: RelDataType,
    +      keyIndex: Array[Int],
    +      orderDirection: Array[Direction],
    +      execConfig: ExecutionConfig): Array[TypeComparator[_]] = {
    +    
    +    var iOrder = 0
    +    for (i <- keyIndex) yield {
    +
    +      val order = if (orderDirection(iOrder) == Direction.ASCENDING) true else false
    +      iOrder += 1
    +      val fieldTypeInfo = FlinkTypeFactory.toTypeInfo(inputType.getFieldList.get(i).getType)
    +      fieldTypeInfo match {
    +        case a: AtomicType[_] => a.createComparator(order, execConfig)
    +        case _ => throw new TableException(s"Unsupported field type $fieldTypeInfo to sort on.")
    +      }
    +    }
    +  }
    +  
    +   /**
    +   * Function creates a RowComparator based on the typed comparators
    +   * @param inputRowType input row type
    +   * @param fieldIdxs the indexes of the fields on which the sorting is done. 
    +   * @param fieldComps the array of typed comparators
    +   * @param fieldOrders the directions of each sort field (true = ASC). 
    +   * @return A Row TypeComparator object 
    +   */
    +  def createRowComparator(
    --- End diff --
    
    This method consists of a single constructor call. I think we do not need to wrap it.


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123742277
  
    --- 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 --
    
    There quite a few: `AggregateFunction`, `SqlTypeName`, `ArrayList`, `JList`, and 10 more or so


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

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

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

    https://github.com/apache/flink/pull/3889#discussion_r118239983
  
    --- 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 --
    
    FlinkTypeFactory is used... am i missing others unused imports to remove?


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

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

Posted by fhueske <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

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

    https://github.com/apache/flink/pull/3889#discussion_r123748629
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala ---
    @@ -0,0 +1,247 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 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}
    +
    +/**
    + * 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)
    +    //drop time from comparison as we sort on time in the states and result emission
    +    val keyIndexesNoTime = keySortFields.slice(1, keySortFields.size)
    +    val booleanOrderings = getSortFieldDirectionBooleanList(collationSort)
    +    val booleanDirectionsNoTime = booleanOrderings.slice(1, booleanOrderings.size)
    +    
    +    val fieldComps = createFieldComparators(
    +      inputType, 
    +      keyIndexesNoTime, 
    +      booleanDirectionsNoTime,
    +      execCfg)
    +    val fieldCompsRefs = fieldComps.asInstanceOf[Array[TypeComparator[AnyRef]]]
    +    
    +    val rowComp = new RowComparator(
    +       inputType.getFieldCount,
    +       keyIndexesNoTime,
    +       fieldCompsRefs,
    +       new Array[TypeSerializer[AnyRef]](0), //used only for object comparisons
    +       booleanDirectionsNoTime)
    +      
    +    val collectionRowComparator = new CollectionRowComparator(rowComp)
    +    
    +    val inputCRowType = CRowTypeInfo(inputTypeInfo)
    + 
    +    new RowTimeSortProcessFunction(
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +  
    +  
    +  /**
    +   * Function creates [org.apache.flink.streaming.api.functions.ProcessFunction] for sorting 
    +   * elements based on proctime 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 createProcTimeSortFunction(
    +    collationSort: RelCollation,
    +    inputType: RelDataType,
    +    inputTypeInfo: TypeInformation[Row],
    +    execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = {
    +
    +    val keySortFields = getSortFieldIndexList(collationSort)
    +    //drop time from comparison
    +    val keyIndexesNoTime = keySortFields.slice(1, keySortFields.size)
    +    val booleanOrderings = getSortFieldDirectionBooleanList(collationSort)
    +    val booleanDirectionsNoTime = booleanOrderings.slice(1, booleanOrderings.size)
    +    
    +    val fieldComps = createFieldComparators(
    +      inputType, 
    +      keyIndexesNoTime, 
    +      booleanDirectionsNoTime,
    +      execCfg)
    +    val fieldCompsRefs = fieldComps.asInstanceOf[Array[TypeComparator[AnyRef]]]
    +    
    +    val rowComp = new RowComparator(
    +       inputType.getFieldCount,
    +       keyIndexesNoTime,
    +       fieldCompsRefs,
    +       new Array[TypeSerializer[AnyRef]](0), //used only for object comparisons
    +       booleanDirectionsNoTime)
    +      
    +    val collectionRowComparator = new CollectionRowComparator(rowComp)
    +    
    +    val inputCRowType = CRowTypeInfo(inputTypeInfo)
    +    
    +    new ProcTimeSortProcessFunction(
    +      inputCRowType,
    +      collectionRowComparator)
    +
    +  }
    +
    +  
    +   /**
    +   * Function creates comparison objects based on the field types
    +   * @param inputType input row type
    +   * @param keyIndex the indexes of the fields on which the sorting is done. 
    +   * @param orderDirection the directions of each sort field. 
    +   * @param execConfig the configuration environment
    +   * @return Array of TypeComparator objects
    +   */
    +  def createFieldComparators(
    --- End diff --
    
    please make functions private that are not accessed from the outside of this class.


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

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

Posted by fhueske <gi...@git.apache.org>.
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).


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