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

LakeShen commented on FLINK-18376:
----------------------------------

Hi [~jark],I find the reason why ArrayIndexOutOfBoundsException in 
RetractableTopNFunction. For example, when use ROW_NUMBER()  function, the sql 
like this :

create view results as select *, ROW_NUMBER() OVER (PARTITION BY name ORDER BY 
cnt DESC) as row_num from test

I will output the data when row_num < 5. Now the user_id is "dema", the 
SortedMap  is : 
9 , 1
8, 3
7, 10.
cnt is 8 , it has three record. when the last record of cnt = 8 becomes 
retract, It will send the first record when  cnt = 7.
But now  curRank(4) + count(10) is 14 , rankEnd - curRank - 1  is -9 , so 
ArrayIndexOutOfBoundsException will be thrown.


> java.lang.ArrayIndexOutOfBoundsException in RetractableTopNFunction
> -------------------------------------------------------------------
>
>                 Key: FLINK-18376
>                 URL: https://issues.apache.org/jira/browse/FLINK-18376
>             Project: Flink
>          Issue Type: Bug
>          Components: Table SQL / Planner
>    Affects Versions: 1.10.0
>            Reporter: LakeShen
>            Priority: Major
>             Fix For: 1.11.1
>
>
> java.lang.ArrayIndexOutOfBoundsException: -1
>       at java.util.ArrayList.elementData(ArrayList.java:422)
>       at java.util.ArrayList.get(ArrayList.java:435)
>       at 
> org.apache.flink.table.runtime.operators.rank.RetractableTopNFunction.retractRecordWithoutRowNumber(RetractableTopNFunction.java:392)
>       at 
> org.apache.flink.table.runtime.operators.rank.RetractableTopNFunction.processElement(RetractableTopNFunction.java:160)
>       at 
> org.apache.flink.table.runtime.operators.rank.RetractableTopNFunction.processElement(RetractableTopNFunction.java:54)
>       at 
> org.apache.flink.streaming.api.operators.KeyedProcessOperator.processElement(KeyedProcessOperator.java:85)
>       at 
> org.apache.flink.streaming.runtime.tasks.OneInputStreamTask$StreamTaskNetworkOutput.emitRecord(OneInputStreamTask.java:173)
>       at 
> org.apache.flink.streaming.runtime.io.StreamTaskNetworkInput.processElement(StreamTaskNetworkInput.java:151)
>       at 
> org.apache.flink.streaming.runtime.io.StreamTaskNetworkInput.emitNext(StreamTaskNetworkInput.java:128)
>       at 
> org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:69)
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:311)
>       at 
> org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:187)
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:487)
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:470)
>       at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:707)
>       at org.apache.flink.runtime.taskmanager.Task.run(Task.java:532)
>       at java.lang.Thread.run(Thread.java:748)



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to