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

    https://github.com/apache/flink/pull/3700#discussion_r110481364
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedSortProcessFunction.scala
 ---
    @@ -0,0 +1,139 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, 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.table.functions.{ Accumulator, AggregateFunction }
    +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.{ ArrayList, LinkedList, List => JList }
    +import org.apache.flink.api.common.typeinfo.BasicTypeInfo
    +import 
org.apache.flink.table.runtime.aggregate.MultiOutputAggregateFunction
    +
    +/**
    + * Process Function used for the aggregate in bounded proc-time OVER window
    + * [[org.apache.flink.streaming.api.datastream.DataStream]]
    + *
    + * @param aggregates the 
[[org.apache.flink.table.functions.aggfunctions.SortAggFunction]]
    + *                   used for this sort aggregation
    + * @param fieldCount Is used to indicate fields in the current element to 
forward
    + * @param aggType It is used to mark the Aggregate type
    + */
    +class ProcTimeUnboundedSortProcessFunction(
    +  private val aggregates: MultiOutputAggregateFunction[_],
    --- End diff --
    
    I don't think the sorting should be done with an aggregation function. An 
aggregation function returns a single value and is IMO not the right tool to 
sort data. 
    
    I would make the design similar to the one you have, but instead of using 
an accumulator, you can use a `ListState` to append new records. When the timer 
fires, you collect all data for the millisecond, and sort with Java's regular 
collections sort with a `Comparator<Row>` that wraps `TypeComparators` for the 
secondary sort fields.


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

Reply via email to