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

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

                Author: ASF GitHub Bot
            Created on: 10/Aug/18 14:41
            Start Date: 10/Aug/18 14:41
    Worklog Time Spent: 10m 
      Work Description: aljoscha closed pull request #5996: [BEAM-4798] Fix 
IndexOutOfBoundsException in Flink runner
URL: https://github.com/apache/beam/pull/5996
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
 
b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
index 32c8caac96b..606c04050ef 100644
--- 
a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
+++ 
b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
@@ -202,7 +202,12 @@ public void 
run(SourceContext<WindowedValue<ValueWithRecordId<OutputT>>> ctx) th
     ReaderInvocationUtil<OutputT, UnboundedSource.UnboundedReader<OutputT>> 
readerInvoker =
         new ReaderInvocationUtil<>(stepName, serializedOptions.get(), 
metricContainer);
 
-    if (localReaders.size() == 1) {
+    if (localReaders.size() == 0) {
+      // It can happen when value of parallelism is greater than number of IO 
readers (for example,
+      // parallelism is 2 and number of Kafka topic partitions is 1). In this 
case, we just fall
+      // through to idle this executor.
+      LOG.info("Number of readers is 0 for this task executor, idle");
+    } else if (localReaders.size() == 1) {
       // the easy case, we just read from one reader
       UnboundedSource.UnboundedReader<OutputT> reader = localReaders.get(0);
 


 

----------------------------------------------------------------
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: 133618)
    Time Spent: 4h 50m  (was: 4h 40m)

> 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
>             Fix For: 2.7.0
>
>          Time Spent: 4h 50m
>  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