You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@nifi.apache.org by "McDermott, Chris Kevin (MSDU - STaTS/StorefrontRemote)" <ch...@hpe.com> on 2016/08/25 20:45:10 UTC

PutKakfa can send a file to its success output although the Kafka put fails

I just wanted to sanity check this before opening a JIRA.

I have PutKafka processor where I’ve set the Max Record Size property to 1MB.

If a transfer larger file to it, I see a bulletin and the following message in the log file, but the flowfile is transferred to the success output.

Caused by: org.apache.kafka.common.errors.RecordTooLargeException: The message is 5800160 bytes when serialized which is larger than the maximum request size you have configured with the max.request.si\
ze configuration.
2016-08-25 20:38:22,041 ERROR [Timer-Driven Process Thread-7] o.apache.nifi.processors.kafka.PutKafka PutKafka[id=3c396e30-7294-467e-b5d7-a3d44c0a04f4] Failed while waiting for acks from Kafka
2016-08-25 20:38:22,042 ERROR [Timer-Driven Process Thread-7] o.apache.nifi.processors.kafka.PutKafka
java.util.concurrent.ExecutionException: org.apache.kafka.common.errors.RecordTooLargeException: The message is 5800160 bytes when serialized which is larger than the maximum request size you have conf\
igured with the max.request.size configuration.
        at org.apache.kafka.clients.producer.KafkaProducer$FutureFailure.<init>(KafkaProducer.java:437) ~[kafka-clients-0.8.2.2.jar:na]
        at org.apache.kafka.clients.producer.KafkaProducer.send(KafkaProducer.java:352) ~[kafka-clients-0.8.2.2.jar:na]
        at org.apache.kafka.clients.producer.KafkaProducer.send(KafkaProducer.java:248) ~[kafka-clients-0.8.2.2.jar:na]
        at org.apache.nifi.processors.kafka.KafkaPublisher.publish(KafkaPublisher.java:137) ~[nifi-kafka-processors-oculus-0.7.x-SNAPSHOT.jar:oculus-0.7.x-SNAPSHOT]
        at org.apache.nifi.processors.kafka.PutKafka$1.process(PutKafka.java:315) [nifi-kafka-processors-oculus-0.7.x-SNAPSHOT.jar:oculus-0.7.x-SNAPSHOT]
        at org.apache.nifi.controller.repository.StandardProcessSession.read(StandardProcessSession.java:1851) [nifi-framework-core-oculus-0.7.x-SNAPSHOT.jar:oculus-0.7.x-SNAPSHOT]
        at org.apache.nifi.controller.repository.StandardProcessSession.read(StandardProcessSession.java:1822) [nifi-framework-core-oculus-0.7.x-SNAPSHOT.jar:oculus-0.7.x-SNAPSHOT]
        at org.apache.nifi.processors.kafka.PutKafka.doRendezvousWithKafka(PutKafka.java:311) [nifi-kafka-processors-oculus-0.7.x-SNAPSHOT.jar:oculus-0.7.x-SNAPSHOT]
        at org.apache.nifi.processors.kafka.PutKafka.rendezvousWithKafka(PutKafka.java:287) [nifi-kafka-processors-oculus-0.7.x-SNAPSHOT.jar:oculus-0.7.x-SNAPSHOT]
        at org.apache.nifi.processors.kafka.AbstractKafkaProcessor.onTrigger(AbstractKafkaProcessor.java:76) [nifi-kafka-processors-oculus-0.7.x-SNAPSHOT.jar:oculus-0.7.x-SNAPSHOT]
        at org.apache.nifi.controller.StandardProcessorNode.onTrigger(StandardProcessorNode.java:1054) [nifi-framework-core-oculus-0.7.x-SNAPSHOT.jar:oculus-0.7.x-SNAPSHOT]
        at org.apache.nifi.controller.tasks.ContinuallyRunProcessorTask.call(ContinuallyRunProcessorTask.java:136) [nifi-framework-core-oculus-0.7.x-SNAPSHOT.jar:oculus-0.7.x-SNAPSHOT]
        at org.apache.nifi.controller.tasks.ContinuallyRunProcessorTask.call(ContinuallyRunProcessorTask.java:47) [nifi-framework-core-oculus-0.7.x-SNAPSHOT.jar:oculus-0.7.x-SNAPSHOT]
        at org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent$1.run(TimerDrivenSchedulingAgent.java:127) [nifi-framework-core-oculus-0.7.x-SNAPSHOT.jar:oculus-0.7.x-SNAPSHOT]
        at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [na:1.8.0_45]
        at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) [na:1.8.0_45]

