Skip to content

Commit 43b2a4b

Browse files
HDDS-1456. Stop the datanode, when any datanode statemachine state is… (#769)
1 parent 3758270 commit 43b2a4b

File tree

9 files changed

+121
-36
lines changed

9 files changed

+121
-36
lines changed

hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java

Lines changed: 28 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -56,6 +56,7 @@
5656
import java.util.Arrays;
5757
import java.util.List;
5858
import java.util.UUID;
59+
import java.util.concurrent.atomic.AtomicBoolean;
5960

6061
import static org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec.getX509Certificate;
6162
import static org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest.getEncodedString;
@@ -84,6 +85,7 @@ public class HddsDatanodeService extends GenericCli implements ServicePlugin {
8485
private HddsDatanodeHttpServer httpServer;
8586
private boolean printBanner;
8687
private String[] args;
88+
private volatile AtomicBoolean isStopped = new AtomicBoolean(false);
8789

8890
public HddsDatanodeService(boolean printBanner, String[] args) {
8991
this.printBanner = printBanner;
@@ -209,7 +211,7 @@ public void start() {
209211
initializeCertificateClient(conf);
210212
}
211213
datanodeStateMachine = new DatanodeStateMachine(datanodeDetails, conf,
212-
dnCertClient);
214+
dnCertClient, this::terminateDatanode);
213215
try {
214216
httpServer = new HddsDatanodeHttpServer(conf);
215217
httpServer.start();
@@ -421,29 +423,37 @@ public void join() {
421423
}
422424
}
423425

426+
public void terminateDatanode() {
427+
stop();
428+
terminate(1);
429+
}
430+
431+
424432
@Override
425433
public void stop() {
426-
if (plugins != null) {
427-
for (ServicePlugin plugin : plugins) {
428-
try {
429-
plugin.stop();
430-
LOG.info("Stopped plug-in {}", plugin);
431-
} catch (Throwable t) {
432-
LOG.warn("ServicePlugin {} could not be stopped", plugin, t);
434+
if (!isStopped.get()) {
435+
isStopped.set(true);
436+
if (plugins != null) {
437+
for (ServicePlugin plugin : plugins) {
438+
try {
439+
plugin.stop();
440+
LOG.info("Stopped plug-in {}", plugin);
441+
} catch (Throwable t) {
442+
LOG.warn("ServicePlugin {} could not be stopped", plugin, t);
443+
}
433444
}
434445
}
435-
}
436-
if (datanodeStateMachine != null) {
437-
datanodeStateMachine.stopDaemon();
438-
}
439-
if (httpServer != null) {
440-
try {
441-
httpServer.stop();
442-
} catch (Exception e) {
443-
LOG.error("Stopping HttpServer is failed.", e);
446+
if (datanodeStateMachine != null) {
447+
datanodeStateMachine.stopDaemon();
448+
}
449+
if (httpServer != null) {
450+
try {
451+
httpServer.stop();
452+
} catch (Exception e) {
453+
LOG.error("Stopping HttpServer is failed.", e);
454+
}
444455
}
445456
}
446-
447457
}
448458

449459
@Override
Lines changed: 27 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,27 @@
1+
/**
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
* <p>
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
* <p>
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
19+
package org.apache.hadoop.ozone;
20+
21+
/**
22+
* Interface which declares a method to stop HddsDatanodeService.
23+
*/
24+
public interface HddsDatanodeStopService {
25+
26+
void stopService();
27+
}

hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java

Lines changed: 13 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -34,6 +34,7 @@
3434
import org.apache.hadoop.hdds.protocol.proto
3535
.StorageContainerDatanodeProtocolProtos.NodeReportProto;
3636
import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
37+
import org.apache.hadoop.ozone.HddsDatanodeStopService;
3738
import org.apache.hadoop.ozone.container.common.report.ReportManager;
3839
import org.apache.hadoop.ozone.container.common.statemachine.commandhandler
3940
.CloseContainerCommandHandler;
@@ -84,6 +85,7 @@ public class DatanodeStateMachine implements Closeable {
8485

8586
private JvmPauseMonitor jvmPauseMonitor;
8687
private CertificateClient dnCertClient;
88+
private final HddsDatanodeStopService hddsDatanodeStopService;
8789

8890
/**
8991
* Constructs a a datanode state machine.
@@ -93,7 +95,9 @@ public class DatanodeStateMachine implements Closeable {
9395
* enabled
9496
*/
9597
public DatanodeStateMachine(DatanodeDetails datanodeDetails,
96-
Configuration conf, CertificateClient certClient) throws IOException {
98+
Configuration conf, CertificateClient certClient,
99+
HddsDatanodeStopService hddsDatanodeStopService) throws IOException {
100+
this.hddsDatanodeStopService = hddsDatanodeStopService;
97101
this.conf = conf;
98102
this.datanodeDetails = datanodeDetails;
99103
executorService = HadoopExecutors.newCachedThreadPool(
@@ -195,6 +199,14 @@ private void start() throws IOException {
195199
LOG.error("Unable to finish the execution.", e);
196200
}
197201
}
202+
203+
// If we have got some exception in stateMachine we set the state to
204+
// shutdown to stop the stateMachine thread. Along with this we should
205+
// also stop the datanode.
206+
if (context.getShutdownOnError()) {
207+
LOG.error("DatanodeStateMachine Shutdown due to an critical error");
208+
hddsDatanodeStopService.stopService();
209+
}
198210
}
199211

200212
/**

hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/StateContext.java

Lines changed: 25 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -73,6 +73,7 @@ public class StateContext {
7373
private final Queue<ContainerAction> containerActions;
7474
private final Queue<PipelineAction> pipelineActions;
7575
private DatanodeStateMachine.DatanodeStates state;
76+
private boolean shutdownOnError = false;
7677

7778
/**
7879
* Starting with a 2 sec heartbeat frequency which will be updated to the
@@ -152,6 +153,22 @@ public void setState(DatanodeStateMachine.DatanodeStates state) {
152153
this.state = state;
153154
}
154155

156+
/**
157+
* Sets the shutdownOnError. This method needs to be called when we
158+
* set DatanodeState to SHUTDOWN when executing a task of a DatanodeState.
159+
* @param value
160+
*/
161+
private void setShutdownOnError(boolean value) {
162+
this.shutdownOnError = value;
163+
}
164+
165+
/**
166+
* Get shutdownStateMachine.
167+
* @return boolean
168+
*/
169+
public boolean getShutdownOnError() {
170+
return shutdownOnError;
171+
}
155172
/**
156173
* Adds the report to report queue.
157174
*
@@ -367,6 +384,14 @@ public void execute(ExecutorService service, long time, TimeUnit unit)
367384
}
368385
this.setState(newState);
369386
}
387+
388+
if (this.state == DatanodeStateMachine.DatanodeStates.SHUTDOWN) {
389+
LOG.error("Critical error occurred in StateMachine, setting " +
390+
"shutDownMachine");
391+
// When some exception occurred, set shutdownStateMachine to true, so
392+
// that we can terminate the datanode.
393+
setShutdownOnError(true);
394+
}
370395
}
371396
}
372397

hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/VolumeSet.java

Lines changed: 6 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -167,8 +167,6 @@ private void initializeVolumeSet() throws IOException {
167167

168168
checkAndSetClusterID(hddsVolume.getClusterID());
169169

170-
volumeMap.put(hddsVolume.getHddsRootDir().getPath(), hddsVolume);
171-
volumeStateMap.get(hddsVolume.getStorageType()).add(hddsVolume);
172170
LOG.info("Added Volume : {} to VolumeSet",
173171
hddsVolume.getHddsRootDir().getPath());
174172

@@ -177,6 +175,8 @@ private void initializeVolumeSet() throws IOException {
177175
throw new IOException("Failed to create HDDS storage dir " +
178176
hddsVolume.getHddsRootDir());
179177
}
178+
volumeMap.put(hddsVolume.getHddsRootDir().getPath(), hddsVolume);
179+
volumeStateMap.get(hddsVolume.getStorageType()).add(hddsVolume);
180180
} catch (IOException e) {
181181
HddsVolume volume = new HddsVolume.Builder(locationString)
182182
.failedVolume(true).build();
@@ -185,12 +185,14 @@ private void initializeVolumeSet() throws IOException {
185185
}
186186
}
187187

188-
checkAllVolumes();
189-
188+
// First checking if we have any volumes, if all volumes are failed the
189+
// volumeMap size will be zero, and we throw Exception.
190190
if (volumeMap.size() == 0) {
191191
throw new DiskOutOfSpaceException("No storage locations configured");
192192
}
193193

194+
checkAllVolumes();
195+
194196
// Ensure volume threads are stopped and scm df is saved during shutdown.
195197
shutdownHook = () -> {
196198
saveVolumeSetUsed();

hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStateMachine.java

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -160,7 +160,7 @@ public void tearDown() throws Exception {
160160
public void testStartStopDatanodeStateMachine() throws IOException,
161161
InterruptedException, TimeoutException {
162162
try (DatanodeStateMachine stateMachine =
163-
new DatanodeStateMachine(getNewDatanodeDetails(), conf, null)) {
163+
new DatanodeStateMachine(getNewDatanodeDetails(), conf, null, null)) {
164164
stateMachine.startDaemon();
165165
SCMConnectionManager connectionManager =
166166
stateMachine.getConnectionManager();
@@ -222,7 +222,7 @@ public void testDatanodeStateContext() throws IOException,
222222
ContainerUtils.writeDatanodeDetailsTo(datanodeDetails, idPath);
223223

224224
try (DatanodeStateMachine stateMachine =
225-
new DatanodeStateMachine(datanodeDetails, conf, null)) {
225+
new DatanodeStateMachine(datanodeDetails, conf, null, null)) {
226226
DatanodeStateMachine.DatanodeStates currentState =
227227
stateMachine.getContext().getState();
228228
Assert.assertEquals(DatanodeStateMachine.DatanodeStates.INIT,
@@ -343,7 +343,7 @@ public void testDatanodeStateMachineWithIdWriteFail() throws Exception {
343343
datanodeDetails.setPort(port);
344344

345345
try (DatanodeStateMachine stateMachine =
346-
new DatanodeStateMachine(datanodeDetails, conf, null)) {
346+
new DatanodeStateMachine(datanodeDetails, conf, null, null)) {
347347
DatanodeStateMachine.DatanodeStates currentState =
348348
stateMachine.getContext().getState();
349349
Assert.assertEquals(DatanodeStateMachine.DatanodeStates.INIT,
@@ -406,7 +406,7 @@ public void testDatanodeStateMachineWithInvalidConfiguration()
406406
perTestConf.setStrings(entry.getKey(), entry.getValue());
407407
LOG.info("Test with {} = {}", entry.getKey(), entry.getValue());
408408
try (DatanodeStateMachine stateMachine = new DatanodeStateMachine(
409-
getNewDatanodeDetails(), perTestConf, null)) {
409+
getNewDatanodeDetails(), perTestConf, null, null)) {
410410
DatanodeStateMachine.DatanodeStates currentState =
411411
stateMachine.getContext().getState();
412412
Assert.assertEquals(DatanodeStateMachine.DatanodeStates.INIT,

hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/volume/TestVolumeSetDiskChecks.java

Lines changed: 7 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -38,6 +38,8 @@
3838
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
3939
import static org.hamcrest.CoreMatchers.is;
4040
import org.junit.After;
41+
42+
import static org.junit.Assert.assertEquals;
4143
import static org.junit.Assert.assertThat;
4244
import static org.junit.Assert.assertTrue;
4345
import org.junit.Rule;
@@ -125,14 +127,14 @@ HddsVolumeChecker getVolumeChecker(Configuration configuration)
125127
}
126128

127129
/**
128-
* Verify that initialization fails if all volumes are bad.
130+
* Verify that all volumes are added to fail list if all volumes are bad.
129131
*/
130132
@Test
131133
public void testAllVolumesAreBad() throws IOException {
132134
final int numVolumes = 5;
133135

134136
conf = getConfWithDataNodeDirs(numVolumes);
135-
thrown.expect(IOException.class);
137+
136138
final VolumeSet volumeSet = new VolumeSet(
137139
UUID.randomUUID().toString(), conf) {
138140
@Override
@@ -141,6 +143,9 @@ HddsVolumeChecker getVolumeChecker(Configuration configuration)
141143
return new DummyChecker(configuration, new Timer(), numVolumes);
142144
}
143145
};
146+
147+
assertEquals(volumeSet.getFailedVolumesList().size(), numVolumes);
148+
assertEquals(volumeSet.getVolumesList().size(), 0);
144149
}
145150

146151
/**

hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/common/TestEndPoint.java

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -175,6 +175,10 @@ public void testGetVersionTask() throws Exception {
175175
@Test
176176
public void testCheckVersionResponse() throws Exception {
177177
OzoneConfiguration conf = SCMTestUtils.getConf();
178+
conf.setBoolean(OzoneConfigKeys.DFS_CONTAINER_IPC_RANDOM_PORT,
179+
true);
180+
conf.setBoolean(OzoneConfigKeys.DFS_CONTAINER_RATIS_IPC_RANDOM_PORT,
181+
true);
178182
try (EndpointStateMachine rpcEndPoint = createEndpoint(conf,
179183
serverAddress, 1000)) {
180184
GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer
@@ -478,7 +482,7 @@ private StateContext heartbeatTaskHelper(InetSocketAddress scmAddress,
478482

479483
// Create a datanode state machine for stateConext used by endpoint task
480484
try (DatanodeStateMachine stateMachine = new DatanodeStateMachine(
481-
TestUtils.randomDatanodeDetails(), conf, null);
485+
TestUtils.randomDatanodeDetails(), conf, null, null);
482486
EndpointStateMachine rpcEndPoint =
483487
createEndpoint(conf, scmAddress, rpcTimeout)) {
484488
HddsProtos.DatanodeDetailsProto datanodeDetailsProto =

hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestMiniOzoneCluster.java

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -175,11 +175,11 @@ public void testContainerRandomPort() throws IOException {
175175
true);
176176
try (
177177
DatanodeStateMachine sm1 = new DatanodeStateMachine(
178-
TestUtils.randomDatanodeDetails(), ozoneConf, null);
178+
TestUtils.randomDatanodeDetails(), ozoneConf, null, null);
179179
DatanodeStateMachine sm2 = new DatanodeStateMachine(
180-
TestUtils.randomDatanodeDetails(), ozoneConf, null);
180+
TestUtils.randomDatanodeDetails(), ozoneConf, null, null);
181181
DatanodeStateMachine sm3 = new DatanodeStateMachine(
182-
TestUtils.randomDatanodeDetails(), ozoneConf, null)
182+
TestUtils.randomDatanodeDetails(), ozoneConf, null, null)
183183
) {
184184
HashSet<Integer> ports = new HashSet<Integer>();
185185
assertTrue(ports.add(sm1.getContainer().getReadChannel().getIPCPort()));
@@ -198,11 +198,11 @@ public void testContainerRandomPort() throws IOException {
198198
ozoneConf.setBoolean(OzoneConfigKeys.DFS_CONTAINER_IPC_RANDOM_PORT, false);
199199
try (
200200
DatanodeStateMachine sm1 = new DatanodeStateMachine(
201-
TestUtils.randomDatanodeDetails(), ozoneConf, null);
201+
TestUtils.randomDatanodeDetails(), ozoneConf, null, null);
202202
DatanodeStateMachine sm2 = new DatanodeStateMachine(
203-
TestUtils.randomDatanodeDetails(), ozoneConf, null);
203+
TestUtils.randomDatanodeDetails(), ozoneConf, null, null);
204204
DatanodeStateMachine sm3 = new DatanodeStateMachine(
205-
TestUtils.randomDatanodeDetails(), ozoneConf, null)
205+
TestUtils.randomDatanodeDetails(), ozoneConf, null, null);
206206
) {
207207
HashSet<Integer> ports = new HashSet<Integer>();
208208
assertTrue(ports.add(sm1.getContainer().getReadChannel().getIPCPort()));

0 commit comments

Comments
 (0)