|
@@ -37,10 +37,12 @@ import java.util.Set;
|
|
|
import java.util.TreeMap;
|
|
|
import java.util.concurrent.Callable;
|
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
|
+import java.util.concurrent.ExecutionException;
|
|
|
import java.util.concurrent.ExecutorService;
|
|
|
import java.util.concurrent.Executors;
|
|
|
import java.util.concurrent.Future;
|
|
|
|
|
|
+import org.apache.hadoop.HadoopIllegalArgumentException;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
@@ -51,6 +53,7 @@ import org.apache.hadoop.fs.LocalFileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
+import org.apache.hadoop.hdfs.DFSUtilClient;
|
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
|
|
@@ -260,8 +263,8 @@ public class DataStorage extends Storage {
|
|
|
}
|
|
|
|
|
|
private StorageDirectory loadStorageDirectory(DataNode datanode,
|
|
|
- NamespaceInfo nsInfo, File dataDir, StartupOption startOpt)
|
|
|
- throws IOException {
|
|
|
+ NamespaceInfo nsInfo, File dataDir, StartupOption startOpt,
|
|
|
+ List<Callable<StorageDirectory>> callables) throws IOException {
|
|
|
StorageDirectory sd = new StorageDirectory(dataDir, null, false);
|
|
|
try {
|
|
|
StorageState curState = sd.analyzeStorage(startOpt, this);
|
|
@@ -287,13 +290,12 @@ public class DataStorage extends Storage {
|
|
|
// 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.
|
|
|
- if (doTransition(sd, nsInfo, startOpt, datanode.getConf())) {
|
|
|
- return sd;
|
|
|
- }
|
|
|
+ if (!doTransition(sd, nsInfo, startOpt, callables, datanode.getConf())) {
|
|
|
|
|
|
- // 3. Update successfully loaded storage.
|
|
|
- setServiceLayoutVersion(getServiceLayoutVersion());
|
|
|
- writeProperties(sd);
|
|
|
+ // 3. Update successfully loaded storage.
|
|
|
+ setServiceLayoutVersion(getServiceLayoutVersion());
|
|
|
+ writeProperties(sd);
|
|
|
+ }
|
|
|
|
|
|
return sd;
|
|
|
} catch (IOException ioe) {
|
|
@@ -325,7 +327,7 @@ public class DataStorage extends Storage {
|
|
|
}
|
|
|
|
|
|
StorageDirectory sd = loadStorageDirectory(
|
|
|
- datanode, nsInfos.get(0), volume, StartupOption.HOTSWAP);
|
|
|
+ datanode, nsInfos.get(0), volume, StartupOption.HOTSWAP, null);
|
|
|
VolumeBuilder builder =
|
|
|
new VolumeBuilder(this, sd);
|
|
|
for (NamespaceInfo nsInfo : nsInfos) {
|
|
@@ -336,12 +338,35 @@ public class DataStorage extends Storage {
|
|
|
|
|
|
final BlockPoolSliceStorage bpStorage = getBlockPoolSliceStorage(nsInfo);
|
|
|
final List<StorageDirectory> dirs = bpStorage.loadBpStorageDirectories(
|
|
|
- nsInfo, bpDataDirs, StartupOption.HOTSWAP, datanode.getConf());
|
|
|
+ nsInfo, bpDataDirs, StartupOption.HOTSWAP, null, datanode.getConf());
|
|
|
builder.addBpStorageDirectories(nsInfo.getBlockPoolID(), dirs);
|
|
|
}
|
|
|
return builder;
|
|
|
}
|
|
|
|
|
|
+ static int getParallelVolumeLoadThreadsNum(int dataDirs, Configuration conf) {
|
|
|
+ final String key
|
|
|
+ = DFSConfigKeys.DFS_DATANODE_PARALLEL_VOLUME_LOAD_THREADS_NUM_KEY;
|
|
|
+ final int n = conf.getInt(key, dataDirs);
|
|
|
+ if (n < 1) {
|
|
|
+ throw new HadoopIllegalArgumentException(key + " = " + n + " < 1");
|
|
|
+ }
|
|
|
+ final int min = Math.min(n, dataDirs);
|
|
|
+ LOG.info("Using " + min + " threads to upgrade data directories ("
|
|
|
+ + key + "=" + n + ", dataDirs=" + dataDirs + ")");
|
|
|
+ return min;
|
|
|
+ }
|
|
|
+
|
|
|
+ static class UpgradeTask {
|
|
|
+ private final StorageLocation dataDir;
|
|
|
+ private final Future<StorageDirectory> future;
|
|
|
+
|
|
|
+ UpgradeTask(StorageLocation dataDir, Future<StorageDirectory> future) {
|
|
|
+ this.dataDir = dataDir;
|
|
|
+ this.future = future;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* 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.
|
|
@@ -356,32 +381,62 @@ public class DataStorage extends Storage {
|
|
|
synchronized List<StorageDirectory> addStorageLocations(DataNode datanode,
|
|
|
NamespaceInfo nsInfo, Collection<StorageLocation> dataDirs,
|
|
|
StartupOption startOpt) throws IOException {
|
|
|
- final List<StorageLocation> successLocations = loadDataStorage(
|
|
|
- datanode, nsInfo, dataDirs, startOpt);
|
|
|
- return loadBlockPoolSliceStorage(
|
|
|
- datanode, nsInfo, successLocations, startOpt);
|
|
|
+ final int numThreads = getParallelVolumeLoadThreadsNum(
|
|
|
+ dataDirs.size(), datanode.getConf());
|
|
|
+ final ExecutorService executor = Executors.newFixedThreadPool(numThreads);
|
|
|
+ try {
|
|
|
+ final List<StorageLocation> successLocations = loadDataStorage(
|
|
|
+ datanode, nsInfo, dataDirs, startOpt, executor);
|
|
|
+ return loadBlockPoolSliceStorage(
|
|
|
+ datanode, nsInfo, successLocations, startOpt, executor);
|
|
|
+ } finally {
|
|
|
+ executor.shutdown();
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
private List<StorageLocation> loadDataStorage(DataNode datanode,
|
|
|
NamespaceInfo nsInfo, Collection<StorageLocation> dataDirs,
|
|
|
- StartupOption startOpt) throws IOException {
|
|
|
+ StartupOption startOpt, ExecutorService executor) throws IOException {
|
|
|
final List<StorageLocation> success = Lists.newArrayList();
|
|
|
+ final List<UpgradeTask> tasks = Lists.newArrayList();
|
|
|
for (StorageLocation dataDir : dataDirs) {
|
|
|
File root = dataDir.getFile();
|
|
|
if (!containsStorageDir(root)) {
|
|
|
try {
|
|
|
// It first ensures the datanode level format is completed.
|
|
|
+ final List<Callable<StorageDirectory>> callables
|
|
|
+ = Lists.newArrayList();
|
|
|
final StorageDirectory sd = loadStorageDirectory(
|
|
|
- datanode, nsInfo, root, startOpt);
|
|
|
- addStorageDir(sd);
|
|
|
+ datanode, nsInfo, root, startOpt, callables);
|
|
|
+ if (callables.isEmpty()) {
|
|
|
+ addStorageDir(sd);
|
|
|
+ success.add(dataDir);
|
|
|
+ } else {
|
|
|
+ for(Callable<StorageDirectory> c : callables) {
|
|
|
+ tasks.add(new UpgradeTask(dataDir, executor.submit(c)));
|
|
|
+ }
|
|
|
+ }
|
|
|
} catch (IOException e) {
|
|
|
LOG.warn("Failed to add storage directory " + dataDir, e);
|
|
|
- continue;
|
|
|
}
|
|
|
} else {
|
|
|
LOG.info("Storage directory " + dataDir + " has already been used.");
|
|
|
+ success.add(dataDir);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ if (!tasks.isEmpty()) {
|
|
|
+ LOG.info("loadDataStorage: " + tasks.size() + " upgrade tasks");
|
|
|
+ for(UpgradeTask t : tasks) {
|
|
|
+ try {
|
|
|
+ addStorageDir(t.future.get());
|
|
|
+ success.add(t.dataDir);
|
|
|
+ } catch (ExecutionException e) {
|
|
|
+ LOG.warn("Failed to upgrade storage directory " + t.dataDir, e);
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ throw DFSUtilClient.toInterruptedIOException("Task interrupted", e);
|
|
|
+ }
|
|
|
}
|
|
|
- success.add(dataDir);
|
|
|
}
|
|
|
|
|
|
return success;
|
|
@@ -389,10 +444,11 @@ public class DataStorage extends Storage {
|
|
|
|
|
|
private List<StorageDirectory> loadBlockPoolSliceStorage(DataNode datanode,
|
|
|
NamespaceInfo nsInfo, Collection<StorageLocation> dataDirs,
|
|
|
- StartupOption startOpt) throws IOException {
|
|
|
+ StartupOption startOpt, ExecutorService executor) throws IOException {
|
|
|
final String bpid = nsInfo.getBlockPoolID();
|
|
|
final BlockPoolSliceStorage bpStorage = getBlockPoolSliceStorage(nsInfo);
|
|
|
final List<StorageDirectory> success = Lists.newArrayList();
|
|
|
+ final List<UpgradeTask> tasks = Lists.newArrayList();
|
|
|
for (StorageLocation dataDir : dataDirs) {
|
|
|
final File curDir = new File(dataDir.getFile(), STORAGE_DIR_CURRENT);
|
|
|
List<File> bpDataDirs = new ArrayList<File>();
|
|
@@ -400,10 +456,17 @@ public class DataStorage extends Storage {
|
|
|
try {
|
|
|
makeBlockPoolDataDir(bpDataDirs, null);
|
|
|
|
|
|
+ final List<Callable<StorageDirectory>> callables = Lists.newArrayList();
|
|
|
final List<StorageDirectory> dirs = bpStorage.recoverTransitionRead(
|
|
|
- nsInfo, bpDataDirs, startOpt, datanode.getConf());
|
|
|
- for(StorageDirectory sd : dirs) {
|
|
|
- success.add(sd);
|
|
|
+ nsInfo, bpDataDirs, startOpt, callables, datanode.getConf());
|
|
|
+ if (callables.isEmpty()) {
|
|
|
+ for(StorageDirectory sd : dirs) {
|
|
|
+ success.add(sd);
|
|
|
+ }
|
|
|
+ } else {
|
|
|
+ for(Callable<StorageDirectory> c : callables) {
|
|
|
+ tasks.add(new UpgradeTask(dataDir, executor.submit(c)));
|
|
|
+ }
|
|
|
}
|
|
|
} catch (IOException e) {
|
|
|
LOG.warn("Failed to add storage directory " + dataDir
|
|
@@ -411,6 +474,20 @@ public class DataStorage extends Storage {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ if (!tasks.isEmpty()) {
|
|
|
+ LOG.info("loadBlockPoolSliceStorage: " + tasks.size() + " upgrade tasks");
|
|
|
+ for(UpgradeTask t : tasks) {
|
|
|
+ try {
|
|
|
+ success.add(t.future.get());
|
|
|
+ } catch (ExecutionException e) {
|
|
|
+ LOG.warn("Failed to upgrade storage directory " + t.dataDir
|
|
|
+ + " for block pool " + bpid, e);
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ throw DFSUtilClient.toInterruptedIOException("Task interrupted", e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
return success;
|
|
|
}
|
|
|
|
|
@@ -655,7 +732,8 @@ public class DataStorage extends Storage {
|
|
|
* @return true if the new properties has been written.
|
|
|
*/
|
|
|
private boolean doTransition(StorageDirectory sd, NamespaceInfo nsInfo,
|
|
|
- StartupOption startOpt, Configuration conf) throws IOException {
|
|
|
+ StartupOption startOpt, List<Callable<StorageDirectory>> callables,
|
|
|
+ Configuration conf) throws IOException {
|
|
|
if (startOpt == StartupOption.ROLLBACK) {
|
|
|
doRollback(sd, nsInfo); // rollback if applicable
|
|
|
}
|
|
@@ -697,7 +775,7 @@ public class DataStorage extends Storage {
|
|
|
// simply update the properties.
|
|
|
upgradeProperties(sd);
|
|
|
} else {
|
|
|
- doUpgradePreFederation(sd, nsInfo, conf);
|
|
|
+ doUpgradePreFederation(sd, nsInfo, callables, conf);
|
|
|
}
|
|
|
return true; // doUgrade already has written properties
|
|
|
}
|
|
@@ -734,7 +812,9 @@ public class DataStorage extends Storage {
|
|
|
* @param sd storage directory
|
|
|
*/
|
|
|
void doUpgradePreFederation(final StorageDirectory sd,
|
|
|
- final NamespaceInfo nsInfo, final Configuration conf) throws IOException {
|
|
|
+ final NamespaceInfo nsInfo,
|
|
|
+ final List<Callable<StorageDirectory>> callables,
|
|
|
+ final Configuration conf) throws IOException {
|
|
|
final int oldLV = getLayoutVersion();
|
|
|
LOG.info("Upgrading storage directory " + sd.getRoot()
|
|
|
+ ".\n old LV = " + oldLV
|
|
@@ -767,10 +847,20 @@ public class DataStorage extends Storage {
|
|
|
bpStorage.format(curDir, nsInfo);
|
|
|
|
|
|
final File toDir = new File(curBpDir, STORAGE_DIR_CURRENT);
|
|
|
- doUgrade(sd, nsInfo, prevDir, tmpDir, bbwDir, toDir, oldLV, conf);
|
|
|
+ if (callables == null) {
|
|
|
+ doUpgrade(sd, nsInfo, prevDir, tmpDir, bbwDir, toDir, oldLV, conf);
|
|
|
+ } else {
|
|
|
+ callables.add(new Callable<StorageDirectory>() {
|
|
|
+ @Override
|
|
|
+ public StorageDirectory call() throws Exception {
|
|
|
+ doUpgrade(sd, nsInfo, prevDir, tmpDir, bbwDir, toDir, oldLV, conf);
|
|
|
+ return sd;
|
|
|
+ }
|
|
|
+ });
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
- private void doUgrade(final StorageDirectory sd,
|
|
|
+ private void doUpgrade(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 {
|