I have not yet tried the ProduceKafka processor.

Anyone see a reason I should not open a JIRA?


Chris McDermott

Remote Business Analytics
STaTS/StoreFront Remote
HPE Storage
Hewlett Packard Enterprise
Mobile: +1 978-697-5315

[cid:image001.png@01D1FEF0.0BF46FF0]

Re: PutKakfa can send a file to its success output although the Kafka put fails

Posted by "McDermott, Chris Kevin (MSDU - STaTS/StorefrontRemote)" <ch...@hpe.com>.
https://issues.apache.org/jira/browse/NIFI-2680

Chris McDermott
 
Remote Business Analytics
STaTS/StoreFront Remote
HPE Storage
Hewlett Packard Enterprise
Mobile: +1 978-697-5315
 


On 8/25/16, 4:50 PM, "Joe Witt" <jo...@gmail.com> wrote:

    Def seems like a good candidate for a bug report.
    
    Thanks
    Joe
    
    On Thu, Aug 25, 2016 at 4:45 PM, McDermott, Chris Kevin (MSDU -
    STaTS/StorefrontRemote) <ch...@hpe.com> wrote:
    
    > I just wanted to sanity check this before opening a JIRA.
    >
    >
    >
    > I have PutKafka processor where I’ve set the Max Record Size property to
    > 1MB.
    >
    >
    >
    > If a transfer larger file to it, I see a bulletin and the following
    > message in the log file, but the flowfile is transferred to the success
    > output.
    >
    >
    >
    > Caused by: org.apache.kafka.common.errors.RecordTooLargeException: The
    > message is 5800160 bytes when serialized which is larger than the maximum
    > request size you have configured with the max.request.si\
    >
    > ze configuration.
    >
    > 2016-08-25 20:38:22,041 ERROR [Timer-Driven Process Thread-7]
    > o.apache.nifi.processors.kafka.PutKafka PutKafka[id=3c396e30-7294-467e-b5d7-a3d44c0a04f4]
    > Failed while waiting for acks from Kafka
    >
    > 2016-08-25 20:38:22,042 ERROR [Timer-Driven Process Thread-7]
    > o.apache.nifi.processors.kafka.PutKafka
    >
    > java.util.concurrent.ExecutionException: org.apache.kafka.common.errors.RecordTooLargeException:
    > The message is 5800160 bytes when serialized which is larger than the
    > maximum request size you have conf\
    >
    > igured with the max.request.size configuration.
    >
    >         at org.apache.kafka.clients.producer.KafkaProducer$
    > FutureFailure.<init>(KafkaProducer.java:437) ~[kafka-clients-0.8.2.2.jar:
    > na]
    >
    >         at org.apache.kafka.clients.producer.KafkaProducer.send(KafkaProducer.java:352)
    > ~[kafka-clients-0.8.2.2.jar:na]
    >
    >         at org.apache.kafka.clients.producer.KafkaProducer.send(KafkaProducer.java:248)
    > ~[kafka-clients-0.8.2.2.jar:na]
    >
    >         at org.apache.nifi.processors.kafka.KafkaPublisher.publish(KafkaPublisher.java:137)
    > ~[nifi-kafka-processors-oculus-0.7.x-SNAPSHOT.jar:oculus-0.7.x-SNAPSHOT]
    >
    >         at org.apache.nifi.processors.kafka.PutKafka$1.process(PutKafka.java:315)
    > [nifi-kafka-processors-oculus-0.7.x-SNAPSHOT.jar:oculus-0.7.x-SNAPSHOT]
    >
    >         at org.apache.nifi.controller.repository.
    > StandardProcessSession.read(StandardProcessSession.java:1851)
    > [nifi-framework-core-oculus-0.7.x-SNAPSHOT.jar:oculus-0.7.x-SNAPSHOT]
    >
    >         at org.apache.nifi.controller.repository.
    > StandardProcessSession.read(StandardProcessSession.java:1822)
    > [nifi-framework-core-oculus-0.7.x-SNAPSHOT.jar:oculus-0.7.x-SNAPSHOT]
    >
    >         at org.apache.nifi.processors.kafka.PutKafka.
    > doRendezvousWithKafka(PutKafka.java:311) [nifi-kafka-processors-oculus-
    > 0.7.x-SNAPSHOT.jar:oculus-0.7.x-SNAPSHOT]
    >
    >         at org.apache.nifi.processors.kafka.PutKafka.
    > rendezvousWithKafka(PutKafka.java:287) [nifi-kafka-processors-oculus-
    > 0.7.x-SNAPSHOT.jar:oculus-0.7.x-SNAPSHOT]
    >
    >         at org.apache.nifi.processors.kafka.AbstractKafkaProcessor.
    > onTrigger(AbstractKafkaProcessor.java:76) [nifi-kafka-processors-oculus-
    > 0.7.x-SNAPSHOT.jar:oculus-0.7.x-SNAPSHOT]
    >
    >         at org.apache.nifi.controller.StandardProcessorNode.onTrigger(
    > StandardProcessorNode.java:1054) [nifi-framework-core-oculus-0.
    > 7.x-SNAPSHOT.jar:oculus-0.7.x-SNAPSHOT]
    >
    >         at org.apache.nifi.controller.tasks.ContinuallyRunProcessorTask.
    > call(ContinuallyRunProcessorTask.java:136) [nifi-framework-core-oculus-0.
    > 7.x-SNAPSHOT.jar:oculus-0.7.x-SNAPSHOT]
    >
    >         at org.apache.nifi.controller.tasks.ContinuallyRunProcessorTask.
    > call(ContinuallyRunProcessorTask.java:47) [nifi-framework-core-oculus-0.
    > 7.x-SNAPSHOT.jar:oculus-0.7.x-SNAPSHOT]
    >
    >         at org.apache.nifi.controller.scheduling.
    > TimerDrivenSchedulingAgent$1.run(TimerDrivenSchedulingAgent.java:127)
    > [nifi-framework-core-oculus-0.7.x-SNAPSHOT.jar:oculus-0.7.x-SNAPSHOT]
    >
    >         at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
    > [na:1.8.0_45]
    >
    >         at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
    > [na:1.8.0_45]
    >
    >
    >
    > I have not yet tried the ProduceKafka processor.
    >
    >
    >
    > Anyone see a reason I should not open a JIRA?
    >
    >
    >
    >
    >
    > Chris McDermott
    >
    >
    >
    > Remote Business Analytics
    >
    > STaTS/StoreFront Remote
    >
    > HPE Storage
    >
    > Hewlett Packard Enterprise
    >
    > Mobile: +1 978-697-5315
    >
    >
    >
    >
    


