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

Ratis Leader election should try for other peers even when ask for votes fails

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Duplicate
    • None
    • None
    • server

    Description

      This bug was simulated using Ozone using Ratis for Data pipeline.
      In this test, one of the nodes was shut down permanently. This can result into a situation where a candidate node is never able to move out of Leader Election phase.

      2018-06-15 07:44:58,246 INFO org.apache.ratis.server.impl.LeaderElection: 0f7b9cd2-4dad-46d7-acbc-57d424492d00_9858 got exception when requesting votes: {}
      java.util.concurrent.ExecutionException: org.apache.ratis.shaded.io.grpc.StatusRuntimeException: UNAVAILABLE: io exception
              at java.util.concurrent.FutureTask.report(FutureTask.java:122)
              at java.util.concurrent.FutureTask.get(FutureTask.java:192)
              at org.apache.ratis.server.impl.LeaderElection.waitForResults(LeaderElection.java:214)
              at org.apache.ratis.server.impl.LeaderElection.askForVotes(LeaderElection.java:146)
              at org.apache.ratis.server.impl.LeaderElection.run(LeaderElection.java:102)
      Caused by: org.apache.ratis.shaded.io.grpc.StatusRuntimeException: UNAVAILABLE: io exception
              at org.apache.ratis.shaded.io.grpc.stub.ClientCalls.toStatusRuntimeException(ClientCalls.java:221)
              at org.apache.ratis.shaded.io.grpc.stub.ClientCalls.getUnchecked(ClientCalls.java:202)
              at org.apache.ratis.shaded.io.grpc.stub.ClientCalls.blockingUnaryCall(ClientCalls.java:131)
              at org.apache.ratis.shaded.proto.grpc.RaftServerProtocolServiceGrpc$RaftServerProtocolServiceBlockingStub.requestVote(RaftServerProtocolServiceGrpc.java:281)
              at org.apache.ratis.grpc.server.RaftServerProtocolClient.requestVote(RaftServerProtocolClient.java:61)
              at org.apache.ratis.grpc.RaftGRpcService.requestVote(RaftGRpcService.java:147)
              at org.apache.ratis.server.impl.LeaderElection.lambda$submitRequests$0(LeaderElection.java:188)
              at java.util.concurrent.FutureTask.run(FutureTask.java:266)
              at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
              at java.util.concurrent.FutureTask.run(FutureTask.java:266)
              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)
      Caused by: org.apache.ratis.shaded.io.netty.channel.AbstractChannel$AnnotatedConnectException: Connection refused: y128.l42scl.hortonworks.com/172.26.32.228:9858
              at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
              at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
              at org.apache.ratis.shaded.io.netty.channel.socket.nio.NioSocketChannel.doFinishConnect(NioSocketChannel.java:323)
              at org.apache.ratis.shaded.io.netty.channel.nio.AbstractNioChannel$AbstractNioUnsafe.finishConnect(AbstractNioChannel.java:340)
              at org.apache.ratis.shaded.io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:633)
              at org.apache.ratis.shaded.io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:580)
              at org.apache.ratis.shaded.io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:497)
              at org.apache.ratis.shaded.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:459)
              at org.apache.ratis.shaded.io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:858)
              at org.apache.ratis.shaded.io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138)
              ... 1 more
      Caused by: java.net.ConnectException: Connection refused
              ... 11 more
      

      This happens because of the following lines of the code during requestVote.

          for (final RaftPeer peer : others) {
            final RequestVoteRequestProto r = server.createRequestVoteRequest(
                peer.getId(), electionTerm, lastEntry);
            service.submit(
                () -> server.getServerRpc().requestVote(r));
            submitted++;
          }
      

      Attachments

        1. RATIS-260.00.patch
          1 kB
          Shashikant Banerjee

        Issue Links

          Activity

            People

              shashikant Shashikant Banerjee
              msingh Mukul Kumar Singh
              Votes:
              0 Vote for this issue
              Watchers:
              4 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: