Details

    • Sub-task
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • None
    • 2.0.0-beta-2, 2.0.0
    • rpc
    • None
    • Reviewed

    Description

      So reading around junit docs (https://github.com/junit-team/junit4/wiki/timeout-for-tests), looks like the reason is result of these two rules:

      • @Test(timeout=X) applies only on the test function, and not on whole test fixture (@After, @Before, etc)
      • Timeout rule applies on whole test fixture

      TestRegionServerReportForDuty just has @Test(timeout=180000) and no Timeout rule unlike we have in so many other tests.
      The test method, in the logs I have, runs in less then 60 sec. So it meets the timeout specified in @Test annotation.
      However, we get stuck in tearDown, and since there is no Timeout rule, it keeps on running until surefire kills the JVM after forkedProcessTimeoutInSeconds (set to 900 sec).
      Let use the "Timeout" rule instead of @Test(timeout=180000).

      However, note that this won't solve the root cause of hangup. It'll just make the test fail neatly rather than getting stuck and requiring surefire plugin to kill the forked JVMs (see HBASE-19803).

      Attachments

        1. HBASE-19866.patch
          2 kB
          Duo Zhang

        Issue Links

          Activity

            As for the root cause, i see that backup master is not stopping.

            Process Thread Dump: Automatic Stack Trace every 60 seconds waiting on M:0;b10d9a4963f1:49296

            M:0;b10d9a4963f1:49296 is backup HMaster thread

            2018-01-25 12:52:52,355 INFO [M:0;b10d9a4963f1:49296] master.HMaster(2006): Adding backup master ZNode /hbase/backup-masters/b10d9a4963f1,49296,1516884770433

            And i think it's stuck on NettyRpcServer?

            Thread 209 (M:0;b10d9a4963f1:49296):
              State: WAITING
              Blocked count: 164
              Waited count: 334
              Waiting on org.apache.hbase.thirdparty.io.netty.channel.group.DefaultChannelGroupFuture@7b59ef17
              Stack:
                java.lang.Object.wait(Native Method)
                java.lang.Object.wait(Object.java:502)
                org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultPromise.awaitUninterruptibly(DefaultPromise.java:254)
                org.apache.hbase.thirdparty.io.netty.channel.group.DefaultChannelGroupFuture.awaitUninterruptibly(DefaultChannelGroupFuture.java:183)
                org.apache.hbase.thirdparty.io.netty.channel.group.DefaultChannelGroupFuture.awaitUninterruptibly(DefaultChannelGroupFuture.java:40)
                org.apache.hadoop.hbase.ipc.NettyRpcServer.stop(NettyRpcServer.java:148)
                org.apache.hadoop.hbase.regionserver.RSRpcServices.stop(RSRpcServices.java:1405)
                org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.java:1134)
                org.apache.hadoop.hbase.master.HMaster.run(HMaster.java:570)
                java.lang.Thread.run(Thread.java:748)
            

            Ping aoxiang stack

            appy Apekshit Sharma added a comment - As for the root cause, i see that backup master is not stopping. Process Thread Dump: Automatic Stack Trace every 60 seconds waiting on M:0;b10d9a4963f1:49296 M:0;b10d9a4963f1:49296 is backup HMaster thread 2018-01-25 12:52:52,355 INFO [M:0;b10d9a4963f1:49296] master.HMaster(2006): Adding backup master ZNode /hbase/backup-masters/b10d9a4963f1,49296,1516884770433 And i think it's stuck on NettyRpcServer? Thread 209 (M:0;b10d9a4963f1:49296): State: WAITING Blocked count: 164 Waited count: 334 Waiting on org.apache.hbase.thirdparty.io.netty.channel.group.DefaultChannelGroupFuture@7b59ef17 Stack: java.lang.Object.wait(Native Method) java.lang.Object.wait(Object.java:502) org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultPromise.awaitUninterruptibly(DefaultPromise.java:254) org.apache.hbase.thirdparty.io.netty.channel.group.DefaultChannelGroupFuture.awaitUninterruptibly(DefaultChannelGroupFuture.java:183) org.apache.hbase.thirdparty.io.netty.channel.group.DefaultChannelGroupFuture.awaitUninterruptibly(DefaultChannelGroupFuture.java:40) org.apache.hadoop.hbase.ipc.NettyRpcServer.stop(NettyRpcServer.java:148) org.apache.hadoop.hbase.regionserver.RSRpcServices.stop(RSRpcServices.java:1405) org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.java:1134) org.apache.hadoop.hbase.master.HMaster.run(HMaster.java:570) java.lang.Thread.run(Thread.java:748) Ping aoxiang stack
            zhangduo Duo Zhang added a comment -

            Is there a full log? I can take a look.

            zhangduo Duo Zhang added a comment - Is there a full log? I can take a look.

            Pushed just the timeout change to branch-2 and master.
            Feel free to keep using this jira for further debugging.

            appy Apekshit Sharma added a comment - Pushed just the timeout change to branch-2 and master. Feel free to keep using this jira for further debugging.
            appy Apekshit Sharma added a comment - - edited

            Deleted my local logs. But i downloaded them from Nightly master. Build Artifacts > test_logs.zip
            Look in any run where it fails (https://builds.apache.org/job/HBase-Find-Flaky-Tests/lastSuccessfulBuild/artifact/dashboard.html)

            appy Apekshit Sharma added a comment - - edited Deleted my local logs. But i downloaded them from Nightly master. Build Artifacts > test_logs.zip Look in any run where it fails ( https://builds.apache.org/job/HBase-Find-Flaky-Tests/lastSuccessfulBuild/artifact/dashboard.html )
            zhangduo Duo Zhang added a comment -

            I need one hour to download the test_logs.zip... The network of my company sucks... Let me try it at home...

            zhangduo Duo Zhang added a comment - I need one hour to download the test_logs.zip... The network of my company sucks... Let me try it at home...
            hudson Hudson added a comment -

            FAILURE: Integrated in Jenkins build HBase-Trunk_matrix #4471 (See https://builds.apache.org/job/HBase-Trunk_matrix/4471/)
            HBASE-19866 Add cateogry based timeout using Timeout Rule to (appy: rev 01c34243fe3ba971fff0fd154f444118597533f1)

            • (edit) hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReportForDuty.java
            hudson Hudson added a comment - FAILURE: Integrated in Jenkins build HBase-Trunk_matrix #4471 (See https://builds.apache.org/job/HBase-Trunk_matrix/4471/ ) HBASE-19866 Add cateogry based timeout using Timeout Rule to (appy: rev 01c34243fe3ba971fff0fd154f444118597533f1) (edit) hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerReportForDuty.java
            zhangduo Duo Zhang added a comment -

            There is a dead lock.

            NettyRpcServer.stop is synchronized, and in the method, we will wait until all the connections to be closed.

            And one of the event loop thread calls RpcServer.authorize which is also synchronized, so it is blocked there and never returns back since we have already locked when calling NettyRpcServer.stop.

            The master thread.

            Thread 209 (M:0;b10d9a4963f1:49296):
              State: WAITING
              Blocked count: 164
              Waited count: 334
              Waiting on org.apache.hbase.thirdparty.io.netty.channel.group.DefaultChannelGroupFuture@7b59ef17
              Stack:
                java.lang.Object.wait(Native Method)
                java.lang.Object.wait(Object.java:502)
                org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultPromise.awaitUninterruptibly(DefaultPromise.java:254)
                org.apache.hbase.thirdparty.io.netty.channel.group.DefaultChannelGroupFuture.awaitUninterruptibly(DefaultChannelGroupFuture.java:183)
                org.apache.hbase.thirdparty.io.netty.channel.group.DefaultChannelGroupFuture.awaitUninterruptibly(DefaultChannelGroupFuture.java:40)
                org.apache.hadoop.hbase.ipc.NettyRpcServer.stop(NettyRpcServer.java:148)
                org.apache.hadoop.hbase.regionserver.RSRpcServices.stop(RSRpcServices.java:1405)
                org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.java:1134)
                org.apache.hadoop.hbase.master.HMaster.run(HMaster.java:570)
                java.lang.Thread.run(Thread.java:748)
            

            The event loop thread.

            Thread 446 (RS-EventLoopGroup-1-4):
              State: BLOCKED
              Blocked count: 1
              Waited count: 0
              Blocked on org.apache.hadoop.hbase.ipc.NettyRpcServer@45c829a7
              Blocked by 209 (M:0;b10d9a4963f1:49296)
              Stack:
                org.apache.hadoop.hbase.ipc.RpcServer.authorize(RpcServer.java:547)
                org.apache.hadoop.hbase.ipc.ServerRpcConnection.authorizeConnection(ServerRpcConnection.java:485)
                org.apache.hadoop.hbase.ipc.ServerRpcConnection.processOneRpc(ServerRpcConnection.java:465)
                org.apache.hadoop.hbase.ipc.NettyServerRpcConnection.process(NettyServerRpcConnection.java:94)
                org.apache.hadoop.hbase.ipc.NettyServerRpcConnection.process(NettyServerRpcConnection.java:78)
                org.apache.hadoop.hbase.ipc.NettyServerRpcConnection.process(NettyServerRpcConnection.java:73)
                org.apache.hadoop.hbase.ipc.NettyRpcServerRequestDecoder.channelRead(NettyRpcServerRequestDecoder.java:62)
                org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362)
                org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348)
                org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:340)
                org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:310)
                org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:297)
                org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.callDecode(ByteToMessageDecoder.java:413)
                org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:265)
                org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362)
                org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348)
                org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:340)
                org.apache.hbase.thirdparty.io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1359)
                org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362)
                org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348)
            
            zhangduo Duo Zhang added a comment - There is a dead lock. NettyRpcServer.stop is synchronized, and in the method, we will wait until all the connections to be closed. And one of the event loop thread calls RpcServer.authorize which is also synchronized, so it is blocked there and never returns back since we have already locked when calling NettyRpcServer.stop. The master thread. Thread 209 (M:0;b10d9a4963f1:49296): State: WAITING Blocked count: 164 Waited count: 334 Waiting on org.apache.hbase.thirdparty.io.netty.channel.group.DefaultChannelGroupFuture@7b59ef17 Stack: java.lang.Object.wait(Native Method) java.lang.Object.wait(Object.java:502) org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultPromise.awaitUninterruptibly(DefaultPromise.java:254) org.apache.hbase.thirdparty.io.netty.channel.group.DefaultChannelGroupFuture.awaitUninterruptibly(DefaultChannelGroupFuture.java:183) org.apache.hbase.thirdparty.io.netty.channel.group.DefaultChannelGroupFuture.awaitUninterruptibly(DefaultChannelGroupFuture.java:40) org.apache.hadoop.hbase.ipc.NettyRpcServer.stop(NettyRpcServer.java:148) org.apache.hadoop.hbase.regionserver.RSRpcServices.stop(RSRpcServices.java:1405) org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.java:1134) org.apache.hadoop.hbase.master.HMaster.run(HMaster.java:570) java.lang.Thread.run(Thread.java:748) The event loop thread. Thread 446 (RS-EventLoopGroup-1-4): State: BLOCKED Blocked count: 1 Waited count: 0 Blocked on org.apache.hadoop.hbase.ipc.NettyRpcServer@45c829a7 Blocked by 209 (M:0;b10d9a4963f1:49296) Stack: org.apache.hadoop.hbase.ipc.RpcServer.authorize(RpcServer.java:547) org.apache.hadoop.hbase.ipc.ServerRpcConnection.authorizeConnection(ServerRpcConnection.java:485) org.apache.hadoop.hbase.ipc.ServerRpcConnection.processOneRpc(ServerRpcConnection.java:465) org.apache.hadoop.hbase.ipc.NettyServerRpcConnection.process(NettyServerRpcConnection.java:94) org.apache.hadoop.hbase.ipc.NettyServerRpcConnection.process(NettyServerRpcConnection.java:78) org.apache.hadoop.hbase.ipc.NettyServerRpcConnection.process(NettyServerRpcConnection.java:73) org.apache.hadoop.hbase.ipc.NettyRpcServerRequestDecoder.channelRead(NettyRpcServerRequestDecoder.java:62) org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362) org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348) org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:340) org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:310) org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:297) org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.callDecode(ByteToMessageDecoder.java:413) org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:265) org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362) org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348) org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:340) org.apache.hbase.thirdparty.io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1359) org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362) org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348)
            zhangduo Duo Zhang added a comment -

            Ping carp84. I think we can use a different lock for authManager?

            zhangduo Duo Zhang added a comment - Ping carp84 . I think we can use a different lock for authManager?
            zhangduo Duo Zhang added a comment -

            A simple patch to eliminate the dead lock.

            Actually we do not need to lock here but the ServiceAuthorizationManager is implemented in hadoop so we can not modify it...
            It uses two volatile field to hold the entries which makes the upper layer must protect it with a lock... What a pity. Maybe we can use our own implementation in the future.

            Thanks.

            zhangduo Duo Zhang added a comment - A simple patch to eliminate the dead lock. Actually we do not need to lock here but the ServiceAuthorizationManager is implemented in hadoop so we can not modify it... It uses two volatile field to hold the entries which makes the upper layer must protect it with a lock... What a pity. Maybe we can use our own implementation in the future. Thanks.
            hadoopqa Hadoop QA added a comment -
            -1 overall



            Vote Subsystem Runtime Comment
            0 reexec 0m 10s Docker mode activated.
                  Prechecks
            0 findbugs 0m 0s Findbugs executables are not available.
            +1 hbaseanti 0m 0s Patch does not have any anti-patterns.
            +1 @author 0m 0s The patch does not contain any @author tags.
            -1 test4tests 0m 0s The patch doesn't appear to include any new or modified tests. Please justify why no new tests are needed for this patch. Also please list what manual steps were performed to verify this patch.
                  master Compile Tests
            +1 mvninstall 4m 45s master passed
            +1 compile 0m 53s master passed
            +1 checkstyle 1m 10s master passed
            +1 shadedjars 6m 13s branch has no errors when building our shaded downstream artifacts.
            +1 javadoc 0m 32s master passed
                  Patch Compile Tests
            +1 mvninstall 4m 47s the patch passed
            +1 compile 0m 43s the patch passed
            +1 javac 0m 43s the patch passed
            +1 checkstyle 1m 9s the patch passed
            +1 whitespace 0m 0s The patch has no whitespace issues.
            +1 shadedjars 5m 16s patch has no errors when building our shaded downstream artifacts.
            +1 hadoopcheck 21m 14s Patch does not cause any errors with Hadoop 2.6.5 2.7.4 or 3.0.0.
            +1 javadoc 0m 33s the patch passed
                  Other Tests
            +1 unit 101m 28s hbase-server in the patch passed.
            +1 asflicense 0m 20s The patch does not generate ASF License warnings.
            143m 1s



            Subsystem Report/Notes
            Docker Client=17.05.0-ce Server=17.05.0-ce Image:yetus/hbase:eee3b01
            JIRA Issue HBASE-19866
            JIRA Patch URL https://issues.apache.org/jira/secure/attachment/12907967/HBASE-19866.patch
            Optional Tests asflicense javac javadoc unit findbugs shadedjars hadoopcheck hbaseanti checkstyle compile
            uname Linux 18b432fbc2a9 3.13.0-139-generic #188-Ubuntu SMP Tue Jan 9 14:43:09 UTC 2018 x86_64 GNU/Linux
            Build tool maven
            Personality /home/jenkins/jenkins-slave/workspace/PreCommit-HBASE-Build/component/dev-support/hbase-personality.sh
            git revision master / a5a8c4f3f2
            maven version: Apache Maven 3.5.2 (138edd61fd100ec658bfa2d307c43b76940a5d7d; 2017-10-18T07:58:13Z)
            Default Java 1.8.0_151
            Test Results https://builds.apache.org/job/PreCommit-HBASE-Build/11211/testReport/
            modules C: hbase-server U: hbase-server
            Console output https://builds.apache.org/job/PreCommit-HBASE-Build/11211/console
            Powered by Apache Yetus 0.6.0 http://yetus.apache.org

            This message was automatically generated.

            hadoopqa Hadoop QA added a comment - -1 overall Vote Subsystem Runtime Comment 0 reexec 0m 10s Docker mode activated.       Prechecks 0 findbugs 0m 0s Findbugs executables are not available. +1 hbaseanti 0m 0s Patch does not have any anti-patterns. +1 @author 0m 0s The patch does not contain any @author tags. -1 test4tests 0m 0s The patch doesn't appear to include any new or modified tests. Please justify why no new tests are needed for this patch. Also please list what manual steps were performed to verify this patch.       master Compile Tests +1 mvninstall 4m 45s master passed +1 compile 0m 53s master passed +1 checkstyle 1m 10s master passed +1 shadedjars 6m 13s branch has no errors when building our shaded downstream artifacts. +1 javadoc 0m 32s master passed       Patch Compile Tests +1 mvninstall 4m 47s the patch passed +1 compile 0m 43s the patch passed +1 javac 0m 43s the patch passed +1 checkstyle 1m 9s the patch passed +1 whitespace 0m 0s The patch has no whitespace issues. +1 shadedjars 5m 16s patch has no errors when building our shaded downstream artifacts. +1 hadoopcheck 21m 14s Patch does not cause any errors with Hadoop 2.6.5 2.7.4 or 3.0.0. +1 javadoc 0m 33s the patch passed       Other Tests +1 unit 101m 28s hbase-server in the patch passed. +1 asflicense 0m 20s The patch does not generate ASF License warnings. 143m 1s Subsystem Report/Notes Docker Client=17.05.0-ce Server=17.05.0-ce Image:yetus/hbase:eee3b01 JIRA Issue HBASE-19866 JIRA Patch URL https://issues.apache.org/jira/secure/attachment/12907967/HBASE-19866.patch Optional Tests asflicense javac javadoc unit findbugs shadedjars hadoopcheck hbaseanti checkstyle compile uname Linux 18b432fbc2a9 3.13.0-139-generic #188-Ubuntu SMP Tue Jan 9 14:43:09 UTC 2018 x86_64 GNU/Linux Build tool maven Personality /home/jenkins/jenkins-slave/workspace/PreCommit-HBASE-Build/component/dev-support/hbase-personality.sh git revision master / a5a8c4f3f2 maven version: Apache Maven 3.5.2 (138edd61fd100ec658bfa2d307c43b76940a5d7d; 2017-10-18T07:58:13Z) Default Java 1.8.0_151 Test Results https://builds.apache.org/job/PreCommit-HBASE-Build/11211/testReport/ modules C: hbase-server U: hbase-server Console output https://builds.apache.org/job/PreCommit-HBASE-Build/11211/console Powered by Apache Yetus 0.6.0 http://yetus.apache.org This message was automatically generated.
            zhangduo Duo Zhang added a comment -

            Any concerns? Thanks.

            zhangduo Duo Zhang added a comment - Any concerns? Thanks.
            yuzhihong@gmail.com Ted Yu added a comment -

            Lgtm

            yuzhihong@gmail.com Ted Yu added a comment - Lgtm
            binlijin Lijin Bin added a comment -

            Lgtm

            binlijin Lijin Bin added a comment - Lgtm
            binlijin Lijin Bin added a comment -

            The lock for authManager was introduced by https://issues.apache.org/jira/browse/HBASE-15118.

             

            binlijin Lijin Bin added a comment - The lock for authManager was introduced by  https://issues.apache.org/jira/browse/HBASE-15118 .  
            zhangduo Duo Zhang added a comment -

            Pushed to master and branch-2. Thanks appy for digging, and thanks all for reviewing.

            zhangduo Duo Zhang added a comment - Pushed to master and branch-2. Thanks appy for digging, and thanks all for reviewing.
            hudson Hudson added a comment -

            FAILURE: Integrated in Jenkins build HBase-Trunk_matrix #4491 (See https://builds.apache.org/job/HBase-Trunk_matrix/4491/)
            HBASE-19866 TestRegionServerReportForDuty doesn't timeout (zhangduo: rev 5b93ea155652292ad06ee3df85e5944191eace9b)

            • (edit) hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
            hudson Hudson added a comment - FAILURE: Integrated in Jenkins build HBase-Trunk_matrix #4491 (See https://builds.apache.org/job/HBase-Trunk_matrix/4491/ ) HBASE-19866 TestRegionServerReportForDuty doesn't timeout (zhangduo: rev 5b93ea155652292ad06ee3df85e5944191eace9b) (edit) hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
            stack Michael Stack added a comment -

            This is a nice fix. I think the authManager lock preceded netty rpc?

            stack Michael Stack added a comment - This is a nice fix. I think the authManager lock preceded netty rpc?

            People

              zhangduo Duo Zhang
              appy Apekshit Sharma
              Votes:
              0 Vote for this issue
              Watchers:
              6 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: