|
@@ -2872,8 +2872,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
throw new IOException("append: lastBlock=" + lastBlock +
|
|
|
" of src=" + src + " is not sufficiently replicated yet.");
|
|
|
}
|
|
|
- return prepareFileForWrite(src, myFile, holder, clientMachine, true,
|
|
|
- iip.getLatestSnapshotId(), logRetryCache);
|
|
|
+ return prepareFileForWrite(src, iip, holder, clientMachine, true,
|
|
|
+ logRetryCache);
|
|
|
} catch (IOException ie) {
|
|
|
NameNode.stateChangeLog.warn("DIR* NameSystem.append: " +ie.getMessage());
|
|
|
throw ie;
|
|
@@ -2895,30 +2895,76 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
* @throws UnresolvedLinkException
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- LocatedBlock prepareFileForWrite(String src, INodeFile file,
|
|
|
- String leaseHolder, String clientMachine,
|
|
|
- boolean writeToEditLog,
|
|
|
- int latestSnapshot, boolean logRetryCache)
|
|
|
- throws IOException {
|
|
|
- file.recordModification(latestSnapshot);
|
|
|
- final INodeFile cons = file.toUnderConstruction(leaseHolder, clientMachine);
|
|
|
+ LocatedBlock prepareFileForWrite(String src, INodesInPath iip,
|
|
|
+ String leaseHolder, String clientMachine, boolean writeToEditLog,
|
|
|
+ boolean logRetryCache) throws IOException {
|
|
|
+ final INodeFile file = iip.getLastINode().asFile();
|
|
|
+ final Quota.Counts delta = verifyQuotaForUCBlock(file, iip);
|
|
|
|
|
|
- leaseManager.addLease(cons.getFileUnderConstructionFeature()
|
|
|
- .getClientName(), src);
|
|
|
-
|
|
|
- LocatedBlock ret = blockManager.convertLastBlockToUnderConstruction(cons);
|
|
|
- if (ret != null) {
|
|
|
- // update the quota: use the preferred block size for UC block
|
|
|
- final long diff = file.getPreferredBlockSize() - ret.getBlockSize();
|
|
|
- dir.updateSpaceConsumed(src, 0, diff * file.getBlockReplication());
|
|
|
+ file.recordModification(iip.getLatestSnapshotId());
|
|
|
+ file.toUnderConstruction(leaseHolder, clientMachine);
|
|
|
+
|
|
|
+ leaseManager.addLease(
|
|
|
+ file.getFileUnderConstructionFeature().getClientName(), src);
|
|
|
+
|
|
|
+ LocatedBlock ret = blockManager.convertLastBlockToUnderConstruction(file);
|
|
|
+ if (ret != null && delta != null) {
|
|
|
+ Preconditions.checkState(delta.get(Quota.DISKSPACE) >= 0,
|
|
|
+ "appending to a block with size larger than the preferred block size");
|
|
|
+ dir.writeLock();
|
|
|
+ try {
|
|
|
+ dir.updateCountNoQuotaCheck(iip, iip.length() - 1,
|
|
|
+ delta.get(Quota.NAMESPACE), delta.get(Quota.DISKSPACE));
|
|
|
+ } finally {
|
|
|
+ dir.writeUnlock();
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
if (writeToEditLog) {
|
|
|
- getEditLog().logOpenFile(src, cons, false, logRetryCache);
|
|
|
+ getEditLog().logOpenFile(src, file, false, logRetryCache);
|
|
|
}
|
|
|
return ret;
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Verify quota when using the preferred block size for UC block. This is
|
|
|
+ * usually used by append and truncate
|
|
|
+ * @throws QuotaExceededException when violating the storage quota
|
|
|
+ * @return expected quota usage update. null means no change or no need to
|
|
|
+ * update quota usage later
|
|
|
+ */
|
|
|
+ private Quota.Counts verifyQuotaForUCBlock(INodeFile file, INodesInPath iip)
|
|
|
+ throws QuotaExceededException {
|
|
|
+ if (!isImageLoaded() || dir.shouldSkipQuotaChecks()) {
|
|
|
+ // Do not check quota if editlog is still being processed
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ if (file.getLastBlock() != null) {
|
|
|
+ final Quota.Counts delta = computeQuotaDeltaForUCBlock(file);
|
|
|
+ dir.readLock();
|
|
|
+ try {
|
|
|
+ FSDirectory.verifyQuota(iip.getINodes(), iip.length() - 1,
|
|
|
+ delta.get(Quota.NAMESPACE), delta.get(Quota.DISKSPACE), null);
|
|
|
+ return delta;
|
|
|
+ } finally {
|
|
|
+ dir.readUnlock();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+
|
|
|
+ /** Compute quota change for converting a complete block to a UC block */
|
|
|
+ private Quota.Counts computeQuotaDeltaForUCBlock(INodeFile file) {
|
|
|
+ final BlockInfo lastBlock = file.getLastBlock();
|
|
|
+ if (lastBlock != null) {
|
|
|
+ final long diff = file.getPreferredBlockSize() - lastBlock.getNumBytes();
|
|
|
+ final short repl = file.getBlockReplication();
|
|
|
+ return Quota.Counts.newInstance(0, diff * repl);
|
|
|
+ } else {
|
|
|
+ return Quota.Counts.newInstance();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Recover lease;
|
|
|
* Immediately revoke the lease of the current lease holder and start lease
|
|
@@ -3318,7 +3364,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
// doesn't match up with what we think is the last block. There are
|
|
|
// four possibilities:
|
|
|
// 1) This is the first block allocation of an append() pipeline
|
|
|
- // which started appending exactly at a block boundary.
|
|
|
+ // which started appending exactly at or exceeding the block boundary.
|
|
|
// In this case, the client isn't passed the previous block,
|
|
|
// so it makes the allocateBlock() call with previous=null.
|
|
|
// We can distinguish this since the last block of the file
|
|
@@ -3343,7 +3389,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
BlockInfo penultimateBlock = pendingFile.getPenultimateBlock();
|
|
|
if (previous == null &&
|
|
|
lastBlockInFile != null &&
|
|
|
- lastBlockInFile.getNumBytes() == pendingFile.getPreferredBlockSize() &&
|
|
|
+ lastBlockInFile.getNumBytes() >= pendingFile.getPreferredBlockSize() &&
|
|
|
lastBlockInFile.isComplete()) {
|
|
|
// Case 1
|
|
|
if (NameNode.stateChangeLog.isDebugEnabled()) {
|