HDDS-257. Hook up VolumeSet#shutdown from HddsDispatcher#shutdown. Contributed by Hanisha Koneru
This commit is contained in:
parent
de894d34f6
commit
ba25d27ddb
@ -81,6 +81,8 @@ public void init() {
|
||||
|
||||
@Override
|
||||
public void shutdown() {
|
||||
// Shutdown the volumes
|
||||
volumeSet.shutdown();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -370,6 +370,4 @@ public enum VolumeState {
|
||||
public void setScmUsageForTesting(GetSpaceUsed scmUsageForTest) {
|
||||
volumeInfo.setScmUsageForTesting(scmUsageForTest);
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
@ -129,4 +129,12 @@ public StorageType getStorageType() {
|
||||
public void setScmUsageForTesting(GetSpaceUsed scmUsageForTest) {
|
||||
usage.setScmUsageForTesting(scmUsageForTest);
|
||||
}
|
||||
|
||||
/**
|
||||
* Only for testing. Do not use otherwise.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public VolumeUsage getUsageForTesting() {
|
||||
return usage;
|
||||
}
|
||||
}
|
||||
|
@ -23,9 +23,11 @@
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.StorageType;
|
||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||
|
||||
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.HDDS_DATANODE_DIR_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
|
||||
import static org.apache.hadoop.util.RunJar.SHUTDOWN_HOOK_PRIORITY;
|
||||
|
||||
import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
|
||||
import org.apache.hadoop.hdds.protocol.proto
|
||||
.StorageContainerDatanodeProtocolProtos;
|
||||
@ -40,6 +42,7 @@
|
||||
import org.apache.hadoop.util.AutoCloseableLock;
|
||||
import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
|
||||
import org.apache.hadoop.util.InstrumentedLock;
|
||||
import org.apache.hadoop.util.ShutdownHookManager;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
@ -89,6 +92,8 @@ public class VolumeSet {
|
||||
private final String datanodeUuid;
|
||||
private String clusterID;
|
||||
|
||||
private Runnable shutdownHook;
|
||||
|
||||
public VolumeSet(String dnUuid, Configuration conf)
|
||||
throws DiskOutOfSpaceException {
|
||||
this(dnUuid, null, conf);
|
||||
@ -155,6 +160,13 @@ private void initializeVolumeSet() throws DiskOutOfSpaceException {
|
||||
if (volumeMap.size() == 0) {
|
||||
throw new DiskOutOfSpaceException("No storage location configured");
|
||||
}
|
||||
|
||||
// Ensure volume threads are stopped and scm df is saved during shutdown.
|
||||
shutdownHook = () -> {
|
||||
shutdown();
|
||||
};
|
||||
ShutdownHookManager.get().addShutdownHook(shutdownHook,
|
||||
SHUTDOWN_HOOK_PRIORITY);
|
||||
}
|
||||
|
||||
/**
|
||||
@ -296,6 +308,10 @@ public void shutdown() {
|
||||
ex);
|
||||
}
|
||||
}
|
||||
|
||||
if (shutdownHook != null) {
|
||||
ShutdownHookManager.get().removeShutdownHook(shutdownHook);
|
||||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
|
@ -24,8 +24,6 @@
|
||||
import org.apache.hadoop.fs.DF;
|
||||
import org.apache.hadoop.fs.GetSpaceUsed;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import static org.apache.hadoop.util.RunJar.SHUTDOWN_HOOK_PRIORITY;
|
||||
import org.apache.hadoop.util.ShutdownHookManager;
|
||||
import org.apache.hadoop.util.Time;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
@ -49,7 +47,6 @@ public class VolumeUsage {
|
||||
private final DF df;
|
||||
private final File scmUsedFile;
|
||||
private GetSpaceUsed scmUsage;
|
||||
private Runnable shutdownHook;
|
||||
|
||||
private static final String DU_CACHE_FILE = "scmUsed";
|
||||
private volatile boolean scmUsedSaved = false;
|
||||
@ -72,15 +69,6 @@ void startScmUsageThread(Configuration conf) throws IOException {
|
||||
.setConf(conf)
|
||||
.setInitialUsed(loadScmUsed())
|
||||
.build();
|
||||
|
||||
// Ensure scm df is saved during shutdown.
|
||||
shutdownHook = () -> {
|
||||
if (!scmUsedSaved) {
|
||||
saveScmUsed();
|
||||
}
|
||||
};
|
||||
ShutdownHookManager.get().addShutdownHook(shutdownHook,
|
||||
SHUTDOWN_HOOK_PRIORITY);
|
||||
}
|
||||
|
||||
long getCapacity() {
|
||||
@ -106,11 +94,6 @@ long getScmUsed() throws IOException{
|
||||
|
||||
public void shutdown() {
|
||||
saveScmUsed();
|
||||
scmUsedSaved = true;
|
||||
|
||||
if (shutdownHook != null) {
|
||||
ShutdownHookManager.get().removeShutdownHook(shutdownHook);
|
||||
}
|
||||
|
||||
if (scmUsage instanceof CachingGetSpaceUsed) {
|
||||
IOUtils.cleanupWithLogger(null, ((CachingGetSpaceUsed) scmUsage));
|
||||
|
@ -34,8 +34,10 @@
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
@ -87,6 +89,7 @@ public void shutdown() throws IOException {
|
||||
for (HddsVolume volume : volumes) {
|
||||
FileUtils.deleteDirectory(volume.getHddsRootDir());
|
||||
}
|
||||
volumeSet.shutdown();
|
||||
}
|
||||
|
||||
private boolean checkVolumeExistsInVolumeSet(String volume) {
|
||||
@ -120,9 +123,6 @@ public void testAddVolume() {
|
||||
|
||||
// Add a volume to VolumeSet
|
||||
String volume3 = baseDir + "disk3";
|
||||
// File dir3 = new File(volume3, "hdds");
|
||||
// File[] files = dir3.listFiles();
|
||||
// System.out.println("------ " + files[0].getPath());
|
||||
boolean success = volumeSet.addVolume(volume3);
|
||||
|
||||
assertTrue(success);
|
||||
@ -204,4 +204,24 @@ public void testVolumeInInconsistentState() throws Exception {
|
||||
File volume = new File(volume3);
|
||||
FileUtils.deleteDirectory(volume);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testShutdown() throws Exception {
|
||||
List<HddsVolume> volumesList = volumeSet.getVolumesList();
|
||||
|
||||
volumeSet.shutdown();
|
||||
|
||||
// Verify that the volumes are shutdown and the volumeUsage is set to null.
|
||||
for (HddsVolume volume : volumesList) {
|
||||
Assert.assertNull(volume.getVolumeInfo().getUsageForTesting());
|
||||
try {
|
||||
// getAvailable() should throw null pointer exception as usage is null.
|
||||
volume.getAvailable();
|
||||
fail("Volume shutdown failed.");
|
||||
} catch (NullPointerException ex) {
|
||||
// Do Nothing. Exception is expected.
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user