|
@@ -745,90 +745,93 @@ public class DataNode extends ReconfigurableBase
|
|
|
* @throws IOException on error. If an IOException is thrown, some new volumes
|
|
|
* may have been successfully added and removed.
|
|
|
*/
|
|
|
- private synchronized void refreshVolumes(String newVolumes) throws IOException {
|
|
|
- Configuration conf = getConf();
|
|
|
- conf.set(DFS_DATANODE_DATA_DIR_KEY, newVolumes);
|
|
|
- ExecutorService service = null;
|
|
|
- int numOldDataDirs = dataDirs.size();
|
|
|
- ChangedVolumes changedVolumes = parseChangedVolumes(newVolumes);
|
|
|
- StringBuilder errorMessageBuilder = new StringBuilder();
|
|
|
- List<String> effectiveVolumes = Lists.newArrayList();
|
|
|
- for (StorageLocation sl : changedVolumes.unchangedLocations) {
|
|
|
- effectiveVolumes.add(sl.toString());
|
|
|
+ private void refreshVolumes(String newVolumes) throws IOException {
|
|
|
+ // Add volumes for each Namespace
|
|
|
+ final List<NamespaceInfo> nsInfos = Lists.newArrayList();
|
|
|
+ for (BPOfferService bpos : blockPoolManager.getAllNamenodeThreads()) {
|
|
|
+ nsInfos.add(bpos.getNamespaceInfo());
|
|
|
}
|
|
|
-
|
|
|
- try {
|
|
|
- if (numOldDataDirs + getFSDataset().getNumFailedVolumes()
|
|
|
- + changedVolumes.newLocations.size()
|
|
|
- - changedVolumes.deactivateLocations.size() <= 0) {
|
|
|
- throw new IOException("Attempt to remove all volumes.");
|
|
|
+ synchronized(this) {
|
|
|
+ Configuration conf = getConf();
|
|
|
+ conf.set(DFS_DATANODE_DATA_DIR_KEY, newVolumes);
|
|
|
+ ExecutorService service = null;
|
|
|
+ int numOldDataDirs = dataDirs.size();
|
|
|
+ ChangedVolumes changedVolumes = parseChangedVolumes(newVolumes);
|
|
|
+ StringBuilder errorMessageBuilder = new StringBuilder();
|
|
|
+ List<String> effectiveVolumes = Lists.newArrayList();
|
|
|
+ for (StorageLocation sl : changedVolumes.unchangedLocations) {
|
|
|
+ effectiveVolumes.add(sl.toString());
|
|
|
}
|
|
|
- if (!changedVolumes.newLocations.isEmpty()) {
|
|
|
- LOG.info("Adding new volumes: {}",
|
|
|
- Joiner.on(",").join(changedVolumes.newLocations));
|
|
|
-
|
|
|
- // Add volumes for each Namespace
|
|
|
- final List<NamespaceInfo> nsInfos = Lists.newArrayList();
|
|
|
- for (BPOfferService bpos : blockPoolManager.getAllNamenodeThreads()) {
|
|
|
- nsInfos.add(bpos.getNamespaceInfo());
|
|
|
+
|
|
|
+ try {
|
|
|
+ if (numOldDataDirs + getFSDataset().getNumFailedVolumes()
|
|
|
+ + changedVolumes.newLocations.size()
|
|
|
+ - changedVolumes.deactivateLocations.size() <= 0) {
|
|
|
+ throw new IOException("Attempt to remove all volumes.");
|
|
|
}
|
|
|
- service = Executors
|
|
|
- .newFixedThreadPool(changedVolumes.newLocations.size());
|
|
|
- List<Future<IOException>> exceptions = Lists.newArrayList();
|
|
|
- for (final StorageLocation location : changedVolumes.newLocations) {
|
|
|
- exceptions.add(service.submit(new Callable<IOException>() {
|
|
|
- @Override
|
|
|
- public IOException call() {
|
|
|
- try {
|
|
|
- data.addVolume(location, nsInfos);
|
|
|
- } catch (IOException e) {
|
|
|
- return e;
|
|
|
+ if (!changedVolumes.newLocations.isEmpty()) {
|
|
|
+ LOG.info("Adding new volumes: {}",
|
|
|
+ Joiner.on(",").join(changedVolumes.newLocations));
|
|
|
+
|
|
|
+ service = Executors
|
|
|
+ .newFixedThreadPool(changedVolumes.newLocations.size());
|
|
|
+ List<Future<IOException>> exceptions = Lists.newArrayList();
|
|
|
+
|
|
|
+ for (final StorageLocation location : changedVolumes.newLocations) {
|
|
|
+ exceptions.add(service.submit(new Callable<IOException>() {
|
|
|
+ @Override
|
|
|
+ public IOException call() {
|
|
|
+ try {
|
|
|
+ data.addVolume(location, nsInfos);
|
|
|
+ } catch (IOException e) {
|
|
|
+ return e;
|
|
|
+ }
|
|
|
+ return null;
|
|
|
}
|
|
|
- return null;
|
|
|
- }
|
|
|
- }));
|
|
|
- }
|
|
|
+ }));
|
|
|
+ }
|
|
|
|
|
|
- for (int i = 0; i < changedVolumes.newLocations.size(); i++) {
|
|
|
- StorageLocation volume = changedVolumes.newLocations.get(i);
|
|
|
- Future<IOException> ioExceptionFuture = exceptions.get(i);
|
|
|
- try {
|
|
|
- IOException ioe = ioExceptionFuture.get();
|
|
|
- if (ioe != null) {
|
|
|
+ for (int i = 0; i < changedVolumes.newLocations.size(); i++) {
|
|
|
+ StorageLocation volume = changedVolumes.newLocations.get(i);
|
|
|
+ Future<IOException> ioExceptionFuture = exceptions.get(i);
|
|
|
+ try {
|
|
|
+ IOException ioe = ioExceptionFuture.get();
|
|
|
+ if (ioe != null) {
|
|
|
+ errorMessageBuilder.append(
|
|
|
+ String.format("FAILED TO ADD: %s: %s%n",
|
|
|
+ volume, ioe.getMessage()));
|
|
|
+ LOG.error("Failed to add volume: {}", volume, ioe);
|
|
|
+ } else {
|
|
|
+ effectiveVolumes.add(volume.toString());
|
|
|
+ LOG.info("Successfully added volume: {}", volume);
|
|
|
+ }
|
|
|
+ } catch (Exception e) {
|
|
|
errorMessageBuilder.append(
|
|
|
- String.format("FAILED TO ADD: %s: %s%n",
|
|
|
- volume, ioe.getMessage()));
|
|
|
- LOG.error("Failed to add volume: {}", volume, ioe);
|
|
|
- } else {
|
|
|
- effectiveVolumes.add(volume.toString());
|
|
|
- LOG.info("Successfully added volume: {}", volume);
|
|
|
+ String.format("FAILED to ADD: %s: %s%n", volume,
|
|
|
+ e.toString()));
|
|
|
+ LOG.error("Failed to add volume: {}", volume, e);
|
|
|
}
|
|
|
- } catch (Exception e) {
|
|
|
- errorMessageBuilder.append(
|
|
|
- String.format("FAILED to ADD: %s: %s%n", volume,
|
|
|
- e.toString()));
|
|
|
- LOG.error("Failed to add volume: {}", volume, e);
|
|
|
}
|
|
|
}
|
|
|
- }
|
|
|
|
|
|
- try {
|
|
|
- removeVolumes(changedVolumes.deactivateLocations);
|
|
|
- } catch (IOException e) {
|
|
|
- errorMessageBuilder.append(e.getMessage());
|
|
|
- LOG.error("Failed to remove volume", e);
|
|
|
- }
|
|
|
+ try {
|
|
|
+ removeVolumes(changedVolumes.deactivateLocations);
|
|
|
+ } catch (IOException e) {
|
|
|
+ errorMessageBuilder.append(e.getMessage());
|
|
|
+ LOG.error("Failed to remove volume", e);
|
|
|
+ }
|
|
|
|
|
|
- if (errorMessageBuilder.length() > 0) {
|
|
|
- throw new IOException(errorMessageBuilder.toString());
|
|
|
- }
|
|
|
- } finally {
|
|
|
- if (service != null) {
|
|
|
- service.shutdown();
|
|
|
+ if (errorMessageBuilder.length() > 0) {
|
|
|
+ throw new IOException(errorMessageBuilder.toString());
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ if (service != null) {
|
|
|
+ service.shutdown();
|
|
|
+ }
|
|
|
+ conf.set(DFS_DATANODE_DATA_DIR_KEY,
|
|
|
+ Joiner.on(",").join(effectiveVolumes));
|
|
|
+ dataDirs = getStorageLocations(conf);
|
|
|
}
|
|
|
- conf.set(DFS_DATANODE_DATA_DIR_KEY,
|
|
|
- Joiner.on(",").join(effectiveVolumes));
|
|
|
- dataDirs = getStorageLocations(conf);
|
|
|
}
|
|
|
}
|
|
|
|