Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HDDS-3498. Shutdown datanode if address is already in use #7256

Merged
merged 17 commits into from
Oct 4, 2024

Conversation

Daniilchik
Copy link
Contributor

What changes were proposed in this pull request?

This PR fixes a bug related to an incomplete datanode process.

What is the link to the Apache JIRA

https://issues.apache.org/jira/browse/HDDS-3498

How was this patch tested?

Manual tests using docker-compose, build-branch workflow.

@sarvekshayr
Copy link
Contributor

sarvekshayr commented Oct 1, 2024

@Daniilchik thank you for working on this.
Could you please share the output log messages from before vs after implementing the change?

@Daniilchik
Copy link
Contributor Author

Daniilchik commented Oct 1, 2024

@sarvekshayr
Before

2024-10-01 11:54:52,322 [4cd909c2-d86f-4176-9900-410c5e2fae15-EndpointStateMachineTaskThread-scm/172.20.0.5:9861-0 ] ERROR statemachine.EndpointStateMachine: Unable to communicate to SCM server at scm/172.20.0.5:9861 for past 0 seconds.
java.io.IOException: Failed to bind to address 0.0.0.0/0.0.0.0:9859
        at org.apache.ratis.thirdparty.io.grpc.netty.NettyServer.start(NettyServer.java:326)
        at org.apache.ratis.thirdparty.io.grpc.internal.ServerImpl.start(ServerImpl.java:185)
        at org.apache.ratis.thirdparty.io.grpc.internal.ServerImpl.start(ServerImpl.java:94)
        at org.apache.hadoop.ozone.container.common.transport.server.XceiverServerGrpc.start(XceiverServerGrpc.java:188)
        at org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer.start(OzoneContainer.java:484)
        at org.apache.hadoop.ozone.container.common.states.endpoint.VersionEndpointTask.call(VersionEndpointTask.java:98)
        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.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
        at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
        at java.base/java.lang.Thread.run(Thread.java:833)
Caused by: org.apache.ratis.thirdparty.io.netty.channel.unix.Errors$NativeIoException: bind(..) failed: Address already in use
2024-10-01 11:55:09,913 [4cd909c2-d86f-4176-9900-410c5e2fae15-DatanodeStateMachineDaemonThread] ERROR datanode.RunningDatanodeState: Error in executing end point task.
java.util.concurrent.ExecutionException: java.util.concurrent.TimeoutException
        at java.base/java.util.concurrent.FutureTask.report(FutureTask.java:122)
        at java.base/java.util.concurrent.FutureTask.get(FutureTask.java:191)
        at org.apache.hadoop.ozone.container.common.states.datanode.RunningDatanodeState.computeNextContainerState(RunningDatanodeState.java:162)
        at org.apache.hadoop.ozone.container.common.states.datanode.RunningDatanodeState.await(RunningDatanodeState.java:202)
        at org.apache.hadoop.ozone.container.common.states.datanode.RunningDatanodeState.await(RunningDatanodeState.java:47)
        at org.apache.hadoop.ozone.container.common.statemachine.StateContext.execute(StateContext.java:641)
        at org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine.startStateMachineThread(DatanodeStateMachine.java:343)
        at org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine.lambda$startDaemon$1(DatanodeStateMachine.java:544)
        at java.base/java.lang.Thread.run(Thread.java:833)
Caused by: java.util.concurrent.TimeoutException
        at java.base/java.util.concurrent.FutureTask.get(FutureTask.java:204)
        at org.apache.hadoop.ozone.container.common.states.datanode.RunningDatanodeState.lambda$execute$0(RunningDatanodeState.java:102)
        at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
        at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:539)
        at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
        at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
        at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
        ... 1 more
2024-10-01 11:56:19,990 [4cd909c2-d86f-4176-9900-410c5e2fae15-DatanodeStateMachineDaemonThread] ERROR datanode.RunningDatanodeState: Error in executing end point task.
java.util.concurrent.ExecutionException: java.util.concurrent.TimeoutException
        at java.base/java.util.concurrent.FutureTask.report(FutureTask.java:122)
        at java.base/java.util.concurrent.FutureTask.get(FutureTask.java:191)
        at org.apache.hadoop.ozone.container.common.states.datanode.RunningDatanodeState.computeNextContainerState(RunningDatanodeState.java:162)
        at org.apache.hadoop.ozone.container.common.states.datanode.RunningDatanodeState.await(RunningDatanodeState.java:202)
        at org.apache.hadoop.ozone.container.common.states.datanode.RunningDatanodeState.await(RunningDatanodeState.java:47)
        at org.apache.hadoop.ozone.container.common.statemachine.StateContext.execute(StateContext.java:641)
        at org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine.startStateMachineThread(DatanodeStateMachine.java:343)
        at org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine.lambda$startDaemon$1(DatanodeStateMachine.java:544)
        at java.base/java.lang.Thread.run(Thread.java:833)
Caused by: java.util.concurrent.TimeoutException
        at java.base/java.util.concurrent.FutureTask.get(FutureTask.java:204)
        at org.apache.hadoop.ozone.container.common.states.datanode.RunningDatanodeState.lambda$execute$0(RunningDatanodeState.java:102)
        at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
        at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:539)
        at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
        at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
        at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
        ... 1 more
...

After

2024-10-01 11:44:19,567 [a7d9f649-6acc-4cce-a190-f478b7ec1764-EndpointStateMachineTaskThread-scm/172.20.0.3:9861-0 ] ERROR statemachine.EndpointStateMachine: Unable to communicate to SCM server at scm/172.20.0.3:9861 for past 0 seconds.
java.io.IOException: Failed to bind to address 0.0.0.0/0.0.0.0:9859
        at org.apache.ratis.thirdparty.io.grpc.netty.NettyServer.start(NettyServer.java:326)
        at org.apache.ratis.thirdparty.io.grpc.internal.ServerImpl.start(ServerImpl.java:185)
        at org.apache.ratis.thirdparty.io.grpc.internal.ServerImpl.start(ServerImpl.java:94)
        at org.apache.hadoop.ozone.container.common.transport.server.XceiverServerGrpc.start(XceiverServerGrpc.java:188)
        at org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer.start(OzoneContainer.java:484)
        at org.apache.hadoop.ozone.container.common.states.endpoint.VersionEndpointTask.call(VersionEndpointTask.java:98)
        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.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
        at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
        at java.base/java.lang.Thread.run(Thread.java:833)
