Uploaded image for project: 'Ratis'
  1. Ratis
  2. RATIS-1497

Avoid using ForkJoinPool.commonPool() in GrpcClientProtocolService

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 2.2.0
    • 2.3.0
    • gRPC, server
    • None

    Description

      TestRaftAsyncWithGrpc times out locally (but not in CI).

      -------------------------------------------------------------------------------
      Test set: org.apache.ratis.grpc.TestRaftAsyncWithGrpc
      -------------------------------------------------------------------------------
      Tests run: 13, Failures: 0, Errors: 7, Skipped: 0, Time elapsed: 490.435 s <<< FAILURE! - in org.apache.ratis.grpc.TestRaftAsyncWithGrpc
      testWithLoadAsync(org.apache.ratis.grpc.TestRaftAsyncWithGrpc)  Time elapsed: 100.017 s  <<< ERROR!
      org.junit.runners.model.TestTimedOutException: test timed out after 100 seconds
      	at java.lang.Thread.sleep(Native Method)
      	at org.apache.ratis.RaftBasicTests.testWithLoad(RaftBasicTests.java:402)
      	at org.apache.ratis.RaftAsyncTests.lambda$testWithLoadAsync$6(RaftAsyncTests.java:271)
      	at org.apache.ratis.RaftAsyncTests$$Lambda$946/230474748.accept(Unknown Source)
      	at org.apache.ratis.server.impl.MiniRaftCluster$Factory$Get.runWithNewCluster(MiniRaftCluster.java:125)
      	at org.apache.ratis.server.impl.MiniRaftCluster$Factory$Get.runWithNewCluster(MiniRaftCluster.java:113)
      	at org.apache.ratis.RaftAsyncTests.testWithLoadAsync(RaftAsyncTests.java:270)
      	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 org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
      	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
      	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
      	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
      	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
      	at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:298)
      	at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:292)
      	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
      	at java.lang.Thread.run(Thread.java:748)
      
      testRequestTimeout(org.apache.ratis.grpc.TestRaftAsyncWithGrpc)  Time elapsed: 100.002 s  <<< ERROR!
      org.junit.runners.model.TestTimedOutException: test timed out after 100 seconds
      	at sun.misc.Unsafe.park(Native Method)
      	at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
      	at java.util.concurrent.CompletableFuture$Signaller.block(CompletableFuture.java:1693)
      	at java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3323)
      	at java.util.concurrent.CompletableFuture.waitingGet(CompletableFuture.java:1729)
      	at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1895)
      	at org.apache.ratis.RaftBasicTests.testRequestTimeout(RaftBasicTests.java:444)
      	at org.apache.ratis.RaftAsyncTests.lambda$testRequestTimeout$11(RaftAsyncTests.java:362)
      	at org.apache.ratis.RaftAsyncTests$$Lambda$985/1750840351.accept(Unknown Source)
      	at org.apache.ratis.server.impl.MiniRaftCluster$Factory$Get.runWithNewCluster(MiniRaftCluster.java:125)
      	at org.apache.ratis.server.impl.MiniRaftCluster$Factory$Get.runWithNewCluster(MiniRaftCluster.java:113)
      	at org.apache.ratis.RaftAsyncTests.testRequestTimeout(RaftAsyncTests.java:362)
      	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 org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
      	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
      	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
      	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
      	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
      	at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:298)
      	at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:292)
      	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
      	at java.lang.Thread.run(Thread.java:748)
      
      testAsyncRequestSemaphore(org.apache.ratis.grpc.TestRaftAsyncWithGrpc)  Time elapsed: 100.001 s  <<< ERROR!
      org.junit.runners.model.TestTimedOutException: test timed out after 100 seconds
      	at java.lang.Thread.sleep(Native Method)
      	at org.apache.ratis.RaftAsyncTests.runTestAsyncRequestSemaphore(RaftAsyncTests.java:234)
      	at org.apache.ratis.RaftAsyncTests$$Lambda$987/96135248.accept(Unknown Source)
      	at org.apache.ratis.server.impl.MiniRaftCluster$Factory$Get.runWithNewCluster(MiniRaftCluster.java:125)
      	at org.apache.ratis.server.impl.MiniRaftCluster$Factory$Get.runWithNewCluster(MiniRaftCluster.java:113)
      	at org.apache.ratis.RaftAsyncTests.testAsyncRequestSemaphore(RaftAsyncTests.java:200)
      	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 org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
      	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
      	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
      	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
      	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
      	at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:298)
      	at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:292)
      	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
      	at java.lang.Thread.run(Thread.java:748)
      
      testBasicAppendEntriesAsync(org.apache.ratis.grpc.TestRaftAsyncWithGrpc)  Time elapsed: 100.006 s  <<< ERROR!
      org.junit.runners.model.TestTimedOutException: test timed out after 100 seconds
      	at sun.misc.Unsafe.park(Native Method)
      	at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
      	at java.util.concurrent.CompletableFuture$Signaller.block(CompletableFuture.java:1693)
      	at java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3323)
      	at java.util.concurrent.CompletableFuture.waitingGet(CompletableFuture.java:1729)
      	at java.util.concurrent.CompletableFuture.join(CompletableFuture.java:1934)
      	at org.apache.ratis.RaftBasicTests.runTestBasicAppendEntries(RaftBasicTests.java:160)
      	at org.apache.ratis.RaftAsyncTests.lambda$runTestBasicAppendEntriesAsync$5(RaftAsyncTests.java:255)
      	at org.apache.ratis.RaftAsyncTests$$Lambda$907/371775439.accept(Unknown Source)
      	at org.apache.ratis.server.impl.MiniRaftCluster$Factory$Get.runWithNewCluster(MiniRaftCluster.java:125)
      	at org.apache.ratis.server.impl.MiniRaftCluster$Factory$Get.runWithNewCluster(MiniRaftCluster.java:113)
      	at org.apache.ratis.RaftAsyncTests.runTestBasicAppendEntriesAsync(RaftAsyncTests.java:254)
      	at org.apache.ratis.RaftAsyncTests.testBasicAppendEntriesAsync(RaftAsyncTests.java:260)
      	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 org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
      	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
      	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
      	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
      	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
      	at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:298)
      	at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:292)
      	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
      	at java.lang.Thread.run(Thread.java:748)
      
      testNoRetryWaitOnNotLeaderException(org.apache.ratis.grpc.TestRaftAsyncWithGrpc)  Time elapsed: 5.486 s  <<< ERROR!
      java.util.concurrent.TimeoutException
      	at java.util.concurrent.CompletableFuture.timedGet(CompletableFuture.java:1771)
      	at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1915)
      	at org.apache.ratis.util.TimeDuration.apply(TimeDuration.java:294)
      	at org.apache.ratis.RaftAsyncTests.runTestNoRetryWaitOnNotLeaderException(RaftAsyncTests.java:476)
      	at org.apache.ratis.server.impl.MiniRaftCluster$Factory$Get.runWithNewCluster(MiniRaftCluster.java:125)
      	at org.apache.ratis.server.impl.MiniRaftCluster$Factory$Get.runWithNewCluster(MiniRaftCluster.java:113)
      	at org.apache.ratis.RaftAsyncTests.testNoRetryWaitOnNotLeaderException(RaftAsyncTests.java:461)
      	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 org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
      	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
      	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
      	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
      	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
      	at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:298)
      	at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:292)
      	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
      	at java.lang.Thread.run(Thread.java:748)
      
      testRequestAsyncWithRetryFailureAfterInitialMessages(org.apache.ratis.grpc.TestRaftAsyncWithGrpc)  Time elapsed: 13.49 s  <<< ERROR!
      java.util.concurrent.TimeoutException
      	at java.util.concurrent.CompletableFuture.timedGet(CompletableFuture.java:1771)
      	at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1915)
      	at org.apache.ratis.RaftTestUtil.assertSuccessReply(RaftTestUtil.java:512)
      	at org.apache.ratis.RaftAsyncTests.runTestRequestAsyncWithRetryFailure(RaftAsyncTests.java:142)
      	at org.apache.ratis.RaftAsyncTests.lambda$runTestRequestAsyncWithRetryFailure$0(RaftAsyncTests.java:123)
      	at org.apache.ratis.server.impl.MiniRaftCluster$Factory$Get.runWithNewCluster(MiniRaftCluster.java:125)
      	at org.apache.ratis.RaftAsyncTests.runTestRequestAsyncWithRetryFailure(RaftAsyncTests.java:123)
      	at org.apache.ratis.RaftAsyncTests.testRequestAsyncWithRetryFailureAfterInitialMessages(RaftAsyncTests.java:118)
      	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 org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
      	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
      	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
      	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
      	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
      	at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:298)
      	at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:292)
      	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
      	at java.lang.Thread.run(Thread.java:748)
      
      testCheckLeadershipFailure(org.apache.ratis.grpc.TestRaftAsyncWithGrpc)  Time elapsed: 55.569 s  <<< ERROR!
      java.lang.IllegalStateException: 
      No leader yet : printing ALL groups
        s0:  RUNNING  FOLLOWER s0@group-B94298B6D27D:t6, leader=s2, voted=s2, raftlog=s0@group-B94298B6D27D-SegmentedRaftLog:OPENED:c-1, conf=5: [s0|rpc:localhost:52827|admin:localhost:52828|client:localhost:52829|dataStream:localhost:52830|priority:0, s1|rpc:localhost:52831|admin:localhost:52832|client:localhost:52833|dataStream:localhost:52834|priority:0, s2|rpc:localhost:52835|admin:localhost:52836|client:localhost:52837|dataStream:localhost:52838|priority:0], old=null RUNNING
        s1:  RUNNING  FOLLOWER s1@group-B94298B6D27D:t6, leader=s2, voted=s2, raftlog=s1@group-B94298B6D27D-SegmentedRaftLog:OPENED:c-1, conf=5: [s0|rpc:localhost:52827|admin:localhost:52828|client:localhost:52829|dataStream:localhost:52830|priority:0, s1|rpc:localhost:52831|admin:localhost:52832|client:localhost:52833|dataStream:localhost:52834|priority:0, s2|rpc:localhost:52835|admin:localhost:52836|client:localhost:52837|dataStream:localhost:52838|priority:0], old=null RUNNING
        s2:  RUNNING  FOLLOWER s2@group-B94298B6D27D:t6, leader=s2, voted=s2, raftlog=s2@group-B94298B6D27D-SegmentedRaftLog:OPENED:c0, conf=5: [s0|rpc:localhost:52827|admin:localhost:52828|client:localhost:52829|dataStream:localhost:52830|priority:0, s1|rpc:localhost:52831|admin:localhost:52832|client:localhost:52833|dataStream:localhost:52834|priority:0, s2|rpc:localhost:52835|admin:localhost:52836|client:localhost:52837|dataStream:localhost:52838|priority:0], old=null RUNNING
      	at org.apache.ratis.server.impl.MiniRaftCluster.newIllegalStateExceptionForNoLeaders(MiniRaftCluster.java:533)
      	at org.apache.ratis.RaftTestUtil.lambda$waitForLeader$1(RaftTestUtil.java:104)
      	at org.apache.ratis.server.impl.MiniRaftCluster.getLeader(MiniRaftCluster.java:567)
      	at org.apache.ratis.server.impl.MiniRaftCluster.getLeader(MiniRaftCluster.java:560)
      	at org.apache.ratis.RaftTestUtil.lambda$waitForLeader$3(RaftTestUtil.java:112)
      	at org.apache.ratis.util.JavaUtils.attempt(JavaUtils.java:166)
      	at org.apache.ratis.util.JavaUtils.attemptRepeatedly(JavaUtils.java:152)
      	at org.apache.ratis.RaftTestUtil.waitForLeader(RaftTestUtil.java:111)
      	at org.apache.ratis.RaftTestUtil.waitForLeader(RaftTestUtil.java:92)
      	at org.apache.ratis.RaftTestUtil.waitForLeader(RaftTestUtil.java:87)
      	at org.apache.ratis.RaftAsyncTests.runTestCheckLeadershipFailure(RaftAsyncTests.java:449)
      	at org.apache.ratis.server.impl.MiniRaftCluster$Factory$Get.runWithNewCluster(MiniRaftCluster.java:125)
      	at org.apache.ratis.server.impl.MiniRaftCluster$Factory$Get.runWithNewCluster(MiniRaftCluster.java:113)
      	at org.apache.ratis.RaftAsyncTests.testCheckLeadershipFailure(RaftAsyncTests.java:414)
      	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 org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
      	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
      	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
      	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
      	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
      	at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:298)
      	at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:292)
      	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
      	at java.lang.Thread.run(Thread.java:748)
      

      Attachments

        1. threaddump.txt
          101 kB
          Attila Doroszlai

        Issue Links

          Activity

            People

              szetszwo Tsz-wo Sze
              adoroszlai Attila Doroszlai
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved:

                Time Tracking

                  Estimated:
                  Original Estimate - Not Specified
                  Not Specified
                  Remaining:
                  Remaining Estimate - 0h
                  0h
                  Logged:
                  Time Spent - 1h 10m
                  1h 10m