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

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

dianfu opened a new pull request #7284: [FLINK-11136] [table] Fix the merge 
logic of DISTINCT aggregates
URL: https://github.com/apache/flink/pull/7284
 
 
   ## What is the purpose of the change
   
   *This pull request fix the merge logic of DISTINCT aggregates.*
   
   ## Brief change log
   
     - *Fix the codegen logic in AggregationCodeGenerator to extract the 
distinct fields for merge*
   
   ## Verifying this change
   
   This change added tests and can be verified as follows:
   
     - *Updated integration tests 
SqlITCase#testDistinctAggWithMergeOnEventTimeSessionGroupWindow*
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (no)
     - The public API, i.e., is any changed class annotated with 
`@Public(Evolving)`: (no)
     - The serializers: (no)
     - The runtime per-record code paths (performance sensitive): (no)
     - Anything that affects deployment or recovery: JobManager (and its 
components), Checkpointing, Yarn/Mesos, ZooKeeper: (no)
     - The S3 file system connector: (no)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (no)
     - If yes, how is the feature documented? (not applicable)
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


> Fix the logical of merge for DISTINCT aggregates
> ------------------------------------------------
>
>                 Key: FLINK-11136
>                 URL: https://issues.apache.org/jira/browse/FLINK-11136
>             Project: Flink
>          Issue Type: Test
>          Components: Table API & SQL
>            Reporter: Dian Fu
>            Assignee: Dian Fu
>            Priority: Major
>              Labels: pull-request-available
>
> The logic of merge for DISTINCT aggregates has bug. For the following query:
> {code:java}
> SELECT
>   c,
>   COUNT(DISTINCT b),
>   SUM(DISTINCT b),
>   SESSION_END(rowtime, INTERVAL '0.005' SECOND)
> FROM MyTable
> GROUP BY SESSION(rowtime, INTERVAL '0.005' SECOND), c{code}
> the following exception will be thrown:
> {code:java}
> Caused by: java.lang.ClassCastException: org.apache.flink.types.Row cannot be 
> cast to java.lang.Integer
> at scala.runtime.BoxesRunTime.unboxToInt(BoxesRunTime.java:101)
> at scala.math.Numeric$IntIsIntegral$.plus(Numeric.scala:58)
> at 
> org.apache.flink.table.functions.aggfunctions.SumAggFunction.accumulate(SumAggFunction.scala:50)
> at GroupingWindowAggregateHelper$18.mergeAccumulatorsPair(Unknown Source)
> at 
> org.apache.flink.table.runtime.aggregate.AggregateAggFunction.merge(AggregateAggFunction.scala:66)
> at 
> org.apache.flink.table.runtime.aggregate.AggregateAggFunction.merge(AggregateAggFunction.scala:33)
> at 
> org.apache.flink.runtime.state.heap.HeapAggregatingState.mergeState(HeapAggregatingState.java:117)
> at 
> org.apache.flink.runtime.state.heap.AbstractHeapMergingState$MergeTransformation.apply(AbstractHeapMergingState.java:102)
> at 
> org.apache.flink.runtime.state.heap.CopyOnWriteStateTable.transform(CopyOnWriteStateTable.java:463)
> at 
> org.apache.flink.runtime.state.heap.CopyOnWriteStateTable.transform(CopyOnWriteStateTable.java:341)
> at 
> org.apache.flink.runtime.state.heap.AbstractHeapMergingState.mergeNamespaces(AbstractHeapMergingState.java:91)
> at 
> org.apache.flink.streaming.runtime.operators.windowing.WindowOperator$2.merge(WindowOperator.java:341)
> at 
> org.apache.flink.streaming.runtime.operators.windowing.WindowOperator$2.merge(WindowOperator.java:311)
> at 
> org.apache.flink.streaming.runtime.operators.windowing.MergingWindowSet.addWindow(MergingWindowSet.java:212)
> at 
> org.apache.flink.streaming.runtime.operators.windowing.WindowOperator.processElement(WindowOperator.java:311)
> at 
> org.apache.flink.streaming.runtime.io.StreamInputProcessor.processInput(StreamInputProcessor.java:202)
> at 
> org.apache.flink.streaming.runtime.tasks.OneInputStreamTask.run(OneInputStreamTask.java:105)
> at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:300)
> at org.apache.flink.runtime.taskmanager.Task.run(Task.java:704)
> at java.lang.Thread.run(Thread.java:745)
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to