|
@@ -18,11 +18,29 @@
|
|
|
|
|
|
package org.apache.hadoop.hdfs.server.datanode;
|
|
package org.apache.hadoop.hdfs.server.datanode;
|
|
|
|
|
|
-import com.google.common.annotations.VisibleForTesting;
|
|
|
|
-import com.google.common.collect.ComparisonChain;
|
|
|
|
-import com.google.common.collect.Lists;
|
|
|
|
-import com.google.common.collect.Maps;
|
|
|
|
-import com.google.common.util.concurrent.Futures;
|
|
|
|
|
|
+import java.io.File;
|
|
|
|
+import java.io.IOException;
|
|
|
|
+import java.io.RandomAccessFile;
|
|
|
|
+import java.nio.channels.FileLock;
|
|
|
|
+import java.nio.channels.OverlappingFileLockException;
|
|
|
|
+import java.util.ArrayList;
|
|
|
|
+import java.util.Collection;
|
|
|
|
+import java.util.Collections;
|
|
|
|
+import java.util.Comparator;
|
|
|
|
+import java.util.HashMap;
|
|
|
|
+import java.util.Iterator;
|
|
|
|
+import java.util.LinkedList;
|
|
|
|
+import java.util.List;
|
|
|
|
+import java.util.Map;
|
|
|
|
+import java.util.Properties;
|
|
|
|
+import java.util.Set;
|
|
|
|
+import java.util.TreeMap;
|
|
|
|
+import java.util.concurrent.Callable;
|
|
|
|
+import java.util.concurrent.ConcurrentHashMap;
|
|
|
|
+import java.util.concurrent.ExecutorService;
|
|
|
|
+import java.util.concurrent.Executors;
|
|
|
|
+import java.util.concurrent.Future;
|
|
|
|
+
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
@@ -47,28 +65,11 @@ import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
|
|
import org.apache.hadoop.util.Daemon;
|
|
import org.apache.hadoop.util.Daemon;
|
|
import org.apache.hadoop.util.DiskChecker;
|
|
import org.apache.hadoop.util.DiskChecker;
|
|
|
|
|
|
-import java.io.File;
|
|
|
|
-import java.io.IOException;
|
|
|
|
-import java.io.RandomAccessFile;
|
|
|
|
-import java.nio.channels.FileLock;
|
|
|
|
-import java.nio.channels.OverlappingFileLockException;
|
|
|
|
-import java.util.ArrayList;
|
|
|
|
-import java.util.Collection;
|
|
|
|
-import java.util.Collections;
|
|
|
|
-import java.util.Comparator;
|
|
|
|
-import java.util.HashMap;
|
|
|
|
-import java.util.Iterator;
|
|
|
|
-import java.util.LinkedList;
|
|
|
|
-import java.util.List;
|
|
|
|
-import java.util.Map;
|
|
|
|
-import java.util.Properties;
|
|
|
|
-import java.util.Set;
|
|
|
|
-import java.util.TreeMap;
|
|
|
|
-import java.util.concurrent.Callable;
|
|
|
|
-import java.util.concurrent.ConcurrentHashMap;
|
|
|
|
-import java.util.concurrent.ExecutorService;
|
|
|
|
-import java.util.concurrent.Executors;
|
|
|
|
-import java.util.concurrent.Future;
|
|
|
|
|
|
+import com.google.common.annotations.VisibleForTesting;
|
|
|
|
+import com.google.common.collect.ComparisonChain;
|
|
|
|
+import com.google.common.collect.Lists;
|
|
|
|
+import com.google.common.collect.Maps;
|
|
|
|
+import com.google.common.util.concurrent.Futures;
|
|
|
|
|
|
/**
|
|
/**
|
|
* Data storage information file.
|
|
* Data storage information file.
|
|
@@ -104,7 +105,7 @@ public class DataStorage extends Storage {
|
|
* upgraded from a pre-UUID version. For compatibility with prior
|
|
* upgraded from a pre-UUID version. For compatibility with prior
|
|
* versions of Datanodes we cannot make this field a UUID.
|
|
* versions of Datanodes we cannot make this field a UUID.
|
|
*/
|
|
*/
|
|
- private String datanodeUuid = null;
|
|
|
|
|
|
+ private volatile String datanodeUuid = null;
|
|
|
|
|
|
// Maps block pool IDs to block pool storage
|
|
// Maps block pool IDs to block pool storage
|
|
private final Map<String, BlockPoolSliceStorage> bpStorageMap
|
|
private final Map<String, BlockPoolSliceStorage> bpStorageMap
|
|
@@ -125,18 +126,28 @@ public class DataStorage extends Storage {
|
|
super(storageInfo);
|
|
super(storageInfo);
|
|
}
|
|
}
|
|
|
|
|
|
- public synchronized String getDatanodeUuid() {
|
|
|
|
|
|
+ public String getDatanodeUuid() {
|
|
return datanodeUuid;
|
|
return datanodeUuid;
|
|
}
|
|
}
|
|
|
|
|
|
- public synchronized void setDatanodeUuid(String newDatanodeUuid) {
|
|
|
|
|
|
+ public void setDatanodeUuid(String newDatanodeUuid) {
|
|
this.datanodeUuid = newDatanodeUuid;
|
|
this.datanodeUuid = newDatanodeUuid;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ private static boolean createStorageID(StorageDirectory sd, int lv) {
|
|
|
|
+ // Clusters previously upgraded from layout versions earlier than
|
|
|
|
+ // ADD_DATANODE_AND_STORAGE_UUIDS failed to correctly generate a
|
|
|
|
+ // new storage ID. We check for that and fix it now.
|
|
|
|
+ final boolean haveValidStorageId = DataNodeLayoutVersion.supports(
|
|
|
|
+ LayoutVersion.Feature.ADD_DATANODE_AND_STORAGE_UUIDS, lv)
|
|
|
|
+ && DatanodeStorage.isValidStorageId(sd.getStorageUuid());
|
|
|
|
+ return createStorageID(sd, !haveValidStorageId);
|
|
|
|
+ }
|
|
|
|
+
|
|
/** Create an ID for this storage.
|
|
/** Create an ID for this storage.
|
|
* @return true if a new storage ID was generated.
|
|
* @return true if a new storage ID was generated.
|
|
* */
|
|
* */
|
|
- public synchronized boolean createStorageID(
|
|
|
|
|
|
+ public static boolean createStorageID(
|
|
StorageDirectory sd, boolean regenerateStorageIds) {
|
|
StorageDirectory sd, boolean regenerateStorageIds) {
|
|
final String oldStorageID = sd.getStorageUuid();
|
|
final String oldStorageID = sd.getStorageUuid();
|
|
if (oldStorageID == null || regenerateStorageIds) {
|
|
if (oldStorageID == null || regenerateStorageIds) {
|
|
@@ -250,7 +261,7 @@ public class DataStorage extends Storage {
|
|
|
|
|
|
private StorageDirectory loadStorageDirectory(DataNode datanode,
|
|
private StorageDirectory loadStorageDirectory(DataNode datanode,
|
|
NamespaceInfo nsInfo, File dataDir, StartupOption startOpt)
|
|
NamespaceInfo nsInfo, File dataDir, StartupOption startOpt)
|
|
- throws IOException {
|
|
|
|
|
|
+ throws IOException {
|
|
StorageDirectory sd = new StorageDirectory(dataDir, null, false);
|
|
StorageDirectory sd = new StorageDirectory(dataDir, null, false);
|
|
try {
|
|
try {
|
|
StorageState curState = sd.analyzeStorage(startOpt, this);
|
|
StorageState curState = sd.analyzeStorage(startOpt, this);
|
|
@@ -263,9 +274,9 @@ public class DataStorage extends Storage {
|
|
throw new IOException("Storage directory " + dataDir
|
|
throw new IOException("Storage directory " + dataDir
|
|
+ " does not exist");
|
|
+ " does not exist");
|
|
case NOT_FORMATTED: // format
|
|
case NOT_FORMATTED: // format
|
|
- LOG.info("Storage directory " + dataDir + " is not formatted for "
|
|
|
|
- + nsInfo.getBlockPoolID());
|
|
|
|
- LOG.info("Formatting ...");
|
|
|
|
|
|
+ LOG.info("Storage directory " + dataDir
|
|
|
|
+ + " is not formatted for namespace " + nsInfo.getNamespaceID()
|
|
|
|
+ + ". Formatting...");
|
|
format(sd, nsInfo, datanode.getDatanodeUuid());
|
|
format(sd, nsInfo, datanode.getDatanodeUuid());
|
|
break;
|
|
break;
|
|
default: // recovery part is common
|
|
default: // recovery part is common
|
|
@@ -276,7 +287,9 @@ public class DataStorage extends Storage {
|
|
// Each storage directory is treated individually.
|
|
// Each storage directory is treated individually.
|
|
// During startup some of them can upgrade or roll back
|
|
// During startup some of them can upgrade or roll back
|
|
// while others could be up-to-date for the regular startup.
|
|
// while others could be up-to-date for the regular startup.
|
|
- doTransition(datanode, sd, nsInfo, startOpt);
|
|
|
|
|
|
+ if (doTransition(sd, nsInfo, startOpt, datanode.getConf())) {
|
|
|
|
+ return sd;
|
|
|
|
+ }
|
|
|
|
|
|
// 3. Update successfully loaded storage.
|
|
// 3. Update successfully loaded storage.
|
|
setServiceLayoutVersion(getServiceLayoutVersion());
|
|
setServiceLayoutVersion(getServiceLayoutVersion());
|
|
@@ -321,20 +334,10 @@ public class DataStorage extends Storage {
|
|
nsInfo.getBlockPoolID(), new File(volume, STORAGE_DIR_CURRENT)));
|
|
nsInfo.getBlockPoolID(), new File(volume, STORAGE_DIR_CURRENT)));
|
|
makeBlockPoolDataDir(bpDataDirs, null);
|
|
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));
|
|
|
|
|
|
+ final BlockPoolSliceStorage bpStorage = getBlockPoolSliceStorage(nsInfo);
|
|
|
|
+ final List<StorageDirectory> dirs = bpStorage.loadBpStorageDirectories(
|
|
|
|
+ nsInfo, bpDataDirs, StartupOption.HOTSWAP, datanode.getConf());
|
|
|
|
+ builder.addBpStorageDirectories(nsInfo.getBlockPoolID(), dirs);
|
|
}
|
|
}
|
|
return builder;
|
|
return builder;
|
|
}
|
|
}
|
|
@@ -347,53 +350,68 @@ public class DataStorage extends Storage {
|
|
* @param nsInfo namespace information
|
|
* @param nsInfo namespace information
|
|
* @param dataDirs array of data storage directories
|
|
* @param dataDirs array of data storage directories
|
|
* @param startOpt startup option
|
|
* @param startOpt startup option
|
|
- * @return a list of successfully loaded volumes.
|
|
|
|
- * @throws IOException
|
|
|
|
|
|
+ * @return a list of successfully loaded storage directories.
|
|
*/
|
|
*/
|
|
@VisibleForTesting
|
|
@VisibleForTesting
|
|
- synchronized List<StorageLocation> addStorageLocations(DataNode datanode,
|
|
|
|
|
|
+ synchronized List<StorageDirectory> addStorageLocations(DataNode datanode,
|
|
NamespaceInfo nsInfo, Collection<StorageLocation> dataDirs,
|
|
NamespaceInfo nsInfo, Collection<StorageLocation> dataDirs,
|
|
StartupOption startOpt) throws IOException {
|
|
StartupOption startOpt) throws IOException {
|
|
- final String bpid = nsInfo.getBlockPoolID();
|
|
|
|
- List<StorageLocation> successVolumes = Lists.newArrayList();
|
|
|
|
|
|
+ final List<StorageLocation> successLocations = loadDataStorage(
|
|
|
|
+ datanode, nsInfo, dataDirs, startOpt);
|
|
|
|
+ return loadBlockPoolSliceStorage(
|
|
|
|
+ datanode, nsInfo, successLocations, startOpt);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private List<StorageLocation> loadDataStorage(DataNode datanode,
|
|
|
|
+ NamespaceInfo nsInfo, Collection<StorageLocation> dataDirs,
|
|
|
|
+ StartupOption startOpt) throws IOException {
|
|
|
|
+ final List<StorageLocation> success = Lists.newArrayList();
|
|
for (StorageLocation dataDir : dataDirs) {
|
|
for (StorageLocation dataDir : dataDirs) {
|
|
File root = dataDir.getFile();
|
|
File root = dataDir.getFile();
|
|
if (!containsStorageDir(root)) {
|
|
if (!containsStorageDir(root)) {
|
|
try {
|
|
try {
|
|
// It first ensures the datanode level format is completed.
|
|
// It first ensures the datanode level format is completed.
|
|
- StorageDirectory sd = loadStorageDirectory(
|
|
|
|
|
|
+ final StorageDirectory sd = loadStorageDirectory(
|
|
datanode, nsInfo, root, startOpt);
|
|
datanode, nsInfo, root, startOpt);
|
|
addStorageDir(sd);
|
|
addStorageDir(sd);
|
|
} catch (IOException e) {
|
|
} catch (IOException e) {
|
|
- LOG.warn(e);
|
|
|
|
|
|
+ LOG.warn("Failed to add storage directory " + dataDir, e);
|
|
continue;
|
|
continue;
|
|
}
|
|
}
|
|
} else {
|
|
} else {
|
|
LOG.info("Storage directory " + dataDir + " has already been used.");
|
|
LOG.info("Storage directory " + dataDir + " has already been used.");
|
|
}
|
|
}
|
|
|
|
+ success.add(dataDir);
|
|
|
|
+ }
|
|
|
|
|
|
|
|
+ return success;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private List<StorageDirectory> loadBlockPoolSliceStorage(DataNode datanode,
|
|
|
|
+ NamespaceInfo nsInfo, Collection<StorageLocation> dataDirs,
|
|
|
|
+ StartupOption startOpt) throws IOException {
|
|
|
|
+ final String bpid = nsInfo.getBlockPoolID();
|
|
|
|
+ final BlockPoolSliceStorage bpStorage = getBlockPoolSliceStorage(nsInfo);
|
|
|
|
+ final List<StorageDirectory> success = Lists.newArrayList();
|
|
|
|
+ for (StorageLocation dataDir : dataDirs) {
|
|
|
|
+ final File curDir = new File(dataDir.getFile(), STORAGE_DIR_CURRENT);
|
|
List<File> bpDataDirs = new ArrayList<File>();
|
|
List<File> bpDataDirs = new ArrayList<File>();
|
|
- bpDataDirs.add(BlockPoolSliceStorage.getBpRoot(bpid, new File(root,
|
|
|
|
- STORAGE_DIR_CURRENT)));
|
|
|
|
|
|
+ bpDataDirs.add(BlockPoolSliceStorage.getBpRoot(bpid, curDir));
|
|
try {
|
|
try {
|
|
makeBlockPoolDataDir(bpDataDirs, null);
|
|
makeBlockPoolDataDir(bpDataDirs, null);
|
|
- BlockPoolSliceStorage bpStorage = this.bpStorageMap.get(bpid);
|
|
|
|
- if (bpStorage == null) {
|
|
|
|
- bpStorage = new BlockPoolSliceStorage(
|
|
|
|
- nsInfo.getNamespaceID(), bpid, nsInfo.getCTime(),
|
|
|
|
- nsInfo.getClusterID());
|
|
|
|
- }
|
|
|
|
|
|
|
|
- bpStorage.recoverTransitionRead(datanode, nsInfo, bpDataDirs, startOpt);
|
|
|
|
- addBlockPoolStorage(bpid, bpStorage);
|
|
|
|
|
|
+ final List<StorageDirectory> dirs = bpStorage.recoverTransitionRead(
|
|
|
|
+ nsInfo, bpDataDirs, startOpt, datanode.getConf());
|
|
|
|
+ for(StorageDirectory sd : dirs) {
|
|
|
|
+ success.add(sd);
|
|
|
|
+ }
|
|
} catch (IOException e) {
|
|
} catch (IOException e) {
|
|
- LOG.warn("Failed to add storage for block pool: " + bpid + " : "
|
|
|
|
- + e.getMessage());
|
|
|
|
- continue;
|
|
|
|
|
|
+ LOG.warn("Failed to add storage directory " + dataDir
|
|
|
|
+ + " for block pool " + bpid, e);
|
|
}
|
|
}
|
|
- successVolumes.add(dataDir);
|
|
|
|
}
|
|
}
|
|
- return successVolumes;
|
|
|
|
|
|
+
|
|
|
|
+ return success;
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -635,17 +653,13 @@ public class DataStorage extends Storage {
|
|
* Upgrade if this.LV > LAYOUT_VERSION
|
|
* Upgrade if this.LV > LAYOUT_VERSION
|
|
* Regular startup if this.LV = LAYOUT_VERSION
|
|
* Regular startup if this.LV = LAYOUT_VERSION
|
|
*
|
|
*
|
|
- * @param datanode Datanode to which this storage belongs to
|
|
|
|
* @param sd storage directory
|
|
* @param sd storage directory
|
|
* @param nsInfo namespace info
|
|
* @param nsInfo namespace info
|
|
* @param startOpt startup option
|
|
* @param startOpt startup option
|
|
- * @throws IOException
|
|
|
|
|
|
+ * @return true if the new properties has been written.
|
|
*/
|
|
*/
|
|
- private void doTransition( DataNode datanode,
|
|
|
|
- StorageDirectory sd,
|
|
|
|
- NamespaceInfo nsInfo,
|
|
|
|
- StartupOption startOpt
|
|
|
|
- ) throws IOException {
|
|
|
|
|
|
+ private boolean doTransition(StorageDirectory sd, NamespaceInfo nsInfo,
|
|
|
|
+ StartupOption startOpt, Configuration conf) throws IOException {
|
|
if (startOpt == StartupOption.ROLLBACK) {
|
|
if (startOpt == StartupOption.ROLLBACK) {
|
|
doRollback(sd, nsInfo); // rollback if applicable
|
|
doRollback(sd, nsInfo); // rollback if applicable
|
|
}
|
|
}
|
|
@@ -674,25 +688,16 @@ public class DataStorage extends Storage {
|
|
+ nsInfo.getClusterID() + "; datanode clusterID = " + getClusterID());
|
|
+ nsInfo.getClusterID() + "; datanode clusterID = " + getClusterID());
|
|
}
|
|
}
|
|
|
|
|
|
- // Clusters previously upgraded from layout versions earlier than
|
|
|
|
- // ADD_DATANODE_AND_STORAGE_UUIDS failed to correctly generate a
|
|
|
|
- // new storage ID. We check for that and fix it now.
|
|
|
|
- boolean haveValidStorageId =
|
|
|
|
- DataNodeLayoutVersion.supports(
|
|
|
|
- LayoutVersion.Feature.ADD_DATANODE_AND_STORAGE_UUIDS, layoutVersion) &&
|
|
|
|
- DatanodeStorage.isValidStorageId(sd.getStorageUuid());
|
|
|
|
-
|
|
|
|
// regular start up.
|
|
// regular start up.
|
|
if (this.layoutVersion == HdfsServerConstants.DATANODE_LAYOUT_VERSION) {
|
|
if (this.layoutVersion == HdfsServerConstants.DATANODE_LAYOUT_VERSION) {
|
|
- createStorageID(sd, !haveValidStorageId);
|
|
|
|
- return; // regular startup
|
|
|
|
|
|
+ createStorageID(sd, layoutVersion);
|
|
|
|
+ return false; // need to write properties
|
|
}
|
|
}
|
|
|
|
|
|
// do upgrade
|
|
// do upgrade
|
|
if (this.layoutVersion > HdfsServerConstants.DATANODE_LAYOUT_VERSION) {
|
|
if (this.layoutVersion > HdfsServerConstants.DATANODE_LAYOUT_VERSION) {
|
|
- doUpgrade(datanode, sd, nsInfo); // upgrade
|
|
|
|
- createStorageID(sd, !haveValidStorageId);
|
|
|
|
- return;
|
|
|
|
|
|
+ doUpgrade(sd, nsInfo, conf); // upgrade
|
|
|
|
+ return true; // doUgrade already has written properties
|
|
}
|
|
}
|
|
|
|
|
|
// layoutVersion < DATANODE_LAYOUT_VERSION. I.e. stored layout version is newer
|
|
// layoutVersion < DATANODE_LAYOUT_VERSION. I.e. stored layout version is newer
|
|
@@ -726,8 +731,8 @@ public class DataStorage extends Storage {
|
|
* @param sd storage directory
|
|
* @param sd storage directory
|
|
* @throws IOException on error
|
|
* @throws IOException on error
|
|
*/
|
|
*/
|
|
- void doUpgrade(DataNode datanode, StorageDirectory sd, NamespaceInfo nsInfo)
|
|
|
|
- throws IOException {
|
|
|
|
|
|
+ void doUpgrade(final StorageDirectory sd, final NamespaceInfo nsInfo,
|
|
|
|
+ final Configuration conf) throws IOException {
|
|
// If the existing on-disk layout version supportes federation, simply
|
|
// If the existing on-disk layout version supportes federation, simply
|
|
// update its layout version.
|
|
// update its layout version.
|
|
if (DataNodeLayoutVersion.supports(
|
|
if (DataNodeLayoutVersion.supports(
|
|
@@ -743,15 +748,16 @@ public class DataStorage extends Storage {
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ final int oldLV = getLayoutVersion();
|
|
LOG.info("Upgrading storage directory " + sd.getRoot()
|
|
LOG.info("Upgrading storage directory " + sd.getRoot()
|
|
- + ".\n old LV = " + this.getLayoutVersion()
|
|
|
|
|
|
+ + ".\n old LV = " + oldLV
|
|
+ "; old CTime = " + this.getCTime()
|
|
+ "; old CTime = " + this.getCTime()
|
|
+ ".\n new LV = " + HdfsServerConstants.DATANODE_LAYOUT_VERSION
|
|
+ ".\n new LV = " + HdfsServerConstants.DATANODE_LAYOUT_VERSION
|
|
+ "; new CTime = " + nsInfo.getCTime());
|
|
+ "; new CTime = " + nsInfo.getCTime());
|
|
|
|
|
|
- File curDir = sd.getCurrentDir();
|
|
|
|
- File prevDir = sd.getPreviousDir();
|
|
|
|
- File bbwDir = new File(sd.getRoot(), Storage.STORAGE_1_BBW);
|
|
|
|
|
|
+ final File curDir = sd.getCurrentDir();
|
|
|
|
+ final File prevDir = sd.getPreviousDir();
|
|
|
|
+ final File bbwDir = new File(sd.getRoot(), Storage.STORAGE_1_BBW);
|
|
|
|
|
|
assert curDir.exists() : "Data node current directory must exist.";
|
|
assert curDir.exists() : "Data node current directory must exist.";
|
|
// Cleanup directory "detach"
|
|
// Cleanup directory "detach"
|
|
@@ -761,21 +767,29 @@ public class DataStorage extends Storage {
|
|
if (prevDir.exists())
|
|
if (prevDir.exists())
|
|
deleteDir(prevDir);
|
|
deleteDir(prevDir);
|
|
// get previous.tmp directory, <SD>/previous.tmp
|
|
// get previous.tmp directory, <SD>/previous.tmp
|
|
- File tmpDir = sd.getPreviousTmp();
|
|
|
|
|
|
+ final File tmpDir = sd.getPreviousTmp();
|
|
assert !tmpDir.exists() :
|
|
assert !tmpDir.exists() :
|
|
"Data node previous.tmp directory must not exist.";
|
|
"Data node previous.tmp directory must not exist.";
|
|
|
|
|
|
// 2. Rename <SD>/current to <SD>/previous.tmp
|
|
// 2. Rename <SD>/current to <SD>/previous.tmp
|
|
rename(curDir, tmpDir);
|
|
rename(curDir, tmpDir);
|
|
|
|
|
|
- // 3. Format BP and hard link blocks from previous directory
|
|
|
|
|
|
+ // 3.1. Format BP
|
|
File curBpDir = BlockPoolSliceStorage.getBpRoot(nsInfo.getBlockPoolID(), curDir);
|
|
File curBpDir = BlockPoolSliceStorage.getBpRoot(nsInfo.getBlockPoolID(), curDir);
|
|
- BlockPoolSliceStorage bpStorage = new BlockPoolSliceStorage(nsInfo.getNamespaceID(),
|
|
|
|
- nsInfo.getBlockPoolID(), nsInfo.getCTime(), nsInfo.getClusterID());
|
|
|
|
|
|
+ BlockPoolSliceStorage bpStorage = getBlockPoolSliceStorage(nsInfo);
|
|
bpStorage.format(curDir, nsInfo);
|
|
bpStorage.format(curDir, nsInfo);
|
|
- linkAllBlocks(datanode, tmpDir, bbwDir, new File(curBpDir,
|
|
|
|
- STORAGE_DIR_CURRENT));
|
|
|
|
-
|
|
|
|
|
|
+
|
|
|
|
+ final File toDir = new File(curBpDir, STORAGE_DIR_CURRENT);
|
|
|
|
+ doUgrade(sd, nsInfo, prevDir, tmpDir, bbwDir, toDir, oldLV, conf);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void doUgrade(final StorageDirectory sd,
|
|
|
|
+ final NamespaceInfo nsInfo, final File prevDir,
|
|
|
|
+ final File tmpDir, final File bbwDir, final File toDir, final int oldLV,
|
|
|
|
+ Configuration conf) throws IOException {
|
|
|
|
+ // 3.2. Link block files from <SD>/previous.tmp to <SD>/current
|
|
|
|
+ linkAllBlocks(tmpDir, bbwDir, toDir, oldLV, conf);
|
|
|
|
+
|
|
// 4. Write version file under <SD>/current
|
|
// 4. Write version file under <SD>/current
|
|
layoutVersion = HdfsServerConstants.DATANODE_LAYOUT_VERSION;
|
|
layoutVersion = HdfsServerConstants.DATANODE_LAYOUT_VERSION;
|
|
clusterID = nsInfo.getClusterID();
|
|
clusterID = nsInfo.getClusterID();
|
|
@@ -784,7 +798,8 @@ public class DataStorage extends Storage {
|
|
// 5. Rename <SD>/previous.tmp to <SD>/previous
|
|
// 5. Rename <SD>/previous.tmp to <SD>/previous
|
|
rename(tmpDir, prevDir);
|
|
rename(tmpDir, prevDir);
|
|
LOG.info("Upgrade of " + sd.getRoot()+ " is complete");
|
|
LOG.info("Upgrade of " + sd.getRoot()+ " is complete");
|
|
- addBlockPoolStorage(nsInfo.getBlockPoolID(), bpStorage);
|
|
|
|
|
|
+
|
|
|
|
+ createStorageID(sd, layoutVersion);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -952,23 +967,22 @@ public class DataStorage extends Storage {
|
|
*
|
|
*
|
|
* @throws IOException If error occurs during hardlink
|
|
* @throws IOException If error occurs during hardlink
|
|
*/
|
|
*/
|
|
- private void linkAllBlocks(DataNode datanode, File fromDir, File fromBbwDir,
|
|
|
|
- File toDir) throws IOException {
|
|
|
|
|
|
+ private static void linkAllBlocks(File fromDir, File fromBbwDir, File toDir,
|
|
|
|
+ int diskLayoutVersion, Configuration conf) throws IOException {
|
|
HardLink hardLink = new HardLink();
|
|
HardLink hardLink = new HardLink();
|
|
// do the link
|
|
// do the link
|
|
- int diskLayoutVersion = this.getLayoutVersion();
|
|
|
|
if (DataNodeLayoutVersion.supports(
|
|
if (DataNodeLayoutVersion.supports(
|
|
LayoutVersion.Feature.APPEND_RBW_DIR, diskLayoutVersion)) {
|
|
LayoutVersion.Feature.APPEND_RBW_DIR, diskLayoutVersion)) {
|
|
// hardlink finalized blocks in tmpDir/finalized
|
|
// hardlink finalized blocks in tmpDir/finalized
|
|
- linkBlocks(datanode, new File(fromDir, STORAGE_DIR_FINALIZED),
|
|
|
|
- new File(toDir, STORAGE_DIR_FINALIZED), diskLayoutVersion, hardLink);
|
|
|
|
|
|
+ linkBlocks(fromDir, toDir, STORAGE_DIR_FINALIZED,
|
|
|
|
+ diskLayoutVersion, hardLink, conf);
|
|
// hardlink rbw blocks in tmpDir/rbw
|
|
// hardlink rbw blocks in tmpDir/rbw
|
|
- linkBlocks(datanode, new File(fromDir, STORAGE_DIR_RBW),
|
|
|
|
- new File(toDir, STORAGE_DIR_RBW), diskLayoutVersion, hardLink);
|
|
|
|
|
|
+ linkBlocks(fromDir, toDir, STORAGE_DIR_RBW,
|
|
|
|
+ diskLayoutVersion, hardLink, conf);
|
|
} else { // pre-RBW version
|
|
} else { // pre-RBW version
|
|
// hardlink finalized blocks in tmpDir
|
|
// hardlink finalized blocks in tmpDir
|
|
- linkBlocks(datanode, fromDir, new File(toDir, STORAGE_DIR_FINALIZED),
|
|
|
|
- diskLayoutVersion, hardLink);
|
|
|
|
|
|
+ linkBlocks(fromDir, new File(toDir, STORAGE_DIR_FINALIZED),
|
|
|
|
+ diskLayoutVersion, hardLink, conf);
|
|
if (fromBbwDir.exists()) {
|
|
if (fromBbwDir.exists()) {
|
|
/*
|
|
/*
|
|
* We need to put the 'blocksBeingWritten' from HDFS 1.x into the rbw
|
|
* We need to put the 'blocksBeingWritten' from HDFS 1.x into the rbw
|
|
@@ -976,11 +990,12 @@ public class DataStorage extends Storage {
|
|
* NOT underneath the 'current' directory in those releases. See
|
|
* NOT underneath the 'current' directory in those releases. See
|
|
* HDFS-3731 for details.
|
|
* HDFS-3731 for details.
|
|
*/
|
|
*/
|
|
- linkBlocks(datanode, fromBbwDir,
|
|
|
|
- new File(toDir, STORAGE_DIR_RBW), diskLayoutVersion, hardLink);
|
|
|
|
|
|
+ linkBlocks(fromBbwDir, new File(toDir, STORAGE_DIR_RBW),
|
|
|
|
+ diskLayoutVersion, hardLink, conf);
|
|
}
|
|
}
|
|
- }
|
|
|
|
- LOG.info( hardLink.linkStats.report() );
|
|
|
|
|
|
+ }
|
|
|
|
+ LOG.info("Linked blocks from " + fromDir + " to " + toDir + ". "
|
|
|
|
+ + hardLink.linkStats.report());
|
|
}
|
|
}
|
|
|
|
|
|
private static class LinkArgs {
|
|
private static class LinkArgs {
|
|
@@ -993,8 +1008,15 @@ public class DataStorage extends Storage {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- static void linkBlocks(DataNode datanode, File from, File to, int oldLV,
|
|
|
|
- HardLink hl) throws IOException {
|
|
|
|
|
|
+ static void linkBlocks(File fromDir, File toDir, String subdir, int oldLV,
|
|
|
|
+ HardLink hl, Configuration conf) throws IOException {
|
|
|
|
+ linkBlocks(new File(fromDir, subdir), new File(toDir, subdir),
|
|
|
|
+ oldLV, hl, conf);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private static void linkBlocks(File from, File to, int oldLV,
|
|
|
|
+ HardLink hl, Configuration conf) throws IOException {
|
|
|
|
+ LOG.info("Start linking block files from " + from + " to " + to);
|
|
boolean upgradeToIdBasedLayout = false;
|
|
boolean upgradeToIdBasedLayout = false;
|
|
// If we are upgrading from a version older than the one where we introduced
|
|
// If we are upgrading from a version older than the one where we introduced
|
|
// block ID-based layout AND we're working with the finalized directory,
|
|
// block ID-based layout AND we're working with the finalized directory,
|
|
@@ -1017,7 +1039,7 @@ public class DataStorage extends Storage {
|
|
removeDuplicateEntries(idBasedLayoutSingleLinks, duplicates);
|
|
removeDuplicateEntries(idBasedLayoutSingleLinks, duplicates);
|
|
}
|
|
}
|
|
|
|
|
|
- int numLinkWorkers = datanode.getConf().getInt(
|
|
|
|
|
|
+ final int numLinkWorkers = conf.getInt(
|
|
DFSConfigKeys.DFS_DATANODE_BLOCK_ID_LAYOUT_UPGRADE_THREADS_KEY,
|
|
DFSConfigKeys.DFS_DATANODE_BLOCK_ID_LAYOUT_UPGRADE_THREADS_KEY,
|
|
DFSConfigKeys.DFS_DATANODE_BLOCK_ID_LAYOUT_UPGRADE_THREADS);
|
|
DFSConfigKeys.DFS_DATANODE_BLOCK_ID_LAYOUT_UPGRADE_THREADS);
|
|
ExecutorService linkWorkers = Executors.newFixedThreadPool(numLinkWorkers);
|
|
ExecutorService linkWorkers = Executors.newFixedThreadPool(numLinkWorkers);
|
|
@@ -1268,13 +1290,19 @@ public class DataStorage extends Storage {
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Add bpStorage into bpStorageMap
|
|
|
|
|
|
+ * Get the BlockPoolSliceStorage from {@link bpStorageMap}.
|
|
|
|
+ * If the object is not found, create a new object and put it to the map.
|
|
*/
|
|
*/
|
|
- private void addBlockPoolStorage(String bpID, BlockPoolSliceStorage bpStorage
|
|
|
|
- ) {
|
|
|
|
- if (!this.bpStorageMap.containsKey(bpID)) {
|
|
|
|
- this.bpStorageMap.put(bpID, bpStorage);
|
|
|
|
|
|
+ synchronized BlockPoolSliceStorage getBlockPoolSliceStorage(
|
|
|
|
+ final NamespaceInfo nsInfo) {
|
|
|
|
+ final String bpid = nsInfo.getBlockPoolID();
|
|
|
|
+ BlockPoolSliceStorage bpStorage = bpStorageMap.get(bpid);
|
|
|
|
+ if (bpStorage == null) {
|
|
|
|
+ bpStorage = new BlockPoolSliceStorage(nsInfo.getNamespaceID(), bpid,
|
|
|
|
+ nsInfo.getCTime(), nsInfo.getClusterID());
|
|
|
|
+ bpStorageMap.put(bpid, bpStorage);
|
|
}
|
|
}
|
|
|
|
+ return bpStorage;
|
|
}
|
|
}
|
|
|
|
|
|
synchronized void removeBlockPoolStorage(String bpId) {
|
|
synchronized void removeBlockPoolStorage(String bpId) {
|