Caused by: org.apache.ratis.thirdparty.io.netty.channel.unix.Errors$NativeIoException: bind(..) failed: Address already in use
2024-10-01 11:44:19,569 [a7d9f649-6acc-4cce-a190-f478b7ec1764-EndpointStateMachineTaskThread-scm/172.20.0.3:9861-0 ] ERROR endpoint.VersionEndpointTask: Failed to bind to address 0.0.0.0/0.0.0.0:9859
2024-10-01 11:44:19,569 [a7d9f649-6acc-4cce-a190-f478b7ec1764-EndpointStateMachineTaskThread-scm/172.20.0.3:9861-0 ] INFO ozoneimpl.OzoneContainer: Attempting to stop container services.
2024-10-01 11:44:19,570 [ContainerMetadataScanner] WARN ozoneimpl.AbstractBackgroundContainerScanner: Background container scan was interrupted.
2024-10-01 11:44:19,570 [ContainerMetadataScanner] INFO ozoneimpl.AbstractBackgroundContainerScanner: Thread[ContainerMetadataScanner,5,main] exiting.
2024-10-01 11:44:19,570 [a7d9f649-6acc-4cce-a190-f478b7ec1764-EndpointStateMachineTaskThread-scm/172.20.0.3:9861-0 ] INFO ozoneimpl.BackgroundContainerDataScanner: ContainerDataScanner(/data/hdds/hdds) is shutting down. 
2024-10-01 11:44:19,570 [ContainerDataScanner(/data/hdds/hdds)] WARN ozoneimpl.AbstractBackgroundContainerScanner: Background container scan was interrupted.
2024-10-01 11:44:19,570 [ContainerDataScanner(/data/hdds/hdds)] INFO ozoneimpl.AbstractBackgroundContainerScanner: ContainerDataScanner(/data/hdds/hdds, DS-e3af78b3-34a7-453f-83e1-1e68169f5ed7) exiting.
2024-10-01 11:44:19,571 [a7d9f649-6acc-4cce-a190-f478b7ec1764-EndpointStateMachineTaskThread-scm/172.20.0.3:9861-0 ] INFO ozoneimpl.OnDemandContainerDataScanner: On-demand container scanner is shutting down.
2024-10-01 11:44:19,572 [a7d9f649-6acc-4cce-a190-f478b7ec1764-EndpointStateMachineTaskThread-scm/172.20.0.3:9861-0 ] INFO ratis.XceiverServerRatis: Stopping XceiverServerRatis a7d9f649-6acc-4cce-a190-f478b7ec1764
2024-10-01 11:44:19,573 [a7d9f649-6acc-4cce-a190-f478b7ec1764-EndpointStateMachineTaskThread-scm/172.20.0.3:9861-0 ] INFO server.RaftServer: a7d9f649-6acc-4cce-a190-f478b7ec1764: close
2024-10-01 11:44:19,574 [a7d9f649-6acc-4cce-a190-f478b7ec1764-EndpointStateMachineTaskThread-scm/172.20.0.3:9861-0 ] INFO server.GrpcService: a7d9f649-6acc-4cce-a190-f478b7ec1764: shutdown server org.apache.ratis.grpc.server.GrpcClientProtocolService now
2024-10-01 11:44:19,576 [a7d9f649-6acc-4cce-a190-f478b7ec1764-EndpointStateMachineTaskThread-scm/172.20.0.3:9861-0 ] INFO server.GrpcService: a7d9f649-6acc-4cce-a190-f478b7ec1764: shutdown server org.apache.ratis.grpc.server.GrpcClientProtocolService successfully
2024-10-01 11:44:19,576 [a7d9f649-6acc-4cce-a190-f478b7ec1764-EndpointStateMachineTaskThread-scm/172.20.0.3:9861-0 ] INFO server.GrpcService: a7d9f649-6acc-4cce-a190-f478b7ec1764: shutdown server GrpcServerProtocolService now
2024-10-01 11:44:19,577 [a7d9f649-6acc-4cce-a190-f478b7ec1764-EndpointStateMachineTaskThread-scm/172.20.0.3:9861-0 ] INFO server.GrpcService: a7d9f649-6acc-4cce-a190-f478b7ec1764: shutdown server GrpcServerProtocolService successfully
2024-10-01 11:44:19,577 [a7d9f649-6acc-4cce-a190-f478b7ec1764-EndpointStateMachineTaskThread-scm/172.20.0.3:9861-0 ] INFO server.GrpcService: a7d9f649-6acc-4cce-a190-f478b7ec1764: shutdown server org.apache.ratis.grpc.server.GrpcAdminProtocolService now
2024-10-01 11:44:19,578 [a7d9f649-6acc-4cce-a190-f478b7ec1764-EndpointStateMachineTaskThread-scm/172.20.0.3:9861-0 ] INFO server.GrpcService: a7d9f649-6acc-4cce-a190-f478b7ec1764: shutdown server org.apache.ratis.grpc.server.GrpcAdminProtocolService successfully
2024-10-01 11:44:19,579 [a7d9f649-6acc-4cce-a190-f478b7ec1764-NettyServerStreamRpc-bossGroup--thread1] INFO logging.LoggingHandler: [id: 0xe7ed575d, L:/0.0.0.0:9855] CLOSE
2024-10-01 11:44:19,579 [a7d9f649-6acc-4cce-a190-f478b7ec1764-NettyServerStreamRpc-bossGroup--thread1] INFO logging.LoggingHandler: [id: 0xe7ed575d, L:/0.0.0.0:9855] INACTIVE
2024-10-01 11:44:19,580 [a7d9f649-6acc-4cce-a190-f478b7ec1764-NettyServerStreamRpc-bossGroup--thread1] INFO logging.LoggingHandler: [id: 0xe7ed575d, L:/0.0.0.0:9855] UNREGISTERED
2024-10-01 11:44:19,596 [JvmPauseMonitor0] INFO util.JvmPauseMonitor: JvmPauseMonitor-a7d9f649-6acc-4cce-a190-f478b7ec1764: Stopped
2024-10-01 11:44:19,620 [a7d9f649-6acc-4cce-a190-f478b7ec1764-EndpointStateMachineTaskThread-scm/172.20.0.3:9861-0 ] INFO volume.HddsVolume: SchemaV3 db is stopped at /data/hdds/hdds/CID-10b15948-55e5-41e3-809b-18d94ea9fe07/DS-e3af78b3-34a7-453f-83e1-1e68169f5ed7/container.db for volume DS-e3af78b3-34a7-453f-83e1-1e68169f5ed7
2024-10-01 11:44:19,620 [a7d9f649-6acc-4cce-a190-f478b7ec1764-EndpointStateMachineTaskThread-scm/172.20.0.3:9861-0 ] INFO utils.BackgroundService: Shutting down service BlockDeletingService
2024-10-01 11:44:19,620 [a7d9f649-6acc-4cce-a190-f478b7ec1764-EndpointStateMachineTaskThread-scm/172.20.0.3:9861-0 ] INFO utils.BackgroundService: Shutting down service StaleRecoveringContainerScrubbingService
2024-10-01 11:44:21,155 [a7d9f649-6acc-4cce-a190-f478b7ec1764-DatanodeStateMachineDaemonThread] ERROR statemachine.StateContext: Critical error occurred in StateMachine, setting shutDownMachine
2024-10-01 11:44:23,115 [a7d9f649-6acc-4cce-a190-f478b7ec1764-DatanodeStateMachineDaemonThread] ERROR statemachine.DatanodeStateMachine: DatanodeStateMachine Shutdown due to an critical error
2024-10-01 11:44:23,126 [a7d9f649-6acc-4cce-a190-f478b7ec1764-DatanodeStateMachineDaemonThread] ERROR statemachine.DatanodeStateMachine: Error attempting to shutdown.
java.lang.InterruptedException
        at java.base/java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:1660)
        at java.base/java.util.concurrent.ThreadPoolExecutor.awaitTermination(ThreadPoolExecutor.java:1464)
        at org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine.executorServiceShutdownGraceful(DatanodeStateMachine.java:455)
        at org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine.close(DatanodeStateMachine.java:428)
        at org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine.stopDaemon(DatanodeStateMachine.java:625)
        at org.apache.hadoop.ozone.HddsDatanodeService.stop(HddsDatanodeService.java:539)
        at org.apache.hadoop.ozone.HddsDatanodeService.terminateDatanode(HddsDatanodeService.java:521)
        at org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine.startStateMachineThread(DatanodeStateMachine.java:372)
        at org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine.lambda$startDaemon$1(DatanodeStateMachine.java:544)
        at java.base/java.lang.Thread.run(Thread.java:833)
