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

ExecuteFlumeSink not working with Sink Type "avro".

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 1.2.0, 1.3.0
    • 1.9.0
    • Extensions
    • None

    Description

      After upgrading NiFi from 1.1.2 to 1.2.0, ExecuteFlumeSink with sink type "avro" stops transmitting data and throws the error quoted below. The problem persists after upgrading to 1.3.0.
      Replacing avro-ipc-1.7.4.jar with avro-ipc-1.8.1.jar in nifi-flume-nar-1.3.0.nar solves the issue.

      {{2017-08-02 10:28:51,142 ERROR [Timer-Driven Process Thread-6] o.a.n.processors.flume.ExecuteFlumeSink ExecuteFlumeSink[id=98da368f-015d-1000-ffff-ffff9
      6768472] ExecuteFlumeSink[id=98da368f-015d-1000-ffff-ffff96768472] failed to process due to org.apache.nifi.processor.exception.ProcessException: Flume
      event delivery failed; rolling back session: {}
      org.apache.nifi.processor.exception.ProcessException: Flume event delivery failed
      at org.apache.nifi.processors.flume.ExecuteFlumeSink.onTrigger(ExecuteFlumeSink.java:145)
      at org.apache.nifi.processors.flume.AbstractFlumeProcessor.onTrigger(AbstractFlumeProcessor.java:148)
      at org.apache.nifi.controller.StandardProcessorNode.onTrigger(StandardProcessorNode.java:1118)
      at org.apache.nifi.controller.tasks.ContinuallyRunProcessorTask.call(ContinuallyRunProcessorTask.java:144)
      at org.apache.nifi.controller.tasks.ContinuallyRunProcessorTask.call(ContinuallyRunProcessorTask.java:47)
      at org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent$1.run(TimerDrivenSchedulingAgent.java:132)
      at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
      at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
      at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
      at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
      at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
      at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
      at java.lang.Thread.run(Thread.java:748)
      Caused by: org.apache.flume.EventDeliveryException: Failed to send events
      at org.apache.flume.sink.AbstractRpcSink.process(AbstractRpcSink.java:392)
      at org.apache.nifi.processors.flume.ExecuteFlumeSink.onTrigger(ExecuteFlumeSink.java:143)
      ... 12 common frames omitted
      Caused by: org.apache.flume.EventDeliveryException: NettyAvroRpcClient

      { host: target.linux.dummy, port: 14100 }

      : Failed to send batch
      at org.apache.flume.api.NettyAvroRpcClient.appendBatch(NettyAvroRpcClient.java:315)
      at org.apache.flume.sink.AbstractRpcSink.process(AbstractRpcSink.java:376)
      ... 13 common frames omitted
      Caused by: org.apache.flume.EventDeliveryException: NettyAvroRpcClient

      { host: target.linux.dummy, port: 14100 }

      : RPC request exception
      at org.apache.flume.api.NettyAvroRpcClient.appendBatch(NettyAvroRpcClient.java:365)
      at org.apache.flume.api.NettyAvroRpcClient.appendBatch(NettyAvroRpcClient.java:303)
      ... 14 common frames omitted
      Caused by: java.util.concurrent.ExecutionException: java.lang.AbstractMethodError: org.apache.avro.specific.SpecificFixed.getSchema()Lorg/apache/avro/S
      chema;
      at java.util.concurrent.FutureTask.report(FutureTask.java:122)
      at java.util.concurrent.FutureTask.get(FutureTask.java:206)
      at org.apache.flume.api.NettyAvroRpcClient.appendBatch(NettyAvroRpcClient.java:357)
      ... 15 common frames omitted
      Caused by: java.lang.AbstractMethodError: org.apache.avro.specific.SpecificFixed.getSchema()Lorg/apache/avro/Schema;
      at org.apache.avro.specific.SpecificFixed.<init>(SpecificFixed.java:36)
      at org.apache.avro.ipc.MD5.<init>(MD5.java:16)
      at org.apache.avro.ipc.Requestor.writeHandshake(Requestor.java:200)
      at org.apache.avro.ipc.Requestor.access$300(Requestor.java:52)
      at org.apache.avro.ipc.Requestor$Request.getBytes(Requestor.java:478)
      at org.apache.avro.ipc.Requestor.request(Requestor.java:147)
      at org.apache.avro.ipc.Requestor.request(Requestor.java:129)
      at org.apache.avro.ipc.specific.SpecificRequestor.invoke(SpecificRequestor.java:84)
      at com.sun.proxy.$Proxy124.appendBatch(Unknown Source)
      at org.apache.flume.api.NettyAvroRpcClient$2.call(NettyAvroRpcClient.java:348)
      at org.apache.flume.api.NettyAvroRpcClient$2.call(NettyAvroRpcClient.java:344)
      at java.util.concurrent.FutureTask.run(FutureTask.java:266)
      ... 3 common frames omitted}}

      Attachments

        Issue Links

          Activity

            People

              Unassigned Unassigned
              schlitte Sebastian Schlitte
              Votes:
              1 Vote for this issue
              Watchers:
              5 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: