|
@@ -52,6 +52,7 @@ import static org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStag
|
|
|
import static org.apache.hadoop.util.ExitUtil.terminate;
|
|
|
|
|
|
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
|
|
+import org.apache.hadoop.fs.DF;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.ReconfigurationProtocolProtos.ReconfigurationProtocolService;
|
|
|
|
|
|
import java.io.BufferedOutputStream;
|
|
@@ -59,6 +60,7 @@ import java.io.ByteArrayInputStream;
|
|
|
import java.io.DataInputStream;
|
|
|
import java.io.DataOutputStream;
|
|
|
import java.io.EOFException;
|
|
|
+import java.io.File;
|
|
|
import java.io.FileInputStream;
|
|
|
import java.io.FileNotFoundException;
|
|
|
import java.io.IOException;
|
|
@@ -116,7 +118,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.checker.DatasetVolumeChecker;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.checker.StorageLocationChecker;
|
|
|
import org.apache.hadoop.hdfs.util.DataTransferThrottler;
|
|
|
-import org.apache.hadoop.util.AutoCloseableLock;
|
|
|
+import org.apache.hadoop.util.*;
|
|
|
import org.apache.hadoop.hdfs.client.BlockReportOptions;
|
|
|
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.net.DomainPeerServer;
|
|
@@ -209,15 +211,7 @@ import org.apache.hadoop.tracing.TraceAdminProtocolPB;
|
|
|
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.DiskErrorException;
|
|
|
-import org.apache.hadoop.util.GenericOptionsParser;
|
|
|
-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.hadoop.util.concurrent.HadoopExecutors;
|
|
|
import org.apache.htrace.core.Tracer;
|
|
|
import org.eclipse.jetty.util.ajax.JSON;
|
|
@@ -2774,6 +2768,51 @@ public class DataNode extends ReconfigurableBase
|
|
|
return makeInstance(dataLocations, conf, resources);
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Get the effective file system where the path is located.
|
|
|
+ * DF is a packaged cross-platform class, it can get volumes
|
|
|
+ * information from current system.
|
|
|
+ * @param path - absolute or fully qualified path
|
|
|
+ * @param conf - the Configuration
|
|
|
+ * @return the effective filesystem of the path
|
|
|
+ */
|
|
|
+ private static String getEffectiveFileSystem(
|
|
|
+ String path, Configuration conf) {
|
|
|
+ try {
|
|
|
+ DF df = new DF(new File(path), conf);
|
|
|
+ return df.getFilesystem();
|
|
|
+ } catch (IOException ioe) {
|
|
|
+ LOG.error("Failed to get filesystem for dir {}", path, ioe);
|
|
|
+ }
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Sometimes we mount different disks for different storage types
|
|
|
+ * as the storage location. It's important to check the volume is
|
|
|
+ * mounted rightly before initializing storage locations.
|
|
|
+ * @param conf - Configuration
|
|
|
+ * @param location - Storage location
|
|
|
+ * @return false if the filesystem of location is configured and mismatch
|
|
|
+ * with effective filesystem.
|
|
|
+ */
|
|
|
+ private static boolean checkFileSystemWithConfigured(
|
|
|
+ Configuration conf, StorageLocation location) {
|
|
|
+ String configFs = StorageType.getConf(
|
|
|
+ conf, location.getStorageType(), "filesystem");
|
|
|
+ if (configFs != null && !configFs.isEmpty()) {
|
|
|
+ String effectiveFs = getEffectiveFileSystem(
|
|
|
+ location.getUri().getPath(), conf);
|
|
|
+ if (effectiveFs == null || !effectiveFs.equals(configFs)) {
|
|
|
+ LOG.error("Filesystem mismatch for storage location {}. " +
|
|
|
+ "Configured is {}, effective is {}.",
|
|
|
+ location.getUri(), configFs, effectiveFs);
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+
|
|
|
public static List<StorageLocation> getStorageLocations(Configuration conf) {
|
|
|
Collection<String> rawLocations =
|
|
|
conf.getTrimmedStringCollection(DFS_DATANODE_DATA_DIR_KEY);
|
|
@@ -2790,8 +2829,9 @@ public class DataNode extends ReconfigurableBase
|
|
|
// Ignore the exception.
|
|
|
continue;
|
|
|
}
|
|
|
-
|
|
|
- locations.add(location);
|
|
|
+ if(checkFileSystemWithConfigured(conf, location)) {
|
|
|
+ locations.add(location);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
return locations;
|