Deadlock in Flume Master during execution of command through flume shell
------------------------------------------------------------------------
Key: FLUME-859
URL: https://issues.apache.org/jira/browse/FLUME-859
Project: Flume
Issue Type: Bug
Components: Master
Affects Versions: v0.9.4
Reporter: Aleksei Sudak
Priority: Blocker
Fix For: v0.9.5
Use case:
- there are 5 physical boxes running Flume Node attached to single Flume
Master. These 5 are used as agents on application side to stream the logs to
the Cloud
- there are 4 physical boxes more running Flume Node attached to the same
single Flume Master. These 4 are used as collectors on Cloud side to write logs
to HDFS
- there are around 200 logical nodes (agents and collectors) configured on
these 9 Flume Nodes.
- during deployment configuration was executed for all of these 200 logical
nodes (sequentially mostly, in some cases 2 in parallel)
- configuration for each logical node consists of the following steps:
unconfig, unmap, decommission, purge, map, config, refreshAll
During execution the following deadlock detected (traces are taken with kill
-SIGQUIT):
Java stack information for the threads listed above:
===================================================
"pool-1-thread-1248":
at
com.cloudera.flume.master.TranslatingConfigurationManager.getLogicalNode(TranslatingConfigurationManager.java:427)
- waiting to lock <0x00000003b03724b0> (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:662)
"exec-thread":
at
com.cloudera.flume.master.StatusManager.getStatus(StatusManager.java:213)
- waiting to lock <0x00000003b03727e8> (a java.util.HashMap)
at
com.cloudera.flume.master.logical.LogicalNameManager.updateNode(LogicalNameManager.java:101)
- locked <0x00000003b0372ac8> (a
com.cloudera.flume.master.logical.LogicalNameManager)
at
com.cloudera.flume.master.logical.LogicalNameManager.update(LogicalNameManager.java:150)
- locked <0x00000003b0372ac8> (a
com.cloudera.flume.master.logical.LogicalNameManager)
at
com.cloudera.flume.master.logical.LogicalConfigurationManager.updateAll(LogicalConfigurationManager.java:236)
at
com.cloudera.flume.master.TranslatingConfigurationManager.unmapLogicalNode(TranslatingConfigurationManager.java:484)
- locked <0x00000003b03724b0> (a
com.cloudera.flume.master.logical.LogicalConfigurationManager)
at
com.cloudera.flume.master.commands.UnmapLogicalNodeForm$1.exec(UnmapLogicalNodeForm.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-37":
at
com.cloudera.flume.master.TranslatingConfigurationManager.getPhysicalNode(TranslatingConfigurationManager.java:474)
- waiting to lock <0x00000003b03724b0> (a
com.cloudera.flume.master.logical.LogicalConfigurationManager)
at
com.cloudera.flume.master.StatusManager.updateHeartbeatStatus(StatusManager.java:97)
- locked <0x00000003b03727e8> (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:662)
Found 1 deadlock.
--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators:
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira