[ 
https://issues.apache.org/jira/browse/CASSANDRA-13137?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16091819#comment-16091819
 ] 

Sotirios Delimanolis commented on CASSANDRA-13137:
--------------------------------------------------

The PR is merged. Pavel is making a release soon. Can we have it included in 
2.2+?

> nodetool disablethrift deadlocks if THsHaDisruptorServer is stopped while a 
> request is being processed
> ------------------------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-13137
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-13137
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>         Environment: 2.2.9
>            Reporter: Sotirios Delimanolis
>
> We are using Thrift with {{rpc_server_type}} set to {{hsha}}. This creates a 
> {{THsHaDisruptorServer}} which is a subclass of 
> [{{TDisruptorServer}}|https://github.com/xedin/disruptor_thrift_server/blob/master/src/main/java/com/thinkaurelius/thrift/TDisruptorServer.java].
> Internally, this spawns {{number_of_cores}} number of selector threads. Each 
> gets a {{RingBuffer}} and {{rpc_max_threads / cores}} number of worker 
> threads (the {{RPC-Thread}} threads). As the server starts receiving 
> requests, each selector thread adds events to its {{RingBuffer}} and the 
> worker threads process them. 
> The _events_ are 
> [{{Message}}|https://github.com/xedin/disruptor_thrift_server/blob/master/src/main/java/com/thinkaurelius/thrift/Message.java]
>  instances, which have preallocated buffers for eventual IO.
> When the thrift server starts up, the corresponding {{ThriftServerThread}} 
> joins on the selector threads, waiting for them to die. It then iterates 
> through all the {{SelectorThread}} objects and calls their {{shutdown}} 
> method which attempts to drain their corresponding {{RingBuffer}}. The [drain 
> ({{drainAndHalt}})|https://github.com/LMAX-Exchange/disruptor/blob/master/src/main/java/com/lmax/disruptor/WorkerPool.java#L147]
>  works by letting the worker pool "consumer" threads catch up to the 
> "producer" index, ie. the selector thread.
> When we execute a {{nodetool disablethrift}}, it attempts to {{stop}} the 
> {{THsHaDisruptorServer}}. That works by setting a {{stopped}} flag to 
> {{true}}. When the selector threads see that, they break from their 
> {{select()}} loop, and clean up their resources, ie. the {{Message}} objects 
> they've created and their buffers. *However*, if one of those {{Message}} 
> objects is currently being used by a worker pool thread to process a request, 
> if it calls [this piece of 
> code|https://github.com/xedin/disruptor_thrift_server/blob/master/src/main/java/com/thinkaurelius/thrift/Message.java#L317],
>  you'll get the following {{NullPointerException}}
> {noformat}
> Jan 18, 2017 6:28:50 PM com.lmax.disruptor.FatalExceptionHandler 
> handleEventException
> SEVERE: Exception processing: 633124 
> com.thinkaurelius.thrift.Message$Invocation@25c9fbeb
> java.lang.NullPointerException
>         at 
> com.thinkaurelius.thrift.Message.getInputTransport(Message.java:338)
>         at com.thinkaurelius.thrift.Message.invoke(Message.java:308)
>         at 
> com.thinkaurelius.thrift.Message$Invocation.execute(Message.java:90)
>         at 
> com.thinkaurelius.thrift.TDisruptorServer$InvocationHandler.onEvent(TDisruptorServer.java:695)
>         at 
> com.thinkaurelius.thrift.TDisruptorServer$InvocationHandler.onEvent(TDisruptorServer.java:689)
>         at com.lmax.disruptor.WorkProcessor.run(WorkProcessor.java:112)
>         at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>         at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>         at java.lang.Thread.run(Thread.java:745)
> {noformat}
> That fails because it tries to dereference one of the {{Message}} "cleaned 
> up", ie. {{null}}, buffers.
> Because that call is outside the {{try}} block, the exception escapes and 
> basically kills the worker pool thread. This has the side effect of 
> "discarding" one of the consumers of a selector's {{RingBuffer}}. 
> *That* has the side effect of preventing the {{ThriftServerThread}} from 
> draining the {{RingBuffer}} (and dying) since the consumers never catch up to 
> the stopped producer. And that finally has the effect of preventing the 
> {{nodetool disablethrift}} from proceeding since it's trying to {{join}} the 
> {{ThriftServerThread}}. Deadlock!
> The {{ThriftServerThread}} thread looks like
> {noformat}
> "Thread-1" #2234 prio=5 os_prio=0 tid=0x00007f4ae6ff1000 nid=0x2eb6 runnable 
> [0x00007f4729174000]
>    java.lang.Thread.State: RUNNABLE
>         at java.lang.Thread.yield(Native Method)
>         at com.lmax.disruptor.WorkerPool.drainAndHalt(WorkerPool.java:147)
>         at 
> com.thinkaurelius.thrift.TDisruptorServer$SelectorThread.shutdown(TDisruptorServer.java:633)
>         at 
> com.thinkaurelius.thrift.TDisruptorServer.gracefullyShutdownInvokerPool(TDisruptorServer.java:301)
>         at 
> com.thinkaurelius.thrift.TDisruptorServer.waitForShutdown(TDisruptorServer.java:280)
>         at 
> org.apache.thrift.server.AbstractNonblockingServer.serve(AbstractNonblockingServer.java:95)
>         at 
> org.apache.cassandra.thrift.ThriftServer$ThriftServerThread.run(ThriftServer.java:137)
> {noformat}
> The {{nodetool disablethrift}} thread looks like
> {noformat}
> "RMI TCP Connection(18183)-127.0.0.1" #12121 daemon prio=5 os_prio=0 
> tid=0x00007f4ac2c61000 nid=0x5805 in Object.wait() [0x00007f4aab7ec000]
>    java.lang.Thread.State: WAITING (on object monitor)
>         at java.lang.Object.wait(Native Method)
>         at java.lang.Thread.join(Thread.java:1249)
>         - locked <0x000000055d3cb010> (a 
> org.apache.cassandra.thrift.ThriftServer$ThriftServerThread)
>         at java.lang.Thread.join(Thread.java:1323)
>         at org.apache.cassandra.thrift.ThriftServer.stop(ThriftServer.java:70)
>         - locked <0x000000055bffb5e0> (a 
> org.apache.cassandra.thrift.ThriftServer)
>         at 
> org.apache.cassandra.service.StorageService.stopRPCServer(StorageService.java:337)
>         at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>         at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>         at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>         at java.lang.reflect.Method.invoke(Method.java:498)
>         at sun.reflect.misc.Trampoline.invoke(MethodUtil.java:71)
>         at sun.reflect.GeneratedMethodAccessor8.invoke(Unknown Source)
>         at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>         at java.lang.reflect.Method.invoke(Method.java:498)
>         at sun.reflect.misc.MethodUtil.invoke(MethodUtil.java:275)
>         at 
> com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:112)
>         at 
> com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:46)
>         at 
> com.sun.jmx.mbeanserver.MBeanIntrospector.invokeM(MBeanIntrospector.java:237)
>         at com.sun.jmx.mbeanserver.PerInterface.invoke(PerInterface.java:138)
>         at com.sun.jmx.mbeanserver.MBeanSupport.invoke(MBeanSupport.java:252)
>         at 
> com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.invoke(DefaultMBeanServerInterceptor.java:819)
>         at 
> com.sun.jmx.mbeanserver.JmxMBeanServer.invoke(JmxMBeanServer.java:801)
>         at 
> javax.management.remote.rmi.RMIConnectionImpl.doOperation(RMIConnectionImpl.java:1468)
>         at 
> javax.management.remote.rmi.RMIConnectionImpl.access$300(RMIConnectionImpl.java:76)
>         at 
> javax.management.remote.rmi.RMIConnectionImpl$PrivilegedOperation.run(RMIConnectionImpl.java:1309)
>         at 
> javax.management.remote.rmi.RMIConnectionImpl.doPrivilegedOperation(RMIConnectionImpl.java:1401)
>         at 
> javax.management.remote.rmi.RMIConnectionImpl.invoke(RMIConnectionImpl.java:829)
>         at sun.reflect.GeneratedMethodAccessor22.invoke(Unknown Source)
>         at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>         at java.lang.reflect.Method.invoke(Method.java:498)
>         at sun.rmi.server.UnicastServerRef.dispatch(UnicastServerRef.java:324)
>         at sun.rmi.transport.Transport$1.run(Transport.java:200)
>         at sun.rmi.transport.Transport$1.run(Transport.java:197)
>         at java.security.AccessController.doPrivileged(Native Method)
>         at sun.rmi.transport.Transport.serviceCall(Transport.java:196)
>         at 
> sun.rmi.transport.tcp.TCPTransport.handleMessages(TCPTransport.java:568)
>         at 
> sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0(TCPTransport.java:826)
>         at 
> sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.lambda$run$0(TCPTransport.java:683)
>         at 
> sun.rmi.transport.tcp.TCPTransport$ConnectionHandler$$Lambda$1/1038375559.run(Unknown
>  Source)
>         at java.security.AccessController.doPrivileged(Native Method)
>         at 
> sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run(TCPTransport.java:682)
>         at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>         at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>         at java.lang.Thread.run(Thread.java:745)
> {noformat}
> Most of the code involved isn't part of Cassandra source, but it's an 
> external dependency that should be fixed.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to