Uploaded image for project: 'Apache NiFi'
  1. Apache NiFi
  2. NIFI-2680

PutKafka processors can fail but sill transfer the flowfile to the success output

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Critical
    • Resolution: Fixed
    • 1.0.0, 0.7.0
    • 1.0.0, 0.7.1
    • None
    • None

    Description

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

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

      This can lead to data loss in the flow.

      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

      Attachments

        Activity

          People

            Unassigned Unassigned
            chris@mcdermott.net Christopher McDermott
            Votes:
            0 Vote for this issue
            Watchers:
            1 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: