|
@@ -38,7 +38,9 @@ import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.XAttrSetFlag;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
|
@@ -413,7 +415,8 @@ public class FSEditLogLoader {
|
|
|
// Update the salient file attributes.
|
|
|
newFile.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID);
|
|
|
newFile.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID);
|
|
|
- updateBlocks(fsDir, addCloseOp, iip, newFile);
|
|
|
+ // TODO whether the file is striped should later be retrieved from iip
|
|
|
+ updateBlocks(fsDir, addCloseOp, iip, newFile, newFile.isStriped());
|
|
|
break;
|
|
|
}
|
|
|
case OP_CLOSE: {
|
|
@@ -433,7 +436,8 @@ public class FSEditLogLoader {
|
|
|
// Update the salient file attributes.
|
|
|
file.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID);
|
|
|
file.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID);
|
|
|
- updateBlocks(fsDir, addCloseOp, iip, file);
|
|
|
+ // TODO whether the file is striped should later be retrieved from iip
|
|
|
+ updateBlocks(fsDir, addCloseOp, iip, file, file.isStriped());
|
|
|
|
|
|
// Now close the file
|
|
|
if (!file.isUnderConstruction() &&
|
|
@@ -491,7 +495,8 @@ public class FSEditLogLoader {
|
|
|
INodesInPath iip = fsDir.getINodesInPath(path, true);
|
|
|
INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path);
|
|
|
// Update in-memory data structures
|
|
|
- updateBlocks(fsDir, updateOp, iip, oldFile);
|
|
|
+ // TODO whether the file is striped should later be retrieved from iip
|
|
|
+ updateBlocks(fsDir, updateOp, iip, oldFile, oldFile.isStriped());
|
|
|
|
|
|
if (toAddRetryCache) {
|
|
|
fsNamesys.addCacheEntry(updateOp.rpcClientId, updateOp.rpcCallId);
|
|
@@ -507,7 +512,8 @@ public class FSEditLogLoader {
|
|
|
}
|
|
|
INodeFile oldFile = INodeFile.valueOf(fsDir.getINode(path), path);
|
|
|
// add the new block to the INodeFile
|
|
|
- addNewBlock(addBlockOp, oldFile);
|
|
|
+ // TODO whether the file is striped should later be retrieved from iip
|
|
|
+ addNewBlock(addBlockOp, oldFile, oldFile.isStriped());
|
|
|
break;
|
|
|
}
|
|
|
case OP_SET_REPLICATION: {
|
|
@@ -787,8 +793,15 @@ public class FSEditLogLoader {
|
|
|
}
|
|
|
case OP_ALLOCATE_BLOCK_ID: {
|
|
|
AllocateBlockIdOp allocateBlockIdOp = (AllocateBlockIdOp) op;
|
|
|
- fsNamesys.getBlockIdManager().setLastAllocatedBlockId(
|
|
|
- allocateBlockIdOp.blockId);
|
|
|
+ if (BlockIdManager.isStripedBlockID(allocateBlockIdOp.blockId)) {
|
|
|
+ // ALLOCATE_BLOCK_ID is added for sequential block id, thus if the id
|
|
|
+ // is negative, it must belong to striped blocks
|
|
|
+ fsNamesys.getBlockIdManager().setLastAllocatedStripedBlockId(
|
|
|
+ allocateBlockIdOp.blockId);
|
|
|
+ } else {
|
|
|
+ fsNamesys.getBlockIdManager().setLastAllocatedContiguousBlockId(
|
|
|
+ allocateBlockIdOp.blockId);
|
|
|
+ }
|
|
|
break;
|
|
|
}
|
|
|
case OP_ROLLING_UPGRADE_START: {
|
|
@@ -940,9 +953,9 @@ public class FSEditLogLoader {
|
|
|
|
|
|
/**
|
|
|
* Add a new block into the given INodeFile
|
|
|
- * TODO support adding striped block
|
|
|
*/
|
|
|
- private void addNewBlock(AddBlockOp op, INodeFile file) throws IOException {
|
|
|
+ private void addNewBlock(AddBlockOp op, INodeFile file, boolean isStriped)
|
|
|
+ throws IOException {
|
|
|
BlockInfo[] oldBlocks = file.getBlocks();
|
|
|
Block pBlock = op.getPenultimateBlock();
|
|
|
Block newBlock= op.getLastBlock();
|
|
@@ -950,7 +963,7 @@ public class FSEditLogLoader {
|
|
|
if (pBlock != null) { // the penultimate block is not null
|
|
|
assert oldBlocks != null && oldBlocks.length > 0;
|
|
|
// compare pBlock with the last block of oldBlocks
|
|
|
- Block oldLastBlock = oldBlocks[oldBlocks.length - 1];
|
|
|
+ BlockInfo oldLastBlock = oldBlocks[oldBlocks.length - 1];
|
|
|
if (oldLastBlock.getBlockId() != pBlock.getBlockId()
|
|
|
|| oldLastBlock.getGenerationStamp() != pBlock.getGenerationStamp()) {
|
|
|
throw new IOException(
|
|
@@ -960,29 +973,33 @@ public class FSEditLogLoader {
|
|
|
}
|
|
|
|
|
|
oldLastBlock.setNumBytes(pBlock.getNumBytes());
|
|
|
- if (oldLastBlock instanceof BlockInfoContiguousUnderConstruction) {
|
|
|
- fsNamesys.getBlockManager().forceCompleteBlock(file,
|
|
|
- (BlockInfoContiguousUnderConstruction) oldLastBlock);
|
|
|
+ if (!oldLastBlock.isComplete()) {
|
|
|
+ fsNamesys.getBlockManager().forceCompleteBlock(file, oldLastBlock);
|
|
|
fsNamesys.getBlockManager().processQueuedMessagesForBlock(pBlock);
|
|
|
}
|
|
|
} else { // the penultimate block is null
|
|
|
Preconditions.checkState(oldBlocks == null || oldBlocks.length == 0);
|
|
|
}
|
|
|
// add the new block
|
|
|
- BlockInfoContiguous newBI = new BlockInfoContiguousUnderConstruction(
|
|
|
- newBlock, file.getPreferredBlockReplication());
|
|
|
- fsNamesys.getBlockManager().addBlockCollection(newBI, file);
|
|
|
- file.addBlock(newBI);
|
|
|
+ final BlockInfo newBlockInfo;
|
|
|
+ if (isStriped) {
|
|
|
+ newBlockInfo = new BlockInfoStripedUnderConstruction(newBlock,
|
|
|
+ HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS);
|
|
|
+ } else {
|
|
|
+ newBlockInfo = new BlockInfoContiguousUnderConstruction(newBlock,
|
|
|
+ file.getPreferredBlockReplication());
|
|
|
+ }
|
|
|
+ fsNamesys.getBlockManager().addBlockCollection(newBlockInfo, file);
|
|
|
+ file.addBlock(newBlockInfo);
|
|
|
fsNamesys.getBlockManager().processQueuedMessagesForBlock(newBlock);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Update in-memory data structures with new block information.
|
|
|
- * TODO support adding striped block
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op,
|
|
|
- INodesInPath iip, INodeFile file) throws IOException {
|
|
|
+ INodesInPath iip, INodeFile file, boolean isStriped) throws IOException {
|
|
|
// Update its block list
|
|
|
BlockInfo[] oldBlocks = file.getBlocks();
|
|
|
Block[] newBlocks = op.getBlocks();
|
|
@@ -1011,11 +1028,10 @@ public class FSEditLogLoader {
|
|
|
oldBlock.getGenerationStamp() != newBlock.getGenerationStamp();
|
|
|
oldBlock.setGenerationStamp(newBlock.getGenerationStamp());
|
|
|
|
|
|
- if (oldBlock instanceof BlockInfoContiguousUnderConstruction &&
|
|
|
+ if (!oldBlock.isComplete() &&
|
|
|
(!isLastBlock || op.shouldCompleteLastBlock())) {
|
|
|
changeMade = true;
|
|
|
- fsNamesys.getBlockManager().forceCompleteBlock(file,
|
|
|
- (BlockInfoContiguousUnderConstruction) oldBlock);
|
|
|
+ fsNamesys.getBlockManager().forceCompleteBlock(file, oldBlock);
|
|
|
}
|
|
|
if (changeMade) {
|
|
|
// The state or gen-stamp of the block has changed. So, we may be
|
|
@@ -1045,13 +1061,18 @@ public class FSEditLogLoader {
|
|
|
// We're adding blocks
|
|
|
for (int i = oldBlocks.length; i < newBlocks.length; i++) {
|
|
|
Block newBlock = newBlocks[i];
|
|
|
- BlockInfoContiguous newBI;
|
|
|
+ final BlockInfo newBI;
|
|
|
if (!op.shouldCompleteLastBlock()) {
|
|
|
// TODO: shouldn't this only be true for the last block?
|
|
|
// what about an old-version fsync() where fsync isn't called
|
|
|
// until several blocks in?
|
|
|
- newBI = new BlockInfoContiguousUnderConstruction(
|
|
|
- newBlock, file.getPreferredBlockReplication());
|
|
|
+ if (isStriped) {
|
|
|
+ newBI = new BlockInfoStripedUnderConstruction(newBlock,
|
|
|
+ HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS);
|
|
|
+ } else {
|
|
|
+ newBI = new BlockInfoContiguousUnderConstruction(newBlock,
|
|
|
+ file.getPreferredBlockReplication());
|
|
|
+ }
|
|
|
} else {
|
|
|
// OP_CLOSE should add finalized blocks. This code path
|
|
|
// is only executed when loading edits written by prior
|