You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@flink.apache.org by Vishal Santoshi <vi...@gmail.com> on 2018/05/12 00:15:07 UTC

A strange exception while consumption using a multi topic Kafka Connector

java.util.ConcurrentModificationException
	at java.util.LinkedList$ListItr.checkForComodification(LinkedList.java:966)
	at java.util.LinkedList$ListItr.next(LinkedList.java:888)
	at org.apache.flink.streaming.connectors.kafka.internal.Kafka09Fetcher.runFetchLoop(Kafka09Fetcher.java:134)

.

.

.



This flink 1.4.0

Any ideas folks ?

Re: A strange exception while consumption using a multi topic Kafka Connector

Posted by Vishal Santoshi <vi...@gmail.com>.
Yep, Thanks. We have a set up where topics are constantly being added on a
kakfa-to-hdfs pipeline.

On Sun, May 13, 2018 at 11:58 AM, Ted Yu <yu...@gmail.com> wrote:

> FLINK-9349 was logged.
>
> FYI
>
> On Sat, May 12, 2018 at 7:52 AM, Ted Yu <yu...@gmail.com> wrote:
>
>> I took a look at ./flink-connectors/flink-co
>> nnector-kafka-0.9/src/main/java/org/apache/flink/streaming/
>> connectors/kafka/internal/Kafka09Fetcher.java
>>
>> It seems the List subscribedPartitionStates was being modified
>> when runFetchLoop iterated the List.
>> This can happen if, e.g., FlinkKafkaConsumer runs the following code
>> concurrently:
>>                 kafkaFetcher.addDiscoveredPart
>> itions(discoveredPartitions);
>>
>> You can log a JIRA.
>> If you have unit test reproducing this, that would be great.
>>
>> FYI
>>
>> On Fri, May 11, 2018 at 5:15 PM, Vishal Santoshi <
>> vishal.santoshi@gmail.com> wrote:
>>
>>> java.util.ConcurrentModificationException
>>> 	at java.util.LinkedList$ListItr.checkForComodification(LinkedList.java:966)
>>> 	at java.util.LinkedList$ListItr.next(LinkedList.java:888)
>>> 	at org.apache.flink.streaming.connectors.kafka.internal.Kafka09Fetcher.runFetchLoop(Kafka09Fetcher.java:134)
>>>
>>> .
>>>
>>> .
>>>
>>> .
>>>
>>>
>>>
>>> This flink 1.4.0
>>>
>>> Any ideas folks ?
>>>
>>
>>
>

Re: A strange exception while consumption using a multi topic Kafka Connector

Posted by Ted Yu <yu...@gmail.com>.
FLINK-9349 was logged.

FYI

On Sat, May 12, 2018 at 7:52 AM, Ted Yu <yu...@gmail.com> wrote:

> I took a look at ./flink-connectors/flink-connector-kafka-0.9/src/main/
> java/org/apache/flink/streaming/connectors/kafka/
> internal/Kafka09Fetcher.java
>
> It seems the List subscribedPartitionStates was being modified
> when runFetchLoop iterated the List.
> This can happen if, e.g., FlinkKafkaConsumer runs the following code
> concurrently:
>                 kafkaFetcher.addDiscoveredPartitions(
> discoveredPartitions);
>
> You can log a JIRA.
> If you have unit test reproducing this, that would be great.
>
> FYI
>
> On Fri, May 11, 2018 at 5:15 PM, Vishal Santoshi <
> vishal.santoshi@gmail.com> wrote:
>
>> java.util.ConcurrentModificationException
>> 	at java.util.LinkedList$ListItr.checkForComodification(LinkedList.java:966)
>> 	at java.util.LinkedList$ListItr.next(LinkedList.java:888)
>> 	at org.apache.flink.streaming.connectors.kafka.internal.Kafka09Fetcher.runFetchLoop(Kafka09Fetcher.java:134)
>>
>> .
>>
>> .
>>
>> .
>>
>>
>>
>> This flink 1.4.0
>>
>> Any ideas folks ?
>>
>
>

Re: A strange exception while consumption using a multi topic Kafka Connector

Posted by Ted Yu <yu...@gmail.com>.
I took a look
at ./flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java

It seems the List subscribedPartitionStates was being modified
when runFetchLoop iterated the List.
This can happen if, e.g., FlinkKafkaConsumer runs the following code
concurrently:
                kafkaFetcher.addDiscoveredPartitions(discoveredPartitions);

You can log a JIRA.
If you have unit test reproducing this, that would be great.

FYI

On Fri, May 11, 2018 at 5:15 PM, Vishal Santoshi <vi...@gmail.com>
wrote:

> java.util.ConcurrentModificationException
> 	at java.util.LinkedList$ListItr.checkForComodification(LinkedList.java:966)
> 	at java.util.LinkedList$ListItr.next(LinkedList.java:888)
> 	at org.apache.flink.streaming.connectors.kafka.internal.Kafka09Fetcher.runFetchLoop(Kafka09Fetcher.java:134)
>
> .
>
> .
>
> .
>
>
>
> This flink 1.4.0
>
> Any ideas folks ?
>

Re: A strange exception while consumption using a multi topic Kafka Connector

Posted by Vishal Santoshi <vi...@gmail.com>.
??

On Fri, May 11, 2018, 8:15 PM Vishal Santoshi <vi...@gmail.com>
wrote:

> java.util.ConcurrentModificationException
> 	at java.util.LinkedList$ListItr.checkForComodification(LinkedList.java:966)
> 	at java.util.LinkedList$ListItr.next(LinkedList.java:888)
> 	at org.apache.flink.streaming.connectors.kafka.internal.Kafka09Fetcher.runFetchLoop(Kafka09Fetcher.java:134)
>
> .
>
> .
>
> .
>
>
>
> This flink 1.4.0
>
> Any ideas folks ?
>