Skip to content

Commit 4b58062

Browse files
committed
HDFS-10843. Update space quota when a UC block is completed rather than committed. Contributed by Erik Krogen.
(cherry picked from commit a5bb88c)
1 parent 2a1e48b commit 4b58062

File tree

6 files changed

+289
-91
lines changed

6 files changed

+289
-91
lines changed

hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

Lines changed: 32 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -68,6 +68,7 @@
6868
import org.apache.hadoop.hdfs.server.blockmanagement.PendingDataNodeMessages.ReportedBlockInfo;
6969
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
7070
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
71+
import org.apache.hadoop.hdfs.server.namenode.INodesInPath;
7172
import org.apache.hadoop.hdfs.server.namenode.NameNode;
7273
import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
7374
import org.apache.hadoop.hdfs.server.namenode.Namesystem;
@@ -622,12 +623,13 @@ private static boolean commitBlock(
622623
*
623624
* @param bc block collection
624625
* @param commitBlock - contains client reported block length and generation
626+
* @param iip - INodes in path to bc
625627
* @return true if the last block is changed to committed state.
626628
* @throws IOException if the block does not have at least a minimal number
627629
* of replicas reported from data-nodes.
628630
*/
629631
public boolean commitOrCompleteLastBlock(BlockCollection bc,
630-
Block commitBlock) throws IOException {
632+
Block commitBlock, INodesInPath iip) throws IOException {
631633
if(commitBlock == null)
632634
return false; // not committing, this is a block allocation retry
633635
BlockInfoContiguous lastBlock = bc.getLastBlock();
@@ -639,19 +641,21 @@ public boolean commitOrCompleteLastBlock(BlockCollection bc,
639641
final boolean b = commitBlock(
640642
(BlockInfoContiguousUnderConstruction) lastBlock, commitBlock);
641643
if(countNodes(lastBlock).liveReplicas() >= minReplication)
642-
completeBlock(bc, bc.numBlocks()-1, false);
644+
completeBlock(bc, bc.numBlocks()-1, iip, false);
643645
return b;
644646
}
645647

646648
/**
647649
* Convert a specified block of the file to a complete block.
648650
* @param bc file
649651
* @param blkIndex block index in the file
652+
* @param iip - INodes in path to file containing curBlock; if null,
653+
* this will be resolved internally
650654
* @throws IOException if the block does not have at least a minimal number
651655
* of replicas reported from data-nodes.
652656
*/
653657
private BlockInfoContiguous completeBlock(final BlockCollection bc,
654-
final int blkIndex, boolean force) throws IOException {
658+
final int blkIndex, INodesInPath iip, boolean force) throws IOException {
655659
if(blkIndex < 0)
656660
return null;
657661
BlockInfoContiguous curBlock = bc.getBlocks()[blkIndex];
@@ -666,7 +670,7 @@ private BlockInfoContiguous completeBlock(final BlockCollection bc,
666670
if(!force && ucBlock.getBlockUCState() != BlockUCState.COMMITTED)
667671
throw new IOException(
668672
"Cannot complete block: block has not been COMMITTED by the client");
669-
BlockInfoContiguous completeBlock = ucBlock.convertToCompleteBlock();
673+
BlockInfoContiguous completeBlock = convertToCompleteBlock(ucBlock, iip);
670674
// replace penultimate block in file
671675
bc.setBlock(blkIndex, completeBlock);
672676

@@ -685,15 +689,34 @@ private BlockInfoContiguous completeBlock(final BlockCollection bc,
685689
}
686690

687691
private BlockInfoContiguous completeBlock(final BlockCollection bc,
688-
final BlockInfoContiguous block, boolean force) throws IOException {
692+
final BlockInfoContiguous block, INodesInPath iip, boolean force)
693+
throws IOException {
689694
BlockInfoContiguous[] fileBlocks = bc.getBlocks();
690695
for(int idx = 0; idx < fileBlocks.length; idx++)
691696
if(fileBlocks[idx] == block) {
692-
return completeBlock(bc, idx, force);
697+
return completeBlock(bc, idx, iip, force);
693698
}
694699
return block;
695700
}
696701

702+
/**
703+
* Convert a specified block of the file to a complete block.
704+
* Skips validity checking and safe mode block total updates; use
705+
* {@link BlockManager#completeBlock} to include these.
706+
* @param curBlock - block to be completed
707+
* @param iip - INodes in path to file containing curBlock; if null,
708+
* this will be resolved internally
709+
* @throws IOException if the block does not have at least a minimal number
710+
* of replicas reported from data-nodes.
711+
*/
712+
private BlockInfoContiguous convertToCompleteBlock(
713+
BlockInfoContiguousUnderConstruction curBlock, INodesInPath iip)
714+
throws IOException {
715+
BlockInfoContiguous complete = curBlock.convertToCompleteBlock();
716+
namesystem.getFSDirectory().updateSpaceForCompleteBlock(curBlock, iip);
717+
return complete;
718+
}
719+
697720
/**
698721
* Force the given block in the given file to be marked as complete,
699722
* regardless of whether enough replicas are present. This is necessary
@@ -702,7 +725,7 @@ private BlockInfoContiguous completeBlock(final BlockCollection bc,
702725
public BlockInfoContiguous forceCompleteBlock(final BlockCollection bc,
703726
final BlockInfoContiguousUnderConstruction block) throws IOException {
704727
block.commitBlock(block);
705-
return completeBlock(bc, block, true);
728+
return completeBlock(bc, block, null, true);
706729
}
707730

708731

@@ -2525,7 +2548,7 @@ private void addStoredBlockImmediate(BlockInfoContiguous storedBlock,
25252548
int numCurrentReplica = countLiveNodes(storedBlock);
25262549
if (storedBlock.getBlockUCState() == BlockUCState.COMMITTED
25272550
&& numCurrentReplica >= minReplication) {
2528-
completeBlock(storedBlock.getBlockCollection(), storedBlock, false);
2551+
completeBlock(storedBlock.getBlockCollection(), storedBlock, null, false);
25292552
} else if (storedBlock.isComplete() && result == AddBlockResult.ADDED) {
25302553
// check whether safe replication is reached for the block
25312554
// only complete blocks are counted towards that.
@@ -2599,7 +2622,7 @@ private Block addStoredBlock(final BlockInfoContiguous block,
25992622

26002623
if(storedBlock.getBlockUCState() == BlockUCState.COMMITTED &&
26012624
numLiveReplicas >= minReplication) {
2602-
storedBlock = completeBlock(bc, storedBlock, false);
2625+
storedBlock = completeBlock(bc, storedBlock, null, false);
26032626
} else if (storedBlock.isComplete() && result == AddBlockResult.ADDED) {
26042627
// check whether safe replication is reached for the block
26052628
// only complete blocks are counted towards that

hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java

Lines changed: 3 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -382,22 +382,21 @@ static Block[] unprotectedSetReplication(
382382
}
383383
INodeFile file = inode.asFile();
384384
final short oldBR = file.getBlockReplication();
385+
long size = file.computeFileSize(true, true);
385386

386387
// before setFileReplication, check for increasing block replication.
387388
// if replication > oldBR, then newBR == replication.
388389
// if replication < oldBR, we don't know newBR yet.
389390
if (replication > oldBR) {
390-
long dsDelta = file.storagespaceConsumed()/oldBR;
391-
fsd.updateCount(iip, 0L, dsDelta, oldBR, replication, true);
391+
fsd.updateCount(iip, 0L, size, oldBR, replication, true);
392392
}
393393

394394
file.setFileReplication(replication, iip.getLatestSnapshotId());
395395

396396
final short newBR = file.getBlockReplication();
397397
// check newBR < oldBR case.
398398
if (newBR < oldBR) {
399-
long dsDelta = file.storagespaceConsumed()/newBR;
400-
fsd.updateCount(iip, 0L, dsDelta, oldBR, newBR, true);
399+
fsd.updateCount(iip, 0L, size, oldBR, newBR, true);
401400
}
402401

403402
if (blockRepls != null) {

hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

Lines changed: 25 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -737,6 +737,31 @@ static void unprotectedUpdateCount(INodesInPath inodesInPath,
737737
}
738738
}
739739

740+
/**
741+
* Update the cached quota space for a block that is being completed.
742+
* Must only be called once, as the block is being completed.
743+
* @param completeBlk - Completed block for which to update space
744+
* @param inodes - INodes in path to file containing completeBlk; if null
745+
* this will be resolved internally
746+
*/
747+
public void updateSpaceForCompleteBlock(BlockInfoContiguous completeBlk,
748+
INodesInPath inodes) throws IOException {
749+
assert namesystem.hasWriteLock();
750+
INodesInPath iip = inodes != null ? inodes :
751+
INodesInPath.fromINode((INodeFile) completeBlk.getBlockCollection());
752+
INodeFile fileINode = iip.getLastINode().asFile();
753+
// Adjust disk space consumption if required
754+
final long diff =
755+
fileINode.getPreferredBlockSize() - completeBlk.getNumBytes();
756+
if (diff > 0) {
757+
try {
758+
updateSpaceConsumed(iip, 0, -diff, fileINode.getFileReplication());
759+
} catch (IOException e) {
760+
LOG.warn("Unexpected exception while updating disk space.", e);
761+
}
762+
}
763+
}
764+
740765
public EnumCounters<StorageType> getStorageTypeDeltas(byte storagePolicyID,
741766
long dsDelta, short oldRep, short newRep) {
742767
EnumCounters<StorageType> typeSpaceDeltas =

hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

Lines changed: 2 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -4219,19 +4219,7 @@ private void commitOrCompleteLastBlock(final INodeFile fileINode,
42194219
final INodesInPath iip, final Block commitBlock) throws IOException {
42204220
assert hasWriteLock();
42214221
Preconditions.checkArgument(fileINode.isUnderConstruction());
4222-
if (!blockManager.commitOrCompleteLastBlock(fileINode, commitBlock)) {
4223-
return;
4224-
}
4225-
4226-
// Adjust disk space consumption if required
4227-
final long diff = fileINode.getPreferredBlockSize() - commitBlock.getNumBytes();
4228-
if (diff > 0) {
4229-
try {
4230-
dir.updateSpaceConsumed(iip, 0, -diff, fileINode.getFileReplication());
4231-
} catch (IOException e) {
4232-
LOG.warn("Unexpected exception while updating disk space.", e);
4233-
}
4234-
}
4222+
blockManager.commitOrCompleteLastBlock(fileINode, commitBlock, iip);
42354223
}
42364224

42374225
private void finalizeINodeFileUnderConstruction(String src,
@@ -4479,7 +4467,6 @@ void commitBlockSynchronization(ExtendedBlock oldBlock,
44794467
/**
44804468
* @param pendingFile open file that needs to be closed
44814469
* @param storedBlock last block
4482-
* @return Path of the file that was closed.
44834470
* @throws IOException on error
44844471
*/
44854472
@VisibleForTesting
@@ -7261,6 +7248,7 @@ public BlockIdManager getBlockIdManager() {
72617248
}
72627249

72637250
/** @return the FSDirectory. */
7251+
@Override
72647252
public FSDirectory getFSDirectory() {
72657253
return dir;
72667254
}

hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -37,6 +37,8 @@ public interface Namesystem extends RwLock, SafeMode {
3737
/** @return the block pool ID */
3838
public String getBlockPoolId();
3939

40+
public FSDirectory getFSDirectory();
41+
4042
public boolean isInStandbyState();
4143

4244
public boolean isGenStampInFuture(Block block);

0 commit comments

Comments
 (0)