|
@@ -44,6 +44,7 @@ import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
|
+import org.apache.hadoop.hdfs.StorageType;
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
|
|
|
import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
|
|
@@ -65,6 +66,7 @@ import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.ReplicaUnderRecovery;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.ReplicaWaitingToBeRecovered;
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
|
|
@@ -188,6 +190,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_DEFAULT);
|
|
|
|
|
|
String[] dataDirs = conf.getTrimmedStrings(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY);
|
|
|
+ Collection<StorageLocation> dataLocations = DataNode.getStorageLocations(conf);
|
|
|
|
|
|
int volsConfigured = (dataDirs == null) ? 0 : dataDirs.length;
|
|
|
int volsFailed = volsConfigured - storage.getNumStorageDirs();
|
|
@@ -209,8 +212,13 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
storage.getNumStorageDirs());
|
|
|
for (int idx = 0; idx < storage.getNumStorageDirs(); idx++) {
|
|
|
final File dir = storage.getStorageDir(idx).getCurrentDir();
|
|
|
- volArray.add(new FsVolumeImpl(this, storage.getStorageID(), dir, conf));
|
|
|
- LOG.info("Added volume - " + dir);
|
|
|
+ // TODO: getStorageTypeFromLocations() is only a temporary workaround and
|
|
|
+ // should be replaced with getting storage type from DataStorage (missing
|
|
|
+ // storage type now) directly.
|
|
|
+ final StorageType storageType = getStorageTypeFromLocations(dataLocations, dir);
|
|
|
+ volArray.add(new FsVolumeImpl(this, storage.getStorageID(), dir, conf,
|
|
|
+ storageType));
|
|
|
+ LOG.info("Added volume - " + dir + ", StorageType: " + storageType);
|
|
|
}
|
|
|
volumeMap = new ReplicaMap(this);
|
|
|
|
|
@@ -231,6 +239,16 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
registerMBean(storage.getStorageID());
|
|
|
}
|
|
|
|
|
|
+ private StorageType getStorageTypeFromLocations(
|
|
|
+ Collection<StorageLocation> dataLocations, File dir) {
|
|
|
+ for (StorageLocation dataLocation : dataLocations) {
|
|
|
+ if (dataLocation.getFile().equals(dir)) {
|
|
|
+ return dataLocation.getStorageType();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return StorageType.DEFAULT;
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Return the total space used by dfs datanode
|
|
|
*/
|