|
@@ -21,8 +21,6 @@ package org.apache.hadoop.hdfs.server.datanode;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_ADDRESS_DEFAULT;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_PERMISSION_DEFAULT;
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_PERMISSION_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_DEFAULT;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_INTERFACE_KEY;
|
|
@@ -67,7 +65,6 @@ import java.io.PrintStream;
|
|
|
import java.lang.management.ManagementFactory;
|
|
|
import java.net.InetSocketAddress;
|
|
|
import java.net.Socket;
|
|
|
-import java.net.URI;
|
|
|
import java.net.UnknownHostException;
|
|
|
import java.nio.channels.ServerSocketChannel;
|
|
|
import java.security.PrivilegedExceptionAction;
|
|
@@ -92,6 +89,7 @@ import java.util.concurrent.ThreadLocalRandom;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
import java.util.concurrent.atomic.AtomicInteger;
|
|
|
|
|
|
+import javax.annotation.Nullable;
|
|
|
import javax.management.ObjectName;
|
|
|
import javax.net.SocketFactory;
|
|
|
|
|
@@ -103,16 +101,14 @@ import org.apache.hadoop.conf.ReconfigurableBase;
|
|
|
import org.apache.hadoop.conf.ReconfigurationException;
|
|
|
import org.apache.hadoop.conf.ReconfigurationTaskStatus;
|
|
|
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
|
|
-import org.apache.hadoop.fs.FileSystem;
|
|
|
-import org.apache.hadoop.fs.LocalFileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.StorageType;
|
|
|
-import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
|
import org.apache.hadoop.hdfs.DFSUtilClient;
|
|
|
import org.apache.hadoop.hdfs.HDFSPolicyProvider;
|
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.checker.StorageLocationChecker;
|
|
|
import org.apache.hadoop.util.AutoCloseableLock;
|
|
|
import org.apache.hadoop.hdfs.client.BlockReportOptions;
|
|
|
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
|
|
@@ -203,7 +199,6 @@ import org.apache.hadoop.tracing.TraceAdminProtocolServerSideTranslatorPB;
|
|
|
import org.apache.hadoop.tracing.TraceUtils;
|
|
|
import org.apache.hadoop.tracing.TracerConfigurationManager;
|
|
|
import org.apache.hadoop.util.Daemon;
|
|
|
-import org.apache.hadoop.util.DiskChecker;
|
|
|
import org.apache.hadoop.util.DiskChecker.DiskErrorException;
|
|
|
import org.apache.hadoop.util.GenericOptionsParser;
|
|
|
import org.apache.hadoop.util.InvalidChecksumSizeException;
|
|
@@ -211,6 +206,7 @@ import org.apache.hadoop.util.JvmPauseMonitor;
|
|
|
import org.apache.hadoop.util.ServicePlugin;
|
|
|
import org.apache.hadoop.util.StringUtils;
|
|
|
import org.apache.hadoop.util.Time;
|
|
|
+import org.apache.hadoop.util.Timer;
|
|
|
import org.apache.hadoop.util.VersionInfo;
|
|
|
import org.apache.htrace.core.Tracer;
|
|
|
import org.eclipse.jetty.util.ajax.JSON;
|
|
@@ -389,6 +385,9 @@ public class DataNode extends ReconfigurableBase
|
|
|
private static final double CONGESTION_RATIO = 1.5;
|
|
|
private DiskBalancer diskBalancer;
|
|
|
|
|
|
+ @Nullable
|
|
|
+ private final StorageLocationChecker storageLocationChecker;
|
|
|
+
|
|
|
|
|
|
private final SocketFactory socketFactory;
|
|
|
|
|
@@ -423,6 +422,7 @@ public class DataNode extends ReconfigurableBase
|
|
|
ThreadLocalRandom.current().nextInt(5000, (int) (5000 * 1.25));
|
|
|
this.socketFactory = NetUtils.getDefaultSocketFactory(conf);
|
|
|
initOOBTimeout();
|
|
|
+ storageLocationChecker = null;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -431,6 +431,7 @@ public class DataNode extends ReconfigurableBase
|
|
|
*/
|
|
|
DataNode(final Configuration conf,
|
|
|
final List<StorageLocation> dataDirs,
|
|
|
+ final StorageLocationChecker storageLocationChecker,
|
|
|
final SecureResources resources) throws IOException {
|
|
|
super(conf);
|
|
|
this.tracer = createTracer(conf);
|
|
@@ -506,6 +507,7 @@ public class DataNode extends ReconfigurableBase
|
|
|
});
|
|
|
|
|
|
initOOBTimeout();
|
|
|
+ this.storageLocationChecker = storageLocationChecker;
|
|
|
}
|
|
|
|
|
|
@Override // ReconfigurableBase
|
|
@@ -1935,6 +1937,10 @@ public class DataNode extends ReconfigurableBase
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ if (storageLocationChecker != null) {
|
|
|
+ storageLocationChecker.shutdownAndWait(1, TimeUnit.SECONDS);
|
|
|
+ }
|
|
|
+
|
|
|
if (pauseMonitor != null) {
|
|
|
pauseMonitor.stop();
|
|
|
}
|
|
@@ -2620,21 +2626,6 @@ public class DataNode extends ReconfigurableBase
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- // Small wrapper around the DiskChecker class that provides means to mock
|
|
|
- // DiskChecker static methods and unittest DataNode#getDataDirsFromURIs.
|
|
|
- static class DataNodeDiskChecker {
|
|
|
- private final FsPermission expectedPermission;
|
|
|
-
|
|
|
- public DataNodeDiskChecker(FsPermission expectedPermission) {
|
|
|
- this.expectedPermission = expectedPermission;
|
|
|
- }
|
|
|
-
|
|
|
- public void checkDir(LocalFileSystem localFS, Path path)
|
|
|
- throws DiskErrorException, IOException {
|
|
|
- DiskChecker.checkDir(localFS, path, expectedPermission);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Make an instance of DataNode after ensuring that at least one of the
|
|
|
* given data directories (and their parent directories, if necessary)
|
|
@@ -2649,44 +2640,18 @@ public class DataNode extends ReconfigurableBase
|
|
|
*/
|
|
|
static DataNode makeInstance(Collection<StorageLocation> dataDirs,
|
|
|
Configuration conf, SecureResources resources) throws IOException {
|
|
|
- LocalFileSystem localFS = FileSystem.getLocal(conf);
|
|
|
- FsPermission permission = new FsPermission(
|
|
|
- conf.get(DFS_DATANODE_DATA_DIR_PERMISSION_KEY,
|
|
|
- DFS_DATANODE_DATA_DIR_PERMISSION_DEFAULT));
|
|
|
- DataNodeDiskChecker dataNodeDiskChecker =
|
|
|
- new DataNodeDiskChecker(permission);
|
|
|
- List<StorageLocation> locations =
|
|
|
- checkStorageLocations(dataDirs, localFS, dataNodeDiskChecker);
|
|
|
+ List<StorageLocation> locations;
|
|
|
+ StorageLocationChecker storageLocationChecker =
|
|
|
+ new StorageLocationChecker(conf, new Timer());
|
|
|
+ try {
|
|
|
+ locations = storageLocationChecker.check(conf, dataDirs);
|
|
|
+ } catch (InterruptedException ie) {
|
|
|
+ throw new IOException("Failed to instantiate DataNode", ie);
|
|
|
+ }
|
|
|
DefaultMetricsSystem.initialize("DataNode");
|
|
|
|
|
|
assert locations.size() > 0 : "number of data directories should be > 0";
|
|
|
- return new DataNode(conf, locations, resources);
|
|
|
- }
|
|
|
-
|
|
|
- // DataNode ctor expects AbstractList instead of List or Collection...
|
|
|
- static List<StorageLocation> checkStorageLocations(
|
|
|
- Collection<StorageLocation> dataDirs,
|
|
|
- LocalFileSystem localFS, DataNodeDiskChecker dataNodeDiskChecker)
|
|
|
- throws IOException {
|
|
|
- ArrayList<StorageLocation> locations = new ArrayList<StorageLocation>();
|
|
|
- StringBuilder invalidDirs = new StringBuilder();
|
|
|
- for (StorageLocation location : dataDirs) {
|
|
|
- final URI uri = location.getUri();
|
|
|
- try {
|
|
|
- dataNodeDiskChecker.checkDir(localFS, new Path(uri));
|
|
|
- locations.add(location);
|
|
|
- } catch (IOException ioe) {
|
|
|
- LOG.warn("Invalid " + DFS_DATANODE_DATA_DIR_KEY + " "
|
|
|
- + location + " : ", ioe);
|
|
|
- invalidDirs.append("\"").append(uri.getPath()).append("\" ");
|
|
|
- }
|
|
|
- }
|
|
|
- if (locations.size() == 0) {
|
|
|
- throw new IOException("All directories in "
|
|
|
- + DFS_DATANODE_DATA_DIR_KEY + " are invalid: "
|
|
|
- + invalidDirs);
|
|
|
- }
|
|
|
- return locations;
|
|
|
+ return new DataNode(conf, locations, storageLocationChecker, resources);
|
|
|
}
|
|
|
|
|
|
@Override
|