|
@@ -58,6 +58,7 @@ import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.DataBlockScanner;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.DataStorage;
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.Replica;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.ReplicaAlreadyExistsException;
|
|
@@ -565,28 +566,33 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
return dstfile;
|
|
|
}
|
|
|
|
|
|
- static File copyBlockFiles(Block b, File srcfile, File destdir)
|
|
|
+ /**
|
|
|
+ * Copy the block and meta files for the given block from the given
|
|
|
+ * @return the new meta file.
|
|
|
+ * @throws IOException
|
|
|
+ */
|
|
|
+ static File copyBlockFiles(ReplicaInfo replicaInfo, File destRoot)
|
|
|
throws IOException {
|
|
|
- final File dstfile = new File(destdir, b.getBlockName());
|
|
|
- final File srcmeta = FsDatasetUtil.getMetaFile(srcfile, b.getGenerationStamp());
|
|
|
- final File dstmeta = FsDatasetUtil.getMetaFile(dstfile, b.getGenerationStamp());
|
|
|
+ final File destDir = DatanodeUtil.idToBlockDir(destRoot, replicaInfo.getBlockId());
|
|
|
+ final File dstFile = new File(destDir, replicaInfo.getBlockName());
|
|
|
+ final File dstMeta = FsDatasetUtil.getMetaFile(dstFile, replicaInfo.getGenerationStamp());
|
|
|
+ final File srcMeta = replicaInfo.getMetaFile();
|
|
|
+ final File srcFile = replicaInfo.getBlockFile();
|
|
|
try {
|
|
|
- FileUtils.copyFile(srcmeta, dstmeta);
|
|
|
+ FileUtils.copyFile(srcMeta, dstMeta);
|
|
|
} catch (IOException e) {
|
|
|
- throw new IOException("Failed to copy meta file for " + b
|
|
|
- + " from " + srcmeta + " to " + dstmeta, e);
|
|
|
+ throw new IOException("Failed to copy " + srcMeta + " to " + dstMeta, e);
|
|
|
}
|
|
|
try {
|
|
|
- FileUtils.copyFile(srcfile, dstfile);
|
|
|
+ FileUtils.copyFile(srcFile, dstFile);
|
|
|
} catch (IOException e) {
|
|
|
- throw new IOException("Failed to copy block file for " + b
|
|
|
- + " from " + srcfile + " to " + dstfile.getAbsolutePath(), e);
|
|
|
+ throw new IOException("Failed to copy " + srcFile + " to " + dstFile, e);
|
|
|
}
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("addBlock: Moved " + srcmeta + " to " + dstmeta
|
|
|
- + " and " + srcfile + " to " + dstfile);
|
|
|
+ LOG.debug("addBlock: Moved " + srcMeta + " to " + dstMeta);
|
|
|
+ LOG.debug("addBlock: Moved " + srcFile + " to " + dstFile);
|
|
|
}
|
|
|
- return dstfile;
|
|
|
+ return dstMeta;
|
|
|
}
|
|
|
|
|
|
static private void truncateBlock(File blockFile, File metaFile,
|
|
@@ -1174,10 +1180,6 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
|
|
|
if (v.isTransientStorage()) {
|
|
|
lazyWriteReplicaTracker.addReplica(bpid, replicaInfo.getBlockId(), v);
|
|
|
-
|
|
|
- // Schedule a checkpoint.
|
|
|
- ((LazyWriter) lazyWriter.getRunnable())
|
|
|
- .addReplicaToLazyWriteQueue(bpid, replicaInfo.getBlockId());
|
|
|
}
|
|
|
}
|
|
|
volumeMap.add(bpid, newReplicaInfo);
|
|
@@ -2188,32 +2190,12 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
nbytes, flags);
|
|
|
}
|
|
|
|
|
|
- private static class BlockIdPair {
|
|
|
- final String bpid;
|
|
|
- final long blockId;
|
|
|
-
|
|
|
- BlockIdPair(final String bpid, final long blockId) {
|
|
|
- this.bpid = bpid;
|
|
|
- this.blockId = blockId;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- private class LazyWriter implements Runnable {
|
|
|
+ class LazyWriter implements Runnable {
|
|
|
private volatile boolean shouldRun = true;
|
|
|
final int checkpointerInterval;
|
|
|
|
|
|
- final private Queue<BlockIdPair> blocksPendingCheckpoint;
|
|
|
-
|
|
|
public LazyWriter(final int checkpointerInterval) {
|
|
|
this.checkpointerInterval = checkpointerInterval;
|
|
|
- blocksPendingCheckpoint = new LinkedList<BlockIdPair>();
|
|
|
- }
|
|
|
-
|
|
|
- // Schedule a replica for writing to persistent storage.
|
|
|
- public synchronized void addReplicaToLazyWriteQueue(
|
|
|
- String bpid, long blockId) {
|
|
|
- LOG.info("Block with blockId=" + blockId + "; bpid=" + bpid + " added to lazy writer queue");
|
|
|
- blocksPendingCheckpoint.add(new BlockIdPair(bpid, blockId));
|
|
|
}
|
|
|
|
|
|
private void moveReplicaToNewVolume(String bpid, long blockId)
|
|
@@ -2221,76 +2203,85 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|
|
|
|
|
LOG.info("LazyWriter invoked to save blockId=" + blockId + "; bpid=" + bpid);
|
|
|
|
|
|
- FsVolumeImpl targetVolume = null;
|
|
|
- Block block = null;
|
|
|
- File blockFile = null;
|
|
|
+ FsVolumeImpl targetVolume;
|
|
|
+ ReplicaInfo replicaInfo;
|
|
|
|
|
|
synchronized (this) {
|
|
|
- block = getStoredBlock(bpid, blockId);
|
|
|
- blockFile = getFile(bpid, blockId);
|
|
|
+ replicaInfo = volumeMap.get(bpid, blockId);
|
|
|
|
|
|
- if (block == null) {
|
|
|
- // The block was deleted before it could be checkpointed.
|
|
|
+ if (replicaInfo == null || !replicaInfo.getVolume().isTransientStorage()) {
|
|
|
+ // The block was either deleted before it could be checkpointed or
|
|
|
+ // it is already on persistent storage. This can occur if a second
|
|
|
+ // replica on persistent storage was found after the lazy write was
|
|
|
+ // scheduled.
|
|
|
return;
|
|
|
}
|
|
|
|
|
|
// Pick a target volume for the block.
|
|
|
targetVolume = volumes.getNextVolume(
|
|
|
- StorageType.DEFAULT, block.getNumBytes());
|
|
|
+ StorageType.DEFAULT, replicaInfo.getNumBytes());
|
|
|
+ }
|
|
|
+
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("LazyWriter starting to save blockId=" + blockId + "; bpid=" + bpid);
|
|
|
}
|
|
|
|
|
|
- LOG.info("LazyWriter starting to save blockId=" + blockId + "; bpid=" + bpid);
|
|
|
lazyWriteReplicaTracker.recordStartLazyPersist(bpid, blockId, targetVolume);
|
|
|
File savedBlockFile = targetVolume.getBlockPoolSlice(bpid)
|
|
|
- .lazyPersistReplica(block, blockFile);
|
|
|
+ .lazyPersistReplica(replicaInfo);
|
|
|
lazyWriteReplicaTracker.recordEndLazyPersist(bpid, blockId, savedBlockFile);
|
|
|
- LOG.info("LazyWriter finished saving blockId=" + blockId + "; bpid=" + bpid +
|
|
|
- " to file " + savedBlockFile);
|
|
|
+
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("LazyWriter finished saving blockId=" + blockId + "; bpid=" + bpid +
|
|
|
+ " to file " + savedBlockFile);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Checkpoint a pending replica to persistent storage now.
|
|
|
+ * If we fail then move the replica to the end of the queue.
|
|
|
* @return true if there is more work to be done, false otherwise.
|
|
|
*/
|
|
|
private boolean saveNextReplica() {
|
|
|
- BlockIdPair blockIdPair = null;
|
|
|
- int moreWorkThreshold = 0;
|
|
|
+ LazyWriteReplicaTracker.ReplicaState replicaState = null;
|
|
|
+ boolean succeeded = false;
|
|
|
|
|
|
try {
|
|
|
synchronized (this) {
|
|
|
- // Dequeue the next replica waiting to be checkpointed.
|
|
|
- blockIdPair = blocksPendingCheckpoint.poll();
|
|
|
- if (blockIdPair == null) {
|
|
|
- LOG.info("LazyWriter has no blocks to persist. " +
|
|
|
- "Thread going to sleep.");
|
|
|
+ replicaState = lazyWriteReplicaTracker.dequeueNextReplicaToPersist();
|
|
|
+ if (replicaState == null) {
|
|
|
return false;
|
|
|
}
|
|
|
}
|
|
|
|
|
|
// Move the replica outside the lock.
|
|
|
- moveReplicaToNewVolume(blockIdPair.bpid, blockIdPair.blockId);
|
|
|
-
|
|
|
+ moveReplicaToNewVolume(replicaState.bpid, replicaState.blockId);
|
|
|
+ succeeded = true;
|
|
|
} catch(IOException ioe) {
|
|
|
- // If we failed, put the block on the queue and let a retry
|
|
|
- // interval elapse before we try again so we don't try to keep
|
|
|
- // checkpointing the same block in a tight loop.
|
|
|
- synchronized (this) {
|
|
|
- blocksPendingCheckpoint.add(blockIdPair);
|
|
|
- ++moreWorkThreshold;
|
|
|
+ LOG.warn("Exception saving replica " + replicaState, ioe);
|
|
|
+ } finally {
|
|
|
+ if (!succeeded && replicaState != null) {
|
|
|
+ lazyWriteReplicaTracker.reenqueueReplica(replicaState);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- synchronized (this) {
|
|
|
- return blocksPendingCheckpoint.size() > moreWorkThreshold;
|
|
|
- }
|
|
|
+ return succeeded;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public void run() {
|
|
|
+ int numSuccessiveFailures = 0;
|
|
|
+
|
|
|
while (fsRunning && shouldRun) {
|
|
|
try {
|
|
|
- if (!saveNextReplica()) {
|
|
|
+ numSuccessiveFailures = saveNextReplica() ? 0 : (numSuccessiveFailures + 1);
|
|
|
+
|
|
|
+ // Sleep if we have no more work to do or if it looks like we are not
|
|
|
+ // making any forward progress. This is to ensure that if all persist
|
|
|
+ // operations are failing we don't keep retrying them in a tight loop.
|
|
|
+ if (numSuccessiveFailures == lazyWriteReplicaTracker.numReplicasNotPersisted()) {
|
|
|
Thread.sleep(checkpointerInterval * 1000);
|
|
|
+ numSuccessiveFailures = 0;
|
|
|
}
|
|
|
} catch (InterruptedException e) {
|
|
|
LOG.info("LazyWriter was interrupted, exiting");
|