Re: PutKakfa can send a file to its success output although the Kafka put fails

Posted by Joe Witt <jo...@gmail.com>.
Def seems like a good candidate for a bug report.

Thanks
Joe

On Thu, Aug 25, 2016 at 4:45 PM, McDermott, Chris Kevin (MSDU -
STaTS/StorefrontRemote) <ch...@hpe.com> wrote:

> I just wanted to sanity check this before opening a JIRA.
>
>
>
> I have PutKafka processor where I’ve set the Max Record Size property to
> 1MB.
>
>
>
> If a transfer larger file to it, I see a bulletin and the following
> message in the log file, but the flowfile is transferred to the success
> output.
>
>
>
> Caused by: org.apache.kafka.common.errors.RecordTooLargeException: The
> message is 5800160 bytes when serialized which is larger than the maximum
> request size you have configured with the max.request.si\
>
> ze configuration.
>
> 2016-08-25 20:38:22,041 ERROR [Timer-Driven Process Thread-7]
> o.apache.nifi.processors.kafka.PutKafka PutKafka[id=3c396e30-7294-467e-b5d7-a3d44c0a04f4]
> Failed while waiting for acks from Kafka
>
> 2016-08-25 20:38:22,042 ERROR [Timer-Driven Process Thread-7]
> o.apache.nifi.processors.kafka.PutKafka
>
> java.util.concurrent.ExecutionException: org.apache.kafka.common.errors.RecordTooLargeException:
> The message is 5800160 bytes when serialized which is larger than the
> maximum request size you have conf\
>
> igured with the max.request.size configuration.
>
>         at org.apache.kafka.clients.producer.KafkaProducer$
> FutureFailure.<init>(KafkaProducer.java:437) ~[kafka-clients-0.8.2.2.jar:
> na]
>
>         at org.apache.kafka.clients.producer.KafkaProducer.send(KafkaProducer.java:352)
> ~[kafka-clients-0.8.2.2.jar:na]
>
>         at org.apache.kafka.clients.producer.KafkaProducer.send(KafkaProducer.java:248)
> ~[kafka-clients-0.8.2.2.jar:na]
>
>         at org.apache.nifi.processors.kafka.KafkaPublisher.publish(KafkaPublisher.java:137)
> ~[nifi-kafka-processors-oculus-0.7.x-SNAPSHOT.jar:oculus-0.7.x-SNAPSHOT]
>
>         at org.apache.nifi.processors.kafka.PutKafka$1.process(PutKafka.java:315)
> [nifi-kafka-processors-oculus-0.7.x-SNAPSHOT.jar:oculus-0.7.x-SNAPSHOT]
>
>         at org.apache.nifi.controller.repository.
> StandardProcessSession.read(StandardProcessSession.java:1851)
> [nifi-framework-core-oculus-0.7.x-SNAPSHOT.jar:oculus-0.7.x-SNAPSHOT]
>
>         at org.apache.nifi.controller.repository.
> StandardProcessSession.read(StandardProcessSession.java:1822)
> [nifi-framework-core-oculus-0.7.x-SNAPSHOT.jar:oculus-0.7.x-SNAPSHOT]
>
>         at org.apache.nifi.processors.kafka.PutKafka.
> doRendezvousWithKafka(PutKafka.java:311) [nifi-kafka-processors-oculus-
> 0.7.x-SNAPSHOT.jar:oculus-0.7.x-SNAPSHOT]
>
>         at org.apache.nifi.processors.kafka.PutKafka.
> rendezvousWithKafka(PutKafka.java:287) [nifi-kafka-processors-oculus-
> 0.7.x-SNAPSHOT.jar:oculus-0.7.x-SNAPSHOT]
>
>         at org.apache.nifi.processors.kafka.AbstractKafkaProcessor.
> onTrigger(AbstractKafkaProcessor.java:76) [nifi-kafka-processors-oculus-
> 0.7.x-SNAPSHOT.jar:oculus-0.7.x-SNAPSHOT]
>
>         at org.apache.nifi.controller.StandardProcessorNode.onTrigger(
> StandardProcessorNode.java:1054) [nifi-framework-core-oculus-0.
> 7.x-SNAPSHOT.jar:oculus-0.7.x-SNAPSHOT]
>
>         at org.apache.nifi.controller.tasks.ContinuallyRunProcessorTask.
> call(ContinuallyRunProcessorTask.java:136) [nifi-framework-core-oculus-0.
> 7.x-SNAPSHOT.jar:oculus-0.7.x-SNAPSHOT]
>
>         at org.apache.nifi.controller.tasks.ContinuallyRunProcessorTask.
> call(ContinuallyRunProcessorTask.java:47) [nifi-framework-core-oculus-0.
> 7.x-SNAPSHOT.jar:oculus-0.7.x-SNAPSHOT]
>
>         at org.apache.nifi.controller.scheduling.
> TimerDrivenSchedulingAgent$1.run(TimerDrivenSchedulingAgent.java:127)
> [nifi-framework-core-oculus-0.7.x-SNAPSHOT.jar:oculus-0.7.x-SNAPSHOT]
>
>         at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> [na:1.8.0_45]
>
>         at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
> [na:1.8.0_45]
>
>
>
> I have not yet tried the ProduceKafka processor.
>
>
>
> Anyone see a reason I should not open a JIRA?
>
>
>
>
>
> Chris McDermott
>
>
>
> Remote Business Analytics
>
> STaTS/StoreFront Remote
>
> HPE Storage
>
> Hewlett Packard Enterprise
>
> Mobile: +1 978-697-5315
>
>
>
>