[ 
https://issues.apache.org/jira/browse/BEAM-4798?focusedWorklogId=132881&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-132881
 ]

ASF GitHub Bot logged work on BEAM-4798:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 09/Aug/18 08:37
            Start Date: 09/Aug/18 08:37
    Worklog Time Spent: 10m 
      Work Description: aljoscha commented on a change in pull request #6177: 
[BEAM-4798] Fix IndexOutOfBoundsException in Flink runner
URL: https://github.com/apache/beam/pull/6177#discussion_r208847912
 
 

 ##########
 File path: 
runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TestCountingSource.java
 ##########
 @@ -109,8 +115,13 @@ public int getShardNumber() {
   @Override
   public List<TestCountingSource> split(int desiredNumSplits, PipelineOptions 
options) {
     List<TestCountingSource> splits = new ArrayList<>();
-    int numSplits = allowSplitting ? desiredNumSplits : 1;
-    for (int i = 0; i < numSplits; i++) {
+    int actualNumSplits;
+    if (fixedNumSplits == -1) {
+      actualNumSplits = desiredNumSplits;
+    } else {
+      actualNumSplits = fixedNumSplits;
+    }
 
 Review comment:
   yes, fixing

----------------------------------------------------------------
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:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 132881)
    Time Spent: 3.5h  (was: 3h 20m)

> IndexOutOfBoundsException when Flink parallelism > 1
> ----------------------------------------------------
>
>                 Key: BEAM-4798
>                 URL: https://issues.apache.org/jira/browse/BEAM-4798
>             Project: Beam
>          Issue Type: Bug
>          Components: runner-flink
>    Affects Versions: 2.4.0, 2.5.0
>            Reporter: Alexey Romanenko
>            Assignee: Alexey Romanenko
>            Priority: Major
>          Time Spent: 3.5h
>  Remaining Estimate: 0h
>
> Running job on Flink in streaming mode and get data from a Kafka topic with 
> parallelism > 1 causes an exception:
> {noformat}
> Caused by: java.lang.IndexOutOfBoundsException: Index: 0, Size: 0
>       at java.util.ArrayList.rangeCheck(ArrayList.java:657)
>       at java.util.ArrayList.get(ArrayList.java:433)
>       at 
> org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSourceWrapper.run(UnboundedSourceWrapper.java:277)
>       at 
> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:87)
>       at 
> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:56)
>       at 
> org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:99)
>       at 
> org.apache.flink.streaming.runtime.tasks.StoppableSourceStreamTask.run(StoppableSourceStreamTask.java:45)
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:306)
>       at org.apache.flink.runtime.taskmanager.Task.run(Task.java:703)
>       at java.lang.Thread.run(Thread.java:748)
> {noformat}
> It happens when number of Kafka topic partitions is less than value of 
> parallelism (number of task slots).
> So, workaround for now can be to set parallelism <= number of topic 
> partitions, thus if parallelism=2 then number_partitions >= 2



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

Reply via email to