|
@@ -258,8 +258,6 @@ import org.apache.hadoop.io.IOUtils;
|
|
import org.apache.hadoop.io.Text;
|
|
import org.apache.hadoop.io.Text;
|
|
import org.apache.hadoop.ipc.RetriableException;
|
|
import org.apache.hadoop.ipc.RetriableException;
|
|
import org.apache.hadoop.ipc.RetryCache;
|
|
import org.apache.hadoop.ipc.RetryCache;
|
|
-import org.apache.hadoop.ipc.RetryCache.CacheEntry;
|
|
|
|
-import org.apache.hadoop.ipc.RetryCache.CacheEntryWithPayload;
|
|
|
|
import org.apache.hadoop.ipc.Server;
|
|
import org.apache.hadoop.ipc.Server;
|
|
import org.apache.hadoop.ipc.StandbyException;
|
|
import org.apache.hadoop.ipc.StandbyException;
|
|
import org.apache.hadoop.metrics2.annotation.Metric;
|
|
import org.apache.hadoop.metrics2.annotation.Metric;
|
|
@@ -342,8 +340,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
|
|
|
private HdfsFileStatus getAuditFileInfo(String path, boolean resolveSymlink)
|
|
private HdfsFileStatus getAuditFileInfo(String path, boolean resolveSymlink)
|
|
throws IOException {
|
|
throws IOException {
|
|
- return (isAuditEnabled() && isExternalInvocation())
|
|
|
|
- ? dir.getFileInfo(path, resolveSymlink, false, false) : null;
|
|
|
|
|
|
+ return dir.getAuditFileInfo(path, resolveSymlink);
|
|
}
|
|
}
|
|
|
|
|
|
private void logAuditEvent(boolean succeeded, String cmd, String src)
|
|
private void logAuditEvent(boolean succeeded, String cmd, String src)
|
|
@@ -1944,175 +1941,26 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
* @throws IOException on error
|
|
* @throws IOException on error
|
|
*/
|
|
*/
|
|
void concat(String target, String [] srcs, boolean logRetryCache)
|
|
void concat(String target, String [] srcs, boolean logRetryCache)
|
|
- throws IOException, UnresolvedLinkException {
|
|
|
|
- if(FSNamesystem.LOG.isDebugEnabled()) {
|
|
|
|
- FSNamesystem.LOG.debug("concat " + Arrays.toString(srcs) +
|
|
|
|
- " to " + target);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- try {
|
|
|
|
- concatInt(target, srcs, logRetryCache);
|
|
|
|
- } catch (AccessControlException e) {
|
|
|
|
- logAuditEvent(false, "concat", Arrays.toString(srcs), target, null);
|
|
|
|
- throw e;
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private void concatInt(String target, String [] srcs,
|
|
|
|
- boolean logRetryCache) throws IOException, UnresolvedLinkException {
|
|
|
|
- // verify args
|
|
|
|
- if(target.isEmpty()) {
|
|
|
|
- throw new IllegalArgumentException("Target file name is empty");
|
|
|
|
- }
|
|
|
|
- if(srcs == null || srcs.length == 0) {
|
|
|
|
- throw new IllegalArgumentException("No sources given");
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- // We require all files be in the same directory
|
|
|
|
- String trgParent =
|
|
|
|
- target.substring(0, target.lastIndexOf(Path.SEPARATOR_CHAR));
|
|
|
|
- for (String s : srcs) {
|
|
|
|
- String srcParent = s.substring(0, s.lastIndexOf(Path.SEPARATOR_CHAR));
|
|
|
|
- if (!srcParent.equals(trgParent)) {
|
|
|
|
- throw new IllegalArgumentException(
|
|
|
|
- "Sources and target are not in the same directory");
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- HdfsFileStatus resultingStat = null;
|
|
|
|
- FSPermissionChecker pc = getPermissionChecker();
|
|
|
|
|
|
+ throws IOException {
|
|
checkOperation(OperationCategory.WRITE);
|
|
checkOperation(OperationCategory.WRITE);
|
|
waitForLoadingFSImage();
|
|
waitForLoadingFSImage();
|
|
|
|
+ HdfsFileStatus stat = null;
|
|
|
|
+ boolean success = false;
|
|
writeLock();
|
|
writeLock();
|
|
try {
|
|
try {
|
|
checkOperation(OperationCategory.WRITE);
|
|
checkOperation(OperationCategory.WRITE);
|
|
checkNameNodeSafeMode("Cannot concat " + target);
|
|
checkNameNodeSafeMode("Cannot concat " + target);
|
|
- concatInternal(pc, target, srcs, logRetryCache);
|
|
|
|
- resultingStat = getAuditFileInfo(target, false);
|
|
|
|
|
|
+ stat = FSDirConcatOp.concat(dir, target, srcs, logRetryCache);
|
|
|
|
+ success = true;
|
|
} finally {
|
|
} finally {
|
|
writeUnlock();
|
|
writeUnlock();
|
|
- }
|
|
|
|
- getEditLog().logSync();
|
|
|
|
- logAuditEvent(true, "concat", Arrays.toString(srcs), target, resultingStat);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /** See {@link #concat(String, String[])} */
|
|
|
|
- private void concatInternal(FSPermissionChecker pc, String target,
|
|
|
|
- String[] srcs, boolean logRetryCache) throws IOException,
|
|
|
|
- UnresolvedLinkException {
|
|
|
|
- assert hasWriteLock();
|
|
|
|
-
|
|
|
|
- // write permission for the target
|
|
|
|
- if (isPermissionEnabled) {
|
|
|
|
- checkPathAccess(pc, target, FsAction.WRITE);
|
|
|
|
-
|
|
|
|
- // and srcs
|
|
|
|
- for(String aSrc: srcs) {
|
|
|
|
- checkPathAccess(pc, aSrc, FsAction.READ); // read the file
|
|
|
|
- checkParentAccess(pc, aSrc, FsAction.WRITE); // for delete
|
|
|
|
|
|
+ if (success) {
|
|
|
|
+ getEditLog().logSync();
|
|
}
|
|
}
|
|
|
|
+ logAuditEvent(success, "concat", Arrays.toString(srcs), target, stat);
|
|
}
|
|
}
|
|
-
|
|
|
|
- // to make sure no two files are the same
|
|
|
|
- Set<INode> si = new HashSet<INode>();
|
|
|
|
-
|
|
|
|
- // we put the following prerequisite for the operation
|
|
|
|
- // replication and blocks sizes should be the same for ALL the blocks
|
|
|
|
-
|
|
|
|
- // check the target
|
|
|
|
- final INodesInPath trgIip = dir.getINodesInPath4Write(target);
|
|
|
|
- if (dir.getEZForPath(trgIip) != null) {
|
|
|
|
- throw new HadoopIllegalArgumentException(
|
|
|
|
- "concat can not be called for files in an encryption zone.");
|
|
|
|
- }
|
|
|
|
- final INodeFile trgInode = INodeFile.valueOf(trgIip.getLastINode(),
|
|
|
|
- target);
|
|
|
|
- if(trgInode.isUnderConstruction()) {
|
|
|
|
- throw new HadoopIllegalArgumentException("concat: target file "
|
|
|
|
- + target + " is under construction");
|
|
|
|
- }
|
|
|
|
- // per design target shouldn't be empty and all the blocks same size
|
|
|
|
- if(trgInode.numBlocks() == 0) {
|
|
|
|
- throw new HadoopIllegalArgumentException("concat: target file "
|
|
|
|
- + target + " is empty");
|
|
|
|
- }
|
|
|
|
- if (trgInode.isWithSnapshot()) {
|
|
|
|
- throw new HadoopIllegalArgumentException("concat: target file "
|
|
|
|
- + target + " is in a snapshot");
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- long blockSize = trgInode.getPreferredBlockSize();
|
|
|
|
-
|
|
|
|
- // check the end block to be full
|
|
|
|
- final BlockInfo last = trgInode.getLastBlock();
|
|
|
|
- if(blockSize != last.getNumBytes()) {
|
|
|
|
- throw new HadoopIllegalArgumentException("The last block in " + target
|
|
|
|
- + " is not full; last block size = " + last.getNumBytes()
|
|
|
|
- + " but file block size = " + blockSize);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- si.add(trgInode);
|
|
|
|
- final short repl = trgInode.getFileReplication();
|
|
|
|
-
|
|
|
|
- // now check the srcs
|
|
|
|
- boolean endSrc = false; // final src file doesn't have to have full end block
|
|
|
|
- for(int i=0; i<srcs.length; i++) {
|
|
|
|
- String src = srcs[i];
|
|
|
|
- if(i==srcs.length-1)
|
|
|
|
- endSrc=true;
|
|
|
|
-
|
|
|
|
- final INodeFile srcInode = INodeFile.valueOf(dir.getINode4Write(src), src);
|
|
|
|
- if(src.isEmpty()
|
|
|
|
- || srcInode.isUnderConstruction()
|
|
|
|
- || srcInode.numBlocks() == 0) {
|
|
|
|
- throw new HadoopIllegalArgumentException("concat: source file " + src
|
|
|
|
- + " is invalid or empty or underConstruction");
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- // check replication and blocks size
|
|
|
|
- if(repl != srcInode.getBlockReplication()) {
|
|
|
|
- throw new HadoopIllegalArgumentException("concat: the source file "
|
|
|
|
- + src + " and the target file " + target
|
|
|
|
- + " should have the same replication: source replication is "
|
|
|
|
- + srcInode.getBlockReplication()
|
|
|
|
- + " but target replication is " + repl);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- //boolean endBlock=false;
|
|
|
|
- // verify that all the blocks are of the same length as target
|
|
|
|
- // should be enough to check the end blocks
|
|
|
|
- final BlockInfo[] srcBlocks = srcInode.getBlocks();
|
|
|
|
- int idx = srcBlocks.length-1;
|
|
|
|
- if(endSrc)
|
|
|
|
- idx = srcBlocks.length-2; // end block of endSrc is OK not to be full
|
|
|
|
- if(idx >= 0 && srcBlocks[idx].getNumBytes() != blockSize) {
|
|
|
|
- throw new HadoopIllegalArgumentException("concat: the source file "
|
|
|
|
- + src + " and the target file " + target
|
|
|
|
- + " should have the same blocks sizes: target block size is "
|
|
|
|
- + blockSize + " but the size of source block " + idx + " is "
|
|
|
|
- + srcBlocks[idx].getNumBytes());
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- si.add(srcInode);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- // make sure no two files are the same
|
|
|
|
- if(si.size() < srcs.length+1) { // trg + srcs
|
|
|
|
- // it means at least two files are the same
|
|
|
|
- throw new HadoopIllegalArgumentException(
|
|
|
|
- "concat: at least two of the source files are the same");
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- if(NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
|
- NameNode.stateChangeLog.debug("DIR* NameSystem.concat: " +
|
|
|
|
- Arrays.toString(srcs) + " to " + target);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- long timestamp = now();
|
|
|
|
- dir.concat(target, srcs, timestamp);
|
|
|
|
- getEditLog().logConcat(target, srcs, timestamp, logRetryCache);
|
|
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* stores the modification and access time for this inode.
|
|
* stores the modification and access time for this inode.
|
|
* The access time is precise up to an hour. The transaction, if needed, is
|
|
* The access time is precise up to an hour. The transaction, if needed, is
|
|
@@ -7240,7 +7088,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
* Client invoked methods are invoked over RPC and will be in
|
|
* Client invoked methods are invoked over RPC and will be in
|
|
* RPC call context even if the client exits.
|
|
* RPC call context even if the client exits.
|
|
*/
|
|
*/
|
|
- private boolean isExternalInvocation() {
|
|
|
|
|
|
+ boolean isExternalInvocation() {
|
|
return Server.isRpcInvocation() || NamenodeWebHdfsMethods.isWebHdfsInvocation();
|
|
return Server.isRpcInvocation() || NamenodeWebHdfsMethods.isWebHdfsInvocation();
|
|
}
|
|
}
|
|
|
|
|