Flume
  1. Flume
  2. FLUME-757

Flume Collector Sink Fails Due to Driver Exception

    Details

    • Type: Bug Bug
    • Status: Open
    • Priority: Major Major
    • Resolution: Unresolved
    • Affects Version/s: v0.9.4
    • Fix Version/s: None
    • Component/s: Sinks+Sources
    • Labels:
      None

      Description

      I am experiencing a weird issue with my Flume configuration. Occasionally when I start my agents they start delivering messages to the collectors and after awhile I receive the following exceptions and stop receiving messages.

      The collectors are configured as follows.

      Source: collectorSource(36892)
      Sink: {regex("^(?:\\n)?(\\d\\d\\d\\d-\\d\\d-\\d\\d\\s\\d\\d:\\d
      d)", 1, "date") => {exDate("date", "yyyy-MM-dd HH:mm") => collectorSink("hdfs://hadoop-namenode1:8020/logs/%

      {dateyear}

      -%

      {datemonth}

      -%

      {dateday}

      /%

      {datehr}

      00/%

      {host}

      ", "log-")}}

      The agents are configured as follows.

      Source: tailDir("/data1/logs", delim="\\n\\d\\d\\d
      d", delimMode="next")
      Sink: agentDFOChain("flume-collector1:36892", "flume-collector2:36892")

      The following exceptions are from my collectors.

      INFO com.cloudera.flume.handlers.debug.InsistentAppendDecorator - Failed due to unexpected runtime exception during append attempt
      java.lang.NullPointerException
      at com.cloudera.flume.handlers.hdfs.CustomDfsSink.close(CustomDfsSink.java:88)
      at com.cloudera.flume.handlers.hdfs.EscapedCustomDfsSink.close(EscapedCustomDfsSink.java:132)
      at com.cloudera.flume.core.CompositeSink.close(CompositeSink.java:56)
      at com.cloudera.flume.core.EventSinkDecorator.close(EventSinkDecorator.java:67)
      at com.cloudera.flume.collector.CollectorSink$RollDetectDeco.close(CollectorSink.java:171)
      at com.cloudera.flume.handlers.rolling.RollSink.close(RollSink.java:331)
      at com.cloudera.flume.core.EventSinkDecorator.close(EventSinkDecorator.java:67)
      at com.cloudera.flume.core.EventSinkDecorator.close(EventSinkDecorator.java:67)
      at com.cloudera.flume.handlers.debug.InsistentOpenDecorator.close(InsistentOpenDecorator.java:175)
      at com.cloudera.flume.core.EventSinkDecorator.close(EventSinkDecorator.java:67)
      at com.cloudera.flume.handlers.debug.StubbornAppendSink.append(StubbornAppendSink.java:96)
      at com.cloudera.flume.core.EventSinkDecorator.append(EventSinkDecorator.java:60)
      at com.cloudera.flume.handlers.debug.InsistentAppendDecorator.append(InsistentAppendDecorator.java:110)
      at com.cloudera.flume.core.EventSinkDecorator.append(EventSinkDecorator.java:60)
      at com.cloudera.flume.handlers.endtoend.AckChecksumChecker.append(AckChecksumChecker.java:113)
      at com.cloudera.flume.core.EventSinkDecorator.append(EventSinkDecorator.java:60)
      at com.cloudera.flume.handlers.batch.UnbatchingDecorator.append(UnbatchingDecorator.java:62)
      at com.cloudera.flume.core.EventSinkDecorator.append(EventSinkDecorator.java:60)
      at com.cloudera.flume.handlers.batch.GunzipDecorator.append(GunzipDecorator.java:81)
      at com.cloudera.flume.collector.CollectorSink.append(CollectorSink.java:222)
      at com.cloudera.flume.core.EventSinkDecorator.append(EventSinkDecorator.java:60)
      at com.cloudera.flume.core.extractors.DateExtractor.append(DateExtractor.java:129)
      at com.cloudera.flume.core.EventSinkDecorator.append(EventSinkDecorator.java:60)
      at com.cloudera.flume.core.extractors.RegexExtractor.append(RegexExtractor.java:88)
      at com.cloudera.flume.core.connector.DirectDriver$PumperThread.run(DirectDriver.java:110)
      ERROR com.cloudera.flume.core.connector.DirectDriver - Closing down due to exception during append calls
      java.lang.NullPointerException
      at com.cloudera.flume.handlers.hdfs.CustomDfsSink.close(CustomDfsSink.java:88)
      at com.cloudera.flume.handlers.hdfs.EscapedCustomDfsSink.close(EscapedCustomDfsSink.java:132)
      at com.cloudera.flume.core.CompositeSink.close(CompositeSink.java:56)
      at com.cloudera.flume.core.EventSinkDecorator.close(EventSinkDecorator.java:67)
      at com.cloudera.flume.collector.CollectorSink$RollDetectDeco.close(CollectorSink.java:171)
      at com.cloudera.flume.handlers.rolling.RollSink.close(RollSink.java:331)
      at com.cloudera.flume.core.EventSinkDecorator.close(EventSinkDecorator.java:67)
      at com.cloudera.flume.core.EventSinkDecorator.close(EventSinkDecorator.java:67)
      at com.cloudera.flume.handlers.debug.InsistentOpenDecorator.close(InsistentOpenDecorator.java:175)
      at com.cloudera.flume.core.EventSinkDecorator.close(EventSinkDecorator.java:67)
      at com.cloudera.flume.handlers.debug.StubbornAppendSink.append(StubbornAppendSink.java:96)
      at com.cloudera.flume.core.EventSinkDecorator.append(EventSinkDecorator.java:60)
      at com.cloudera.flume.handlers.debug.InsistentAppendDecorator.append(InsistentAppendDecorator.java:110)
      at com.cloudera.flume.core.EventSinkDecorator.append(EventSinkDecorator.java:60)
      at com.cloudera.flume.handlers.endtoend.AckChecksumChecker.append(AckChecksumChecker.java:113)
      at com.cloudera.flume.core.EventSinkDecorator.append(EventSinkDecorator.java:60)
      at com.cloudera.flume.handlers.batch.UnbatchingDecorator.append(UnbatchingDecorator.java:62)
      at com.cloudera.flume.core.EventSinkDecorator.append(EventSinkDecorator.java:60)
      at com.cloudera.flume.handlers.batch.GunzipDecorator.append(GunzipDecorator.java:81)
      at com.cloudera.flume.collector.CollectorSink.append(CollectorSink.java:222)
      at com.cloudera.flume.core.EventSinkDecorator.append(EventSinkDecorator.java:60)
      at com.cloudera.flume.core.extractors.DateExtractor.append(DateExtractor.java:129)
      at com.cloudera.flume.core.EventSinkDecorator.append(EventSinkDecorator.java:60)
      at com.cloudera.flume.core.extractors.RegexExtractor.append(RegexExtractor.java:88)
      at com.cloudera.flume.core.connector.DirectDriver$PumperThread.run(DirectDriver.java:110)

      The following are from my agents

      ERROR com.cloudera.flume.core.connector.DirectDriver - Expected IDLE but timed out in state ACTIVE
      INFO com.cloudera.flume.agent.diskfailover.DiskFailoverDeco - Closing disk failover log, subsink still making progre
      ERROR com.cloudera.flume.agent.LogicalNode - Forcing driver to exit uncleanly
      INFO com.cloudera.flume.agent.LogicalNode - Node config successfully set to com.cloudera.flume.conf.FlumeConfigData@19a0feb
      WARN com.cloudera.flume.handlers.debug.LazyOpenDecorator - Closing a lazy sink that was not logically opened
      INFO com.cloudera.flume.handlers.text.TailDirSource - added file /data1/mutator/logs/one.log
      INFO com.cloudera.flume.handlers.rolling.RollSink - Created RollSink: trigger=[TimeTrigger: maxAge=10000 tagger=com.
      cloudera.flume.handlers.rolling.ProcessTagger@6279d] checkPeriodMs = 250 spec='NaiveFileFailover'
      INFO com.cloudera.flume.handlers.rolling.RollSink - opening RollSink 'NaiveFileFailover'
      ERROR com.cloudera.flume.agent.diskfailover.DiskFailoverDeco - WAL drain thread interrupted
      java.lang.InterruptedException
      at java.lang.Object.wait(Native Method)
      at com.cloudera.flume.core.connector.DirectDriver.waitForAtLeastState(DirectDriver.java:308)
      at com.cloudera.flume.agent.diskfailover.DiskFailoverDeco.ensureClosedDrainDriver(DiskFailoverDeco.java:129)
      at com.cloudera.flume.agent.diskfailover.DiskFailoverDeco.close(DiskFailoverDeco.java:177)
      at com.cloudera.flume.core.BackOffFailOverSink.close(BackOffFailOverSink.java:165)
      at com.cloudera.flume.core.CompositeSink.close(CompositeSink.java:56)
      at com.cloudera.flume.agent.AgentFailChainSink.close(AgentFailChainSink.java:98)
      at com.cloudera.flume.core.CompositeSink.close(CompositeSink.java:56)
      at com.cloudera.flume.core.connector.DirectDriver$PumperThread.run(DirectDriver.java:126)

        Activity

        Hide
        Cameron Gandevia added a comment - - edited

        We have around 50 nodes writing to two collectors (Although only one at a time ever seems to receive anything)

        FLUME-762 seems to have solved recovering from the NullPointerException but our collector now fails with InterruptExceptions which I have detailed in FLUME-798.
        What is the reason flume doesn't attempt to recover from InterruptExceptions?

        Show
        Cameron Gandevia added a comment - - edited We have around 50 nodes writing to two collectors (Although only one at a time ever seems to receive anything) FLUME-762 seems to have solved recovering from the NullPointerException but our collector now fails with InterruptExceptions which I have detailed in FLUME-798 . What is the reason flume doesn't attempt to recover from InterruptExceptions?
        Hide
        Matthew Rathbone added a comment -

        Hey Jon,

        This has started happening at all times of the day pretty recently.

        There are currently 22 different 'log types', but this is growing. That doesn't seem like too many though right?

        Is there a patch that might help with this? It's causing pretty big issues for us.

        Also seeing this issue in the current TRUNK.

        Show
        Matthew Rathbone added a comment - Hey Jon, This has started happening at all times of the day pretty recently. There are currently 22 different 'log types', but this is growing. That doesn't seem like too many though right? Is there a patch that might help with this? It's causing pretty big issues for us. Also seeing this issue in the current TRUNK.
        Hide
        Jonathan Hsieh added a comment -

        FLUME-762 may solve the recovery part of the problem but from what you describe, it is a situation where even that may not be able to recover.

        @Cameron. Interesting and good stuff! How man hosts do you have sending to your collector?

        Because you have %

        {host}

        in your collector config, each host and each date is create a whole slew of new file handles to hdfs (each filename is another open file handle!). host information is saved with the data written out to hdfs so you can always group/filter host info on a post processing phase.

        Curious – what is the default file handles value and what did you set it to?

        @Mathew - How many %

        {logtype}

        's are there? A change around midnight may be a place where the collector may have 2x the normal hdfs connections open (wrapping up yesterday and creating new files for today).

        Show
        Jonathan Hsieh added a comment - FLUME-762 may solve the recovery part of the problem but from what you describe, it is a situation where even that may not be able to recover. @Cameron. Interesting and good stuff! How man hosts do you have sending to your collector? Because you have % {host} in your collector config, each host and each date is create a whole slew of new file handles to hdfs (each filename is another open file handle!). host information is saved with the data written out to hdfs so you can always group/filter host info on a post processing phase. Curious – what is the default file handles value and what did you set it to? @Mathew - How many % {logtype} 's are there? A change around midnight may be a place where the collector may have 2x the normal hdfs connections open (wrapping up yesterday and creating new files for today).
        Hide
        Cameron Gandevia added a comment -

        To reproduce this problem reduce the number of file handles in your HDFS cluster, start the flume agents tailing a directory with a ton of files. HDFS data nodes will fail with out of file handle errors and the collector will NullPointer

        Show
        Cameron Gandevia added a comment - To reproduce this problem reduce the number of file handles in your HDFS cluster, start the flume agents tailing a directory with a ton of files. HDFS data nodes will fail with out of file handle errors and the collector will NullPointer
        Hide
        Cameron Gandevia added a comment -

        Hey starting to take a look at this issue.

        What I have initially discovered is the writer is actually Null. I quickly tested synchronizing the open/close functions but the writer still ends up being null.
        Looks like the race condition is in the collection of sfWriters in EscapedCustomDfsSink.

        I will look into it further and post comments.

        Just curious how people are running the trunk build of flume for any length of time. I cannot get the collectors to run for more than 5 mins.

        Show
        Cameron Gandevia added a comment - Hey starting to take a look at this issue. What I have initially discovered is the writer is actually Null. I quickly tested synchronizing the open/close functions but the writer still ends up being null. Looks like the race condition is in the collection of sfWriters in EscapedCustomDfsSink. I will look into it further and post comments. Just curious how people are running the trunk build of flume for any length of time. I cannot get the collectors to run for more than 5 mins.
        Hide
        Jonathan Hsieh added a comment -

        I looks like there is a race condition in CustomDFSSink where pathManager is null when it is attempting to close. The only way this could happen is if the open call and close call are being called concurrently.

          @Override
          public void close() throws IOException {
            LOG.info("Closing HDFS file: " + pathManager.getOpenPath());  /// NPE here.
            writer.flush();
            LOG.info("done writing raw file to hdfs");
            writer.close();
            pathManager.close();
            writer = null;
          }
        

        First thought is that we need to get a local instance of the pathManager and then do null checks on it, or to possibly put synchronization on open and close calls.

          @Override
          public void close() throws IOException {
            LOG.info("Closing HDFS file: " + pathManager.getOpenPath());
            writer.flush();
            LOG.info("done writing raw file to hdfs");
            writer.close();
            pathManager.close();
            writer = null;
          }
        

        Want to give it a try?

        Show
        Jonathan Hsieh added a comment - I looks like there is a race condition in CustomDFSSink where pathManager is null when it is attempting to close. The only way this could happen is if the open call and close call are being called concurrently. @Override public void close() throws IOException { LOG.info( "Closing HDFS file: " + pathManager.getOpenPath()); /// NPE here. writer.flush(); LOG.info( "done writing raw file to hdfs" ); writer.close(); pathManager.close(); writer = null ; } First thought is that we need to get a local instance of the pathManager and then do null checks on it, or to possibly put synchronization on open and close calls. @Override public void close() throws IOException { LOG.info( "Closing HDFS file: " + pathManager.getOpenPath()); writer.flush(); LOG.info( "done writing raw file to hdfs" ); writer.close(); pathManager.close(); writer = null ; } Want to give it a try?
        Hide
        Cameron Gandevia added a comment -

        I think it can be replicated by deleting the log files as they are being tailed. I will confirm this later tonight and reply.

        Show
        Cameron Gandevia added a comment - I think it can be replicated by deleting the log files as they are being tailed. I will confirm this later tonight and reply.
        Hide
        Matthew Rathbone added a comment - - edited

        I'm getting almost the exact same issue (on 0.9.4). We were NOT getting this problem in version 0.9.3.

        I get slightly different log output though. Firstly, my configuration is:

        collectorSource(9108)
        collectorSink(\"s3n://bucket/flumed/%

        {logtype}

        /dt=%Y-%m-%d/\", \"%Hlog-\", 900000)

        The (only) errors I see in my console are:

        2011-09-01 23:01:26,953 ERROR com.cloudera.flume.core.connector.DirectDriver: Closing down due to exception during append calls
        2011-09-01 23:10:19,620 ERROR com.cloudera.flume.core.connector.DirectDriver: Exiting driver logicalNode logs-2-19 in error state CollectorSource | Collector because null

        The funny thing is that this happens every night at about the same time of day (23:XX)

        I'm totally willing and able to take a look at this, but I could really do with some direction on how to replicate it?

        Show
        Matthew Rathbone added a comment - - edited I'm getting almost the exact same issue (on 0.9.4). We were NOT getting this problem in version 0.9.3. I get slightly different log output though. Firstly, my configuration is: collectorSource(9108) collectorSink(\"s3n://bucket/flumed/% {logtype} /dt=%Y-%m-%d/\", \"%Hlog-\", 900000) The (only) errors I see in my console are: 2011-09-01 23:01:26,953 ERROR com.cloudera.flume.core.connector.DirectDriver: Closing down due to exception during append calls 2011-09-01 23:10:19,620 ERROR com.cloudera.flume.core.connector.DirectDriver: Exiting driver logicalNode logs-2-19 in error state CollectorSource | Collector because null The funny thing is that this happens every night at about the same time of day (23:XX) I'm totally willing and able to take a look at this, but I could really do with some direction on how to replicate it?

          People

          • Assignee:
            Unassigned
            Reporter:
            Cameron Gandevia
          • Votes:
            2 Vote for this issue
            Watchers:
            6 Start watching this issue

            Dates

            • Created:
              Updated:

              Development