|
@@ -83,7 +83,9 @@ public class FSImage implements Closeable {
|
|
|
|
|
|
final private Configuration conf;
|
|
|
|
|
|
- private final NNStorageRetentionManager archivalManager;
|
|
|
+ private final NNStorageRetentionManager archivalManager;
|
|
|
+
|
|
|
+ private SaveNamespaceContext curSaveNamespaceContext = null;
|
|
|
|
|
|
|
|
|
/**
|
|
@@ -715,14 +717,15 @@ public class FSImage implements Closeable {
|
|
|
/**
|
|
|
* Save the contents of the FS image to the file.
|
|
|
*/
|
|
|
- void saveFSImage(FSNamesystem source, StorageDirectory sd, long txid)
|
|
|
+ void saveFSImage(SaveNamespaceContext context, StorageDirectory sd)
|
|
|
throws IOException {
|
|
|
+ long txid = context.getTxId();
|
|
|
File newFile = NNStorage.getStorageFile(sd, NameNodeFile.IMAGE_NEW, txid);
|
|
|
File dstFile = NNStorage.getStorageFile(sd, NameNodeFile.IMAGE, txid);
|
|
|
|
|
|
- FSImageFormat.Saver saver = new FSImageFormat.Saver();
|
|
|
+ FSImageFormat.Saver saver = new FSImageFormat.Saver(context);
|
|
|
FSImageCompression compression = FSImageCompression.createCompression(conf);
|
|
|
- saver.save(newFile, txid, source, compression);
|
|
|
+ saver.save(newFile, compression);
|
|
|
|
|
|
MD5FileUtils.saveMD5File(dstFile, saver.getSavedDigest());
|
|
|
storage.setMostRecentCheckpointTxId(txid);
|
|
@@ -740,25 +743,24 @@ public class FSImage implements Closeable {
|
|
|
* and writing it out.
|
|
|
*/
|
|
|
private class FSImageSaver implements Runnable {
|
|
|
+ private final SaveNamespaceContext context;
|
|
|
private StorageDirectory sd;
|
|
|
- private List<StorageDirectory> errorSDs;
|
|
|
- private final long txid;
|
|
|
- private final FSNamesystem source;
|
|
|
-
|
|
|
- FSImageSaver(FSNamesystem source, StorageDirectory sd,
|
|
|
- List<StorageDirectory> errorSDs, long txid) {
|
|
|
- this.source = source;
|
|
|
+
|
|
|
+ public FSImageSaver(SaveNamespaceContext context, StorageDirectory sd) {
|
|
|
+ this.context = context;
|
|
|
this.sd = sd;
|
|
|
- this.errorSDs = errorSDs;
|
|
|
- this.txid = txid;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
public void run() {
|
|
|
try {
|
|
|
- saveFSImage(source, sd, txid);
|
|
|
+ saveFSImage(context, sd);
|
|
|
+ } catch (SaveNamespaceCancelledException snce) {
|
|
|
+ LOG.info("Cancelled image saving for " + sd.getRoot() +
|
|
|
+ ": " + snce.getMessage());
|
|
|
+ // don't report an error on the storage dir!
|
|
|
} catch (Throwable t) {
|
|
|
LOG.error("Unable to save image for " + sd.getRoot(), t);
|
|
|
- errorSDs.add(sd);
|
|
|
+ context.reportErrorOnStorageDirectory(sd);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -784,7 +786,7 @@ public class FSImage implements Closeable {
|
|
|
* Save the contents of the FS image to a new image file in each of the
|
|
|
* current storage directories.
|
|
|
*/
|
|
|
- void saveNamespace(FSNamesystem source) throws IOException {
|
|
|
+ synchronized void saveNamespace(FSNamesystem source) throws IOException {
|
|
|
assert editLog != null : "editLog must be initialized";
|
|
|
storage.attemptRestoreRemovedStorage();
|
|
|
|
|
@@ -800,46 +802,71 @@ public class FSImage implements Closeable {
|
|
|
} finally {
|
|
|
if (editLogWasOpen) {
|
|
|
editLog.startLogSegment(imageTxId + 1, true);
|
|
|
- // Take this opportunity to note the current transaction
|
|
|
+ // Take this opportunity to note the current transaction.
|
|
|
+ // Even if the namespace save was cancelled, this marker
|
|
|
+ // is only used to determine what transaction ID is required
|
|
|
+ // for startup. So, it doesn't hurt to update it unnecessarily.
|
|
|
storage.writeTransactionIdFileToStorage(imageTxId + 1);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
}
|
|
|
|
|
|
- protected void saveFSImageInAllDirs(FSNamesystem source, long txid)
|
|
|
- throws IOException {
|
|
|
- if (storage.getNumStorageDirs(NameNodeDirType.IMAGE) == 0) {
|
|
|
- throw new IOException("No image directories available!");
|
|
|
- }
|
|
|
-
|
|
|
- List<StorageDirectory> errorSDs =
|
|
|
- Collections.synchronizedList(new ArrayList<StorageDirectory>());
|
|
|
-
|
|
|
- List<Thread> saveThreads = new ArrayList<Thread>();
|
|
|
- // save images into current
|
|
|
- for (Iterator<StorageDirectory> it
|
|
|
- = storage.dirIterator(NameNodeDirType.IMAGE); it.hasNext();) {
|
|
|
- StorageDirectory sd = it.next();
|
|
|
- FSImageSaver saver = new FSImageSaver(source, sd, errorSDs, txid);
|
|
|
- Thread saveThread = new Thread(saver, saver.toString());
|
|
|
- saveThreads.add(saveThread);
|
|
|
- saveThread.start();
|
|
|
+ void cancelSaveNamespace(String reason)
|
|
|
+ throws InterruptedException {
|
|
|
+ SaveNamespaceContext ctx = curSaveNamespaceContext;
|
|
|
+ if (ctx != null) {
|
|
|
+ ctx.cancel(reason); // waits until complete
|
|
|
}
|
|
|
- waitForThreads(saveThreads);
|
|
|
- saveThreads.clear();
|
|
|
- storage.reportErrorsOnDirectories(errorSDs);
|
|
|
+ }
|
|
|
|
|
|
+
|
|
|
+ protected synchronized void saveFSImageInAllDirs(FSNamesystem source, long txid)
|
|
|
+ throws IOException {
|
|
|
if (storage.getNumStorageDirs(NameNodeDirType.IMAGE) == 0) {
|
|
|
- throw new IOException(
|
|
|
- "Failed to save in any storage directories while saving namespace.");
|
|
|
+ throw new IOException("No image directories available!");
|
|
|
}
|
|
|
-
|
|
|
- renameCheckpoint(txid);
|
|
|
|
|
|
- // Since we now have a new checkpoint, we can clean up some
|
|
|
- // old edit logs and checkpoints.
|
|
|
- purgeOldStorage();
|
|
|
+ SaveNamespaceContext ctx = new SaveNamespaceContext(
|
|
|
+ source, txid);
|
|
|
+ curSaveNamespaceContext = ctx;
|
|
|
+
|
|
|
+ try {
|
|
|
+ List<Thread> saveThreads = new ArrayList<Thread>();
|
|
|
+ // save images into current
|
|
|
+ for (Iterator<StorageDirectory> it
|
|
|
+ = storage.dirIterator(NameNodeDirType.IMAGE); it.hasNext();) {
|
|
|
+ StorageDirectory sd = it.next();
|
|
|
+ FSImageSaver saver = new FSImageSaver(ctx, sd);
|
|
|
+ Thread saveThread = new Thread(saver, saver.toString());
|
|
|
+ saveThreads.add(saveThread);
|
|
|
+ saveThread.start();
|
|
|
+ }
|
|
|
+ waitForThreads(saveThreads);
|
|
|
+ saveThreads.clear();
|
|
|
+ storage.reportErrorsOnDirectories(ctx.getErrorSDs());
|
|
|
+
|
|
|
+ if (storage.getNumStorageDirs(NameNodeDirType.IMAGE) == 0) {
|
|
|
+ throw new IOException(
|
|
|
+ "Failed to save in any storage directories while saving namespace.");
|
|
|
+ }
|
|
|
+ if (ctx.isCancelled()) {
|
|
|
+ deleteCancelledCheckpoint(txid);
|
|
|
+ ctx.checkCancelled(); // throws
|
|
|
+ assert false : "should have thrown above!";
|
|
|
+ }
|
|
|
+
|
|
|
+ renameCheckpoint(txid);
|
|
|
+
|
|
|
+ // Since we now have a new checkpoint, we can clean up some
|
|
|
+ // old edit logs and checkpoints.
|
|
|
+ purgeOldStorage();
|
|
|
+ } finally {
|
|
|
+ // Notify any threads waiting on the checkpoint to be canceled
|
|
|
+ // that it is complete.
|
|
|
+ ctx.markComplete();
|
|
|
+ ctx = null;
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -873,6 +900,24 @@ public class FSImage implements Closeable {
|
|
|
}
|
|
|
if(al != null) storage.reportErrorsOnDirectories(al);
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Deletes the checkpoint file in every storage directory,
|
|
|
+ * since the checkpoint was cancelled.
|
|
|
+ */
|
|
|
+ private void deleteCancelledCheckpoint(long txid) throws IOException {
|
|
|
+ ArrayList<StorageDirectory> al = Lists.newArrayList();
|
|
|
+
|
|
|
+ for (StorageDirectory sd : storage.dirIterable(NameNodeDirType.IMAGE)) {
|
|
|
+ File ckpt = NNStorage.getStorageFile(sd, NameNodeFile.IMAGE_NEW, txid);
|
|
|
+ if (ckpt.exists() && !ckpt.delete()) {
|
|
|
+ LOG.warn("Unable to delete cancelled checkpoint in " + sd);
|
|
|
+ al.add(sd);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ storage.reportErrorsOnDirectories(al);
|
|
|
+ }
|
|
|
+
|
|
|
|
|
|
private void renameCheckpointInDir(StorageDirectory sd, long txid)
|
|
|
throws IOException {
|
|
@@ -1055,4 +1100,5 @@ public class FSImage implements Closeable {
|
|
|
public synchronized long getLastAppliedTxId() {
|
|
|
return lastAppliedTxId;
|
|
|
}
|
|
|
+
|
|
|
}
|