You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@flume.apache.org by Dan Everton <da...@iocaine.org> on 2011/07/27 04:47:01 UTC

Flume master deadlocks after "exec refreshAll"

Just had the Flume master deadlock after executing a refreshAll command.
I'll raise this as in issue in JIRA once the migration is complete, but
here's the stack trace details if someone wants to take a look now:

Found one Java-level deadlock:
=============================
"pool-1-thread-495":
  waiting to lock monitor 0x000000005e883880 (object 0x00002aaaba41c540,
  a com.cloudera.flume.master.logical.LogicalConfigurationManager),
  which is held by "exec-thread"
"exec-thread":
  waiting to lock monitor 0x000000005f5e83c8 (object 0x00002aaaba436a00,
  a java.util.HashMap),
  which is held by "pool-1-thread-13"
"pool-1-thread-13":
  waiting to lock monitor 0x000000005e883880 (object 0x00002aaaba41c540,
  a com.cloudera.flume.master.logical.LogicalConfigurationManager),
  which is held by "exec-thread"

Java stack information for the threads listed above:
===================================================
"pool-1-thread-495":
        at
        com.cloudera.flume.master.TranslatingConfigurationManager.getLogicalNode(TranslatingConfigurationManager.java:427)
        - waiting to lock <0x00002aaaba41c540> (a
        com.cloudera.flume.master.logical.LogicalConfigurationManager)
        at
        com.cloudera.flume.master.MasterClientServer.getLogicalNodes(MasterClientServer.java:83)
        at
        com.cloudera.flume.master.MasterClientServerThrift.getLogicalNodes(MasterClientServerThrift.java:62)
        at
        com.cloudera.flume.conf.thrift.ThriftFlumeClientServer$Processor$getLogicalNodes.process(ThriftFlumeClientServer.java:714)
        at
        com.cloudera.flume.conf.thrift.ThriftFlumeClientServer$Processor.process(ThriftFlumeClientServer.java:640)
        at
        org.apache.thrift.server.TSaneThreadPoolServer$WorkerProcess.run(TSaneThreadPoolServer.java:280)
        at
        java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
        at
        java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
        at java.lang.Thread.run(Thread.java:619)
"exec-thread":
        at
        com.cloudera.flume.master.StatusManager.getStatus(StatusManager.java:214)
        - waiting to lock <0x00002aaaba436a00> (a java.util.HashMap)
        at
        com.cloudera.flume.master.logical.LogicalNameManager.updateNode(LogicalNameManager.java:101)
        - locked <0x00002aaaba436da0> (a
        com.cloudera.flume.master.logical.LogicalNameManager)
        at
        com.cloudera.flume.master.logical.LogicalNameManager.update(LogicalNameManager.java:150)
        - locked <0x00002aaaba436da0> (a
        com.cloudera.flume.master.logical.LogicalNameManager)
        at
        com.cloudera.flume.master.logical.LogicalConfigurationManager.updateAll(LogicalConfigurationManager.java:236)
        at
        com.cloudera.flume.master.TranslatingConfigurationManager.addLogicalNode(TranslatingConfigurationManager.java:450)
        - locked <0x00002aaaba41c540> (a
        com.cloudera.flume.master.logical.LogicalConfigurationManager)
        at
        com.cloudera.flume.master.commands.CreateLogicalNodeForm$1.exec(CreateLogicalNodeForm.java:71)
        at
        com.cloudera.flume.master.CommandManager.exec(CommandManager.java:266)
        at
        com.cloudera.flume.master.CommandManager.handleCommand(CommandManager.java:205)
        at
        com.cloudera.flume.master.CommandManager$ExecThread.run(CommandManager.java:236)
"pool-1-thread-13":
        at
        com.cloudera.flume.master.TranslatingConfigurationManager.getPhysicalNode(TranslatingConfigurationManager.java:474)
        - waiting to lock <0x00002aaaba41c540> (a
        com.cloudera.flume.master.logical.LogicalConfigurationManager)
        at
        com.cloudera.flume.master.StatusManager.updateHeartbeatStatus(StatusManager.java:97)
        - locked <0x00002aaaba436a00> (a java.util.HashMap)
        at
        com.cloudera.flume.master.MasterClientServer.heartbeat(MasterClientServer.java:117)
        at
        com.cloudera.flume.master.MasterClientServerThrift.heartbeat(MasterClientServerThrift.java:75)
        at
        com.cloudera.flume.conf.thrift.ThriftFlumeClientServer$Processor$heartbeat.process(ThriftFlumeClientServer.java:661)
        at
        com.cloudera.flume.conf.thrift.ThriftFlumeClientServer$Processor.process(ThriftFlumeClientServer.java:640)
        at
        org.apache.thrift.server.TSaneThreadPoolServer$WorkerProcess.run(TSaneThreadPoolServer.java:280)
        at
        java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
        at
        java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
        at java.lang.Thread.run(Thread.java:619)

Found 1 deadlock.