Flume
  1. Flume
  2. FLUME-637

Flume Node deadlocks on getting reports/metrics info.

    Details

    • Type: Bug Bug
    • Status: Open
    • Priority: Critical Critical
    • Resolution: Unresolved
    • Affects Version/s: v0.9.4
    • Fix Version/s: v0.9.5
    • Component/s: None
    • Labels:
      None

      Description

      There is a deadlock condition that can occur when attempting to gather report information. In the comments, we show a RollSink.getMetrics call, a MasterReportPusher.getReport and several httpServlet threads all blocked at a RollSink.

        Issue Links

          Activity

          Hide
          Jonathan Hsieh added a comment -

          On an agent, this results in the following error message. (which has been reported by many people)

          2011-05-05 00:00:24,954 INFO com.cloudera.flume.agent.durability.NaiveFileWALManager: File lives in /tmp/flume-flume/agent/agentjenkins/writing/log.00000024.20110505-000014873-0700.218629927164233.seq
          2011-05-05 00:00:24,954 INFO com.cloudera.flume.handlers.hdfs.SeqfileEventSink: constructed new seqfile event sink: file=/tmp/flume-flume/agent/agentjenkins/writing/log.00000024.20110505-000024954-0700.218640008171233.seq
          2011-05-05 00:00:24,955 INFO com.cloudera.flume.handlers.endtoend.AckListener$Empty: Empty Ack Listener began log.00000024.20110505-000024954-0700.218640008171233.seq
          2011-05-05 00:00:31,320 INFO com.cloudera.flume.agent.WALAckManager: Retransmitting log.00000024.20110504-031329620-0700.143824674444233.seq after being stale for 60387ms
          2011-05-05 00:00:31,324 WARN com.cloudera.flume.agent.durability.NaiveFileWALManager: There was a race that happend with SENT vs SENDING states
          
          Show
          Jonathan Hsieh added a comment - On an agent, this results in the following error message. (which has been reported by many people) 2011-05-05 00:00:24,954 INFO com.cloudera.flume.agent.durability.NaiveFileWALManager: File lives in /tmp/flume-flume/agent/agentjenkins/writing/log.00000024.20110505-000014873-0700.218629927164233.seq 2011-05-05 00:00:24,954 INFO com.cloudera.flume.handlers.hdfs.SeqfileEventSink: constructed new seqfile event sink: file=/tmp/flume-flume/agent/agentjenkins/writing/log.00000024.20110505-000024954-0700.218640008171233.seq 2011-05-05 00:00:24,955 INFO com.cloudera.flume.handlers.endtoend.AckListener$Empty: Empty Ack Listener began log.00000024.20110505-000024954-0700.218640008171233.seq 2011-05-05 00:00:31,320 INFO com.cloudera.flume.agent.WALAckManager: Retransmitting log.00000024.20110504-031329620-0700.143824674444233.seq after being stale for 60387ms 2011-05-05 00:00:31,324 WARN com.cloudera.flume.agent.durability.NaiveFileWALManager: There was a race that happend with SENT vs SENDING states
          Hide
          Jonathan Hsieh added a comment -

          Here is a sample of the blocked threads on the agent.

          Thread 153903 (1621610227@qtp-2005327631-17):
            State: BLOCKED
            Blocked count: 1
            Waited count: 0
            Blocked on com.cloudera.flume.collector.CollectorSink$1@724f782e
            Blocked by 107418 (Roll-TriggerThread-133)
            Stack:
              com.cloudera.flume.handlers.rolling.RollSink.getMetrics(RollSink.java:256)
              com.cloudera.flume.reporter.ReportUtil.getFlattenedReport(ReportUtil.java:92)
              com.cloudera.flume.reporter.ReportUtil.getFlattenedReport(ReportUtil.java:96)
              com.cloudera.flume.reporter.ReportUtil.getFlattenedReport(ReportUtil.java:96)
              com.cloudera.flume.reporter.ReportUtil.getFlattenedReport(ReportUtil.java:96)
              com.cloudera.flume.reporter.ReportUtil.getFlattenedReport(ReportUtil.java:96)
              com.cloudera.flume.reporter.ReportUtil.getFlattenedReport(ReportUtil.java:96)
              com.cloudera.flume.reporter.ReportUtil.getFlattenedReport(ReportUtil.java:96)
              com.cloudera.flume.reporter.ReportUtil.getFlattenedReport(ReportUtil.java:96)
              com.cloudera.flume.reporter.ReportUtil.getFlattenedReport(ReportUtil.java:96)
              com.cloudera.flume.reporter.ReportUtil.getFlattenedReport(ReportUtil.java:96)
              com.cloudera.flume.reporter.ReportUtil.getFlattenedReport(ReportUtil.java:96)
              com.cloudera.flume.agent.FlumeNode.report(FlumeNode.java:391)
              org.apache.jsp.flumeagent_jsp._jspService(flumeagent_jsp.java:57)
              org.apache.jasper.runtime.HttpJspBase.service(HttpJspBase.java:94)
              javax.servlet.http.HttpServlet.service(HttpServlet.java:802)
              org.mortbay.jetty.servlet.ServletHolder.handle(ServletHolder.java:511)
              org.mortbay.jetty.servlet.ServletHandler.handle(ServletHandler.java:390)
              org.mortbay.jetty.security.SecurityHandler.handle(SecurityHandler.java:216)
              org.mortbay.jetty.servlet.SessionHandler.handle(SessionHandler.java:182)
          Thread 107418 (Roll-TriggerThread-133):
            State: WAITING
            Blocked count: 10846
            Waited count: 213933
            Waiting on java.util.LinkedList@4cf877ca
            Stack:
              java.lang.Object.wait(Native Method)
              java.lang.Object.wait(Object.java:485)
              org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.waitForAckedSeqno(DFSClient.java:3468)
              org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.flushInternal(DFSClient.java:3457)
              org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.closeInternal(DFSClient.java:3549)
              org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.close(DFSClient.java:3488)
              org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:61)
              org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:86)
              com.cloudera.flume.handlers.hdfs.CustomDfsSink.close(CustomDfsSink.java:87)
              com.cloudera.flume.handlers.hdfs.EscapedCustomDfsSink.close(EscapedCustomDfsSink.java:130)
              com.cloudera.flume.core.FanOutSink.close(FanOutSink.java:85)
              com.cloudera.flume.core.CompositeSink.close(CompositeSink.java:56)
              com.cloudera.flume.core.EventSinkDecorator.close(EventSinkDecorator.java:67)
              com.cloudera.flume.collector.CollectorSink$RollDetectDeco.close(CollectorSink.java:166)
              com.cloudera.flume.handlers.rolling.RollSink.rotate(RollSink.java:188)
              com.cloudera.flume.handlers.rolling.RollSink$TriggerThread.run(RollSink.java:126)
          Thread 103359 (logicalNode collectjenkins-103359):
            State: BLOCKED
            Blocked count: 3975
            Waited count: 603667
            Blocked on com.cloudera.flume.collector.CollectorSink$1@724f782e
            Blocked by 107418 (Roll-TriggerThread-133)
            Stack:
              com.cloudera.flume.handlers.rolling.RollSink.append(RollSink.java:173)
              com.cloudera.flume.core.EventSinkDecorator.append(EventSinkDecorator.java:60)
              com.cloudera.flume.core.MaskDecorator.append(MaskDecorator.java:43)
              com.cloudera.flume.core.EventSinkDecorator.append(EventSinkDecorator.java:60)
              com.cloudera.flume.handlers.debug.InsistentOpenDecorator.append(InsistentOpenDecorator.java:164)
              com.cloudera.flume.core.EventSinkDecorator.append(EventSinkDecorator.java:60)
              com.cloudera.flume.handlers.debug.StubbornAppendSink.append(StubbornAppendSink.java:72)
              com.cloudera.flume.core.EventSinkDecorator.append(EventSinkDecorator.java:60)
              com.cloudera.flume.handlers.debug.InsistentAppendDecorator.append(InsistentAppendDecorator.java:108)
              com.cloudera.flume.core.EventSinkDecorator.append(EventSinkDecorator.java:60)
              com.cloudera.flume.handlers.endtoend.AckChecksumChecker.append(AckChecksumChecker.java:172)
              com.cloudera.flume.collector.CollectorSink.append(CollectorSink.java:217)
              com.cloudera.flume.core.EventSinkDecorator.append(EventSinkDecorator.java:60)
              com.cloudera.flume.handlers.debug.LazyOpenDecorator.append(LazyOpenDecorator.java:75)
              com.cloudera.flume.core.connector.DirectDriver$PumperThread.run(DirectDriver.java:93)
          Thread 10 (Thread-1):
            State: BLOCKED
            Blocked count: 8850
            Waited count: 375495
            Blocked on com.cloudera.flume.collector.CollectorSink$1@724f782e
            Blocked by 107418 (Roll-TriggerThread-133)
            Stack:
              com.cloudera.flume.handlers.rolling.RollSink.getReport(RollSink.java:276)
              com.cloudera.flume.core.EventSink$Base.getReports(EventSink.java:172)
              com.cloudera.flume.handlers.rolling.RollSink.getReports(RollSink.java:286)
              com.cloudera.flume.core.EventSinkDecorator.getReports(EventSinkDecorator.java:97)
              com.cloudera.flume.core.EventSinkDecorator.getReports(EventSinkDecorator.java:97)
              com.cloudera.flume.core.EventSinkDecorator.getReports(EventSinkDecorator.java:97)
              com.cloudera.flume.core.EventSinkDecorator.getReports(EventSinkDecorator.java:97)
              com.cloudera.flume.core.EventSinkDecorator.getReports(EventSinkDecorator.java:97)
              com.cloudera.flume.collector.CollectorSink.getReports(CollectorSink.java:253)
              com.cloudera.flume.core.EventSinkDecorator.getReports(EventSinkDecorator.java:97)
              com.cloudera.flume.agent.LogicalNode.getReports(LogicalNode.java:386)
              com.cloudera.flume.reporter.MasterReportPusher$PusherThread.querySrcSinkReports(MasterReportPusher.java:102)
              com.cloudera.flume.reporter.MasterReportPusher$PusherThread.sendReports(MasterReportPusher.java:110)
              com.cloudera.flume.reporter.MasterReportPusher$PusherThread.run(MasterReportPusher.java:119)
          
          
          Show
          Jonathan Hsieh added a comment - Here is a sample of the blocked threads on the agent. Thread 153903 (1621610227@qtp-2005327631-17): State: BLOCKED Blocked count: 1 Waited count: 0 Blocked on com.cloudera.flume.collector.CollectorSink$1@724f782e Blocked by 107418 (Roll-TriggerThread-133) Stack: com.cloudera.flume.handlers.rolling.RollSink.getMetrics(RollSink.java:256) com.cloudera.flume.reporter.ReportUtil.getFlattenedReport(ReportUtil.java:92) com.cloudera.flume.reporter.ReportUtil.getFlattenedReport(ReportUtil.java:96) com.cloudera.flume.reporter.ReportUtil.getFlattenedReport(ReportUtil.java:96) com.cloudera.flume.reporter.ReportUtil.getFlattenedReport(ReportUtil.java:96) com.cloudera.flume.reporter.ReportUtil.getFlattenedReport(ReportUtil.java:96) com.cloudera.flume.reporter.ReportUtil.getFlattenedReport(ReportUtil.java:96) com.cloudera.flume.reporter.ReportUtil.getFlattenedReport(ReportUtil.java:96) com.cloudera.flume.reporter.ReportUtil.getFlattenedReport(ReportUtil.java:96) com.cloudera.flume.reporter.ReportUtil.getFlattenedReport(ReportUtil.java:96) com.cloudera.flume.reporter.ReportUtil.getFlattenedReport(ReportUtil.java:96) com.cloudera.flume.reporter.ReportUtil.getFlattenedReport(ReportUtil.java:96) com.cloudera.flume.agent.FlumeNode.report(FlumeNode.java:391) org.apache.jsp.flumeagent_jsp._jspService(flumeagent_jsp.java:57) org.apache.jasper.runtime.HttpJspBase.service(HttpJspBase.java:94) javax.servlet.http.HttpServlet.service(HttpServlet.java:802) org.mortbay.jetty.servlet.ServletHolder.handle(ServletHolder.java:511) org.mortbay.jetty.servlet.ServletHandler.handle(ServletHandler.java:390) org.mortbay.jetty.security.SecurityHandler.handle(SecurityHandler.java:216) org.mortbay.jetty.servlet.SessionHandler.handle(SessionHandler.java:182) Thread 107418 (Roll-TriggerThread-133): State: WAITING Blocked count: 10846 Waited count: 213933 Waiting on java.util.LinkedList@4cf877ca Stack: java.lang. Object .wait(Native Method) java.lang. Object .wait( Object .java:485) org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.waitForAckedSeqno(DFSClient.java:3468) org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.flushInternal(DFSClient.java:3457) org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.closeInternal(DFSClient.java:3549) org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.close(DFSClient.java:3488) org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:61) org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:86) com.cloudera.flume.handlers.hdfs.CustomDfsSink.close(CustomDfsSink.java:87) com.cloudera.flume.handlers.hdfs.EscapedCustomDfsSink.close(EscapedCustomDfsSink.java:130) com.cloudera.flume.core.FanOutSink.close(FanOutSink.java:85) com.cloudera.flume.core.CompositeSink.close(CompositeSink.java:56) com.cloudera.flume.core.EventSinkDecorator.close(EventSinkDecorator.java:67) com.cloudera.flume.collector.CollectorSink$RollDetectDeco.close(CollectorSink.java:166) com.cloudera.flume.handlers.rolling.RollSink.rotate(RollSink.java:188) com.cloudera.flume.handlers.rolling.RollSink$TriggerThread.run(RollSink.java:126) Thread 103359 (logicalNode collectjenkins-103359): State: BLOCKED Blocked count: 3975 Waited count: 603667 Blocked on com.cloudera.flume.collector.CollectorSink$1@724f782e Blocked by 107418 (Roll-TriggerThread-133) Stack: com.cloudera.flume.handlers.rolling.RollSink.append(RollSink.java:173) com.cloudera.flume.core.EventSinkDecorator.append(EventSinkDecorator.java:60) com.cloudera.flume.core.MaskDecorator.append(MaskDecorator.java:43) com.cloudera.flume.core.EventSinkDecorator.append(EventSinkDecorator.java:60) com.cloudera.flume.handlers.debug.InsistentOpenDecorator.append(InsistentOpenDecorator.java:164) com.cloudera.flume.core.EventSinkDecorator.append(EventSinkDecorator.java:60) com.cloudera.flume.handlers.debug.StubbornAppendSink.append(StubbornAppendSink.java:72) com.cloudera.flume.core.EventSinkDecorator.append(EventSinkDecorator.java:60) com.cloudera.flume.handlers.debug.InsistentAppendDecorator.append(InsistentAppendDecorator.java:108) com.cloudera.flume.core.EventSinkDecorator.append(EventSinkDecorator.java:60) com.cloudera.flume.handlers.endtoend.AckChecksumChecker.append(AckChecksumChecker.java:172) com.cloudera.flume.collector.CollectorSink.append(CollectorSink.java:217) com.cloudera.flume.core.EventSinkDecorator.append(EventSinkDecorator.java:60) com.cloudera.flume.handlers.debug.LazyOpenDecorator.append(LazyOpenDecorator.java:75) com.cloudera.flume.core.connector.DirectDriver$PumperThread.run(DirectDriver.java:93) Thread 10 ( Thread -1): State: BLOCKED Blocked count: 8850 Waited count: 375495 Blocked on com.cloudera.flume.collector.CollectorSink$1@724f782e Blocked by 107418 (Roll-TriggerThread-133) Stack: com.cloudera.flume.handlers.rolling.RollSink.getReport(RollSink.java:276) com.cloudera.flume.core.EventSink$Base.getReports(EventSink.java:172) com.cloudera.flume.handlers.rolling.RollSink.getReports(RollSink.java:286) com.cloudera.flume.core.EventSinkDecorator.getReports(EventSinkDecorator.java:97) com.cloudera.flume.core.EventSinkDecorator.getReports(EventSinkDecorator.java:97) com.cloudera.flume.core.EventSinkDecorator.getReports(EventSinkDecorator.java:97) com.cloudera.flume.core.EventSinkDecorator.getReports(EventSinkDecorator.java:97) com.cloudera.flume.core.EventSinkDecorator.getReports(EventSinkDecorator.java:97) com.cloudera.flume.collector.CollectorSink.getReports(CollectorSink.java:253) com.cloudera.flume.core.EventSinkDecorator.getReports(EventSinkDecorator.java:97) com.cloudera.flume.agent.LogicalNode.getReports(LogicalNode.java:386) com.cloudera.flume.reporter.MasterReportPusher$PusherThread.querySrcSinkReports(MasterReportPusher.java:102) com.cloudera.flume.reporter.MasterReportPusher$PusherThread.sendReports(MasterReportPusher.java:110) com.cloudera.flume.reporter.MasterReportPusher$PusherThread.run(MasterReportPusher.java:119)
          Show
          Jonathan Hsieh added a comment - This may be the root cause of these problems: http://pastebin.com/UjKAGnd3 https://groups.google.com/a/cloudera.org/group/flume-user/browse_thread/thread/c07a2f62fec08b44/1bfd45707fd15726?lnk=gst&q=NaiveFileWALManager+error#1bfd45707fd15726https://groups.google.com/a/cloudera.org/group/flume-user/browse_thread/thread/c07a2f62fec08b44/1bfd45707fd15726?lnk=gst&q=NaiveFileWALManager+error#1bfd45707fd15726
          Hide
          Jonathan Hsieh added a comment -

          Here's how I think the hang occurs:

          A rotate holds a lock on the RollSink. rotate closes the current subsink and opens a new subsink. If either the close or the open fail and hang (because of retrying or bad connections), the lock on RollSink remains held.

          RollSink.getMetrics and getReports, as well as parts of append hold the RollSink lock. These threads are thus blocked until the RollSink clears.

          Show
          Jonathan Hsieh added a comment - Here's how I think the hang occurs: A rotate holds a lock on the RollSink. rotate closes the current subsink and opens a new subsink. If either the close or the open fail and hang (because of retrying or bad connections), the lock on RollSink remains held. RollSink.getMetrics and getReports, as well as parts of append hold the RollSink lock. These threads are thus blocked until the RollSink clears.
          Hide
          Jonathan Hsieh added a comment -

          I think that flume-543 is likely related.

          Show
          Jonathan Hsieh added a comment - I think that flume-543 is likely related.
          Hide
          Jonathan Hsieh added a comment -

          More (likely) story. Since the lock is held by rotate, and retry is a separate thread, eventually, retry attempts to move data between states. A chunk's state isn't protected it results in the illegal state exception.

          Show
          Jonathan Hsieh added a comment - More (likely) story. Since the lock is held by rotate, and retry is a separate thread, eventually, retry attempts to move data between states. A chunk's state isn't protected it results in the illegal state exception.
          Hide
          Jonathan Hsieh added a comment -

          I think this story is going to include some explicit locks – either tryLock(time,unit) or lockInterruptibly(). Fun.

          Show
          Jonathan Hsieh added a comment - I think this story is going to include some explicit locks – either tryLock(time,unit) or lockInterruptibly(). Fun.
          Hide
          Jonathan Hsieh added a comment -

          So part of this problem is solved by making the two remaining synchronized getMetrics calls unsynchrnoized: RollSink, and EventSink.Base.

          Show
          Jonathan Hsieh added a comment - So part of this problem is solved by making the two remaining synchronized getMetrics calls unsynchrnoized: RollSink, and EventSink.Base.
          Hide
          Jonathan Hsieh added a comment -

          Since the interesting comments and traces are here, I will create a new issue for unsynchronizing the getMetrics call, and rename this jira somethign like: RollSink holds lock if unable to reopen downstream sink.

          Show
          Jonathan Hsieh added a comment - Since the interesting comments and traces are here, I will create a new issue for unsynchronizing the getMetrics call, and rename this jira somethign like: RollSink holds lock if unable to reopen downstream sink.
          Hide
          Disabled imported user added a comment - - edited

          I think I'm seeing a similar issue, and I've attached the stacks output in case it helps: https://gist.github.com/1018157

          I also have a question: given the fact that this bug currently exists and is not yet fixed, what's the best way to get my agent nodes in a working state again?

          Show
          Disabled imported user added a comment - - edited I think I'm seeing a similar issue, and I've attached the stacks output in case it helps: https://gist.github.com/1018157 I also have a question: given the fact that this bug currently exists and is not yet fixed, what's the best way to get my agent nodes in a working state again?
          Hide
          Disabled imported user added a comment -

          Ugh, I forgot to wipe hostname information from my output before attaching. Can someone delete my attachment for me? I don't have delete privileges.

          You can see the stacks output here post-delete: https://gist.github.com/1018157

          Show
          Disabled imported user added a comment - Ugh, I forgot to wipe hostname information from my output before attaching. Can someone delete my attachment for me? I don't have delete privileges. You can see the stacks output here post-delete: https://gist.github.com/1018157
          Hide
          Jonathan Hsieh added a comment -

          removed.

          Show
          Jonathan Hsieh added a comment - removed.
          Hide
          Disabled imported user added a comment -

          Thanks Jonathan. Do you have any thoughts on how to get the agent node into a good state again? I've tried restarting the node as well as stopping the node, removing the contents of the write-ahead log directory, and then restarting the node, but it reenters the deadlock state immediately.

          Show
          Disabled imported user added a comment - Thanks Jonathan. Do you have any thoughts on how to get the agent node into a good state again? I've tried restarting the node as well as stopping the node, removing the contents of the write-ahead log directory, and then restarting the node, but it reenters the deadlock state immediately.
          Hide
          Jonathan Hsieh added a comment -

          Christopher,

          What mode are you in? (i suspect that dfo and e2e mode are less susceptible, and BE is more susceptible to this)?

          This is essentially a blocker for the next release. I've isolated why it happens, need to figure out what to do to prevent it from being a problem.

          The condition I can reproduce it with has to do with when a down-stream connection isn't available. So if you have an agent trying to send to a destination that isn't up. It will fail. Eventually and roll happens, which grabs a lock that effectively blocks the reports until the connection makes it through. The lock isn't released until a connection makes it through which blocks the other threads.

          Jon.

          Show
          Jonathan Hsieh added a comment - Christopher, What mode are you in? (i suspect that dfo and e2e mode are less susceptible, and BE is more susceptible to this)? This is essentially a blocker for the next release. I've isolated why it happens, need to figure out what to do to prevent it from being a problem. The condition I can reproduce it with has to do with when a down-stream connection isn't available. So if you have an agent trying to send to a destination that isn't up. It will fail. Eventually and roll happens, which grabs a lock that effectively blocks the reports until the connection makes it through. The lock isn't released until a connection makes it through which blocks the other threads. Jon.
          Hide
          Disabled imported user added a comment -

          Jon,

          I run both BE and E2E nodes. It's unclear to me which node triggered the deadlock.

          I suspect but have not been able to confirm that this started when an HDFS file opened by a Flume collector was deleted by another job. I'm not sure how this affects what the collector sends back to the agent. Is this scenario a subset of the scenario you describe?

          Thanks for the quick responses!

          Christopher

          Show
          Disabled imported user added a comment - Jon, I run both BE and E2E nodes. It's unclear to me which node triggered the deadlock. I suspect but have not been able to confirm that this started when an HDFS file opened by a Flume collector was deleted by another job. I'm not sure how this affects what the collector sends back to the agent. Is this scenario a subset of the scenario you describe? Thanks for the quick responses! Christopher
          Hide
          Jonathan Hsieh added a comment -

          I now have a fairly generic solution that preserves existing semantics (at least in existing tests) but new correctly allows the roller to escape from what essentially is a deadlock. It is a bit cumbersome because use an executor service to turn a synchronous method into something that is asynchronously interruptible. Need to write more tests to validate this approach.

          Show
          Jonathan Hsieh added a comment - I now have a fairly generic solution that preserves existing semantics (at least in existing tests) but new correctly allows the roller to escape from what essentially is a deadlock. It is a bit cumbersome because use an executor service to turn a synchronous method into something that is asynchronously interruptible. Need to write more tests to validate this approach.
          Hide
          Jonathan Hsieh added a comment -

          Christopher,

          I believe that issue you mention has been fixed by FLUME-595. (all threads aren't spawned anymore and the info that used to spawn them are stored in a queue).

          Jon.

          Show
          Jonathan Hsieh added a comment - Christopher, I believe that issue you mention has been fixed by FLUME-595 . (all threads aren't spawned anymore and the info that used to spawn them are stored in a queue). Jon.

            People

            • Assignee:
              Unassigned
              Reporter:
              Jonathan Hsieh
            • Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

              • Created:
                Updated:

                Development