You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ratis.apache.org by "Mukul Kumar Singh (JIRA)" <ji...@apache.org> on 2018/11/09 07:54:00 UTC

[jira] [Created] (RATIS-404) Deadlock in ratis between appendEntries and RaftLogWorker

Mukul Kumar Singh created RATIS-404:
---------------------------------------

             Summary: Deadlock in ratis between appendEntries and RaftLogWorker
                 Key: RATIS-404
                 URL: https://issues.apache.org/jira/browse/RATIS-404
             Project: Ratis
          Issue Type: Bug
          Components: server
    Affects Versions: 0.3.0
            Reporter: Mukul Kumar Singh
             Fix For: 0.3.0


The deadlock happens when the RaftLogWorker queue is completely full.  This happens when the following thread is trying to enqueue holding onto the RaftServerImpl lock.

{code}
"grpc-default-executor-18" #459 daemon prio=5 os_prio=0 tid=0x00007f8cd4a4a000 nid=0x5f6 waiting on condition [0x00007f8c31df2000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x0000000098dd53d0> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
        at java.util.concurrent.ArrayBlockingQueue.put(ArrayBlockingQueue.java:353)
        at org.apache.ratis.server.storage.RaftLogWorker.addIOTask(RaftLogWorker.java:186)
        at org.apache.ratis.server.storage.RaftLogWorker.writeLogEntry(RaftLogWorker.java:300)
        at org.apache.ratis.server.storage.SegmentedRaftLog.appendEntry(SegmentedRaftLog.java:302)
        at org.apache.ratis.server.storage.SegmentedRaftLog.append(SegmentedRaftLog.java:379)
        at org.apache.ratis.server.impl.RaftServerImpl.appendEntriesAsync(RaftServerImpl.java:914)
        - locked <0x000000009893b638> (a org.apache.ratis.server.impl.RaftServerImpl)
        at org.apache.ratis.server.impl.RaftServerImpl.appendEntriesAsync(RaftServerImpl.java:821)
        at org.apache.ratis.server.impl.RaftServerProxy.lambda$appendEntriesAsync$18(RaftServerProxy.java:434)
        at org.apache.ratis.server.impl.RaftServerProxy$$Lambda$310/1439556067.apply(Unknown Source)
        at org.apache.ratis.server.impl.RaftServerProxy.lambda$null$5(RaftServerProxy.java:309)
        at org.apache.ratis.server.impl.RaftServerProxy$$Lambda$176/355487796.get(Unknown Source)
        at org.apache.ratis.util.JavaUtils.callAsUnchecked(JavaUtils.java:82)
        at org.apache.ratis.server.impl.RaftServerProxy.lambda$submitRequest$6(RaftServerProxy.java:309)
        at org.apache.ratis.server.impl.RaftServerProxy$$Lambda$175/1025132044.apply(Unknown Source)
        at java.util.concurrent.CompletableFuture.uniComposeStage(CompletableFuture.java:981)
        at java.util.concurrent.CompletableFuture.thenCompose(CompletableFuture.java:2124)
        at org.apache.ratis.server.impl.RaftServerProxy.submitRequest(RaftServerProxy.java:308)
        at org.apache.ratis.server.impl.RaftServerProxy.appendEntriesAsync(RaftServerProxy.java:434)
        at org.apache.ratis.grpc.server.GrpcServerProtocolService$1.onNext(GrpcServerProtocolService.java:76)
        at org.apache.ratis.grpc.server.GrpcServerProtocolService$1.onNext(GrpcServerProtocolService.java:66)
        at org.apache.ratis.thirdparty.io.grpc.stub.ServerCalls$StreamingServerCallHandler$StreamingServerCallListener.onMessage(ServerCalls.java:248)
        at org.apache.ratis.thirdparty.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.messagesAvailable(ServerCallImpl.java:263)
        at org.apache.ratis.thirdparty.io.grpc.internal.ServerImpl$JumpToApplicationThreadServerStreamListener$1MessagesAvailable.runInContext(ServerImpl.java:683)
        at org.apache.ratis.thirdparty.io.grpc.internal.ContextRunnable.run(ContextRunnable.java:37)
        at org.apache.ratis.thirdparty.io.grpc.internal.SerializingExecutor.run(SerializingExecutor.java:123)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)
{code}


The RaftLogWorker thread is in turn blocked on locking the RaftServerImpl lock as in the following trace.

{code}
"c5a4d441-cb73-47a2-94b5-fc8233061955-RaftLogWorker" #440 daemon prio=5 os_prio=0 tid=0x00000000026a2000 nid=0x5e3 waiting for monitor entry [0x00007f8c884aa000]
   java.lang.Thread.State: BLOCKED (on object monitor)
        at org.apache.ratis.server.impl.RaftServerImpl.lambda$appendEntriesAsync$21(RaftServerImpl.java:925)
        - waiting to lock <0x000000009893b638> (a org.apache.ratis.server.impl.RaftServerImpl)
        at org.apache.ratis.server.impl.RaftServerImpl$$Lambda$316/47202155.apply(Unknown Source)
        at java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:602)
        at java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
        at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
        at java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1962)
        at org.apache.ratis.server.storage.SegmentedRaftLog$Task.done(SegmentedRaftLog.java:83)
        at org.apache.ratis.server.storage.RaftLogWorker.run(RaftLogWorker.java:220)
        at java.lang.Thread.run(Thread.java:748)
{code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)