2024-10-01 11:44:23,131 [a7d9f649-6acc-4cce-a190-f478b7ec1764-DatanodeStateMachineDaemonThread] INFO ozoneimpl.OzoneContainer: Attempting to stop container services.
2024-10-01 11:44:23,133 [a7d9f649-6acc-4cce-a190-f478b7ec1764-DatanodeStateMachineDaemonThread] INFO utils.BackgroundService: Shutting down service BlockDeletingService
2024-10-01 11:44:23,134 [a7d9f649-6acc-4cce-a190-f478b7ec1764-DatanodeStateMachineDaemonThread] INFO utils.BackgroundService: Shutting down service StaleRecoveringContainerScrubbingService
2024-10-01 11:44:23,134 [a7d9f649-6acc-4cce-a190-f478b7ec1764-DatanodeStateMachineDaemonThread] INFO statemachine.DatanodeStateMachine: Ozone container server stopped.
2024-10-01 11:44:23,148 [a7d9f649-6acc-4cce-a190-f478b7ec1764-DatanodeStateMachineDaemonThread] INFO handler.ContextHandler: Stopped o.e.j.w.WebAppContext@5fe46d52{hddsDatanode,/,null,STOPPED}{jar:file:/opt/hadoop/share/ozone/lib/hdds-container-service-1.5.0-SNAPSHOT.jar!/webapps/hddsDatanode}
2024-10-01 11:44:23,153 [a7d9f649-6acc-4cce-a190-f478b7ec1764-DatanodeStateMachineDaemonThread] INFO server.session: node0 Stopped scavenging
2024-10-01 11:44:23,156 [a7d9f649-6acc-4cce-a190-f478b7ec1764-DatanodeStateMachineDaemonThread] INFO server.AbstractConnector: Stopped ServerConnector@75bd28d{HTTP/1.1, (http/1.1)}{0.0.0.0:9882}
2024-10-01 11:44:23,157 [a7d9f649-6acc-4cce-a190-f478b7ec1764-DatanodeStateMachineDaemonThread] INFO handler.ContextHandler: Stopped o.e.j.s.ServletContextHandler@4fea840f{static,/static,jar:file:/opt/hadoop/share/ozone/lib/hdds-container-service-1.5.0-SNAPSHOT.jar!/webapps/static,STOPPED}
2024-10-01 11:44:23,157 [a7d9f649-6acc-4cce-a190-f478b7ec1764-DatanodeStateMachineDaemonThread] INFO handler.ContextHandler: Stopped o.e.j.s.ServletContextHandler@6b4a4e40{logs,/logs,file:///var/log/hadoop/,STOPPED}
2024-10-01 11:44:23,159 [a7d9f649-6acc-4cce-a190-f478b7ec1764-DatanodeStateMachineDaemonThread] ERROR http.HttpServer2: Error while stopping web server for webapp hddsDatanode
java.lang.InterruptedException
        at java.base/java.util.concurrent.locks.AbstractQueuedSynchronizer.tryAcquireSharedNanos(AbstractQueuedSynchronizer.java:1081)
        at java.base/java.util.concurrent.CountDownLatch.await(CountDownLatch.java:276)
        at org.eclipse.jetty.server.AbstractConnector.doStop(AbstractConnector.java:373)
        at org.eclipse.jetty.server.AbstractNetworkConnector.doStop(AbstractNetworkConnector.java:88)
        at org.eclipse.jetty.server.ServerConnector.doStop(ServerConnector.java:246)
        at org.eclipse.jetty.util.component.AbstractLifeCycle.stop(AbstractLifeCycle.java:94)
        at org.eclipse.jetty.server.Server.doStop(Server.java:459)
        at org.eclipse.jetty.util.component.AbstractLifeCycle.stop(AbstractLifeCycle.java:94)
        at org.apache.hadoop.hdds.server.http.HttpServer2.stop(HttpServer2.java:1363)
        at org.apache.hadoop.hdds.server.http.BaseHttpServer.stop(BaseHttpServer.java:339)
        at org.apache.hadoop.ozone.HddsDatanodeService.stop(HddsDatanodeService.java:543)
        at org.apache.hadoop.ozone.HddsDatanodeService.terminateDatanode(HddsDatanodeService.java:521)
        at org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine.startStateMachineThread(DatanodeStateMachine.java:372)
        at org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine.lambda$startDaemon$1(DatanodeStateMachine.java:544)
        at java.base/java.lang.Thread.run(Thread.java:833)
2024-10-01 11:44:23,160 [a7d9f649-6acc-4cce-a190-f478b7ec1764-DatanodeStateMachineDaemonThread] ERROR ozone.HddsDatanodeService: Stopping HttpServer is failed.
java.lang.InterruptedException
        at java.base/java.util.concurrent.locks.AbstractQueuedSynchronizer.tryAcquireSharedNanos(AbstractQueuedSynchronizer.java:1081)
        at java.base/java.util.concurrent.CountDownLatch.await(CountDownLatch.java:276)
        at org.eclipse.jetty.server.AbstractConnector.doStop(AbstractConnector.java:373)
        at org.eclipse.jetty.server.AbstractNetworkConnector.doStop(AbstractNetworkConnector.java:88)
        at org.eclipse.jetty.server.ServerConnector.doStop(ServerConnector.java:246)
        at org.eclipse.jetty.util.component.AbstractLifeCycle.stop(AbstractLifeCycle.java:94)
        at org.eclipse.jetty.server.Server.doStop(Server.java:459)
        at org.eclipse.jetty.util.component.AbstractLifeCycle.stop(AbstractLifeCycle.java:94)
        at org.apache.hadoop.hdds.server.http.HttpServer2.stop(HttpServer2.java:1363)
        at org.apache.hadoop.hdds.server.http.BaseHttpServer.stop(BaseHttpServer.java:339)
        at org.apache.hadoop.ozone.HddsDatanodeService.stop(HddsDatanodeService.java:543)
        at org.apache.hadoop.ozone.HddsDatanodeService.terminateDatanode(HddsDatanodeService.java:521)
        at org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine.startStateMachineThread(DatanodeStateMachine.java:372)
        at org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine.lambda$startDaemon$1(DatanodeStateMachine.java:544)
        at java.base/java.lang.Thread.run(Thread.java:833)
2024-10-01 11:44:23,160 [a7d9f649-6acc-4cce-a190-f478b7ec1764-DatanodeStateMachineDaemonThread] INFO ozone.HddsDatanodeClientProtocolServer: Stopping the RPC server for Client Protocol
2024-10-01 11:44:23,160 [a7d9f649-6acc-4cce-a190-f478b7ec1764-DatanodeStateMachineDaemonThread] INFO ipc.Server: Stopping server on 19864
2024-10-01 11:44:23,163 [IPC Server listener on 19864] INFO ipc.Server: Stopping IPC Server listener on 19864
2024-10-01 11:44:23,163 [IPC Server Responder] INFO ipc.Server: Stopping IPC Server Responder
2024-10-01 11:44:23,175 [shutdown-hook-0] INFO ozone.HddsDatanodeService: SHUTDOWN_MSG: 
/************************************************************
SHUTDOWN_MSG: Shutting down HddsDatanodeService at e18429beef12/172.20.0.4
************************************************************/

"Unable to communicate to {} server at {} for past {} seconds.",
serverName,
getAddress().getHostString() + ":" + getAddress().getPort(),
address.getAddress() + ":" + address.getPort(),
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please use log4j templates instead of concatenation.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

@@ -108,6 +109,10 @@ public EndpointStateMachine.EndPointStates call() throws Exception {
rpcEndPoint.setState(EndpointStateMachine.EndPointStates.SHUTDOWN);
} catch (IOException ex) {
rpcEndPoint.logIfNeeded(ex);
if (ex.getCause() instanceof NativeIoException) {
LOG.error(ex.getMessage());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Shouldn't we log stacktrace here?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fixed

Copy link
Contributor

@myskov myskov left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @Daniilchik for working on the patch.

A few thoughts:

  • NativeIoException.class should not be exposed to VersionEndpointTask, as it is an internal detail of OzoneContainer. I believe OzoneContainer.start() should throw a dedicated exception for this scenario, perhaps something like ContainerStartException.
  • rpcEndPoint.logIfNeeded(ex) should not be called in the event of a port binding error. This call is responsible for the log message 'Unable to communicate with SCM server at scm/172.20.0.3:9861', but that’s not relevant in this situation.
  • ozoneContainer.stop() should not be invoked directly. Simply calling rpcEndPoint.setState(EndpointStateMachine.EndPointStates.SHUTDOWN) is sufficient to initiate the shutdown process.

@Daniilchik
Copy link
Contributor Author

@myskov thanks for help, all done.

@adoroszlai adoroszlai changed the title HDDS-3498. Address already in use Should shutdown the datanode with FATAL log and point out the port and configure key HDDS-3498. Shutdown datanode if address is already in use Oct 2, 2024
@Daniilchik Daniilchik force-pushed the HDDS-3498 branch 2 times, most recently from 8c47c15 to c151608 Compare October 2, 2024 10:32
@Daniilchik Daniilchik closed this Oct 2, 2024
@myskov
Copy link
Contributor

myskov commented Oct 2, 2024

@Daniilchik please do not use force push in opened PRs. It makes it difficult to review deltas between commits.

@Daniilchik Daniilchik deleted the HDDS-3498 branch October 2, 2024 10:41
@Daniilchik Daniilchik reopened this Oct 2, 2024
@Daniilchik
Copy link
Contributor Author

Daniilchik commented Oct 2, 2024

@Daniilchik please do not use force push in opened PRs. It makes it difficult to review deltas between commits.

Sorry about that, I won’t do it again.

Copy link
Contributor

@sarvekshayr sarvekshayr left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you please update the log output to reflect these changes?

@@ -107,7 +107,8 @@ public EndpointStateMachine.EndPointStates call() throws Exception {
} catch (DiskOutOfSpaceException ex) {
rpcEndPoint.setState(EndpointStateMachine.EndPointStates.SHUTDOWN);
} catch (IOException ex) {
rpcEndPoint.logIfNeeded(ex);
LOG.error(ex.getCause().getMessage(), ex);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LOG.error(ex) should already print message from exception. I would rather put here thoughtful and distinct message so it will be much easier to pinpoint place of the failure and understand when and why it failed.

server.start();
} catch (IOException e) {
LOG.error("Failed to bind to address", e);
if (e.getMessage().contains("Failed to bind to address")) {
Copy link
Contributor

@ivanzlenko ivanzlenko Oct 2, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we have any actual reason to start a service if we can't start a server? Shouldn't we just handle all IOException like that?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If we will handle all IOException like that we will end up with shutting down datanodes due to any network failures.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we identify how actual retryable exceptions looks like? Is netty throwing specifically only IOExceptions here?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actually it is better just handle correctly server.start part and shutdown everything here cause we can't start server. There is no point to live afterwards.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Discussed it with Daniil - we need to return to this code at some point in time to refactor it.

try {
server.start();
} catch (IOException e) {
LOG.error("Failed to bind to address", e);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Failed to bind to address - you are writing this text for any IOException. What if it was other issue preventing us from starting server?
Message something like "Can't start whatever server due to error occurred during startup process:" will be more appropriate here.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

serverName,
address.getAddress(),
address.getPort(),
TimeUnit.MILLISECONDS.toSeconds(this.getMissedCount() * getScmHeartbeatInterval(this.conf)),
Copy link
Contributor

@ivanzlenko ivanzlenko Oct 2, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would rather calculate it beforehand in a dedicated variable just to improve code readability.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

@Daniilchik Daniilchik requested a review from myskov October 2, 2024 16:35
@Daniilchik
Copy link
Contributor Author

Daniilchik commented Oct 2, 2024

Could you please update the log output to reflect these changes?

ERROR server.XceiverServerGrpc: Error while starting the server
java.io.IOException: Failed to bind to address 0.0.0.0/0.0.0.0:9859
at org.apache.ratis.thirdparty.io.grpc.netty.NettyServer.start(NettyServer.java:326)
at org.apache.ratis.thirdparty.io.grpc.internal.ServerImpl.start(ServerImpl.java:185)
at org.apache.ratis.thirdparty.io.grpc.internal.ServerImpl.start(ServerImpl.java:94)
at org.apache.hadoop.ozone.container.common.transport.server.XceiverServerGrpc.start(XceiverServerGrpc.java:190)
at org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer.start(OzoneContainer.java:484)
at org.apache.hadoop.ozone.container.common.states.endpoint.VersionEndpointTask.call(VersionEndpointTask.java:98)
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.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
at java.base/java.lang.Thread.run(Thread.java:833)
Caused by: org.apache.ratis.thirdparty.io.netty.channel.unix.Errors$NativeIoException: bind(..) failed: Address already in use
2024-10-02 23:03:41,281 [19255126-07c6-45a7-aae6-1d0705f31d0c-DatanodeStateMachineDaemonThread] ERROR statemachine.StateContext: Critical error occurred in StateMachine, setting shutDownMachine
2024-10-02 23:03:42,837 [19255126-07c6-45a7-aae6-1d0705f31d0c-DatanodeStateMachineDaemonThread] ERROR statemachine.DatanodeStateMachine: DatanodeStateMachine Shutdown due to an critical error
2024-10-02 23:03:42,853 [19255126-07c6-45a7-aae6-1d0705f31d0c-DatanodeStateMachineDaemonThread] ERROR statemachine.DatanodeStateMachine: Error attempting to shutdown.
...
/************************************************************
SHUTDOWN_MSG: Shutting down HddsDatanodeService at f0eb3f5c02d3/172.20.0.6
************************************************************/
2024-10-02 23:04:42,942 [shutdown-hook-0] INFO ozone.HddsDatanodeService: Interrupted during StorageContainerManager join.
2024-10-02 23:04:42,942 [Thread-31] WARN util.ShutdownHookManager: ShutdownHook 'HddsDatanodeService$$Lambda$530/0x0000000801134400' timeout, java.util.concurrent.TimeoutException
java.util.concurrent.TimeoutException
at java.base/java.util.concurrent.FutureTask.get(FutureTask.java:204)
at org.apache.hadoop.ozone.util.ShutdownHookManager.executeShutdown(ShutdownHookManager.java:129)
at org.apache.hadoop.ozone.util.ShutdownHookManager.access$300(ShutdownHookManager.java:76)
at org.apache.hadoop.ozone.util.ShutdownHookManager$1.run(ShutdownHookManager.java:102)

@Daniilchik Daniilchik requested a review from myskov October 3, 2024 14:59
Copy link
Contributor

@myskov myskov left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The most recent version looks good to me. @ivanzlenko please take a look.

long missedDurationSeconds = TimeUnit.MILLISECONDS.toSeconds(
this.getMissedCount() * getScmHeartbeatInterval(this.conf)
);
LOG.error(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why you changed it to error?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you for the remark. I’ll change it back. I initially changed it at the beginning of working on the task.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

server.start();
} catch (IOException e) {
LOG.error("Failed to bind to address", e);
if (e.getMessage().contains("Failed to bind to address")) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Discussed it with Daniil - we need to return to this code at some point in time to refactor it.

@myskov
Copy link
Contributor

myskov commented Oct 4, 2024

@Daniilchik thank you for working on the patch. Thanks @sarvekshayr and @ivanzlenko for reviewing.

@myskov myskov merged commit 30da31f into apache:master Oct 4, 2024
40 checks passed
sarvekshayr pushed a commit to sarvekshayr/ozone that referenced this pull request Oct 7, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants