|
@@ -202,6 +202,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
final Map<String, DatanodeStorage> storageMap;
|
|
|
final FsDatasetAsyncDiskService asyncDiskService;
|
|
|
final FsDatasetCache cacheManager;
|
|
|
+ private final Configuration conf;
|
|
|
private final int validVolsRequired;
|
|
|
|
|
|
final ReplicaMap volumeMap;
|
|
@@ -216,6 +217,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
) throws IOException {
|
|
|
this.datanode = datanode;
|
|
|
this.dataStorage = storage;
|
|
|
+ this.conf = conf;
|
|
|
// The number of volumes required for operation is the total number
|
|
|
// of volumes minus the number of failed volumes we can tolerate.
|
|
|
final int volFailuresTolerated =
|
|
@@ -242,38 +244,76 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
}
|
|
|
|
|
|
storageMap = new HashMap<String, DatanodeStorage>();
|
|
|
- final List<FsVolumeImpl> volArray = new ArrayList<FsVolumeImpl>(
|
|
|
- storage.getNumStorageDirs());
|
|
|
- for (int idx = 0; idx < storage.getNumStorageDirs(); idx++) {
|
|
|
- Storage.StorageDirectory sd = storage.getStorageDir(idx);
|
|
|
- final File dir = sd.getCurrentDir();
|
|
|
- final StorageType storageType = getStorageTypeFromLocations(dataLocations, sd.getRoot());
|
|
|
- volArray.add(new FsVolumeImpl(this, sd.getStorageUuid(), dir, conf,
|
|
|
- storageType));
|
|
|
- LOG.info("Added volume - " + dir + ", StorageType: " + storageType);
|
|
|
- storageMap.put(sd.getStorageUuid(),
|
|
|
- new DatanodeStorage(sd.getStorageUuid(), DatanodeStorage.State.NORMAL, storageType));
|
|
|
- }
|
|
|
volumeMap = new ReplicaMap(this);
|
|
|
-
|
|
|
@SuppressWarnings("unchecked")
|
|
|
final VolumeChoosingPolicy<FsVolumeImpl> blockChooserImpl =
|
|
|
ReflectionUtils.newInstance(conf.getClass(
|
|
|
DFSConfigKeys.DFS_DATANODE_FSDATASET_VOLUME_CHOOSING_POLICY_KEY,
|
|
|
RoundRobinVolumeChoosingPolicy.class,
|
|
|
VolumeChoosingPolicy.class), conf);
|
|
|
- volumes = new FsVolumeList(volArray, volsFailed, blockChooserImpl);
|
|
|
- volumes.initializeReplicaMaps(volumeMap);
|
|
|
+ volumes = new FsVolumeList(volsFailed, blockChooserImpl);
|
|
|
+ asyncDiskService = new FsDatasetAsyncDiskService(datanode);
|
|
|
|
|
|
- File[] roots = new File[storage.getNumStorageDirs()];
|
|
|
for (int idx = 0; idx < storage.getNumStorageDirs(); idx++) {
|
|
|
- roots[idx] = storage.getStorageDir(idx).getCurrentDir();
|
|
|
+ addVolume(dataLocations, storage.getStorageDir(idx));
|
|
|
}
|
|
|
- asyncDiskService = new FsDatasetAsyncDiskService(datanode, roots);
|
|
|
+
|
|
|
cacheManager = new FsDatasetCache(this);
|
|
|
registerMBean(datanode.getDatanodeUuid());
|
|
|
}
|
|
|
|
|
|
+ private void addVolume(Collection<StorageLocation> dataLocations,
|
|
|
+ Storage.StorageDirectory sd) throws IOException {
|
|
|
+ final File dir = sd.getCurrentDir();
|
|
|
+ final StorageType storageType =
|
|
|
+ getStorageTypeFromLocations(dataLocations, sd.getRoot());
|
|
|
+
|
|
|
+ // If IOException raises from FsVolumeImpl() or getVolumeMap(), there is
|
|
|
+ // nothing needed to be rolled back to make various data structures, e.g.,
|
|
|
+ // storageMap and asyncDiskService, consistent.
|
|
|
+ FsVolumeImpl fsVolume = new FsVolumeImpl(
|
|
|
+ this, sd.getStorageUuid(), dir, this.conf, storageType);
|
|
|
+ fsVolume.getVolumeMap(volumeMap);
|
|
|
+
|
|
|
+ volumes.addVolume(fsVolume);
|
|
|
+ storageMap.put(sd.getStorageUuid(),
|
|
|
+ new DatanodeStorage(sd.getStorageUuid(),
|
|
|
+ DatanodeStorage.State.NORMAL,
|
|
|
+ storageType));
|
|
|
+ asyncDiskService.addVolume(sd.getCurrentDir());
|
|
|
+
|
|
|
+ LOG.info("Added volume - " + dir + ", StorageType: " + storageType);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Add an array of StorageLocation to FsDataset.
|
|
|
+ *
|
|
|
+ * @pre dataStorage must have these volumes.
|
|
|
+ * @param volumes
|
|
|
+ * @throws IOException
|
|
|
+ */
|
|
|
+ @Override
|
|
|
+ public synchronized void addVolumes(Collection<StorageLocation> volumes)
|
|
|
+ throws IOException {
|
|
|
+ final Collection<StorageLocation> dataLocations =
|
|
|
+ DataNode.getStorageLocations(this.conf);
|
|
|
+ Map<String, Storage.StorageDirectory> allStorageDirs =
|
|
|
+ new HashMap<String, Storage.StorageDirectory>();
|
|
|
+ for (int idx = 0; idx < dataStorage.getNumStorageDirs(); idx++) {
|
|
|
+ Storage.StorageDirectory sd = dataStorage.getStorageDir(idx);
|
|
|
+ allStorageDirs.put(sd.getRoot().getAbsolutePath(), sd);
|
|
|
+ }
|
|
|
+
|
|
|
+ for (StorageLocation vol : volumes) {
|
|
|
+ String key = vol.getFile().getAbsolutePath();
|
|
|
+ if (!allStorageDirs.containsKey(key)) {
|
|
|
+ LOG.warn("Attempt to add an invalid volume: " + vol.getFile());
|
|
|
+ } else {
|
|
|
+ addVolume(dataLocations, allStorageDirs.get(key));
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
private StorageType getStorageTypeFromLocations(
|
|
|
Collection<StorageLocation> dataLocations, File dir) {
|
|
|
for (StorageLocation dataLocation : dataLocations) {
|