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

GrpcClientProtocolClient#close Interrupts itself

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • None
    • 0.5.0
    • gRPC
    • None

    Description

      GrpcClientProtocolClient#close throws InterruptedException. This happens when GrpcClientProtocolClient#close is called from a TimeoutScheduler thread. GrpcClientProtocolClient#close calls scheduler.close() which interrupts all the timeout scheduler threads including the thread executing the close routine. This leads to InterruptedException when channel.awaitTermination is called.

       

      19/10/09 07:40:33 ERROR client.GrpcClientProtocolClient: Unexpected exception while waiting for channel termination
      java.lang.InterruptedException
              at java.util.concurrent.locks.AbstractQueuedSynchronizer.tryAcquireSharedNanos(AbstractQueuedSynchronizer.java:1326)
              at java.util.concurrent.CountDownLatch.await(CountDownLatch.java:277)
              at org.apache.ratis.thirdparty.io.grpc.internal.ManagedChannelImpl.awaitTermination(ManagedChannelImpl.java:763)
              at org.apache.ratis.thirdparty.io.grpc.internal.ForwardingManagedChannel.awaitTermination(ForwardingManagedChannel.java:57)
              at org.apache.ratis.thirdparty.io.grpc.internal.ManagedChannelOrphanWrapper.awaitTermination(ManagedChannelOrphanWrapper.java:70)
              at org.apache.ratis.grpc.client.GrpcClientProtocolClient.close(GrpcClientProtocolClient.java:146)
              at org.apache.ratis.util.PeerProxyMap$PeerAndProxy.lambda$close$1(PeerProxyMap.java:74)
              at org.apache.ratis.util.LifeCycle.lambda$checkStateAndClose$2(LifeCycle.java:231)
              at org.apache.ratis.util.LifeCycle.checkStateAndClose(LifeCycle.java:251)
              at org.apache.ratis.util.LifeCycle.checkStateAndClose(LifeCycle.java:229)
              at org.apache.ratis.util.PeerProxyMap$PeerAndProxy.close(PeerProxyMap.java:70)
              at org.apache.ratis.util.PeerProxyMap.resetProxy(PeerProxyMap.java:127)
              at org.apache.ratis.util.PeerProxyMap.handleException(PeerProxyMap.java:136)
              at org.apache.ratis.client.impl.RaftClientRpcWithProxy.handleException(RaftClientRpcWithProxy.java:47)
              at org.apache.ratis.client.impl.RaftClientImpl.handleIOException(RaftClientImpl.java:372)
              at org.apache.ratis.client.impl.OrderedAsync.lambda$sendRequest$10(OrderedAsync.java:236)
              at java.util.concurrent.CompletableFuture.uniExceptionally(CompletableFuture.java:870)
              at java.util.concurrent.CompletableFuture$UniExceptionally.tryFire(CompletableFuture.java:852)
              at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
              at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
              at org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers.lambda$timeoutCheck$3(GrpcClientProtocolClient.java:324)
              at java.util.Optional.ifPresent(Optional.java:159)
              at org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers.handleReplyFuture(GrpcClientProtocolClient.java:329)
              at org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers.timeoutCheck(GrpcClientProtocolClient.java:324)
              at org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers.lambda$onNext$1(GrpcClientProtocolClient.java:318)
              at org.apache.ratis.util.TimeoutScheduler.lambda$onTimeout$0(TimeoutScheduler.java:113)
              at org.apache.ratis.util.TimeoutScheduler.lambda$onTimeout$1(TimeoutScheduler.java:133)
              at org.apache.ratis.util.LogUtils.runAndLog(LogUtils.java:50)
              at org.apache.ratis.util.LogUtils$1.run(LogUtils.java:91)
              at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
              at java.util.concurrent.FutureTask.run(FutureTask.java:266)
              at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
              at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
              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)
      

       

       

      Attachments

        1. RATIS-705.002.patch
          6 kB
          Lokesh Jain
        2. RATIS-705.001.patch
          7 kB
          Lokesh Jain

        Activity

          People

            ljain Lokesh Jain
            nilotpalnandi Nilotpal Nandi
            Votes:
            0 Vote for this issue
            Watchers:
            5 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: