|
@@ -104,6 +104,7 @@ import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress.Counter;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step;
|
|
|
import org.apache.hadoop.hdfs.util.Holder;
|
|
|
+import org.apache.hadoop.io.erasurecode.ECSchema;
|
|
|
import org.apache.hadoop.util.ChunkedArrayList;
|
|
|
|
|
|
import com.google.common.base.Joiner;
|
|
@@ -416,8 +417,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);
|
|
|
- // TODO whether the file is striped should later be retrieved from iip
|
|
|
- updateBlocks(fsDir, addCloseOp, iip, newFile, fsDir.isInECZone(iip));
|
|
|
+ updateBlocks(fsDir, addCloseOp, iip, newFile,
|
|
|
+ fsDir.getECSchema(iip), fsDir.isInECZone(iip));
|
|
|
break;
|
|
|
}
|
|
|
case OP_CLOSE: {
|
|
@@ -437,8 +438,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);
|
|
|
- // TODO whether the file is striped should later be retrieved from iip
|
|
|
- updateBlocks(fsDir, addCloseOp, iip, file, fsDir.isInECZone(iip));
|
|
|
+ updateBlocks(fsDir, addCloseOp, iip, file,
|
|
|
+ fsDir.getECSchema(iip), fsDir.isInECZone(iip));
|
|
|
|
|
|
// Now close the file
|
|
|
if (!file.isUnderConstruction() &&
|
|
@@ -496,9 +497,9 @@ public class FSEditLogLoader {
|
|
|
INodesInPath iip = fsDir.getINodesInPath(path, true);
|
|
|
INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path);
|
|
|
// Update in-memory data structures
|
|
|
- // TODO whether the file is striped should later be retrieved from iip
|
|
|
- updateBlocks(fsDir, updateOp, iip, oldFile, fsDir.isInECZone(iip));
|
|
|
-
|
|
|
+ updateBlocks(fsDir, updateOp, iip, oldFile,
|
|
|
+ fsDir.getECSchema(iip), fsDir.isInECZone(iip));
|
|
|
+
|
|
|
if (toAddRetryCache) {
|
|
|
fsNamesys.addCacheEntry(updateOp.rpcClientId, updateOp.rpcCallId);
|
|
|
}
|
|
@@ -514,8 +515,8 @@ public class FSEditLogLoader {
|
|
|
INodesInPath iip = fsDir.getINodesInPath(path, true);
|
|
|
INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path);
|
|
|
// add the new block to the INodeFile
|
|
|
- // TODO whether the file is striped should later be retrieved from iip
|
|
|
- addNewBlock(addBlockOp, oldFile, fsDir.isInECZone(iip));
|
|
|
+ addNewBlock(addBlockOp, oldFile,
|
|
|
+ fsDir.getECSchema(iip), fsDir.isInECZone(iip));
|
|
|
break;
|
|
|
}
|
|
|
case OP_SET_REPLICATION: {
|
|
@@ -956,7 +957,8 @@ public class FSEditLogLoader {
|
|
|
/**
|
|
|
* Add a new block into the given INodeFile
|
|
|
*/
|
|
|
- private void addNewBlock(AddBlockOp op, INodeFile file, boolean isStriped)
|
|
|
+ private void addNewBlock(AddBlockOp op, INodeFile file,
|
|
|
+ ECSchema schema, boolean isStriped)
|
|
|
throws IOException {
|
|
|
BlockInfo[] oldBlocks = file.getBlocks();
|
|
|
Block pBlock = op.getPenultimateBlock();
|
|
@@ -985,8 +987,7 @@ public class FSEditLogLoader {
|
|
|
// add the new block
|
|
|
final BlockInfo newBlockInfo;
|
|
|
if (isStriped) {
|
|
|
- newBlockInfo = new BlockInfoStripedUnderConstruction(newBlock,
|
|
|
- HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS);
|
|
|
+ newBlockInfo = new BlockInfoStripedUnderConstruction(newBlock, schema);
|
|
|
} else {
|
|
|
newBlockInfo = new BlockInfoContiguousUnderConstruction(newBlock,
|
|
|
file.getPreferredBlockReplication());
|
|
@@ -1001,7 +1002,8 @@ public class FSEditLogLoader {
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op,
|
|
|
- INodesInPath iip, INodeFile file, boolean isStriped) throws IOException {
|
|
|
+ INodesInPath iip, INodeFile file, ECSchema schema,
|
|
|
+ boolean isStriped) throws IOException {
|
|
|
// Update its block list
|
|
|
BlockInfo[] oldBlocks = file.getBlocks();
|
|
|
Block[] newBlocks = op.getBlocks();
|
|
@@ -1069,8 +1071,7 @@ public class FSEditLogLoader {
|
|
|
// what about an old-version fsync() where fsync isn't called
|
|
|
// until several blocks in?
|
|
|
if (isStriped) {
|
|
|
- newBI = new BlockInfoStripedUnderConstruction(newBlock,
|
|
|
- HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS);
|
|
|
+ newBI = new BlockInfoStripedUnderConstruction(newBlock, schema);
|
|
|
} else {
|
|
|
newBI = new BlockInfoContiguousUnderConstruction(newBlock,
|
|
|
file.getPreferredBlockReplication());
|
|
@@ -1080,8 +1081,9 @@ public class FSEditLogLoader {
|
|
|
// is only executed when loading edits written by prior
|
|
|
// versions of Hadoop. Current versions always log
|
|
|
// OP_ADD operations as each block is allocated.
|
|
|
+ // TODO: ECSchema can be restored from persisted file (HDFS-7859).
|
|
|
newBI = isStriped ? new BlockInfoStriped(newBlock,
|
|
|
- HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS) :
|
|
|
+ ErasureCodingSchemaManager.getSystemDefaultSchema()) :
|
|
|
new BlockInfoContiguous(newBlock, file.getPreferredBlockReplication());
|
|
|
}
|
|
|
fsNamesys.getBlockManager().addBlockCollectionWithCheck(newBI, file);
|