|
@@ -142,7 +142,6 @@ import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
|
|
|
import org.apache.hadoop.fs.CacheFlag;
|
|
|
import org.apache.hadoop.fs.ContentSummary;
|
|
|
import org.apache.hadoop.fs.CreateFlag;
|
|
|
-import org.apache.hadoop.fs.FileAlreadyExistsException;
|
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.FsServerDefaults;
|
|
@@ -185,7 +184,6 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
|
-import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
|
|
|
import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
|
|
|
import org.apache.hadoop.hdfs.protocol.RollingUpgradeException;
|
|
|
import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
|
|
@@ -250,7 +248,6 @@ import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.StorageReport;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
|
|
|
-import org.apache.hadoop.io.EnumSetWritable;
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
|
import org.apache.hadoop.io.Text;
|
|
|
import org.apache.hadoop.ipc.RetriableException;
|
|
@@ -2173,175 +2170,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
return stat;
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Append to an existing file for append.
|
|
|
- * <p>
|
|
|
- *
|
|
|
- * The method returns the last block of the file if this is a partial block,
|
|
|
- * which can still be used for writing more data. The client uses the returned
|
|
|
- * block locations to form the data pipeline for this block.<br>
|
|
|
- * The method returns null if the last block is full. The client then
|
|
|
- * allocates a new block with the next call using
|
|
|
- * {@link ClientProtocol#addBlock}.
|
|
|
- * <p>
|
|
|
- *
|
|
|
- * For description of parameters and exceptions thrown see
|
|
|
- * {@link ClientProtocol#append(String, String, EnumSetWritable)}
|
|
|
- *
|
|
|
- * @return the last block locations if the block is partial or null otherwise
|
|
|
- */
|
|
|
- private LocatedBlock appendFileInternal(FSPermissionChecker pc,
|
|
|
- INodesInPath iip, String holder, String clientMachine, boolean newBlock,
|
|
|
- boolean logRetryCache) throws IOException {
|
|
|
- assert hasWriteLock();
|
|
|
- // Verify that the destination does not exist as a directory already.
|
|
|
- final INode inode = iip.getLastINode();
|
|
|
- final String src = iip.getPath();
|
|
|
- if (inode != null && inode.isDirectory()) {
|
|
|
- throw new FileAlreadyExistsException("Cannot append to directory " + src
|
|
|
- + "; already exists as a directory.");
|
|
|
- }
|
|
|
- if (isPermissionEnabled) {
|
|
|
- dir.checkPathAccess(pc, iip, FsAction.WRITE);
|
|
|
- }
|
|
|
-
|
|
|
- try {
|
|
|
- if (inode == null) {
|
|
|
- throw new FileNotFoundException("failed to append to non-existent file "
|
|
|
- + src + " for client " + clientMachine);
|
|
|
- }
|
|
|
- INodeFile myFile = INodeFile.valueOf(inode, src, true);
|
|
|
- final BlockStoragePolicy lpPolicy =
|
|
|
- blockManager.getStoragePolicy("LAZY_PERSIST");
|
|
|
- if (lpPolicy != null &&
|
|
|
- lpPolicy.getId() == myFile.getStoragePolicyID()) {
|
|
|
- throw new UnsupportedOperationException(
|
|
|
- "Cannot append to lazy persist file " + src);
|
|
|
- }
|
|
|
- // Opening an existing file for append - may need to recover lease.
|
|
|
- recoverLeaseInternal(RecoverLeaseOp.APPEND_FILE, iip, src, holder,
|
|
|
- clientMachine, false);
|
|
|
-
|
|
|
- final BlockInfo lastBlock = myFile.getLastBlock();
|
|
|
- // Check that the block has at least minimum replication.
|
|
|
- if(lastBlock != null && lastBlock.isComplete() &&
|
|
|
- !getBlockManager().isSufficientlyReplicated(lastBlock)) {
|
|
|
- throw new IOException("append: lastBlock=" + lastBlock +
|
|
|
- " of src=" + src + " is not sufficiently replicated yet.");
|
|
|
- }
|
|
|
- return prepareFileForAppend(src, iip, holder, clientMachine, newBlock,
|
|
|
- true, logRetryCache);
|
|
|
- } catch (IOException ie) {
|
|
|
- NameNode.stateChangeLog.warn("DIR* NameSystem.append: " +ie.getMessage());
|
|
|
- throw ie;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Convert current node to under construction.
|
|
|
- * Recreate in-memory lease record.
|
|
|
- *
|
|
|
- * @param src path to the file
|
|
|
- * @param leaseHolder identifier of the lease holder on this file
|
|
|
- * @param clientMachine identifier of the client machine
|
|
|
- * @param newBlock if the data is appended to a new block
|
|
|
- * @param writeToEditLog whether to persist this change to the edit log
|
|
|
- * @param logRetryCache whether to record RPC ids in editlog for retry cache
|
|
|
- * rebuilding
|
|
|
- * @return the last block locations if the block is partial or null otherwise
|
|
|
- * @throws UnresolvedLinkException
|
|
|
- * @throws IOException
|
|
|
- */
|
|
|
- LocatedBlock prepareFileForAppend(String src, INodesInPath iip,
|
|
|
- String leaseHolder, String clientMachine, boolean newBlock,
|
|
|
- boolean writeToEditLog, boolean logRetryCache) throws IOException {
|
|
|
- final INodeFile file = iip.getLastINode().asFile();
|
|
|
- final QuotaCounts delta = verifyQuotaForUCBlock(file, iip);
|
|
|
-
|
|
|
- file.recordModification(iip.getLatestSnapshotId());
|
|
|
- file.toUnderConstruction(leaseHolder, clientMachine);
|
|
|
-
|
|
|
- leaseManager.addLease(
|
|
|
- file.getFileUnderConstructionFeature().getClientName(), file.getId());
|
|
|
-
|
|
|
- LocatedBlock ret = null;
|
|
|
- if (!newBlock) {
|
|
|
- ret = blockManager.convertLastBlockToUnderConstruction(file, 0);
|
|
|
- if (ret != null && delta != null) {
|
|
|
- Preconditions.checkState(delta.getStorageSpace() >= 0,
|
|
|
- "appending to a block with size larger than the preferred block size");
|
|
|
- dir.writeLock();
|
|
|
- try {
|
|
|
- dir.updateCountNoQuotaCheck(iip, iip.length() - 1, delta);
|
|
|
- } finally {
|
|
|
- dir.writeUnlock();
|
|
|
- }
|
|
|
- }
|
|
|
- } else {
|
|
|
- BlockInfo lastBlock = file.getLastBlock();
|
|
|
- if (lastBlock != null) {
|
|
|
- ExtendedBlock blk = new ExtendedBlock(this.getBlockPoolId(), lastBlock);
|
|
|
- ret = new LocatedBlock(blk, new DatanodeInfo[0]);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- if (writeToEditLog) {
|
|
|
- if (NameNodeLayoutVersion.supports(Feature.APPEND_NEW_BLOCK,
|
|
|
- getEffectiveLayoutVersion())) {
|
|
|
- getEditLog().logAppendFile(src, file, newBlock, logRetryCache);
|
|
|
- } else {
|
|
|
- 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 QuotaCounts 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 QuotaCounts delta = computeQuotaDeltaForUCBlock(file);
|
|
|
- dir.readLock();
|
|
|
- try {
|
|
|
- FSDirectory.verifyQuota(iip, iip.length() - 1, delta, null);
|
|
|
- return delta;
|
|
|
- } finally {
|
|
|
- dir.readUnlock();
|
|
|
- }
|
|
|
- }
|
|
|
- return null;
|
|
|
- }
|
|
|
-
|
|
|
- /** Compute quota change for converting a complete block to a UC block */
|
|
|
- private QuotaCounts computeQuotaDeltaForUCBlock(INodeFile file) {
|
|
|
- final QuotaCounts delta = new QuotaCounts.Builder().build();
|
|
|
- final BlockInfo lastBlock = file.getLastBlock();
|
|
|
- if (lastBlock != null) {
|
|
|
- final long diff = file.getPreferredBlockSize() - lastBlock.getNumBytes();
|
|
|
- final short repl = file.getPreferredBlockReplication();
|
|
|
- delta.addStorageSpace(diff * repl);
|
|
|
- final BlockStoragePolicy policy = dir.getBlockStoragePolicySuite()
|
|
|
- .getPolicy(file.getStoragePolicyID());
|
|
|
- List<StorageType> types = policy.chooseStorageTypes(repl);
|
|
|
- for (StorageType t : types) {
|
|
|
- if (t.supportTypeQuota()) {
|
|
|
- delta.addTypeSpace(t, diff);
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
- return delta;
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Recover lease;
|
|
|
* Immediately revoke the lease of the current lease holder and start lease
|
|
@@ -2487,62 +2315,45 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
/**
|
|
|
* Append to an existing file in the namespace.
|
|
|
*/
|
|
|
- LastBlockWithStatus appendFile(String src, String holder,
|
|
|
+ LastBlockWithStatus appendFile(String srcArg, String holder,
|
|
|
String clientMachine, EnumSet<CreateFlag> flag, boolean logRetryCache)
|
|
|
throws IOException {
|
|
|
boolean newBlock = flag.contains(CreateFlag.NEW_BLOCK);
|
|
|
if (newBlock) {
|
|
|
requireEffectiveLayoutVersionForFeature(Feature.APPEND_NEW_BLOCK);
|
|
|
}
|
|
|
- try {
|
|
|
- return appendFileInt(src, holder, clientMachine, newBlock, logRetryCache);
|
|
|
- } catch (AccessControlException e) {
|
|
|
- logAuditEvent(false, "append", src);
|
|
|
- throw e;
|
|
|
- }
|
|
|
- }
|
|
|
|
|
|
- private LastBlockWithStatus appendFileInt(final String srcArg, String holder,
|
|
|
- String clientMachine, boolean newBlock, boolean logRetryCache)
|
|
|
- throws IOException {
|
|
|
- String src = srcArg;
|
|
|
NameNode.stateChangeLog.debug(
|
|
|
"DIR* NameSystem.appendFile: src={}, holder={}, clientMachine={}",
|
|
|
- src, holder, clientMachine);
|
|
|
- boolean skipSync = false;
|
|
|
- LocatedBlock lb = null;
|
|
|
- HdfsFileStatus stat = null;
|
|
|
- FSPermissionChecker pc = getPermissionChecker();
|
|
|
- byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
|
|
|
- writeLock();
|
|
|
+ srcArg, holder, clientMachine);
|
|
|
try {
|
|
|
+ boolean skipSync = false;
|
|
|
+ LastBlockWithStatus lbs = null;
|
|
|
+ final FSPermissionChecker pc = getPermissionChecker();
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
- checkNameNodeSafeMode("Cannot append to file" + src);
|
|
|
- src = dir.resolvePath(pc, src, pathComponents);
|
|
|
- final INodesInPath iip = dir.getINodesInPath4Write(src);
|
|
|
- lb = appendFileInternal(pc, iip, holder, clientMachine, newBlock,
|
|
|
- logRetryCache);
|
|
|
- stat = FSDirStatAndListingOp.getFileInfo(dir, src, false,
|
|
|
- FSDirectory.isReservedRawName(srcArg), true);
|
|
|
- } catch (StandbyException se) {
|
|
|
- skipSync = true;
|
|
|
- throw se;
|
|
|
- } finally {
|
|
|
- writeUnlock();
|
|
|
- // There might be transactions logged while trying to recover the lease.
|
|
|
- // They need to be sync'ed even when an exception was thrown.
|
|
|
- if (!skipSync) {
|
|
|
- getEditLog().logSync();
|
|
|
+ writeLock();
|
|
|
+ try {
|
|
|
+ checkOperation(OperationCategory.WRITE);
|
|
|
+ checkNameNodeSafeMode("Cannot append to file" + srcArg);
|
|
|
+ lbs = FSDirAppendOp.appendFile(this, srcArg, pc, holder, clientMachine,
|
|
|
+ newBlock, logRetryCache);
|
|
|
+ } catch (StandbyException se) {
|
|
|
+ skipSync = true;
|
|
|
+ throw se;
|
|
|
+ } finally {
|
|
|
+ writeUnlock();
|
|
|
+ // There might be transactions logged while trying to recover the lease
|
|
|
+ // They need to be sync'ed even when an exception was thrown.
|
|
|
+ if (!skipSync) {
|
|
|
+ getEditLog().logSync();
|
|
|
+ }
|
|
|
}
|
|
|
+ logAuditEvent(true, "append", srcArg);
|
|
|
+ return lbs;
|
|
|
+ } catch (AccessControlException e) {
|
|
|
+ logAuditEvent(false, "append", srcArg);
|
|
|
+ throw e;
|
|
|
}
|
|
|
- if (lb != null) {
|
|
|
- NameNode.stateChangeLog.debug(
|
|
|
- "DIR* NameSystem.appendFile: file {} for {} at {} block {} block" +
|
|
|
- " size {}", src, holder, clientMachine, lb.getBlock(),
|
|
|
- lb.getBlock().getNumBytes());
|
|
|
- }
|
|
|
- logAuditEvent(true, "append", srcArg);
|
|
|
- return new LastBlockWithStatus(lb, stat);
|
|
|
}
|
|
|
|
|
|
ExtendedBlock getExtendedBlock(Block blk) {
|