|
@@ -28,6 +28,7 @@ import java.util.EnumMap;
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
import org.apache.hadoop.fs.permission.PermissionStatus;
|
|
import org.apache.hadoop.fs.permission.PermissionStatus;
|
|
|
|
+import org.apache.hadoop.hdfs.protocol.Block;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
|
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
|
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
|
|
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
|
|
@@ -57,6 +58,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TimesOp;
|
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateMasterKeyOp;
|
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateMasterKeyOp;
|
|
import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
|
|
import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
|
|
import org.apache.hadoop.hdfs.util.Holder;
|
|
import org.apache.hadoop.hdfs.util.Holder;
|
|
|
|
+
|
|
import com.google.common.base.Joiner;
|
|
import com.google.common.base.Joiner;
|
|
|
|
|
|
@InterfaceAudience.Private
|
|
@InterfaceAudience.Private
|
|
@@ -137,82 +139,84 @@ public class FSEditLogLoader {
|
|
numEdits++;
|
|
numEdits++;
|
|
incrOpCount(op.opCode, opCounts);
|
|
incrOpCount(op.opCode, opCounts);
|
|
switch (op.opCode) {
|
|
switch (op.opCode) {
|
|
- case OP_ADD:
|
|
|
|
- case OP_CLOSE: {
|
|
|
|
|
|
+ case OP_ADD: {
|
|
AddCloseOp addCloseOp = (AddCloseOp)op;
|
|
AddCloseOp addCloseOp = (AddCloseOp)op;
|
|
|
|
|
|
- // versions > 0 support per file replication
|
|
|
|
- // get name and replication
|
|
|
|
- final short replication = fsNamesys.getBlockManager(
|
|
|
|
- ).adjustReplication(addCloseOp.replication);
|
|
|
|
-
|
|
|
|
- long blockSize = addCloseOp.blockSize;
|
|
|
|
- BlockInfo blocks[] = new BlockInfo[addCloseOp.blocks.length];
|
|
|
|
- for (int i = 0; i < addCloseOp.blocks.length; i++) {
|
|
|
|
- if(addCloseOp.opCode == FSEditLogOpCodes.OP_ADD
|
|
|
|
- && i == addCloseOp.blocks.length-1) {
|
|
|
|
- blocks[i] = new BlockInfoUnderConstruction(addCloseOp.blocks[i],
|
|
|
|
- replication);
|
|
|
|
- } else {
|
|
|
|
- blocks[i] = new BlockInfo(addCloseOp.blocks[i], replication);
|
|
|
|
|
|
+ // See if the file already exists (persistBlocks call)
|
|
|
|
+ INodeFile oldFile = getINodeFile(fsDir, addCloseOp.path);
|
|
|
|
+ if (oldFile == null) { // this is OP_ADD on a new file
|
|
|
|
+ // versions > 0 support per file replication
|
|
|
|
+ // get name and replication
|
|
|
|
+ final short replication = fsNamesys.getBlockManager(
|
|
|
|
+ ).adjustReplication(addCloseOp.replication);
|
|
|
|
+ PermissionStatus permissions = fsNamesys.getUpgradePermission();
|
|
|
|
+ if (addCloseOp.permissions != null) {
|
|
|
|
+ permissions = addCloseOp.permissions;
|
|
}
|
|
}
|
|
- }
|
|
|
|
-
|
|
|
|
- PermissionStatus permissions = fsNamesys.getUpgradePermission();
|
|
|
|
- if (addCloseOp.permissions != null) {
|
|
|
|
- permissions = addCloseOp.permissions;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
-
|
|
|
|
- // Older versions of HDFS does not store the block size in inode.
|
|
|
|
- // If the file has more than one block, use the size of the
|
|
|
|
- // first block as the blocksize. Otherwise use the default
|
|
|
|
- // block size.
|
|
|
|
- if (-8 <= logVersion && blockSize == 0) {
|
|
|
|
- if (blocks.length > 1) {
|
|
|
|
- blockSize = blocks[0].getNumBytes();
|
|
|
|
- } else {
|
|
|
|
- long first = ((blocks.length == 1)? blocks[0].getNumBytes(): 0);
|
|
|
|
- blockSize = Math.max(fsNamesys.getDefaultBlockSize(), first);
|
|
|
|
|
|
+ long blockSize = addCloseOp.blockSize;
|
|
|
|
+
|
|
|
|
+ if (FSNamesystem.LOG.isDebugEnabled()) {
|
|
|
|
+ FSNamesystem.LOG.debug(op.opCode + ": " + addCloseOp.path +
|
|
|
|
+ " numblocks : " + addCloseOp.blocks.length +
|
|
|
|
+ " clientHolder " + addCloseOp.clientName +
|
|
|
|
+ " clientMachine " + addCloseOp.clientMachine);
|
|
}
|
|
}
|
|
- }
|
|
|
|
|
|
|
|
|
|
+ // Older versions of HDFS does not store the block size in inode.
|
|
|
|
+ // If the file has more than one block, use the size of the
|
|
|
|
+ // first block as the blocksize. Otherwise use the default
|
|
|
|
+ // block size.
|
|
|
|
+ if (-8 <= logVersion && blockSize == 0) {
|
|
|
|
+ if (addCloseOp.blocks.length > 1) {
|
|
|
|
+ blockSize = addCloseOp.blocks[0].getNumBytes();
|
|
|
|
+ } else {
|
|
|
|
+ long first = ((addCloseOp.blocks.length == 1)?
|
|
|
|
+ addCloseOp.blocks[0].getNumBytes(): 0);
|
|
|
|
+ blockSize = Math.max(fsNamesys.getDefaultBlockSize(), first);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
|
|
- // The open lease transaction re-creates a file if necessary.
|
|
|
|
- // Delete the file if it already exists.
|
|
|
|
- if (FSNamesystem.LOG.isDebugEnabled()) {
|
|
|
|
- FSNamesystem.LOG.debug(op.opCode + ": " + addCloseOp.path +
|
|
|
|
- " numblocks : " + blocks.length +
|
|
|
|
- " clientHolder " + addCloseOp.clientName +
|
|
|
|
- " clientMachine " + addCloseOp.clientMachine);
|
|
|
|
|
|
+ // TODO: We should do away with this add-then-replace dance.
|
|
|
|
+
|
|
|
|
+ // add to the file tree
|
|
|
|
+ INodeFile node = (INodeFile)fsDir.unprotectedAddFile(
|
|
|
|
+ addCloseOp.path, permissions,
|
|
|
|
+ replication, addCloseOp.mtime,
|
|
|
|
+ addCloseOp.atime, blockSize);
|
|
|
|
+
|
|
|
|
+ fsNamesys.prepareFileForWrite(addCloseOp.path, node,
|
|
|
|
+ addCloseOp.clientName, addCloseOp.clientMachine, null);
|
|
|
|
+ } else { // This is OP_ADD on an existing file
|
|
|
|
+ if (!oldFile.isUnderConstruction()) {
|
|
|
|
+ // This is a call to append() on an already-closed file.
|
|
|
|
+ fsNamesys.prepareFileForWrite(addCloseOp.path, oldFile,
|
|
|
|
+ addCloseOp.clientName, addCloseOp.clientMachine, null);
|
|
|
|
+ oldFile = getINodeFile(fsDir, addCloseOp.path);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ updateBlocks(fsDir, addCloseOp, oldFile);
|
|
}
|
|
}
|
|
-
|
|
|
|
- fsDir.unprotectedDelete(addCloseOp.path, addCloseOp.mtime);
|
|
|
|
-
|
|
|
|
- // add to the file tree
|
|
|
|
- INodeFile node = (INodeFile)fsDir.unprotectedAddFile(
|
|
|
|
- addCloseOp.path, permissions,
|
|
|
|
- blocks, replication,
|
|
|
|
- addCloseOp.mtime, addCloseOp.atime, blockSize);
|
|
|
|
- if (addCloseOp.opCode == FSEditLogOpCodes.OP_ADD) {
|
|
|
|
- //
|
|
|
|
- // Replace current node with a INodeUnderConstruction.
|
|
|
|
- // Recreate in-memory lease record.
|
|
|
|
- //
|
|
|
|
- INodeFileUnderConstruction cons = new INodeFileUnderConstruction(
|
|
|
|
- node.getLocalNameBytes(),
|
|
|
|
- node.getReplication(),
|
|
|
|
- node.getModificationTime(),
|
|
|
|
- node.getPreferredBlockSize(),
|
|
|
|
- node.getBlocks(),
|
|
|
|
- node.getPermissionStatus(),
|
|
|
|
- addCloseOp.clientName,
|
|
|
|
- addCloseOp.clientMachine,
|
|
|
|
- null);
|
|
|
|
- fsDir.replaceNode(addCloseOp.path, node, cons);
|
|
|
|
- fsNamesys.leaseManager.addLease(cons.getClientName(),
|
|
|
|
- addCloseOp.path);
|
|
|
|
|
|
+ break;
|
|
|
|
+ }
|
|
|
|
+ case OP_CLOSE: {
|
|
|
|
+ AddCloseOp addCloseOp = (AddCloseOp)op;
|
|
|
|
+
|
|
|
|
+ INodeFile oldFile = getINodeFile(fsDir, addCloseOp.path);
|
|
|
|
+ if (oldFile == null) {
|
|
|
|
+ throw new IOException("Operation trying to close non-existent file " +
|
|
|
|
+ addCloseOp.path);
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ // Update in-memory data structures
|
|
|
|
+ updateBlocks(fsDir, addCloseOp, oldFile);
|
|
|
|
+
|
|
|
|
+ // Now close the file
|
|
|
|
+ INodeFileUnderConstruction ucFile = (INodeFileUnderConstruction) oldFile;
|
|
|
|
+ // TODO: we could use removeLease(holder, path) here, but OP_CLOSE
|
|
|
|
+ // doesn't seem to serialize the holder... unclear why!
|
|
|
|
+ fsNamesys.leaseManager.removeLeaseWithPrefixPath(addCloseOp.path);
|
|
|
|
+ INodeFile newFile = ucFile.convertToInodeFile();
|
|
|
|
+ fsDir.replaceNode(addCloseOp.path, ucFile, newFile);
|
|
break;
|
|
break;
|
|
}
|
|
}
|
|
case OP_SET_REPLICATION: {
|
|
case OP_SET_REPLICATION: {
|
|
@@ -404,7 +408,88 @@ public class FSEditLogLoader {
|
|
}
|
|
}
|
|
return numEdits;
|
|
return numEdits;
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
|
|
+ private static INodeFile getINodeFile(FSDirectory fsDir, String path)
|
|
|
|
+ throws IOException {
|
|
|
|
+ INode inode = fsDir.getINode(path);
|
|
|
|
+ if (inode != null) {
|
|
|
|
+ if (!(inode instanceof INodeFile)) {
|
|
|
|
+ throw new IOException("Operation trying to get non-file " + path);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ return (INodeFile)inode;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Update in-memory data structures with new block information.
|
|
|
|
+ * @throws IOException
|
|
|
|
+ */
|
|
|
|
+ private void updateBlocks(FSDirectory fsDir, AddCloseOp addCloseOp,
|
|
|
|
+ INodeFile file) throws IOException {
|
|
|
|
+
|
|
|
|
+ // Update the salient file attributes.
|
|
|
|
+ file.setAccessTime(addCloseOp.atime);
|
|
|
|
+ file.setModificationTimeForce(addCloseOp.mtime);
|
|
|
|
+
|
|
|
|
+ // Update its block list
|
|
|
|
+ BlockInfo[] oldBlocks = file.getBlocks();
|
|
|
|
+
|
|
|
|
+ // Are we only updating the last block's gen stamp.
|
|
|
|
+ boolean isGenStampUpdate = oldBlocks.length == addCloseOp.blocks.length;
|
|
|
|
+
|
|
|
|
+ // First, update blocks in common
|
|
|
|
+ for (int i = 0; i < oldBlocks.length && i < addCloseOp.blocks.length; i++) {
|
|
|
|
+ BlockInfo oldBlock = oldBlocks[i];
|
|
|
|
+ Block newBlock = addCloseOp.blocks[i];
|
|
|
|
+
|
|
|
|
+ boolean isLastBlock = i == oldBlocks.length - 1;
|
|
|
|
+ if (oldBlock.getBlockId() != newBlock.getBlockId() ||
|
|
|
|
+ (oldBlock.getGenerationStamp() != newBlock.getGenerationStamp() &&
|
|
|
|
+ !(isGenStampUpdate && isLastBlock))) {
|
|
|
|
+ throw new IOException("Mismatched block IDs or generation stamps, " +
|
|
|
|
+ "attempting to replace block " + oldBlock + " with " + newBlock +
|
|
|
|
+ " as block # " + i + "/" + addCloseOp.blocks.length + " of " +
|
|
|
|
+ addCloseOp.path);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ oldBlock.setNumBytes(newBlock.getNumBytes());
|
|
|
|
+ oldBlock.setGenerationStamp(newBlock.getGenerationStamp());
|
|
|
|
+
|
|
|
|
+ if (oldBlock instanceof BlockInfoUnderConstruction &&
|
|
|
|
+ (!isLastBlock || addCloseOp.opCode == FSEditLogOpCodes.OP_CLOSE)) {
|
|
|
|
+ fsNamesys.getBlockManager().forceCompleteBlock(
|
|
|
|
+ (INodeFileUnderConstruction)file,
|
|
|
|
+ (BlockInfoUnderConstruction)oldBlock);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ if (addCloseOp.blocks.length < oldBlocks.length) {
|
|
|
|
+ // We're removing a block from the file, e.g. abandonBlock(...)
|
|
|
|
+ if (!file.isUnderConstruction()) {
|
|
|
|
+ throw new IOException("Trying to remove a block from file " +
|
|
|
|
+ addCloseOp.path + " which is not under construction.");
|
|
|
|
+ }
|
|
|
|
+ if (addCloseOp.blocks.length != oldBlocks.length - 1) {
|
|
|
|
+ throw new IOException("Trying to remove more than one block from file "
|
|
|
|
+ + addCloseOp.path);
|
|
|
|
+ }
|
|
|
|
+ fsDir.unprotectedRemoveBlock(addCloseOp.path,
|
|
|
|
+ (INodeFileUnderConstruction)file, oldBlocks[oldBlocks.length - 1]);
|
|
|
|
+ } else if (addCloseOp.blocks.length > oldBlocks.length) {
|
|
|
|
+ // We're adding blocks
|
|
|
|
+ for (int i = oldBlocks.length; i < addCloseOp.blocks.length; i++) {
|
|
|
|
+ Block newBlock = addCloseOp.blocks[i];
|
|
|
|
+ BlockInfo newBI = new BlockInfoUnderConstruction(newBlock, file.getReplication());
|
|
|
|
+ fsNamesys.getBlockManager().addINode(newBI, file);
|
|
|
|
+ file.addBlock(newBI);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ if (addCloseOp.blocks.length > 0) {
|
|
|
|
+ fsNamesys.notifyGenStampUpdate(
|
|
|
|
+ addCloseOp.blocks[addCloseOp.blocks.length - 1].getGenerationStamp());
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
|
|
private static void dumpOpCounts(
|
|
private static void dumpOpCounts(
|
|
EnumMap<FSEditLogOpCodes, Holder<Integer>> opCounts) {
|
|
EnumMap<FSEditLogOpCodes, Holder<Integer>> opCounts) {
|