You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "Jun Wang (Jira)" <ji...@apache.org> on 2020/05/04 13:38:00 UTC

[jira] [Commented] (KAFKA-8120) Getting NegativeArraySizeException when using Kafka Connect to send data to Kafka

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

Jun Wang commented on KAFKA-8120:
---------------------------------

Steps to reproduce and verify with kafka 2.5

1. run attached gen-mongodump.sh to create a dummy monodump file arround 1.2G

2. increase the initial heap size to 4G in bin/connect-standalone.sh
{noformat}
if [ "x$KAFKA_HEAP_OPTS" = "x" ]; then
 export KAFKA_HEAP_OPTS="-Xms4G -Xmx8G"
fi{noformat}
3. add batch.size=1 parameter in config/connect-file-source.properties

 
{noformat}
batch.size=1
 
{noformat}
4. start zookeeper-server and kafka-server if not yet

5. start kafka connect in standalone mode with following parameters

 
{noformat}
bin/connect-standalone.sh config/connect-standalone.properties config/connect-file-source.properties config/connect-file-sink.properties
 
{noformat}
within one minute, you will see following error.
{noformat}
[2020-05-02 21:17:45,128] ERROR WorkerSourceTask{id=local-file-source-0} Task threw an uncaught and unrecoverable exception (org.apache.kafka.connect.runtime.WorkerTask:186)
java.lang.NegativeArraySizeException
 at org.apache.kafka.connect.file.FileStreamSourceTask.poll(FileStreamSourceTask.java:141)
 at org.apache.kafka.connect.runtime.WorkerSourceTask.poll(WorkerSourceTask.java:272)
 at org.apache.kafka.connect.runtime.WorkerSourceTask.execute(WorkerSourceTask.java:239)
 at org.apache.kafka.connect.runtime.WorkerTask.doRun(WorkerTask.java:184)
 at org.apache.kafka.connect.runtime.WorkerTask.run(WorkerTask.java:234)
 at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
 at java.util.concurrent.FutureTask.run(FutureTask.java:266)
 at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
 at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
 at java.lang.Thread.run(Thread.java:748)
[2020-05-02 21:17:45,131] ERROR WorkerSourceTask{id=local-file-source-0} Task is being killed and will not recover until manually restarted (org.apache.kafka.connect.runtime.WorkerTask:187){noformat}
6. Run extra test with attached test.txt, In this test case kafka connect only process 6 lines out of 100 lines file.
This extra test case is not rated to the issue reported. but I found and fixed.

PR is ready https://github.com/apache/kafka/pull/8612

 

> Getting NegativeArraySizeException when using Kafka Connect to send data to Kafka
> ---------------------------------------------------------------------------------
>
>                 Key: KAFKA-8120
>                 URL: https://issues.apache.org/jira/browse/KAFKA-8120
>             Project: Kafka
>          Issue Type: Bug
>          Components: KafkaConnect
>    Affects Versions: 2.1.1
>         Environment: Ubuntu 16.04 LTS
>            Reporter: Prashant Shahi
>            Assignee: Jun Wang
>            Priority: Major
>         Attachments: gen-mongodump.sh.txt, test.txt
>
>
>  
> I have a large MongoDump JSON which I tried pushing to Kafka using Kafka Connect.
> I am getting the following Exception after around 16k messages been pushed. After the exception, the program doesn't get killed or exit, but now no more messages are pushed.
> {code:java}
> [2019-03-15 08:48:13,812] ERROR WorkerSourceTask{id=od-test18-0} Task threw an uncaught and unrecoverable exception (org.apache.kafka.connect.runtime.WorkerTask:177) java.lang.NegativeArraySizeException at org.apache.kafka.connect.file.FileStreamSourceTask.poll(FileStreamSourceTask.java:141) at org.apache.kafka.connect.runtime.WorkerSourceTask.poll(WorkerSourceTask.java:244) at org.apache.kafka.connect.runtime.WorkerSourceTask.execute(WorkerSourceTask.java:220) at org.apache.kafka.connect.runtime.WorkerTask.doRun(WorkerTask.java:175) at org.apache.kafka.connect.runtime.WorkerTask.run(WorkerTask.java:219) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:748) [2019-03-15 08:48:13,814] ERROR WorkerSourceTask{id=od-test18-0} Task is being killed and will not recover until manually restarted (org.apache.kafka.connect.runtime.WorkerTask:178){code}
>  



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