|
@@ -18,9 +18,12 @@
|
|
|
|
|
|
package org.apache.hadoop.hdfs.server.datanode;
|
|
|
|
|
|
+import com.google.common.annotations.VisibleForTesting;
|
|
|
import com.google.common.collect.Lists;
|
|
|
+import com.google.common.collect.Maps;
|
|
|
import com.google.common.util.concurrent.Futures;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
+import org.apache.hadoop.classification.InterfaceStability;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.FileUtil;
|
|
@@ -188,176 +191,215 @@ public class DataStorage extends Storage {
|
|
|
}
|
|
|
return null;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
- * {{@inheritDoc org.apache.hadoop.hdfs.server.common.Storage#writeAll()}}
|
|
|
+ * VolumeBuilder holds the metadata (e.g., the storage directories) of the
|
|
|
+ * prepared volume returned from {@link prepareVolume()}. Calling {@link build()}
|
|
|
+ * to add the metadata to {@link DataStorage} so that this prepared volume can
|
|
|
+ * be active.
|
|
|
*/
|
|
|
- private void writeAll(Collection<StorageDirectory> dirs) throws IOException {
|
|
|
- this.layoutVersion = getServiceLayoutVersion();
|
|
|
- for (StorageDirectory dir : dirs) {
|
|
|
- writeProperties(dir);
|
|
|
+ @InterfaceAudience.Private
|
|
|
+ @InterfaceStability.Unstable
|
|
|
+ static public class VolumeBuilder {
|
|
|
+ private DataStorage storage;
|
|
|
+ /** Volume level storage directory. */
|
|
|
+ private StorageDirectory sd;
|
|
|
+ /** Mapping from block pool ID to an array of storage directories. */
|
|
|
+ private Map<String, List<StorageDirectory>> bpStorageDirMap =
|
|
|
+ Maps.newHashMap();
|
|
|
+
|
|
|
+ @VisibleForTesting
|
|
|
+ public VolumeBuilder(DataStorage storage, StorageDirectory sd) {
|
|
|
+ this.storage = storage;
|
|
|
+ this.sd = sd;
|
|
|
+ }
|
|
|
+
|
|
|
+ public final StorageDirectory getStorageDirectory() {
|
|
|
+ return this.sd;
|
|
|
+ }
|
|
|
+
|
|
|
+ private void addBpStorageDirectories(String bpid,
|
|
|
+ List<StorageDirectory> dirs) {
|
|
|
+ bpStorageDirMap.put(bpid, dirs);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Add loaded metadata of a data volume to {@link DataStorage}.
|
|
|
+ */
|
|
|
+ public void build() {
|
|
|
+ assert this.sd != null;
|
|
|
+ synchronized (storage) {
|
|
|
+ for (Map.Entry<String, List<StorageDirectory>> e :
|
|
|
+ bpStorageDirMap.entrySet()) {
|
|
|
+ final String bpid = e.getKey();
|
|
|
+ BlockPoolSliceStorage bpStorage = this.storage.bpStorageMap.get(bpid);
|
|
|
+ assert bpStorage != null;
|
|
|
+ for (StorageDirectory bpSd : e.getValue()) {
|
|
|
+ bpStorage.addStorageDir(bpSd);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ storage.addStorageDir(sd);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private StorageDirectory loadStorageDirectory(DataNode datanode,
|
|
|
+ NamespaceInfo nsInfo, File dataDir, StartupOption startOpt)
|
|
|
+ throws IOException {
|
|
|
+ StorageDirectory sd = new StorageDirectory(dataDir, null, false);
|
|
|
+ try {
|
|
|
+ StorageState curState = sd.analyzeStorage(startOpt, this);
|
|
|
+ // sd is locked but not opened
|
|
|
+ switch (curState) {
|
|
|
+ case NORMAL:
|
|
|
+ break;
|
|
|
+ case NON_EXISTENT:
|
|
|
+ LOG.info("Storage directory " + dataDir + " does not exist");
|
|
|
+ throw new IOException("Storage directory " + dataDir
|
|
|
+ + " does not exist");
|
|
|
+ case NOT_FORMATTED: // format
|
|
|
+ LOG.info("Storage directory " + dataDir + " is not formatted for "
|
|
|
+ + nsInfo.getBlockPoolID());
|
|
|
+ LOG.info("Formatting ...");
|
|
|
+ format(sd, nsInfo, datanode.getDatanodeUuid());
|
|
|
+ break;
|
|
|
+ default: // recovery part is common
|
|
|
+ sd.doRecover(curState);
|
|
|
+ }
|
|
|
+
|
|
|
+ // 2. Do transitions
|
|
|
+ // Each storage directory is treated individually.
|
|
|
+ // During startup some of them can upgrade or roll back
|
|
|
+ // while others could be up-to-date for the regular startup.
|
|
|
+ doTransition(datanode, sd, nsInfo, startOpt);
|
|
|
+
|
|
|
+ // 3. Update successfully loaded storage.
|
|
|
+ setServiceLayoutVersion(getServiceLayoutVersion());
|
|
|
+ writeProperties(sd);
|
|
|
+
|
|
|
+ return sd;
|
|
|
+ } catch (IOException ioe) {
|
|
|
+ sd.unlock();
|
|
|
+ throw ioe;
|
|
|
}
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Add a list of volumes to be managed by DataStorage. If the volume is empty,
|
|
|
- * format it, otherwise recover it from previous transitions if required.
|
|
|
+ * Prepare a storage directory. It creates a builder which can be used to add
|
|
|
+ * to the volume. If the volume cannot be added, it is OK to discard the
|
|
|
+ * builder later.
|
|
|
*
|
|
|
- * @param datanode the reference to DataNode.
|
|
|
- * @param nsInfo namespace information
|
|
|
- * @param dataDirs array of data storage directories
|
|
|
- * @param startOpt startup option
|
|
|
- * @throws IOException
|
|
|
+ * @param datanode DataNode object.
|
|
|
+ * @param volume the root path of a storage directory.
|
|
|
+ * @param nsInfos an array of namespace infos.
|
|
|
+ * @return a VolumeBuilder that holds the metadata of this storage directory
|
|
|
+ * and can be added to DataStorage later.
|
|
|
+ * @throws IOException if encounters I/O errors.
|
|
|
+ *
|
|
|
+ * Note that if there is IOException, the state of DataStorage is not modified.
|
|
|
*/
|
|
|
- synchronized void addStorageLocations(DataNode datanode,
|
|
|
- NamespaceInfo nsInfo, Collection<StorageLocation> dataDirs,
|
|
|
- StartupOption startOpt)
|
|
|
- throws IOException {
|
|
|
- // Similar to recoverTransitionRead, it first ensures the datanode level
|
|
|
- // format is completed.
|
|
|
- List<StorageLocation> tmpDataDirs =
|
|
|
- new ArrayList<StorageLocation>(dataDirs);
|
|
|
- addStorageLocations(datanode, nsInfo, tmpDataDirs, startOpt, false, true);
|
|
|
-
|
|
|
- Collection<File> bpDataDirs = new ArrayList<File>();
|
|
|
- String bpid = nsInfo.getBlockPoolID();
|
|
|
- for (StorageLocation dir : dataDirs) {
|
|
|
- File dnRoot = dir.getFile();
|
|
|
- File bpRoot = BlockPoolSliceStorage.getBpRoot(bpid, new File(dnRoot,
|
|
|
- STORAGE_DIR_CURRENT));
|
|
|
- bpDataDirs.add(bpRoot);
|
|
|
- }
|
|
|
- // mkdir for the list of BlockPoolStorage
|
|
|
- makeBlockPoolDataDir(bpDataDirs, null);
|
|
|
- BlockPoolSliceStorage bpStorage = this.bpStorageMap.get(bpid);
|
|
|
- if (bpStorage == null) {
|
|
|
- bpStorage = new BlockPoolSliceStorage(
|
|
|
- nsInfo.getNamespaceID(), bpid, nsInfo.getCTime(),
|
|
|
- nsInfo.getClusterID());
|
|
|
+ public VolumeBuilder prepareVolume(DataNode datanode, File volume,
|
|
|
+ List<NamespaceInfo> nsInfos) throws IOException {
|
|
|
+ if (containsStorageDir(volume)) {
|
|
|
+ final String errorMessage = "Storage directory is in use";
|
|
|
+ LOG.warn(errorMessage + ".");
|
|
|
+ throw new IOException(errorMessage);
|
|
|
}
|
|
|
|
|
|
- bpStorage.recoverTransitionRead(datanode, nsInfo, bpDataDirs, startOpt);
|
|
|
- addBlockPoolStorage(bpid, bpStorage);
|
|
|
+ StorageDirectory sd = loadStorageDirectory(
|
|
|
+ datanode, nsInfos.get(0), volume, StartupOption.HOTSWAP);
|
|
|
+ VolumeBuilder builder =
|
|
|
+ new VolumeBuilder(this, sd);
|
|
|
+ for (NamespaceInfo nsInfo : nsInfos) {
|
|
|
+ List<File> bpDataDirs = Lists.newArrayList();
|
|
|
+ bpDataDirs.add(BlockPoolSliceStorage.getBpRoot(
|
|
|
+ nsInfo.getBlockPoolID(), new File(volume, STORAGE_DIR_CURRENT)));
|
|
|
+ makeBlockPoolDataDir(bpDataDirs, null);
|
|
|
+
|
|
|
+ BlockPoolSliceStorage bpStorage;
|
|
|
+ final String bpid = nsInfo.getBlockPoolID();
|
|
|
+ synchronized (this) {
|
|
|
+ bpStorage = this.bpStorageMap.get(bpid);
|
|
|
+ if (bpStorage == null) {
|
|
|
+ bpStorage = new BlockPoolSliceStorage(
|
|
|
+ nsInfo.getNamespaceID(), bpid, nsInfo.getCTime(),
|
|
|
+ nsInfo.getClusterID());
|
|
|
+ addBlockPoolStorage(bpid, bpStorage);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ builder.addBpStorageDirectories(
|
|
|
+ bpid, bpStorage.loadBpStorageDirectories(
|
|
|
+ datanode, nsInfo, bpDataDirs, StartupOption.HOTSWAP));
|
|
|
+ }
|
|
|
+ return builder;
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Add a list of volumes to be managed by this DataStorage. If the volume is
|
|
|
- * empty, it formats the volume, otherwise it recovers it from previous
|
|
|
- * transitions if required.
|
|
|
- *
|
|
|
- * If isInitialize is false, only the directories that have finished the
|
|
|
- * doTransition() process will be added into DataStorage.
|
|
|
+ * Add a list of volumes to be managed by DataStorage. If the volume is empty,
|
|
|
+ * format it, otherwise recover it from previous transitions if required.
|
|
|
*
|
|
|
* @param datanode the reference to DataNode.
|
|
|
* @param nsInfo namespace information
|
|
|
* @param dataDirs array of data storage directories
|
|
|
* @param startOpt startup option
|
|
|
- * @param isInitialize whether it is called when DataNode starts up.
|
|
|
+ * @return a list of successfully loaded volumes.
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- private synchronized void addStorageLocations(DataNode datanode,
|
|
|
+ @VisibleForTesting
|
|
|
+ synchronized List<StorageLocation> addStorageLocations(DataNode datanode,
|
|
|
NamespaceInfo nsInfo, Collection<StorageLocation> dataDirs,
|
|
|
- StartupOption startOpt, boolean isInitialize, boolean ignoreExistingDirs)
|
|
|
- throws IOException {
|
|
|
- Set<String> existingStorageDirs = new HashSet<String>();
|
|
|
- for (int i = 0; i < getNumStorageDirs(); i++) {
|
|
|
- existingStorageDirs.add(getStorageDir(i).getRoot().getAbsolutePath());
|
|
|
- }
|
|
|
-
|
|
|
- // 1. For each data directory calculate its state and check whether all is
|
|
|
- // consistent before transitioning. Format and recover.
|
|
|
- ArrayList<StorageState> dataDirStates =
|
|
|
- new ArrayList<StorageState>(dataDirs.size());
|
|
|
- List<StorageDirectory> addedStorageDirectories =
|
|
|
- new ArrayList<StorageDirectory>();
|
|
|
- for(Iterator<StorageLocation> it = dataDirs.iterator(); it.hasNext();) {
|
|
|
- File dataDir = it.next().getFile();
|
|
|
- if (existingStorageDirs.contains(dataDir.getAbsolutePath())) {
|
|
|
- LOG.info("Storage directory " + dataDir + " has already been used.");
|
|
|
- it.remove();
|
|
|
- continue;
|
|
|
- }
|
|
|
- StorageDirectory sd = new StorageDirectory(dataDir);
|
|
|
- StorageState curState;
|
|
|
- try {
|
|
|
- curState = sd.analyzeStorage(startOpt, this);
|
|
|
- // sd is locked but not opened
|
|
|
- switch (curState) {
|
|
|
- case NORMAL:
|
|
|
- break;
|
|
|
- case NON_EXISTENT:
|
|
|
- // ignore this storage
|
|
|
- LOG.info("Storage directory " + dataDir + " does not exist");
|
|
|
- it.remove();
|
|
|
+ StartupOption startOpt) throws IOException {
|
|
|
+ final String bpid = nsInfo.getBlockPoolID();
|
|
|
+ List<StorageLocation> successVolumes = Lists.newArrayList();
|
|
|
+ for (StorageLocation dataDir : dataDirs) {
|
|
|
+ File root = dataDir.getFile();
|
|
|
+ if (!containsStorageDir(root)) {
|
|
|
+ try {
|
|
|
+ // It first ensures the datanode level format is completed.
|
|
|
+ StorageDirectory sd = loadStorageDirectory(
|
|
|
+ datanode, nsInfo, root, startOpt);
|
|
|
+ addStorageDir(sd);
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.warn(e);
|
|
|
continue;
|
|
|
- case NOT_FORMATTED: // format
|
|
|
- LOG.info("Storage directory " + dataDir + " is not formatted for "
|
|
|
- + nsInfo.getBlockPoolID());
|
|
|
- LOG.info("Formatting ...");
|
|
|
- format(sd, nsInfo, datanode.getDatanodeUuid());
|
|
|
- break;
|
|
|
- default: // recovery part is common
|
|
|
- sd.doRecover(curState);
|
|
|
}
|
|
|
- } catch (IOException ioe) {
|
|
|
- sd.unlock();
|
|
|
- LOG.warn("Ignoring storage directory " + dataDir
|
|
|
- + " due to an exception", ioe);
|
|
|
- //continue with other good dirs
|
|
|
- continue;
|
|
|
- }
|
|
|
- if (isInitialize) {
|
|
|
- addStorageDir(sd);
|
|
|
+ } else {
|
|
|
+ LOG.info("Storage directory " + dataDir + " has already been used.");
|
|
|
}
|
|
|
- addedStorageDirectories.add(sd);
|
|
|
- dataDirStates.add(curState);
|
|
|
- }
|
|
|
|
|
|
- if (dataDirs.size() == 0 || dataDirStates.size() == 0) {
|
|
|
- // none of the data dirs exist
|
|
|
- if (ignoreExistingDirs) {
|
|
|
- return;
|
|
|
- }
|
|
|
- throw new IOException(
|
|
|
- "All specified directories are not accessible or do not exist.");
|
|
|
- }
|
|
|
-
|
|
|
- // 2. Do transitions
|
|
|
- // Each storage directory is treated individually.
|
|
|
- // During startup some of them can upgrade or rollback
|
|
|
- // while others could be up-to-date for the regular startup.
|
|
|
- for (Iterator<StorageDirectory> it = addedStorageDirectories.iterator();
|
|
|
- it.hasNext(); ) {
|
|
|
- StorageDirectory sd = it.next();
|
|
|
+ List<File> bpDataDirs = new ArrayList<File>();
|
|
|
+ bpDataDirs.add(BlockPoolSliceStorage.getBpRoot(bpid, new File(root,
|
|
|
+ STORAGE_DIR_CURRENT)));
|
|
|
try {
|
|
|
- doTransition(datanode, sd, nsInfo, startOpt);
|
|
|
- createStorageID(sd);
|
|
|
- } catch (IOException e) {
|
|
|
- if (!isInitialize) {
|
|
|
- sd.unlock();
|
|
|
- it.remove();
|
|
|
- continue;
|
|
|
+ makeBlockPoolDataDir(bpDataDirs, null);
|
|
|
+ BlockPoolSliceStorage bpStorage = this.bpStorageMap.get(bpid);
|
|
|
+ if (bpStorage == null) {
|
|
|
+ bpStorage = new BlockPoolSliceStorage(
|
|
|
+ nsInfo.getNamespaceID(), bpid, nsInfo.getCTime(),
|
|
|
+ nsInfo.getClusterID());
|
|
|
}
|
|
|
- unlockAll();
|
|
|
- throw e;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- // 3. Update all successfully loaded storages. Some of them might have just
|
|
|
- // been formatted.
|
|
|
- this.writeAll(addedStorageDirectories);
|
|
|
|
|
|
- // 4. Make newly loaded storage directories visible for service.
|
|
|
- if (!isInitialize) {
|
|
|
- this.storageDirs.addAll(addedStorageDirectories);
|
|
|
+ bpStorage.recoverTransitionRead(datanode, nsInfo, bpDataDirs, startOpt);
|
|
|
+ addBlockPoolStorage(bpid, bpStorage);
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.warn("Failed to add storage for block pool: " + bpid + " : "
|
|
|
+ + e.getMessage());
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+ successVolumes.add(dataDir);
|
|
|
}
|
|
|
+ return successVolumes;
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Remove volumes from DataStorage.
|
|
|
+ * Remove volumes from DataStorage. All volumes are removed even when the
|
|
|
+ * IOException is thrown.
|
|
|
+ *
|
|
|
* @param locations a collection of volumes.
|
|
|
+ * @throws IOException if I/O error when unlocking storage directory.
|
|
|
*/
|
|
|
- synchronized void removeVolumes(Collection<StorageLocation> locations) {
|
|
|
+ synchronized void removeVolumes(Collection<StorageLocation> locations)
|
|
|
+ throws IOException {
|
|
|
if (locations.isEmpty()) {
|
|
|
return;
|
|
|
}
|
|
@@ -371,6 +413,7 @@ public class DataStorage extends Storage {
|
|
|
bpsStorage.removeVolumes(dataDirs);
|
|
|
}
|
|
|
|
|
|
+ StringBuilder errorMsgBuilder = new StringBuilder();
|
|
|
for (Iterator<StorageDirectory> it = this.storageDirs.iterator();
|
|
|
it.hasNext(); ) {
|
|
|
StorageDirectory sd = it.next();
|
|
@@ -382,13 +425,18 @@ public class DataStorage extends Storage {
|
|
|
LOG.warn(String.format(
|
|
|
"I/O error attempting to unlock storage directory %s.",
|
|
|
sd.getRoot()), e);
|
|
|
+ errorMsgBuilder.append(String.format("Failed to remove %s: %s\n",
|
|
|
+ sd.getRoot(), e.getMessage()));
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
+ if (errorMsgBuilder.length() > 0) {
|
|
|
+ throw new IOException(errorMsgBuilder.toString());
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Analyze storage directories.
|
|
|
+ * Analyze storage directories for a specific block pool.
|
|
|
* Recover from previous transitions if required.
|
|
|
* Perform fs state transition if necessary depending on the namespace info.
|
|
|
* Read storage info.
|
|
@@ -396,60 +444,25 @@ public class DataStorage extends Storage {
|
|
|
* This method should be synchronized between multiple DN threads. Only the
|
|
|
* first DN thread does DN level storage dir recoverTransitionRead.
|
|
|
*
|
|
|
- * @param nsInfo namespace information
|
|
|
- * @param dataDirs array of data storage directories
|
|
|
- * @param startOpt startup option
|
|
|
- * @throws IOException
|
|
|
- */
|
|
|
- synchronized void recoverTransitionRead(DataNode datanode,
|
|
|
- NamespaceInfo nsInfo, Collection<StorageLocation> dataDirs,
|
|
|
- StartupOption startOpt)
|
|
|
- throws IOException {
|
|
|
- if (initialized) {
|
|
|
- // DN storage has been initialized, no need to do anything
|
|
|
- return;
|
|
|
- }
|
|
|
- LOG.info("DataNode version: " + HdfsConstants.DATANODE_LAYOUT_VERSION
|
|
|
- + " and NameNode layout version: " + nsInfo.getLayoutVersion());
|
|
|
-
|
|
|
- this.storageDirs = new ArrayList<StorageDirectory>(dataDirs.size());
|
|
|
- addStorageLocations(datanode, nsInfo, dataDirs, startOpt, true, false);
|
|
|
-
|
|
|
- // mark DN storage is initialized
|
|
|
- this.initialized = true;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * recoverTransitionRead for a specific block pool
|
|
|
- *
|
|
|
* @param datanode DataNode
|
|
|
- * @param bpID Block pool Id
|
|
|
* @param nsInfo Namespace info of namenode corresponding to the block pool
|
|
|
* @param dataDirs Storage directories
|
|
|
* @param startOpt startup option
|
|
|
* @throws IOException on error
|
|
|
*/
|
|
|
- void recoverTransitionRead(DataNode datanode, String bpID, NamespaceInfo nsInfo,
|
|
|
+ void recoverTransitionRead(DataNode datanode, NamespaceInfo nsInfo,
|
|
|
Collection<StorageLocation> dataDirs, StartupOption startOpt) throws IOException {
|
|
|
- // First ensure datanode level format/snapshot/rollback is completed
|
|
|
- recoverTransitionRead(datanode, nsInfo, dataDirs, startOpt);
|
|
|
-
|
|
|
- // Create list of storage directories for the block pool
|
|
|
- Collection<File> bpDataDirs = new ArrayList<File>();
|
|
|
- for(StorageLocation dir : dataDirs) {
|
|
|
- File dnRoot = dir.getFile();
|
|
|
- File bpRoot = BlockPoolSliceStorage.getBpRoot(bpID, new File(dnRoot,
|
|
|
- STORAGE_DIR_CURRENT));
|
|
|
- bpDataDirs.add(bpRoot);
|
|
|
+ if (this.initialized) {
|
|
|
+ LOG.info("DataNode version: " + HdfsConstants.DATANODE_LAYOUT_VERSION
|
|
|
+ + " and NameNode layout version: " + nsInfo.getLayoutVersion());
|
|
|
+ this.storageDirs = new ArrayList<StorageDirectory>(dataDirs.size());
|
|
|
+ // mark DN storage is initialized
|
|
|
+ this.initialized = true;
|
|
|
}
|
|
|
|
|
|
- // mkdir for the list of BlockPoolStorage
|
|
|
- makeBlockPoolDataDir(bpDataDirs, null);
|
|
|
- BlockPoolSliceStorage bpStorage = new BlockPoolSliceStorage(
|
|
|
- nsInfo.getNamespaceID(), bpID, nsInfo.getCTime(), nsInfo.getClusterID());
|
|
|
-
|
|
|
- bpStorage.recoverTransitionRead(datanode, nsInfo, bpDataDirs, startOpt);
|
|
|
- addBlockPoolStorage(bpID, bpStorage);
|
|
|
+ if (addStorageLocations(datanode, nsInfo, dataDirs, startOpt).isEmpty()) {
|
|
|
+ throw new IOException("All specified directories are failed to load.");
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -665,12 +678,15 @@ public class DataStorage extends Storage {
|
|
|
// meaningful at BlockPoolSliceStorage level.
|
|
|
|
|
|
// regular start up.
|
|
|
- if (this.layoutVersion == HdfsConstants.DATANODE_LAYOUT_VERSION)
|
|
|
+ if (this.layoutVersion == HdfsConstants.DATANODE_LAYOUT_VERSION) {
|
|
|
+ createStorageID(sd);
|
|
|
return; // regular startup
|
|
|
+ }
|
|
|
|
|
|
// do upgrade
|
|
|
if (this.layoutVersion > HdfsConstants.DATANODE_LAYOUT_VERSION) {
|
|
|
doUpgrade(datanode, sd, nsInfo); // upgrade
|
|
|
+ createStorageID(sd);
|
|
|
return;
|
|
|
}
|
|
|
|