Uploaded image for project: 'Flume'
  1. Flume
  2. FLUME-2519

Avro source closes connection when unable to process an event batch

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Not A Problem
    • 1.5.0
    • None
    • None
    • None

    Description

      I have had this issue a few times now while sending log events from one agent to another (avro sink). The flume receiver fails to process an event and then closes the connection. The sender receives RPC call FAILED response and it retries the event and ends up in an infinite loop of "Failed to send events" error.

      Following are the logs from sender and receiver
      Sender
      ---------
      [SinkRunner-PollingRunner-LoadBalancingSinkProcessor] (org.apache.flume.sink.LoadBalancingSinkProcessor.process:158) - Sink failed to consume event. Attempting next sink if
      available.
      org.apache.flume.EventDeliveryException: Failed to send events
      at org.apache.flume.sink.AbstractRpcSink.process(AbstractRpcSink.java:392)
      at org.apache.flume.sink.LoadBalancingSinkProcessor.process(LoadBalancingSinkProcessor.java:154)
      at org.apache.flume.SinkRunner$PollingRunner.run(SinkRunner.java:147)
      at java.lang.Thread.run(Thread.java:745)
      Caused by: org.apache.flume.EventDeliveryException: NettyAvroRpcClient

      { host: ec2-54-188-251-126.us-west-2.compute.amazonaws.com, port: 44445 }

      : Failed to send batch
      at org.apache.flume.api.NettyAvroRpcClient.appendBatch(NettyAvroRpcClient.java:311)
      at org.apache.flume.sink.AbstractRpcSink.process(AbstractRpcSink.java:376)
      ... 3 more
      Caused by: org.apache.flume.EventDeliveryException: NettyAvroRpcClient

      { host: ec2-54-188-251-126.us-west-2.compute.amazonaws.com, port: 44445 }

      : Avro RPC call returned Status: FAILED
      at org.apache.flume.api.NettyAvroRpcClient.waitForStatusOK(NettyAvroRpcClient.java:387)
      at org.apache.flume.api.NettyAvroRpcClient.appendBatch(NettyAvroRpcClient.java:370)
      at org.apache.flume.api.NettyAvroRpcClient.appendBatch(NettyAvroRpcClient.java:299)
      ... 4 more

      Receiver
      ------------
      23 Oct 2014 12:25:22,595 ERROR New I/O worker #30 (org.apache.flume.source.AvroSource.appendBatch:380) - Avro source avrosource: Unable to process event batch. Exception follows.
      org.apache.flume.ChannelException: Unable to put batch on required channel: FileChannel filechannel-invalid

      { dataDirs: [/mnt1/data/flume/filechannel-invalid/data] }

      at org.apache.flume.channel.ChannelProcessor.processEventBatch(ChannelProcessor.java:200)
      at org.apache.flume.source.AvroSource.appendBatch(AvroSource.java:378)
      at sun.reflect.GeneratedMethodAccessor10.invoke(Unknown Source)
      at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
      at java.lang.reflect.Method.invoke(Method.java:606)
      at org.apache.avro.ipc.specific.SpecificResponder.respond(SpecificResponder.java:88)
      at org.apache.avro.ipc.Responder.respond(Responder.java:149)
      at org.apache.avro.ipc.NettyServer$NettyServerAvroHandler.messageReceived(NettyServer.java:188)
      at org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:70)
      at org.apache.avro.ipc.NettyServer$NettyServerAvroHandler.handleUpstream(NettyServer.java:173)
      at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:558)
      at org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:786)
      at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:296)
      at org.jboss.netty.handler.codec.frame.FrameDecoder.unfoldAndFireMessageReceived(FrameDecoder.java:458)
      at org.jboss.netty.handler.codec.frame.FrameDecoder.callDecode(FrameDecoder.java:439)
      at org.jboss.netty.handler.codec.frame.FrameDecoder.messageReceived(FrameDecoder.java:311)
      at org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:70)
      at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:558)
      at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:553)
      at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:268)
      at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:255)
      at org.jboss.netty.channel.socket.nio.NioWorker.read(NioWorker.java:84)
      at org.jboss.netty.channel.socket.nio.AbstractNioWorker.processSelectedKeys(AbstractNioWorker.java:471)
      at org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:332)
      at org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:35)
      at org.jboss.netty.util.ThreadRenamingRunnable.run(ThreadRenamingRunnable.java:102)
      at org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
      at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
      at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
      at java.lang.Thread.run(Thread.java:745)
      Caused by: java.lang.NullPointerException
      at org.apache.flume.channel.file.proto.ProtosFactory$FlumeEventHeader$Builder.setValue(ProtosFactory.java:7415)
      at org.apache.flume.channel.file.Put.writeProtos(Put.java:85)
      at org.apache.flume.channel.file.TransactionEventRecord.toByteBuffer(TransactionEventRecord.java:174)
      at org.apache.flume.channel.file.Log.put(Log.java:611)
      at org.apache.flume.channel.file.FileChannel$FileBackedTransaction.doPut(FileChannel.java:458)
      at org.apache.flume.channel.BasicTransactionSemantics.put(BasicTransactionSemantics.java:93)
      at org.apache.flume.channel.BasicChannelSemantics.put(BasicChannelSemantics.java:80)
      at org.apache.flume.channel.ChannelProcessor.processEventBatch(ChannelProcessor.java:189)
      ... 29 more
      23 Oct 2014 12:25:22,679 INFO New I/O worker #30 (org.apache.avro.ipc.NettyServer$NettyServerAvroHandler.handleUpstream:171) - [id: 0x1b64c49f, /54.221.18.120:51345 :> /10.196.7.161:44445] DISCONNECT
      ED
      23 Oct 2014 12:25:22,679 INFO New I/O worker #30 (org.apache.avro.ipc.NettyServer$NettyServerAvroHandler.handleUpstream:171) - [id: 0x1b64c49f, /54.221.18.120:51345 :> /10.196.7.161:44445] UNBOUND
      23 Oct 2014 12:25:22,679 INFO New I/O worker #30 (org.apache.avro.ipc.NettyServer$NettyServerAvroHandler.handleUpstream:171) - [id: 0x1b64c49f, /54.221.18.120:51345 :> /10.196.7.161:44445] CLOSED
      23 Oct 2014 12:25:22,679 INFO New I/O worker #30 (org.apache.avro.ipc.NettyServer$NettyServerAvroHandler.channelClosed:209) - Connection to /54.221.18.120:51345 disconnected.

      Attachments

        Activity

          People

            Unassigned Unassigned
            bijithkumar Bijith Kumar
            Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: