|
@@ -129,21 +129,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
|
|
|
private File addBlock(Block b, File src, boolean createOk,
|
|
|
boolean resetIdx) throws IOException {
|
|
|
if (numBlocks < maxBlocksPerDir) {
|
|
|
- File dest = new File(dir, b.getBlockName());
|
|
|
- File metaData = getMetaFile(src, b.getGenerationStamp());
|
|
|
- File newmeta = getMetaFile(dest, b.getGenerationStamp());
|
|
|
- if ( ! metaData.renameTo( newmeta ) ||
|
|
|
- ! src.renameTo( dest ) ) {
|
|
|
- throw new IOException( "could not move files for " + b +
|
|
|
- " from " + src + " to " +
|
|
|
- dest.getAbsolutePath() + " or from"
|
|
|
- + metaData + " to " + newmeta);
|
|
|
- }
|
|
|
- if (DataNode.LOG.isDebugEnabled()) {
|
|
|
- DataNode.LOG.debug("addBlock: Moved " + metaData + " to " + newmeta);
|
|
|
- DataNode.LOG.debug("addBlock: Moved " + src + " to " + dest);
|
|
|
- }
|
|
|
-
|
|
|
+ final File dest = moveBlockFiles(b, src, dir);
|
|
|
numBlocks += 1;
|
|
|
return dest;
|
|
|
}
|
|
@@ -1289,6 +1275,26 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
|
|
|
return info.unlinkBlock(numLinks);
|
|
|
}
|
|
|
|
|
|
+ private static File moveBlockFiles(Block b, File srcfile, File destdir
|
|
|
+ ) throws IOException {
|
|
|
+ final File dstfile = new File(destdir, b.getBlockName());
|
|
|
+ final File srcmeta = getMetaFile(srcfile, b.getGenerationStamp());
|
|
|
+ final File dstmeta = getMetaFile(dstfile, b.getGenerationStamp());
|
|
|
+ if (!srcmeta.renameTo(dstmeta)) {
|
|
|
+ throw new IOException("Failed to move meta file for " + b
|
|
|
+ + " from " + srcmeta + " to " + dstmeta);
|
|
|
+ }
|
|
|
+ if (!srcfile.renameTo(dstfile)) {
|
|
|
+ throw new IOException("Failed to move block file for " + b
|
|
|
+ + " from " + srcfile + " to " + dstfile.getAbsolutePath());
|
|
|
+ }
|
|
|
+ if (DataNode.LOG.isDebugEnabled()) {
|
|
|
+ DataNode.LOG.debug("addBlock: Moved " + srcmeta + " to " + dstmeta);
|
|
|
+ DataNode.LOG.debug("addBlock: Moved " + srcfile + " to " + dstfile);
|
|
|
+ }
|
|
|
+ return dstfile;
|
|
|
+ }
|
|
|
+
|
|
|
static private void truncateBlock(File blockFile, File metaFile,
|
|
|
long oldlen, long newlen) throws IOException {
|
|
|
DataNode.LOG.info("truncateBlock: blockFile=" + blockFile
|
|
@@ -1619,6 +1625,56 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
|
|
|
return rbw;
|
|
|
}
|
|
|
|
|
|
+ @Override // FSDatasetInterface
|
|
|
+ public synchronized ReplicaInPipelineInterface convertTemporaryToRbw(
|
|
|
+ final ExtendedBlock b) throws IOException {
|
|
|
+ final long blockId = b.getBlockId();
|
|
|
+ final long expectedGs = b.getGenerationStamp();
|
|
|
+ final long visible = b.getNumBytes();
|
|
|
+ DataNode.LOG.info("Covert the temporary replica " + b
|
|
|
+ + " to RBW, visible length is " + visible);
|
|
|
+
|
|
|
+ // get replica
|
|
|
+ final ReplicaInfo r = volumeMap.get(b.getBlockPoolId(), blockId);
|
|
|
+ if (r == null) {
|
|
|
+ throw new ReplicaNotFoundException(
|
|
|
+ ReplicaNotFoundException.NON_EXISTENT_REPLICA + b);
|
|
|
+ }
|
|
|
+ // check the replica's state
|
|
|
+ if (r.getState() != ReplicaState.TEMPORARY) {
|
|
|
+ throw new ReplicaNotFoundException(
|
|
|
+ "r.getState() != ReplicaState.TEMPORARY, r=" + r);
|
|
|
+ }
|
|
|
+ // check generation stamp
|
|
|
+ if (r.getGenerationStamp() != expectedGs) {
|
|
|
+ throw new ReplicaNotFoundException(
|
|
|
+ "r.getGenerationStamp() != expectedGs = " + expectedGs + ", r=" + r);
|
|
|
+ }
|
|
|
+ // check length
|
|
|
+ final long numBytes = r.getNumBytes();
|
|
|
+ if (numBytes < visible) {
|
|
|
+ throw new ReplicaNotFoundException(numBytes + " = numBytes < visible = "
|
|
|
+ + visible + ", r=" + r);
|
|
|
+ }
|
|
|
+ // check volume
|
|
|
+ final FSVolume v = r.getVolume();
|
|
|
+ if (v == null) {
|
|
|
+ throw new IOException("r.getVolume() = null, temp=" + r);
|
|
|
+ }
|
|
|
+
|
|
|
+ // move block files to the rbw directory
|
|
|
+ final File dest = moveBlockFiles(b.getLocalBlock(), r.getBlockFile(),
|
|
|
+ v.getRbwDir(b.getBlockPoolId()));
|
|
|
+ // create RBW
|
|
|
+ final ReplicaBeingWritten rbw = new ReplicaBeingWritten(
|
|
|
+ blockId, numBytes, expectedGs,
|
|
|
+ v, dest.getParentFile(), Thread.currentThread());
|
|
|
+ rbw.setBytesAcked(visible);
|
|
|
+ // overwrite the RBW in the volume map
|
|
|
+ volumeMap.add(b.getBlockPoolId(), rbw);
|
|
|
+ return rbw;
|
|
|
+ }
|
|
|
+
|
|
|
@Override // FSDatasetInterface
|
|
|
public synchronized ReplicaInPipelineInterface createTemporary(ExtendedBlock b)
|
|
|
throws IOException {
|
|
@@ -1815,14 +1871,24 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
|
|
|
*/
|
|
|
@Override // FSDatasetInterface
|
|
|
public boolean isValidBlock(ExtendedBlock b) {
|
|
|
- ReplicaInfo replicaInfo = volumeMap.get(b.getBlockPoolId(),
|
|
|
+ return isValid(b, ReplicaState.FINALIZED);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Check whether the given block is a valid RBW.
|
|
|
+ */
|
|
|
+ @Override // {@link FSDatasetInterface}
|
|
|
+ public boolean isValidRbw(final ExtendedBlock b) {
|
|
|
+ return isValid(b, ReplicaState.RBW);
|
|
|
+ }
|
|
|
+
|
|
|
+ /** Does the block exist and have the given state? */
|
|
|
+ private boolean isValid(final ExtendedBlock b, final ReplicaState state) {
|
|
|
+ final ReplicaInfo replicaInfo = volumeMap.get(b.getBlockPoolId(),
|
|
|
b.getLocalBlock());
|
|
|
-
|
|
|
- if (replicaInfo == null ||
|
|
|
- replicaInfo.getState() != ReplicaState.FINALIZED) {
|
|
|
- return false;
|
|
|
- }
|
|
|
- return replicaInfo.getBlockFile().exists();
|
|
|
+ return replicaInfo != null
|
|
|
+ && replicaInfo.getState() == state
|
|
|
+ && replicaInfo.getBlockFile().exists();
|
|
|
}
|
|
|
|
|
|
/**
|