|
@@ -145,7 +145,6 @@ import org.apache.hadoop.hdfs.DFSUtil;
|
|
|
import org.apache.hadoop.hdfs.HAUtil;
|
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
|
import org.apache.hadoop.hdfs.StorageType;
|
|
|
-import org.apache.hadoop.hdfs.XAttrHelper;
|
|
|
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
|
|
@@ -1567,6 +1566,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
src = FSDirectory.resolvePath(src, pathComponents, dir);
|
|
|
checkOwner(pc, src);
|
|
|
dir.setPermission(src, permission);
|
|
|
+ getEditLog().logSetPermissions(src, permission);
|
|
|
resultingStat = getAuditFileInfo(src, false);
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
@@ -1612,6 +1612,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
}
|
|
|
}
|
|
|
dir.setOwner(src, username, group);
|
|
|
+ getEditLog().logSetOwner(src, username, group);
|
|
|
resultingStat = getAuditFileInfo(src, false);
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
@@ -1742,7 +1743,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
if (isReadOp) {
|
|
|
continue;
|
|
|
}
|
|
|
- dir.setTimes(src, inode, -1, now, false, iip.getLatestSnapshotId());
|
|
|
+ boolean changed = dir.setTimes(inode, -1, now, false,
|
|
|
+ iip.getLatestSnapshotId());
|
|
|
+ if (changed) {
|
|
|
+ getEditLog().logTimes(src, -1, now);
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
final long fileSize = iip.isSnapshot() ?
|
|
@@ -1953,7 +1958,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
Arrays.toString(srcs) + " to " + target);
|
|
|
}
|
|
|
|
|
|
- dir.concat(target,srcs, logRetryCache);
|
|
|
+ long timestamp = now();
|
|
|
+ dir.concat(target, srcs, timestamp);
|
|
|
+ getEditLog().logConcat(target, srcs, timestamp, logRetryCache);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -1994,7 +2001,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
final INodesInPath iip = dir.getINodesInPath4Write(src);
|
|
|
final INode inode = iip.getLastINode();
|
|
|
if (inode != null) {
|
|
|
- dir.setTimes(src, inode, mtime, atime, true, iip.getLatestSnapshotId());
|
|
|
+ boolean changed = dir.setTimes(inode, mtime, atime, true,
|
|
|
+ iip.getLatestSnapshotId());
|
|
|
+ if (changed) {
|
|
|
+ getEditLog().logTimes(src, mtime, atime);
|
|
|
+ }
|
|
|
resultingStat = getAuditFileInfo(src, false);
|
|
|
} else {
|
|
|
throw new FileNotFoundException("File/Directory " + src + " does not exist.");
|
|
@@ -2115,6 +2126,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
final Block[] blocks = dir.setReplication(src, replication, blockRepls);
|
|
|
isFile = blocks != null;
|
|
|
if (isFile) {
|
|
|
+ getEditLog().logSetReplication(src, replication);
|
|
|
blockManager.setReplication(blockRepls[0], blockRepls[1], src, blocks);
|
|
|
}
|
|
|
} finally {
|
|
@@ -2740,7 +2752,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
INodesInPath inodesInPath = INodesInPath.fromINode(pendingFile);
|
|
|
saveAllocatedBlock(src, inodesInPath, newBlock, targets);
|
|
|
|
|
|
- dir.persistNewBlock(src, pendingFile);
|
|
|
+ persistNewBlock(src, pendingFile);
|
|
|
offset = pendingFile.computeFileSize();
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
@@ -2960,7 +2972,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
NameNode.stateChangeLog.debug("BLOCK* NameSystem.abandonBlock: "
|
|
|
+ b + " is removed from pendingCreates");
|
|
|
}
|
|
|
- dir.persistBlocks(src, file, false);
|
|
|
+ persistBlocks(src, file, false);
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
|
}
|
|
@@ -3260,7 +3272,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
false, false);
|
|
|
}
|
|
|
|
|
|
- if (dir.renameTo(src, dst, logRetryCache)) {
|
|
|
+ long mtime = now();
|
|
|
+ if (dir.renameTo(src, dst, mtime)) {
|
|
|
+ getEditLog().logRename(src, dst, mtime, logRetryCache);
|
|
|
return true;
|
|
|
}
|
|
|
return false;
|
|
@@ -3325,7 +3339,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
false);
|
|
|
}
|
|
|
|
|
|
- dir.renameTo(src, dst, logRetryCache, options);
|
|
|
+ long mtime = now();
|
|
|
+ dir.renameTo(src, dst, mtime, options);
|
|
|
+ getEditLog().logRename(src, dst, mtime, logRetryCache, options);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -3408,10 +3424,17 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
checkPermission(pc, src, false, null, FsAction.WRITE, null,
|
|
|
FsAction.ALL, true, false);
|
|
|
}
|
|
|
+ long mtime = now();
|
|
|
// Unlink the target directory from directory tree
|
|
|
- if (!dir.delete(src, collectedBlocks, removedINodes, logRetryCache)) {
|
|
|
+ long filesRemoved = dir.delete(src, collectedBlocks, removedINodes,
|
|
|
+ mtime);
|
|
|
+ if (filesRemoved < 0) {
|
|
|
return false;
|
|
|
}
|
|
|
+ getEditLog().logDelete(src, mtime, logRetryCache);
|
|
|
+ incrDeletedFileCount(filesRemoved);
|
|
|
+ // Blocks/INodes will be handled later
|
|
|
+ removePathAndBlocks(src, null, null);
|
|
|
ret = true;
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
@@ -3721,7 +3744,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
*
|
|
|
* Note: This does not support ".inodes" relative path.
|
|
|
*/
|
|
|
- void setQuota(String path, long nsQuota, long dsQuota)
|
|
|
+ void setQuota(String path, long nsQuota, long dsQuota)
|
|
|
throws IOException, UnresolvedLinkException {
|
|
|
checkSuperuserPrivilege();
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
@@ -3729,7 +3752,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
checkNameNodeSafeMode("Cannot set quota on " + path);
|
|
|
- dir.setQuota(path, nsQuota, dsQuota);
|
|
|
+ INodeDirectory changed = dir.setQuota(path, nsQuota, dsQuota);
|
|
|
+ if (changed != null) {
|
|
|
+ final Quota.Counts q = changed.getQuotaCounts();
|
|
|
+ getEditLog().logSetQuota(path,
|
|
|
+ q.get(Quota.NAMESPACE), q.get(Quota.DISKSPACE));
|
|
|
+ }
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
|
}
|
|
@@ -3770,7 +3798,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
pendingFile.getFileUnderConstructionFeature().updateLengthOfLastBlock(
|
|
|
pendingFile, lastBlockLength);
|
|
|
}
|
|
|
- dir.persistBlocks(src, pendingFile, false);
|
|
|
+ persistBlocks(src, pendingFile, false);
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
|
}
|
|
@@ -3963,7 +3991,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
final INodeFile newFile = pendingFile.toCompleteFile(now());
|
|
|
|
|
|
// close file and persist block allocations for this file
|
|
|
- dir.closeFile(src, newFile);
|
|
|
+ closeFile(src, newFile);
|
|
|
|
|
|
blockManager.checkReplication(newFile);
|
|
|
}
|
|
@@ -4114,7 +4142,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
src = closeFileCommitBlocks(iFile, storedBlock);
|
|
|
} else {
|
|
|
// If this commit does not want to close the file, persist blocks
|
|
|
- src = persistBlocks(iFile, false);
|
|
|
+ src = iFile.getFullPathName();
|
|
|
+ persistBlocks(src, iFile, false);
|
|
|
}
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
@@ -4152,21 +4181,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
return src;
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Persist the block list for the given file.
|
|
|
- *
|
|
|
- * @param pendingFile
|
|
|
- * @return Path to the given file.
|
|
|
- * @throws IOException
|
|
|
- */
|
|
|
- @VisibleForTesting
|
|
|
- String persistBlocks(INodeFile pendingFile, boolean logRetryCache)
|
|
|
- throws IOException {
|
|
|
- String src = pendingFile.getFullPathName();
|
|
|
- dir.persistBlocks(src, pendingFile, logRetryCache);
|
|
|
- return src;
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Renew the lease(s) held by the given client
|
|
|
*/
|
|
@@ -4350,6 +4364,45 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
hasResourcesAvailable = nnResourceChecker.hasAvailableDiskSpace();
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Persist the block list for the inode.
|
|
|
+ * @param path
|
|
|
+ * @param file
|
|
|
+ * @param logRetryCache
|
|
|
+ */
|
|
|
+ private void persistBlocks(String path, INodeFile file,
|
|
|
+ boolean logRetryCache) {
|
|
|
+ assert hasWriteLock();
|
|
|
+ Preconditions.checkArgument(file.isUnderConstruction());
|
|
|
+ getEditLog().logUpdateBlocks(path, file, logRetryCache);
|
|
|
+ if(NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
+ NameNode.stateChangeLog.debug("persistBlocks: " + path
|
|
|
+ + " with " + file.getBlocks().length + " blocks is persisted to" +
|
|
|
+ " the file system");
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ void incrDeletedFileCount(long count) {
|
|
|
+ NameNode.getNameNodeMetrics().incrFilesDeleted(count);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Close file.
|
|
|
+ * @param path
|
|
|
+ * @param file
|
|
|
+ */
|
|
|
+ private void closeFile(String path, INodeFile file) {
|
|
|
+ assert hasWriteLock();
|
|
|
+ dir.waitForReady();
|
|
|
+ // file is closed
|
|
|
+ getEditLog().logCloseFile(path, file);
|
|
|
+ if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
+ NameNode.stateChangeLog.debug("closeFile: "
|
|
|
+ +path+" with "+ file.getBlocks().length
|
|
|
+ +" blocks is persisted to the file system");
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Periodically calls hasAvailableResources of NameNodeResourceChecker, and if
|
|
|
* there are found to be insufficient resources available, causes the NN to
|
|
@@ -4682,6 +4735,21 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
getBlockManager().getDatanodeManager().setBalancerBandwidth(bandwidth);
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Persist the new block (the last block of the given file).
|
|
|
+ * @param path
|
|
|
+ * @param file
|
|
|
+ */
|
|
|
+ private void persistNewBlock(String path, INodeFile file) {
|
|
|
+ Preconditions.checkArgument(file.isUnderConstruction());
|
|
|
+ getEditLog().logAddBlock(path, file);
|
|
|
+ if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
+ NameNode.stateChangeLog.debug("persistNewBlock: "
|
|
|
+ + path + " with new block " + file.getLastBlock().toString()
|
|
|
+ + ", current total block count is " + file.getBlocks().length);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* SafeModeInfo contains information related to the safe mode.
|
|
|
* <p>
|
|
@@ -6090,7 +6158,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
blockinfo.setExpectedLocations(storages);
|
|
|
|
|
|
String src = pendingFile.getFullPathName();
|
|
|
- dir.persistBlocks(src, pendingFile, logRetryCache);
|
|
|
+ persistBlocks(src, pendingFile, logRetryCache);
|
|
|
}
|
|
|
|
|
|
// rename was successful. If any part of the renamed subtree had
|
|
@@ -7718,7 +7786,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
checkNameNodeSafeMode("Cannot modify ACL entries on " + src);
|
|
|
src = FSDirectory.resolvePath(src, pathComponents, dir);
|
|
|
checkOwner(pc, src);
|
|
|
- dir.modifyAclEntries(src, aclSpec);
|
|
|
+ List<AclEntry> newAcl = dir.modifyAclEntries(src, aclSpec);
|
|
|
+ getEditLog().logSetAcl(src, newAcl);
|
|
|
resultingStat = getAuditFileInfo(src, false);
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
@@ -7739,7 +7808,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
checkNameNodeSafeMode("Cannot remove ACL entries on " + src);
|
|
|
src = FSDirectory.resolvePath(src, pathComponents, dir);
|
|
|
checkOwner(pc, src);
|
|
|
- dir.removeAclEntries(src, aclSpec);
|
|
|
+ List<AclEntry> newAcl = dir.removeAclEntries(src, aclSpec);
|
|
|
+ getEditLog().logSetAcl(src, newAcl);
|
|
|
resultingStat = getAuditFileInfo(src, false);
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
@@ -7760,7 +7830,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
checkNameNodeSafeMode("Cannot remove default ACL entries on " + src);
|
|
|
src = FSDirectory.resolvePath(src, pathComponents, dir);
|
|
|
checkOwner(pc, src);
|
|
|
- dir.removeDefaultAcl(src);
|
|
|
+ List<AclEntry> newAcl = dir.removeDefaultAcl(src);
|
|
|
+ getEditLog().logSetAcl(src, newAcl);
|
|
|
resultingStat = getAuditFileInfo(src, false);
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
@@ -7782,6 +7853,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
src = FSDirectory.resolvePath(src, pathComponents, dir);
|
|
|
checkOwner(pc, src);
|
|
|
dir.removeAcl(src);
|
|
|
+ getEditLog().logSetAcl(src, AclFeature.EMPTY_ENTRY_LIST);
|
|
|
resultingStat = getAuditFileInfo(src, false);
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
@@ -7802,7 +7874,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
checkNameNodeSafeMode("Cannot set ACL on " + src);
|
|
|
src = FSDirectory.resolvePath(src, pathComponents, dir);
|
|
|
checkOwner(pc, src);
|
|
|
- dir.setAcl(src, aclSpec);
|
|
|
+ List<AclEntry> newAcl = dir.setAcl(src, aclSpec);
|
|
|
+ getEditLog().logSetAcl(src, newAcl);
|
|
|
resultingStat = getAuditFileInfo(src, false);
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
@@ -7878,7 +7951,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
checkOwner(pc, src);
|
|
|
checkPathAccess(pc, src, FsAction.WRITE);
|
|
|
}
|
|
|
- dir.setXAttr(src, xAttr, flag, logRetryCache);
|
|
|
+ dir.setXAttr(src, xAttr, flag);
|
|
|
+ getEditLog().logSetXAttr(src, xAttr, logRetryCache);
|
|
|
resultingStat = getAuditFileInfo(src, false);
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
@@ -7999,7 +8073,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
checkPathAccess(pc, src, FsAction.WRITE);
|
|
|
}
|
|
|
|
|
|
- dir.removeXAttr(src, xAttr);
|
|
|
+ XAttr removedXAttr = dir.removeXAttr(src, xAttr);
|
|
|
+ if (removedXAttr != null) {
|
|
|
+ getEditLog().logRemoveXAttr(src, removedXAttr);
|
|
|
+ }
|
|
|
resultingStat = getAuditFileInfo(src, false);
|
|
|
} catch (AccessControlException e) {
|
|
|
logAuditEvent(false, "removeXAttr", src);
|