|
@@ -32,6 +32,8 @@ import org.apache.hadoop.fs.XAttr;
|
|
import org.apache.hadoop.fs.permission.AclEntry;
|
|
import org.apache.hadoop.fs.permission.AclEntry;
|
|
import org.apache.hadoop.fs.permission.FsAction;
|
|
import org.apache.hadoop.fs.permission.FsAction;
|
|
import org.apache.hadoop.fs.permission.PermissionStatus;
|
|
import org.apache.hadoop.fs.permission.PermissionStatus;
|
|
|
|
+import com.google.protobuf.ByteString;
|
|
|
|
+import org.apache.hadoop.fs.InvalidPathException;
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
|
|
import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
|
|
@@ -173,34 +175,34 @@ class FSDirWriteFileOp {
|
|
final byte storagePolicyID;
|
|
final byte storagePolicyID;
|
|
String clientMachine;
|
|
String clientMachine;
|
|
|
|
|
|
- byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
|
|
|
|
- src = fsn.dir.resolvePath(pc, src, pathComponents);
|
|
|
|
- FileState fileState = analyzeFileState(fsn, src, fileId, clientName,
|
|
|
|
- previous, onRetryBlock);
|
|
|
|
- final INodeFile pendingFile = fileState.inode;
|
|
|
|
- // Check if the penultimate block is minimally replicated
|
|
|
|
- if (!fsn.checkFileProgress(src, pendingFile, false)) {
|
|
|
|
- throw new NotReplicatedYetException("Not replicated yet: " + src);
|
|
|
|
- }
|
|
|
|
|
|
+ FSDirectory fsd = fsn.getFSDirectory();
|
|
|
|
+ try (ROTransaction tx = fsd.newROTransaction().begin()) {
|
|
|
|
+ FileState fileState = analyzeFileState(tx, fsn, src, fileId, clientName,
|
|
|
|
+ previous, onRetryBlock);
|
|
|
|
+ final FlatINode pendingFile = fileState.inode;
|
|
|
|
+ FlatINodeFileFeature f = pendingFile.feature(FlatINodeFileFeature.class);
|
|
|
|
+ // Check if the penultimate block is minimally replicated
|
|
|
|
+ if (!fsn.checkFileProgress(src, f, false)) {
|
|
|
|
+ throw new NotReplicatedYetException("Not replicated yet: " + src);
|
|
|
|
+ }
|
|
|
|
|
|
- if (onRetryBlock[0] != null && onRetryBlock[0].getLocations().length > 0) {
|
|
|
|
- // This is a retry. No need to generate new locations.
|
|
|
|
- // Use the last block if it has locations.
|
|
|
|
- return null;
|
|
|
|
- }
|
|
|
|
- if (pendingFile.getBlocks().length >= fsn.maxBlocksPerFile) {
|
|
|
|
- throw new IOException("File has reached the limit on maximum number of"
|
|
|
|
- + " blocks (" + DFSConfigKeys.DFS_NAMENODE_MAX_BLOCKS_PER_FILE_KEY
|
|
|
|
- + "): " + pendingFile.getBlocks().length + " >= "
|
|
|
|
- + fsn.maxBlocksPerFile);
|
|
|
|
|
|
+ if (onRetryBlock[0] != null && onRetryBlock[0].getLocations().length > 0) {
|
|
|
|
+ // This is a retry. No need to generate new locations.
|
|
|
|
+ // Use the last block if it has locations.
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+ if (f.numBlocks() >= fsn.maxBlocksPerFile) {
|
|
|
|
+ throw new IOException("File has reached the limit on maximum number of"
|
|
|
|
+ + " blocks (" + DFSConfigKeys.DFS_NAMENODE_MAX_BLOCKS_PER_FILE_KEY
|
|
|
|
+ + "): " + f.numBlocks() + " >= " + fsn.maxBlocksPerFile);
|
|
|
|
+ }
|
|
|
|
+ blockSize = f.blockSize();
|
|
|
|
+ clientMachine = f.clientMachine();
|
|
|
|
+ replication = f.replication();
|
|
|
|
+ storagePolicyID = f.storagePolicyId();
|
|
|
|
+ return new ValidateAddBlockResult(blockSize, replication, storagePolicyID,
|
|
|
|
+ clientMachine);
|
|
}
|
|
}
|
|
- blockSize = pendingFile.getPreferredBlockSize();
|
|
|
|
- clientMachine = pendingFile.getFileUnderConstructionFeature()
|
|
|
|
- .getClientMachine();
|
|
|
|
- replication = pendingFile.getFileReplication();
|
|
|
|
- storagePolicyID = pendingFile.getStoragePolicyID();
|
|
|
|
- return new ValidateAddBlockResult(blockSize, replication, storagePolicyID,
|
|
|
|
- clientMachine);
|
|
|
|
}
|
|
}
|
|
|
|
|
|
static LocatedBlock makeLocatedBlock(FSNamesystem fsn, Block blk,
|
|
static LocatedBlock makeLocatedBlock(FSNamesystem fsn, Block blk,
|
|
@@ -226,39 +228,50 @@ class FSDirWriteFileOp {
|
|
// Run the full analysis again, since things could have changed
|
|
// Run the full analysis again, since things could have changed
|
|
// while chooseTarget() was executing.
|
|
// while chooseTarget() was executing.
|
|
LocatedBlock[] onRetryBlock = new LocatedBlock[1];
|
|
LocatedBlock[] onRetryBlock = new LocatedBlock[1];
|
|
- FileState fileState = analyzeFileState(fsn, src, fileId, clientName,
|
|
|
|
- previous, onRetryBlock);
|
|
|
|
- final INodeFile pendingFile = fileState.inode;
|
|
|
|
- src = fileState.path;
|
|
|
|
-
|
|
|
|
- if (onRetryBlock[0] != null) {
|
|
|
|
- if (onRetryBlock[0].getLocations().length > 0) {
|
|
|
|
- // This is a retry. Just return the last block if having locations.
|
|
|
|
- return onRetryBlock[0];
|
|
|
|
- } else {
|
|
|
|
- // add new chosen targets to already allocated block and return
|
|
|
|
- BlockInfoContiguous lastBlockInFile = pendingFile.getLastBlock();
|
|
|
|
- ((BlockInfoContiguousUnderConstruction) lastBlockInFile)
|
|
|
|
- .setExpectedLocations(targets);
|
|
|
|
- offset = pendingFile.computeFileSize();
|
|
|
|
- return makeLocatedBlock(fsn, lastBlockInFile, targets, offset);
|
|
|
|
|
|
+ FSDirectory fsd = fsn.getFSDirectory();
|
|
|
|
+ BlockManager bm = fsn.getBlockManager();
|
|
|
|
+ try (RWTransaction tx = fsd.newRWTransaction().begin()) {
|
|
|
|
+ FileState fileState = analyzeFileState(tx, fsn, src, fileId, clientName,
|
|
|
|
+ previous, onRetryBlock);
|
|
|
|
+ final FlatINode inode = fileState.inode;
|
|
|
|
+ FlatINodeFileFeature file = inode.feature(FlatINodeFileFeature.class);
|
|
|
|
+ src = fileState.path;
|
|
|
|
+
|
|
|
|
+ if (onRetryBlock[0] != null) {
|
|
|
|
+ if (onRetryBlock[0].getLocations().length > 0) {
|
|
|
|
+ // This is a retry. Just return the last block if having locations.
|
|
|
|
+ return onRetryBlock[0];
|
|
|
|
+ } else {
|
|
|
|
+ // add new chosen targets to already allocated block and return
|
|
|
|
+ Block lastBlock = file.lastBlock();
|
|
|
|
+ BlockInfoContiguous lastBlockInFile = bm.getStoredBlock(lastBlock);
|
|
|
|
+ ((BlockInfoContiguousUnderConstruction) lastBlockInFile)
|
|
|
|
+ .setExpectedLocations(targets);
|
|
|
|
+ offset = file.fileSize();
|
|
|
|
+ return makeLocatedBlock(fsn, lastBlockInFile, targets, offset);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
- }
|
|
|
|
|
|
|
|
- // commit the last block and complete it if it has minimum replicas
|
|
|
|
- fsn.commitOrCompleteLastBlock(pendingFile, fileState.iip,
|
|
|
|
- ExtendedBlock.getLocalBlock(previous));
|
|
|
|
|
|
+ // commit the last block and complete it if it has minimum replicas
|
|
|
|
+ FlatINodeFileFeature.Builder newFile = fsn.commitOrCompleteLastBlock(
|
|
|
|
+ file, ExtendedBlock.getLocalBlock(previous));
|
|
|
|
|
|
- // allocate new block, record block locations in INode.
|
|
|
|
- Block newBlock = fsn.createNewBlock();
|
|
|
|
- INodesInPath inodesInPath = INodesInPath.fromINode(pendingFile);
|
|
|
|
- saveAllocatedBlock(fsn, src, inodesInPath, newBlock, targets);
|
|
|
|
|
|
+ // allocate new block, record block locations in INode.
|
|
|
|
+ Block newBlock = fsn.createNewBlock();
|
|
|
|
+ saveAllocatedBlock(fsn, src, inode, newBlock, targets);
|
|
|
|
+ FlatINode newInode = persistNewBlock(tx, src, inode, newFile, newBlock);
|
|
|
|
+ offset = newInode.<FlatINodeFileFeature>feature(
|
|
|
|
+ FlatINodeFileFeature.class).fileSize();
|
|
|
|
|
|
- persistNewBlock(fsn, src, pendingFile);
|
|
|
|
- offset = pendingFile.computeFileSize();
|
|
|
|
|
|
+ // TODO: Update quota
|
|
|
|
+ // check quota limits and updated space consumed
|
|
|
|
+// fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(),
|
|
|
|
+// fileINode.getFileReplication(), true);
|
|
|
|
|
|
- // Return located block
|
|
|
|
- return makeLocatedBlock(fsn, newBlock, targets, offset);
|
|
|
|
|
|
+ tx.commit();
|
|
|
|
+ // Return located block
|
|
|
|
+ return makeLocatedBlock(fsn, newBlock, targets, offset);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
static DatanodeStorageInfo[] chooseTargetForNewBlock(
|
|
static DatanodeStorageInfo[] chooseTargetForNewBlock(
|
|
@@ -553,72 +566,56 @@ class FSDirWriteFileOp {
|
|
* Add a block to the file. Returns a reference to the added block.
|
|
* Add a block to the file. Returns a reference to the added block.
|
|
*/
|
|
*/
|
|
private static BlockInfoContiguous addBlock(
|
|
private static BlockInfoContiguous addBlock(
|
|
- FSDirectory fsd, String path, INodesInPath inodesInPath, Block block,
|
|
|
|
|
|
+ BlockManager bm, String path, FlatINode inode, Block block,
|
|
DatanodeStorageInfo[] targets) throws IOException {
|
|
DatanodeStorageInfo[] targets) throws IOException {
|
|
- fsd.writeLock();
|
|
|
|
- try {
|
|
|
|
- final INodeFile fileINode = inodesInPath.getLastINode().asFile();
|
|
|
|
- Preconditions.checkState(fileINode.isUnderConstruction());
|
|
|
|
-
|
|
|
|
- // check quota limits and updated space consumed
|
|
|
|
- fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(),
|
|
|
|
- fileINode.getFileReplication(), true);
|
|
|
|
-
|
|
|
|
- // associate new last block for the file
|
|
|
|
- BlockInfoContiguousUnderConstruction blockInfo =
|
|
|
|
|
|
+ FlatINodeFileFeature f = inode.feature(FlatINodeFileFeature.class);
|
|
|
|
+ Preconditions.checkState(f.inConstruction());
|
|
|
|
+ // associate new last block for the file
|
|
|
|
+ BlockInfoContiguousUnderConstruction blockInfo =
|
|
new BlockInfoContiguousUnderConstruction(
|
|
new BlockInfoContiguousUnderConstruction(
|
|
- block,
|
|
|
|
- fileINode.getFileReplication(),
|
|
|
|
|
|
+ block, f.replication(),
|
|
HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION,
|
|
HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION,
|
|
targets);
|
|
targets);
|
|
- fsd.getBlockManager().addBlockCollection(blockInfo, fileINode);
|
|
|
|
- fileINode.addBlock(blockInfo);
|
|
|
|
|
|
|
|
- if(NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
|
- NameNode.stateChangeLog.debug("DIR* FSDirectory.addBlock: "
|
|
|
|
- + path + " with " + block
|
|
|
|
- + " block is added to the in-memory "
|
|
|
|
- + "file system");
|
|
|
|
- }
|
|
|
|
- return blockInfo;
|
|
|
|
- } finally {
|
|
|
|
- fsd.writeUnlock();
|
|
|
|
|
|
+ bm.addBlockCollection(blockInfo, inode.id());
|
|
|
|
+ if(NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
|
+ NameNode.stateChangeLog.debug("DIR* FSDirectory.addBlock: " + path
|
|
|
|
+ + " with " + block + " block is added to the in-memory file system");
|
|
}
|
|
}
|
|
|
|
+ return blockInfo;
|
|
}
|
|
}
|
|
|
|
|
|
private static FileState analyzeFileState(
|
|
private static FileState analyzeFileState(
|
|
- FSNamesystem fsn, String src, long fileId, String clientName,
|
|
|
|
|
|
+ Transaction tx, FSNamesystem fsn, String src, long fileId, String clientName,
|
|
ExtendedBlock previous, LocatedBlock[] onRetryBlock)
|
|
ExtendedBlock previous, LocatedBlock[] onRetryBlock)
|
|
throws IOException {
|
|
throws IOException {
|
|
assert fsn.hasReadLock();
|
|
assert fsn.hasReadLock();
|
|
-
|
|
|
|
|
|
+ BlockManager bm = fsn.getBlockManager();
|
|
checkBlock(fsn, previous);
|
|
checkBlock(fsn, previous);
|
|
onRetryBlock[0] = null;
|
|
onRetryBlock[0] = null;
|
|
- fsn.checkNameNodeSafeMode("Cannot add block to " + src);
|
|
|
|
-
|
|
|
|
- // have we exceeded the configured limit of fs objects.
|
|
|
|
- fsn.checkFsObjectLimit();
|
|
|
|
|
|
|
|
Block previousBlock = ExtendedBlock.getLocalBlock(previous);
|
|
Block previousBlock = ExtendedBlock.getLocalBlock(previous);
|
|
- final INode inode;
|
|
|
|
- final INodesInPath iip;
|
|
|
|
- if (fileId == HdfsConstants.GRANDFATHER_INODE_ID) {
|
|
|
|
|
|
+ final Resolver.Result paths;
|
|
|
|
+ if (true || fileId == HdfsConstants.GRANDFATHER_INODE_ID) {
|
|
// Older clients may not have given us an inode ID to work with.
|
|
// Older clients may not have given us an inode ID to work with.
|
|
// In this case, we have to try to resolve the path and hope it
|
|
// In this case, we have to try to resolve the path and hope it
|
|
// hasn't changed or been deleted since the file was opened for write.
|
|
// hasn't changed or been deleted since the file was opened for write.
|
|
- iip = fsn.dir.getINodesInPath4Write(src);
|
|
|
|
- inode = iip.getLastINode();
|
|
|
|
|
|
+ paths = Resolver.resolve(tx, src);
|
|
} else {
|
|
} else {
|
|
// Newer clients pass the inode ID, so we can just get the inode
|
|
// Newer clients pass the inode ID, so we can just get the inode
|
|
// directly.
|
|
// directly.
|
|
- inode = fsn.dir.getInode(fileId);
|
|
|
|
- iip = INodesInPath.fromINode(inode);
|
|
|
|
- if (inode != null) {
|
|
|
|
- src = iip.getPath();
|
|
|
|
- }
|
|
|
|
|
|
+ paths = Resolver.resolveById(tx, fileId);
|
|
|
|
+ }
|
|
|
|
+ if (paths.invalidPath()) {
|
|
|
|
+ throw new InvalidPathException(src);
|
|
|
|
+ } else if (paths.notFound()) {
|
|
|
|
+ throw new FileNotFoundException(src);
|
|
}
|
|
}
|
|
- final INodeFile file = fsn.checkLease(src, clientName, inode, fileId);
|
|
|
|
- BlockInfoContiguous lastBlockInFile = file.getLastBlock();
|
|
|
|
|
|
+ FlatINode inode = paths.inodesInPath().getLastINode();
|
|
|
|
+ fsn.checkLease(src, clientName, inode);
|
|
|
|
+ FlatINodeFileFeature pendingFile = inode.feature(FlatINodeFileFeature.class);
|
|
|
|
+ BlockInfoContiguous lastBlockInFile = pendingFile.lastBlock() == null ?
|
|
|
|
+ null : bm.getStoredBlock(pendingFile.lastBlock());
|
|
if (!Block.matchingIdAndGenStamp(previousBlock, lastBlockInFile)) {
|
|
if (!Block.matchingIdAndGenStamp(previousBlock, lastBlockInFile)) {
|
|
// The block that the client claims is the current last block
|
|
// The block that the client claims is the current last block
|
|
// doesn't match up with what we think is the last block. There are
|
|
// doesn't match up with what we think is the last block. There are
|
|
@@ -646,10 +643,11 @@ class FSDirWriteFileOp {
|
|
// changed the namesystem state yet.
|
|
// changed the namesystem state yet.
|
|
// We run this analysis again in Part II where case 4 is impossible.
|
|
// We run this analysis again in Part II where case 4 is impossible.
|
|
|
|
|
|
- BlockInfoContiguous penultimateBlock = file.getPenultimateBlock();
|
|
|
|
|
|
+ BlockInfoContiguous penultimateBlock = bm.getStoredBlock(
|
|
|
|
+ pendingFile.penultimateBlock());
|
|
if (previous == null &&
|
|
if (previous == null &&
|
|
lastBlockInFile != null &&
|
|
lastBlockInFile != null &&
|
|
- lastBlockInFile.getNumBytes() >= file.getPreferredBlockSize() &&
|
|
|
|
|
|
+ lastBlockInFile.getNumBytes() >= pendingFile.blockSize() &&
|
|
lastBlockInFile.isComplete()) {
|
|
lastBlockInFile.isComplete()) {
|
|
// Case 1
|
|
// Case 1
|
|
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
@@ -668,15 +666,15 @@ class FSDirWriteFileOp {
|
|
|
|
|
|
// Case 2
|
|
// Case 2
|
|
// Return the last block.
|
|
// Return the last block.
|
|
- NameNode.stateChangeLog.info("BLOCK* allocateBlock: caught retry for " +
|
|
|
|
- "allocation of a new block in " + src + ". Returning previously" +
|
|
|
|
- " allocated block " + lastBlockInFile);
|
|
|
|
- long offset = file.computeFileSize();
|
|
|
|
|
|
+ NameNode.stateChangeLog.info("BLOCK* allocateBlock: " +
|
|
|
|
+ "caught retry for allocation of a new block in " +
|
|
|
|
+ src + ". Returning previously allocated block " + lastBlockInFile);
|
|
|
|
+ long offset = pendingFile.fileSize();
|
|
BlockInfoContiguousUnderConstruction lastBlockUC =
|
|
BlockInfoContiguousUnderConstruction lastBlockUC =
|
|
(BlockInfoContiguousUnderConstruction) lastBlockInFile;
|
|
(BlockInfoContiguousUnderConstruction) lastBlockInFile;
|
|
onRetryBlock[0] = makeLocatedBlock(fsn, lastBlockInFile,
|
|
onRetryBlock[0] = makeLocatedBlock(fsn, lastBlockInFile,
|
|
lastBlockUC.getExpectedStorageLocations(), offset);
|
|
lastBlockUC.getExpectedStorageLocations(), offset);
|
|
- return new FileState(file, src, iip);
|
|
|
|
|
|
+ return new FileState(inode, src);
|
|
} else {
|
|
} else {
|
|
// Case 3
|
|
// Case 3
|
|
throw new IOException("Cannot allocate block in " + src + ": " +
|
|
throw new IOException("Cannot allocate block in " + src + ": " +
|
|
@@ -684,7 +682,7 @@ class FSDirWriteFileOp {
|
|
"last block in file " + lastBlockInFile);
|
|
"last block in file " + lastBlockInFile);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
- return new FileState(file, src, iip);
|
|
|
|
|
|
+ return new FileState(inode, src);
|
|
}
|
|
}
|
|
|
|
|
|
static boolean completeFile(FSNamesystem fsn, FSPermissionChecker pc,
|
|
static boolean completeFile(FSNamesystem fsn, FSPermissionChecker pc,
|
|
@@ -779,15 +777,25 @@ class FSDirWriteFileOp {
|
|
/**
|
|
/**
|
|
* Persist the new block (the last block of the given file).
|
|
* Persist the new block (the last block of the given file).
|
|
*/
|
|
*/
|
|
- private static void persistNewBlock(
|
|
|
|
- FSNamesystem fsn, String path, INodeFile file) {
|
|
|
|
- Preconditions.checkArgument(file.isUnderConstruction());
|
|
|
|
- fsn.getEditLog().logAddBlock(path, file);
|
|
|
|
|
|
+ private static FlatINode persistNewBlock(
|
|
|
|
+ RWTransaction tx, String path, FlatINode inode,
|
|
|
|
+ FlatINodeFileFeature.Builder newFile, Block newBlock) {
|
|
|
|
+ Preconditions.checkArgument(newFile.inConstruction());
|
|
|
|
+ newFile.addBlock(newBlock);
|
|
|
|
+ FlatINodeFileFeature newFeature = FlatINodeFileFeature.wrap(newFile.build());
|
|
|
|
+
|
|
|
|
+ FlatINode.Builder builder = new FlatINode.Builder()
|
|
|
|
+ .mergeFrom(inode).replaceFeature(newFeature);
|
|
|
|
+ ByteString newFileBytes = builder.build();
|
|
|
|
+ FlatINode newInode = FlatINode.wrap(newFileBytes);
|
|
|
|
+ tx.putINode(inode.id(), newFileBytes);
|
|
|
|
+ tx.logAddBlock(path, newFeature);
|
|
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
- NameNode.stateChangeLog.debug("persistNewBlock: "
|
|
|
|
- + path + " with new block " + file.getLastBlock().toString()
|
|
|
|
- + ", current total block count is " + file.getBlocks().length);
|
|
|
|
|
|
+ NameNode.stateChangeLog.debug("persistNewBlock: " + path
|
|
|
|
+ + " with new block " + newBlock + ", current total block count is "
|
|
|
|
+ + newFeature.numBlocks());
|
|
}
|
|
}
|
|
|
|
+ return newInode;
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -795,19 +803,18 @@ class FSDirWriteFileOp {
|
|
*
|
|
*
|
|
* @param fsn FSNamesystem
|
|
* @param fsn FSNamesystem
|
|
* @param src path to the file
|
|
* @param src path to the file
|
|
- * @param inodesInPath representing each of the components of src.
|
|
|
|
- * The last INode is the INode for {@code src} file.
|
|
|
|
|
|
+ * @param inode the file
|
|
* @param newBlock newly allocated block to be save
|
|
* @param newBlock newly allocated block to be save
|
|
* @param targets target datanodes where replicas of the new block is placed
|
|
* @param targets target datanodes where replicas of the new block is placed
|
|
* @throws QuotaExceededException If addition of block exceeds space quota
|
|
* @throws QuotaExceededException If addition of block exceeds space quota
|
|
*/
|
|
*/
|
|
private static void saveAllocatedBlock(
|
|
private static void saveAllocatedBlock(
|
|
- FSNamesystem fsn, String src, INodesInPath inodesInPath, Block newBlock,
|
|
|
|
|
|
+ FSNamesystem fsn, String src, FlatINode inode, Block newBlock,
|
|
DatanodeStorageInfo[] targets)
|
|
DatanodeStorageInfo[] targets)
|
|
throws IOException {
|
|
throws IOException {
|
|
assert fsn.hasWriteLock();
|
|
assert fsn.hasWriteLock();
|
|
- BlockInfoContiguous b = addBlock(fsn.dir, src, inodesInPath, newBlock,
|
|
|
|
- targets);
|
|
|
|
|
|
+ BlockManager bm = fsn.getBlockManager();
|
|
|
|
+ BlockInfoContiguous b = addBlock(bm, src, inode, newBlock, targets);
|
|
NameNode.stateChangeLog.info("BLOCK* allocate " + b + " for " + src);
|
|
NameNode.stateChangeLog.info("BLOCK* allocate " + b + " for " + src);
|
|
DatanodeStorageInfo.incrementBlocksScheduled(targets);
|
|
DatanodeStorageInfo.incrementBlocksScheduled(targets);
|
|
}
|
|
}
|
|
@@ -842,14 +849,12 @@ class FSDirWriteFileOp {
|
|
}
|
|
}
|
|
|
|
|
|
private static class FileState {
|
|
private static class FileState {
|
|
- final INodeFile inode;
|
|
|
|
|
|
+ final FlatINode inode;
|
|
final String path;
|
|
final String path;
|
|
- final INodesInPath iip;
|
|
|
|
|
|
|
|
- FileState(INodeFile inode, String fullPath, INodesInPath iip) {
|
|
|
|
|
|
+ FileState(FlatINode inode, String fullPath) {
|
|
this.inode = inode;
|
|
this.inode = inode;
|
|
this.path = fullPath;
|
|
this.path = fullPath;
|
|
- this.iip = iip;
|
|
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|