[
https://issues.apache.org/jira/browse/HDDS-3116?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17050463#comment-17050463
]
Stephen O'Donnell commented on HDDS-3116:
-----------------------------------------
The code gets somewhat hard to following when it gets into Ratis, but from what
I gather Ratis attempts to create a new RaftServerImpl.
This is actually a CompleteableFuture and the future is stored into the ImplMap.
The future gets this exception when it is execute, which it stores away for
later (capture with an extra debug message I added:
{code}
2020-03-03 16:12:53,098 [pool-10-thread-1] ERROR ratis.XceiverServerRatis:
Caught a NPE:
java.lang.NullPointerException
at
org.apache.hadoop.ozone.container.common.transport.server.ratis.XceiverServerRatis.sendPipelineReport(XceiverServerRatis.java:766)
at
org.apache.hadoop.ozone.container.common.transport.server.ratis.XceiverServerRatis.notifyGroupAdd(XceiverServerRatis.java:742)
at
org.apache.hadoop.ozone.container.common.transport.server.ratis.ContainerStateMachine.initialize(ContainerStateMachine.java:218)
at
org.apache.ratis.server.impl.ServerState.initStatemachine(ServerState.java:160)
at org.apache.ratis.server.impl.ServerState.<init>(ServerState.java:112)
at
org.apache.ratis.server.impl.RaftServerImpl.<init>(RaftServerImpl.java:112)
at
org.apache.ratis.server.impl.RaftServerProxy.lambda$newRaftServerImpl$2(RaftServerProxy.java:208)
at
java.base/java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1700)
at
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
at
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
at java.base/java.lang.Thread.run(Thread.java:834)
{code}
Then the Datanode State Machine Thread keeps trying over and over to start the
Impl, but as the future failed, it just repeats the same exception over and
over. The impl is never recreated, which is why once it fails it can never
recover even though the OzoneContainer inside the DataodeStateMachine is now
populated:
{code}
2020-03-03 16:12:56,813 [Datanode State Machine Thread - 0] WARN
statemachine.EndpointStateMachine: Unable to communicate to SCM server at
scm:9861 for past 0 seconds.
java.io.IOException: java.lang.NullPointerException
at org.apache.ratis.util.IOUtils.asIOException(IOUtils.java:54)
at org.apache.ratis.util.IOUtils.toIOException(IOUtils.java:61)
at org.apache.ratis.util.IOUtils.getFromFuture(IOUtils.java:70)
at
org.apache.ratis.server.impl.RaftServerProxy.getImpls(RaftServerProxy.java:284)
at
org.apache.ratis.server.impl.RaftServerProxy.start(RaftServerProxy.java:296)
at
org.apache.hadoop.ozone.container.common.transport.server.ratis.XceiverServerRatis.start(XceiverServerRatis.java:421)
at
org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer.start(OzoneContainer.java:237)
at
org.apache.hadoop.ozone.container.common.states.endpoint.VersionEndpointTask.call(VersionEndpointTask.java:113)
at
org.apache.hadoop.ozone.container.common.states.endpoint.VersionEndpointTask.call(VersionEndpointTask.java:42)
at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
at
java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
at
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
at
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
at java.base/java.lang.Thread.run(Thread.java:834)
Caused by: java.lang.NullPointerException
at
org.apache.hadoop.ozone.container.common.transport.server.ratis.XceiverServerRatis.sendPipelineReport(XceiverServerRatis.java:771)
at
org.apache.hadoop.ozone.container.common.transport.server.ratis.XceiverServerRatis.notifyGroupAdd(XceiverServerRatis.java:742)
at
org.apache.hadoop.ozone.container.common.transport.server.ratis.ContainerStateMachine.initialize(ContainerStateMachine.java:218)
at
org.apache.ratis.server.impl.ServerState.initStatemachine(ServerState.java:160)
at org.apache.ratis.server.impl.ServerState.<init>(ServerState.java:112)
at
org.apache.ratis.server.impl.RaftServerImpl.<init>(RaftServerImpl.java:112)
at
org.apache.ratis.server.impl.RaftServerProxy.lambda$newRaftServerImpl$2(RaftServerProxy.java:208)
at
java.base/java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1700)
{code}
It feels like this is a seperate problem within ratis, but it all stems from
the same race condition.
> Datanode sometimes fails to start with NPE when starting Ratis xceiver server
> -----------------------------------------------------------------------------
>
> Key: HDDS-3116
> URL: https://issues.apache.org/jira/browse/HDDS-3116
> Project: Hadoop Distributed Data Store
> Issue Type: Bug
> Components: Ozone Datanode
> Affects Versions: 0.5.0
> Reporter: Stephen O'Donnell
> Assignee: Nanda kumar
> Priority: Major
> Attachments: full_logs.txt
>
>
> While working on a network Topology test (HDDS-3084) which does the following:
> 1. Start a cluster with 6 DNs and 2 racks.
> 2. Create a volume, bucket and a single key.
> 3. Stop one rack of hosts using "docker-compose down"
> 4. Read the data from the single key
> 5. Start the 3 down hosts
> 6. Stop the other 3 hosts
> 7. Attempt to read the key again.
> At step 5 I sometimes see this stack trace in one of the DNs and it fails to
> full come up:
> {code}
> 2020-03-02 13:01:31,887 [Datanode State Machine Thread - 0] INFO
> ozoneimpl.OzoneContainer: Attempting to start container services.
> 2020-03-02 13:01:31,887 [Datanode State Machine Thread - 0] INFO
> ozoneimpl.OzoneContainer: Background container scanner has been disabled.
> 2020-03-02 13:01:31,887 [Datanode State Machine Thread - 0] INFO
> ratis.XceiverServerRatis: Starting XceiverServerRatis
> 8c1178dd-c44d-49d1-b899-cc3e40ae8f23 at port 9858
> 2020-03-02 13:01:31,887 [Datanode State Machine Thread - 0] WARN
> statemachine.EndpointStateMachine: Unable to communicate to SCM server at
> scm:9861 for past 15000 seconds.
> java.io.IOException: java.lang.NullPointerException
> at org.apache.ratis.util.IOUtils.asIOException(IOUtils.java:54)
> at org.apache.ratis.util.IOUtils.toIOException(IOUtils.java:61)
> at org.apache.ratis.util.IOUtils.getFromFuture(IOUtils.java:70)
> at
> org.apache.ratis.server.impl.RaftServerProxy.getImpls(RaftServerProxy.java:284)
> at
> org.apache.ratis.server.impl.RaftServerProxy.start(RaftServerProxy.java:296)
> at
> org.apache.hadoop.ozone.container.common.transport.server.ratis.XceiverServerRatis.start(XceiverServerRatis.java:418)
> at
> org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer.start(OzoneContainer.java:232)
> at
> org.apache.hadoop.ozone.container.common.states.endpoint.VersionEndpointTask.call(VersionEndpointTask.java:113)
> at
> org.apache.hadoop.ozone.container.common.states.endpoint.VersionEndpointTask.call(VersionEndpointTask.java:42)
> at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
> at
> java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
> at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
> at
> java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
> at
> java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
> at java.base/java.lang.Thread.run(Thread.java:834)
> Caused by: java.lang.NullPointerException
> at
> org.apache.hadoop.ozone.container.common.transport.server.ratis.XceiverServerRatis.sendPipelineReport(XceiverServerRatis.java:757)
> at
> org.apache.hadoop.ozone.container.common.transport.server.ratis.XceiverServerRatis.notifyGroupAdd(XceiverServerRatis.java:739)
> at
> org.apache.hadoop.ozone.container.common.transport.server.ratis.ContainerStateMachine.initialize(ContainerStateMachine.java:218)
> at
> org.apache.ratis.server.impl.ServerState.initStatemachine(ServerState.java:160)
> at org.apache.ratis.server.impl.ServerState.<init>(ServerState.java:112)
> at
> org.apache.ratis.server.impl.RaftServerImpl.<init>(RaftServerImpl.java:112)
> at
> org.apache.ratis.server.impl.RaftServerProxy.lambda$newRaftServerImpl$2(RaftServerProxy.java:208)
> at
> java.base/java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1700)
> ... 3 more
> {code}
> The DN does not recover from this automatically, although I confirmed that a
> full cluster restart fixed it (docker-compose stop; docker-compose start). I
> will try to confirm if a restart of the stuck DN would fix it or not too.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]