chosen;
+ checkOperation(OperationCategory.READ);
readLock();
try {
- checkOperation(OperationCategory.WRITE);
+ checkOperation(OperationCategory.READ);
//check safe mode
if (isInSafeMode()) {
throw new SafeModeException("Cannot add datanode; src=" + src
@@ -2462,6 +2434,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
boolean abandonBlock(ExtendedBlock b, String src, String holder)
throws LeaseExpiredException, FileNotFoundException,
UnresolvedLinkException, IOException {
+ checkOperation(OperationCategory.WRITE);
writeLock();
try {
checkOperation(OperationCategory.WRITE);
@@ -2539,6 +2512,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
throws SafeModeException, UnresolvedLinkException, IOException {
checkBlock(last);
boolean success = false;
+ checkOperation(OperationCategory.WRITE);
writeLock();
try {
checkOperation(OperationCategory.WRITE);
@@ -2691,11 +2665,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
try {
return renameToInt(src, dst);
} catch (AccessControlException e) {
- if (isAuditEnabled() && isExternalInvocation()) {
- logAuditEvent(false, UserGroupInformation.getCurrentUser(),
- getRemoteIp(),
- "rename", src, dst, null);
- }
+ logAuditEvent(false, "rename", src, dst, null);
throw e;
}
}
@@ -2708,29 +2678,29 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
NameNode.stateChangeLog.debug("DIR* NameSystem.renameTo: " + src +
" to " + dst);
}
+ FSPermissionChecker pc = getPermissionChecker();
+ checkOperation(OperationCategory.WRITE);
writeLock();
try {
checkOperation(OperationCategory.WRITE);
- status = renameToInternal(src, dst);
- if (status && isAuditEnabled() && isExternalInvocation()) {
- resultingStat = dir.getFileInfo(dst, false);
+ status = renameToInternal(pc, src, dst);
+ if (status) {
+ resultingStat = getAuditFileInfo(dst, false);
}
} finally {
writeUnlock();
}
getEditLog().logSync();
- if (status && isAuditEnabled() && isExternalInvocation()) {
- logAuditEvent(UserGroupInformation.getCurrentUser(),
- getRemoteIp(),
- "rename", src, dst, resultingStat);
+ if (status) {
+ logAuditEvent(true, "rename", src, dst, resultingStat);
}
return status;
}
/** @deprecated See {@link #renameTo(String, String)} */
@Deprecated
- private boolean renameToInternal(String src, String dst)
+ private boolean renameToInternal(FSPermissionChecker pc, String src, String dst)
throws IOException, UnresolvedLinkException {
assert hasWriteLock();
if (isInSafeMode()) {
@@ -2746,8 +2716,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
// of rewriting the dst
String actualdst = dir.isDir(dst)?
dst + Path.SEPARATOR + new Path(src).getName(): dst;
- checkParentAccess(src, FsAction.WRITE);
- checkAncestorAccess(actualdst, FsAction.WRITE);
+ checkParentAccess(pc, src, FsAction.WRITE);
+ checkAncestorAccess(pc, actualdst, FsAction.WRITE);
}
if (dir.renameTo(src, dst)) {
@@ -2765,29 +2735,27 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
NameNode.stateChangeLog.debug("DIR* NameSystem.renameTo: with options - "
+ src + " to " + dst);
}
+ FSPermissionChecker pc = getPermissionChecker();
+ checkOperation(OperationCategory.WRITE);
writeLock();
try {
checkOperation(OperationCategory.WRITE);
-
- renameToInternal(src, dst, options);
- if (isAuditEnabled() && isExternalInvocation()) {
- resultingStat = dir.getFileInfo(dst, false);
- }
+ renameToInternal(pc, src, dst, options);
+ resultingStat = getAuditFileInfo(dst, false);
} finally {
writeUnlock();
}
getEditLog().logSync();
- if (isAuditEnabled() && isExternalInvocation()) {
+ if (resultingStat != null) {
StringBuilder cmd = new StringBuilder("rename options=");
for (Rename option : options) {
cmd.append(option.value()).append(" ");
}
- logAuditEvent(UserGroupInformation.getCurrentUser(), getRemoteIp(),
- cmd.toString(), src, dst, resultingStat);
+ logAuditEvent(true, cmd.toString(), src, dst, resultingStat);
}
}
- private void renameToInternal(String src, String dst,
+ private void renameToInternal(FSPermissionChecker pc, String src, String dst,
Options.Rename... options) throws IOException {
assert hasWriteLock();
if (isInSafeMode()) {
@@ -2797,8 +2765,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
throw new InvalidPathException("Invalid name: " + dst);
}
if (isPermissionEnabled) {
- checkParentAccess(src, FsAction.WRITE);
- checkAncestorAccess(dst, FsAction.WRITE);
+ checkParentAccess(pc, src, FsAction.WRITE);
+ checkAncestorAccess(pc, dst, FsAction.WRITE);
}
dir.renameTo(src, dst, options);
@@ -2816,11 +2784,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
try {
return deleteInt(src, recursive);
} catch (AccessControlException e) {
- if (isAuditEnabled() && isExternalInvocation()) {
- logAuditEvent(false, UserGroupInformation.getCurrentUser(),
- getRemoteIp(),
- "delete", src, null, null);
- }
+ logAuditEvent(false, "delete", src);
throw e;
}
}
@@ -2832,14 +2796,16 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
NameNode.stateChangeLog.debug("DIR* NameSystem.delete: " + src);
}
boolean status = deleteInternal(src, recursive, true);
- if (status && isAuditEnabled() && isExternalInvocation()) {
- logAuditEvent(UserGroupInformation.getCurrentUser(),
- getRemoteIp(),
- "delete", src, null, null);
+ if (status) {
+ logAuditEvent(true, "delete", src);
}
return status;
}
+ private FSPermissionChecker getPermissionChecker()
+ throws AccessControlException {
+ return new FSPermissionChecker(fsOwnerShortUserName, supergroup);
+ }
/**
* Remove a file/directory from the namespace.
*
@@ -2856,7 +2822,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
throws AccessControlException, SafeModeException, UnresolvedLinkException,
IOException {
BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
-
+ FSPermissionChecker pc = getPermissionChecker();
+ checkOperation(OperationCategory.WRITE);
writeLock();
try {
checkOperation(OperationCategory.WRITE);
@@ -2867,7 +2834,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
throw new IOException(src + " is non empty");
}
if (enforcePermission && isPermissionEnabled) {
- checkPermission(src, false, null, FsAction.WRITE, null, FsAction.ALL);
+ checkPermission(pc, src, false, null, FsAction.WRITE, null, FsAction.ALL);
}
// Unlink the target directory from directory tree
if (!dir.delete(src, collectedBlocks)) {
@@ -2984,9 +2951,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
throws AccessControlException, UnresolvedLinkException,
StandbyException, IOException {
HdfsFileStatus stat = null;
-
+ FSPermissionChecker pc = getPermissionChecker();
+ checkOperation(OperationCategory.READ);
readLock();
-
try {
checkOperation(OperationCategory.READ);
@@ -2994,24 +2961,16 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
throw new InvalidPathException("Invalid file name: " + src);
}
if (isPermissionEnabled) {
- checkTraverse(src);
+ checkTraverse(pc, src);
}
stat = dir.getFileInfo(src, resolveLink);
} catch (AccessControlException e) {
- if (isAuditEnabled() && isExternalInvocation()) {
- logAuditEvent(false, UserGroupInformation.getCurrentUser(),
- getRemoteIp(),
- "getfileinfo", src, null, null);
- }
+ logAuditEvent(false, "getfileinfo", src);
throw e;
} finally {
readUnlock();
}
- if (isAuditEnabled() && isExternalInvocation()) {
- logAuditEvent(UserGroupInformation.getCurrentUser(),
- getRemoteIp(),
- "getfileinfo", src, null, null);
- }
+ logAuditEvent(true, "getfileinfo", src);
return stat;
}
@@ -3023,35 +2982,33 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
try {
return mkdirsInt(src, permissions, createParent);
} catch (AccessControlException e) {
- if (isAuditEnabled() && isExternalInvocation()) {
- logAuditEvent(false, UserGroupInformation.getCurrentUser(),
- getRemoteIp(),
- "mkdirs", src, null, null);
- }
+ logAuditEvent(false, "mkdirs", src);
throw e;
}
}
private boolean mkdirsInt(String src, PermissionStatus permissions,
boolean createParent) throws IOException, UnresolvedLinkException {
+ HdfsFileStatus resultingStat = null;
boolean status = false;
if(NameNode.stateChangeLog.isDebugEnabled()) {
NameNode.stateChangeLog.debug("DIR* NameSystem.mkdirs: " + src);
}
+ FSPermissionChecker pc = getPermissionChecker();
+ checkOperation(OperationCategory.WRITE);
writeLock();
try {
checkOperation(OperationCategory.WRITE);
-
- status = mkdirsInternal(src, permissions, createParent);
+ status = mkdirsInternal(pc, src, permissions, createParent);
+ if (status) {
+ resultingStat = dir.getFileInfo(src, false);
+ }
} finally {
writeUnlock();
}
getEditLog().logSync();
- if (status && isAuditEnabled() && isExternalInvocation()) {
- final HdfsFileStatus stat = dir.getFileInfo(src, false);
- logAuditEvent(UserGroupInformation.getCurrentUser(),
- getRemoteIp(),
- "mkdirs", src, null, stat);
+ if (status) {
+ logAuditEvent(true, "mkdirs", src, null, resultingStat);
}
return status;
}
@@ -3059,7 +3016,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
/**
* Create all the necessary directories
*/
- private boolean mkdirsInternal(String src,
+ private boolean mkdirsInternal(FSPermissionChecker pc, String src,
PermissionStatus permissions, boolean createParent)
throws IOException, UnresolvedLinkException {
assert hasWriteLock();
@@ -3067,7 +3024,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
throw new SafeModeException("Cannot create directory " + src, safeMode);
}
if (isPermissionEnabled) {
- checkTraverse(src);
+ checkTraverse(pc, src);
}
if (dir.isDir(src)) {
// all the users of mkdirs() are used to expect 'true' even if
@@ -3078,7 +3035,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
throw new InvalidPathException(src);
}
if (isPermissionEnabled) {
- checkAncestorAccess(src, FsAction.WRITE);
+ checkAncestorAccess(pc, src, FsAction.WRITE);
}
if (!createParent) {
verifyParentDir(src);
@@ -3097,12 +3054,13 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
ContentSummary getContentSummary(String src) throws AccessControlException,
FileNotFoundException, UnresolvedLinkException, StandbyException {
+ FSPermissionChecker pc = getPermissionChecker();
+ checkOperation(OperationCategory.READ);
readLock();
try {
checkOperation(OperationCategory.READ);
-
if (isPermissionEnabled) {
- checkPermission(src, false, null, null, null, FsAction.READ_EXECUTE);
+ checkPermission(pc, src, false, null, null, null, FsAction.READ_EXECUTE);
}
return dir.getContentSummary(src);
} finally {
@@ -3117,15 +3075,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
*/
void setQuota(String path, long nsQuota, long dsQuota)
throws IOException, UnresolvedLinkException {
+ checkSuperuserPrivilege();
+ checkOperation(OperationCategory.WRITE);
writeLock();
try {
checkOperation(OperationCategory.WRITE);
if (isInSafeMode()) {
throw new SafeModeException("Cannot set quota on " + path, safeMode);
}
- if (isPermissionEnabled) {
- checkSuperuserPrivilege();
- }
dir.setQuota(path, nsQuota, dsQuota);
} finally {
writeUnlock();
@@ -3143,6 +3100,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
void fsync(String src, String clientName, long lastBlockLength)
throws IOException, UnresolvedLinkException {
NameNode.stateChangeLog.info("BLOCK* fsync: " + src + " for " + clientName);
+ checkOperation(OperationCategory.WRITE);
writeLock();
try {
checkOperation(OperationCategory.WRITE);
@@ -3347,6 +3305,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
String[] newtargetstorages)
throws IOException, UnresolvedLinkException {
String src = "";
+ checkOperation(OperationCategory.WRITE);
writeLock();
try {
checkOperation(OperationCategory.WRITE);
@@ -3450,6 +3409,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
* Renew the lease(s) held by the given client
*/
void renewLease(String holder) throws IOException {
+ checkOperation(OperationCategory.WRITE);
writeLock();
try {
checkOperation(OperationCategory.WRITE);
@@ -3481,11 +3441,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
try {
return getListingInt(src, startAfter, needLocation);
} catch (AccessControlException e) {
- if (isAuditEnabled() && isExternalInvocation()) {
- logAuditEvent(false, UserGroupInformation.getCurrentUser(),
- getRemoteIp(),
- "listStatus", src, null, null);
- }
+ logAuditEvent(false, "listStatus", src);
throw e;
}
}
@@ -3494,22 +3450,20 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
boolean needLocation)
throws AccessControlException, UnresolvedLinkException, IOException {
DirectoryListing dl;
+ FSPermissionChecker pc = getPermissionChecker();
+ checkOperation(OperationCategory.READ);
readLock();
try {
checkOperation(OperationCategory.READ);
if (isPermissionEnabled) {
if (dir.isDir(src)) {
- checkPathAccess(src, FsAction.READ_EXECUTE);
+ checkPathAccess(pc, src, FsAction.READ_EXECUTE);
} else {
- checkTraverse(src);
+ checkTraverse(pc, src);
}
}
- if (isAuditEnabled() && isExternalInvocation()) {
- logAuditEvent(UserGroupInformation.getCurrentUser(),
- getRemoteIp(),
- "listStatus", src, null, null);
- }
+ logAuditEvent(true, "listStatus", src);
dl = dir.getListing(src, startAfter, needLocation);
} finally {
readUnlock();
@@ -3721,42 +3675,49 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
return stats;
}
- /**
- * Total raw bytes including non-dfs used space.
- */
@Override // FSNamesystemMBean
+ @Metric({"CapacityTotal",
+ "Total raw capacity of data nodes in bytes"})
public long getCapacityTotal() {
return datanodeStatistics.getCapacityTotal();
}
- @Metric
+ @Metric({"CapacityTotalGB",
+ "Total raw capacity of data nodes in GB"})
public float getCapacityTotalGB() {
return DFSUtil.roundBytesToGB(getCapacityTotal());
}
- /**
- * Total used space by data nodes
- */
@Override // FSNamesystemMBean
+ @Metric({"CapacityUsed",
+ "Total used capacity across all data nodes in bytes"})
public long getCapacityUsed() {
return datanodeStatistics.getCapacityUsed();
}
- @Metric
+ @Metric({"CapacityUsedGB",
+ "Total used capacity across all data nodes in GB"})
public float getCapacityUsedGB() {
return DFSUtil.roundBytesToGB(getCapacityUsed());
}
- @Override
+ @Override // FSNamesystemMBean
+ @Metric({"CapacityRemaining", "Remaining capacity in bytes"})
public long getCapacityRemaining() {
return datanodeStatistics.getCapacityRemaining();
}
- @Metric
+ @Metric({"CapacityRemainingGB", "Remaining capacity in GB"})
public float getCapacityRemainingGB() {
return DFSUtil.roundBytesToGB(getCapacityRemaining());
}
+ @Metric({"CapacityUsedNonDFS",
+ "Total space used by data nodes for non DFS purposes in bytes"})
+ public long getCapacityUsedNonDFS() {
+ return datanodeStatistics.getCapacityUsedNonDFS();
+ }
+
/**
* Total number of connections.
*/
@@ -3777,10 +3738,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
}
DatanodeInfo[] datanodeReport(final DatanodeReportType type
- ) throws AccessControlException {
+ ) throws AccessControlException, StandbyException {
checkSuperuserPrivilege();
+ checkOperation(OperationCategory.UNCHECKED);
readLock();
try {
+ checkOperation(OperationCategory.UNCHECKED);
final DatanodeManager dm = getBlockManager().getDatanodeManager();
final List results = dm.getDatanodeListForReport(type);
@@ -3803,9 +3766,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
* @throws IOException if
*/
void saveNamespace() throws AccessControlException, IOException {
+ checkSuperuserPrivilege();
+ checkOperation(OperationCategory.UNCHECKED);
readLock();
try {
- checkSuperuserPrivilege();
+ checkOperation(OperationCategory.UNCHECKED);
if (!isInSafeMode()) {
throw new IOException("Safe mode should be turned ON " +
"in order to create namespace image.");
@@ -3823,10 +3788,13 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
*
* @throws AccessControlException if superuser privilege is violated.
*/
- boolean restoreFailedStorage(String arg) throws AccessControlException {
+ boolean restoreFailedStorage(String arg) throws AccessControlException,
+ StandbyException {
+ checkSuperuserPrivilege();
+ checkOperation(OperationCategory.UNCHECKED);
writeLock();
try {
- checkSuperuserPrivilege();
+ checkOperation(OperationCategory.UNCHECKED);
// if it is disabled - enable it and vice versa.
if(arg.equals("check"))
@@ -3846,10 +3814,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
}
void finalizeUpgrade() throws IOException {
+ checkSuperuserPrivilege();
+ checkOperation(OperationCategory.WRITE);
writeLock();
try {
checkOperation(OperationCategory.WRITE);
- checkSuperuserPrivilege();
getFSImage().finalizeUpgrade();
} finally {
writeUnlock();
@@ -4585,10 +4554,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
}
CheckpointSignature rollEditLog() throws IOException {
+ checkSuperuserPrivilege();
+ checkOperation(OperationCategory.JOURNAL);
writeLock();
try {
checkOperation(OperationCategory.JOURNAL);
- checkSuperuserPrivilege();
if (isInSafeMode()) {
throw new SafeModeException("Log not rolled", safeMode);
}
@@ -4603,6 +4573,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
NamenodeRegistration bnReg, // backup node
NamenodeRegistration nnReg) // active name-node
throws IOException {
+ checkOperation(OperationCategory.CHECKPOINT);
writeLock();
try {
checkOperation(OperationCategory.CHECKPOINT);
@@ -4621,6 +4592,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
void endCheckpoint(NamenodeRegistration registration,
CheckpointSignature sig) throws IOException {
+ checkOperation(OperationCategory.CHECKPOINT);
readLock();
try {
checkOperation(OperationCategory.CHECKPOINT);
@@ -4639,61 +4611,64 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
return new PermissionStatus(fsOwner.getShortUserName(), supergroup, permission);
}
- private FSPermissionChecker checkOwner(String path
- ) throws AccessControlException, UnresolvedLinkException {
- return checkPermission(path, true, null, null, null, null);
+ private void checkOwner(FSPermissionChecker pc, String path)
+ throws AccessControlException, UnresolvedLinkException {
+ checkPermission(pc, path, true, null, null, null, null);
}
- private FSPermissionChecker checkPathAccess(String path, FsAction access
- ) throws AccessControlException, UnresolvedLinkException {
- return checkPermission(path, false, null, null, access, null);
+ private void checkPathAccess(FSPermissionChecker pc,
+ String path, FsAction access) throws AccessControlException,
+ UnresolvedLinkException {
+ checkPermission(pc, path, false, null, null, access, null);
}
- private FSPermissionChecker checkParentAccess(String path, FsAction access
- ) throws AccessControlException, UnresolvedLinkException {
- return checkPermission(path, false, null, access, null, null);
+ private void checkParentAccess(FSPermissionChecker pc,
+ String path, FsAction access) throws AccessControlException,
+ UnresolvedLinkException {
+ checkPermission(pc, path, false, null, access, null, null);
}
- private FSPermissionChecker checkAncestorAccess(String path, FsAction access
- ) throws AccessControlException, UnresolvedLinkException {
- return checkPermission(path, false, access, null, null, null);
+ private void checkAncestorAccess(FSPermissionChecker pc,
+ String path, FsAction access) throws AccessControlException,
+ UnresolvedLinkException {
+ checkPermission(pc, path, false, access, null, null, null);
}
- private FSPermissionChecker checkTraverse(String path
- ) throws AccessControlException, UnresolvedLinkException {
- return checkPermission(path, false, null, null, null, null);
+ private void checkTraverse(FSPermissionChecker pc, String path)
+ throws AccessControlException, UnresolvedLinkException {
+ checkPermission(pc, path, false, null, null, null, null);
}
@Override
- public void checkSuperuserPrivilege() throws AccessControlException {
+ public void checkSuperuserPrivilege()
+ throws AccessControlException {
if (isPermissionEnabled) {
- FSPermissionChecker.checkSuperuserPrivilege(fsOwner, supergroup);
+ FSPermissionChecker pc = getPermissionChecker();
+ pc.checkSuperuserPrivilege();
}
}
/**
- * Check whether current user have permissions to access the path.
- * For more details of the parameters, see
- * {@link FSPermissionChecker#checkPermission(String, INodeDirectory, boolean, FsAction, FsAction, FsAction, FsAction)}.
+ * Check whether current user have permissions to access the path. For more
+ * details of the parameters, see
+ * {@link FSPermissionChecker#checkPermission()}.
*/
- private FSPermissionChecker checkPermission(String path, boolean doCheckOwner,
- FsAction ancestorAccess, FsAction parentAccess, FsAction access,
- FsAction subAccess) throws AccessControlException, UnresolvedLinkException {
- FSPermissionChecker pc = new FSPermissionChecker(
- fsOwner.getShortUserName(), supergroup);
- if (!pc.isSuper) {
+ private void checkPermission(FSPermissionChecker pc,
+ String path, boolean doCheckOwner, FsAction ancestorAccess,
+ FsAction parentAccess, FsAction access, FsAction subAccess)
+ throws AccessControlException, UnresolvedLinkException {
+ if (!pc.isSuperUser()) {
dir.waitForReady();
readLock();
try {
- pc.checkPermission(path, dir.rootDir, doCheckOwner,
- ancestorAccess, parentAccess, access, subAccess);
+ pc.checkPermission(path, dir.rootDir, doCheckOwner, ancestorAccess,
+ parentAccess, access, subAccess);
} finally {
readUnlock();
- }
+ }
}
- return pc;
}
-
+
/**
* Check to see if we have exceeded the limit on the number
* of inodes.
@@ -4906,6 +4881,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
* Client is reporting some bad block locations.
*/
void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
+ checkOperation(OperationCategory.WRITE);
writeLock();
try {
checkOperation(OperationCategory.WRITE);
@@ -4940,6 +4916,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
LocatedBlock updateBlockForPipeline(ExtendedBlock block,
String clientName) throws IOException {
LocatedBlock locatedBlock;
+ checkOperation(OperationCategory.WRITE);
writeLock();
try {
checkOperation(OperationCategory.WRITE);
@@ -4971,6 +4948,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
void updatePipeline(String clientName, ExtendedBlock oldBlock,
ExtendedBlock newBlock, DatanodeID[] newNodes)
throws IOException {
+ checkOperation(OperationCategory.WRITE);
writeLock();
try {
checkOperation(OperationCategory.WRITE);
@@ -5098,8 +5076,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
*/
void releaseBackupNode(NamenodeRegistration registration)
throws IOException {
+ checkOperation(OperationCategory.WRITE);
writeLock();
try {
+ checkOperation(OperationCategory.WRITE);
if(getFSImage().getStorage().getNamespaceID()
!= registration.getNamespaceID())
throw new IOException("Incompatible namespaceIDs: "
@@ -5137,16 +5117,15 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
*/
Collection listCorruptFileBlocks(String path,
String[] cookieTab) throws IOException {
-
+ checkSuperuserPrivilege();
+ checkOperation(OperationCategory.READ);
readLock();
try {
checkOperation(OperationCategory.READ);
-
if (!isPopulatingReplQueues()) {
throw new IOException("Cannot run listCorruptFileBlocks because " +
"replication queues have not been initialized.");
}
- checkSuperuserPrivilege();
// print a limited # of corrupt files per call
int count = 0;
ArrayList corruptFiles = new ArrayList();
@@ -5232,6 +5211,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
Token getDelegationToken(Text renewer)
throws IOException {
Token token;
+ checkOperation(OperationCategory.WRITE);
writeLock();
try {
checkOperation(OperationCategory.WRITE);
@@ -5248,7 +5228,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
return null;
}
- UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+ UserGroupInformation ugi = getRemoteUser();
String user = ugi.getUserName();
Text owner = new Text(user);
Text realUser = null;
@@ -5278,6 +5258,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
long renewDelegationToken(Token token)
throws InvalidToken, IOException {
long expiryTime;
+ checkOperation(OperationCategory.WRITE);
writeLock();
try {
checkOperation(OperationCategory.WRITE);
@@ -5289,7 +5270,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
throw new IOException(
"Delegation Token can be renewed only with kerberos or web authentication");
}
- String renewer = UserGroupInformation.getCurrentUser().getShortUserName();
+ String renewer = getRemoteUser().getShortUserName();
expiryTime = dtSecretManager.renewToken(token, renewer);
DelegationTokenIdentifier id = new DelegationTokenIdentifier();
ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier());
@@ -5310,6 +5291,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
*/
void cancelDelegationToken(Token token)
throws IOException {
+ checkOperation(OperationCategory.WRITE);
writeLock();
try {
checkOperation(OperationCategory.WRITE);
@@ -5317,7 +5299,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
if (isInSafeMode()) {
throw new SafeModeException("Cannot cancel delegation token", safeMode);
}
- String canceller = UserGroupInformation.getCurrentUser().getUserName();
+ String canceller = getRemoteUser().getUserName();
DelegationTokenIdentifier id = dtSecretManager
.cancelToken(token, canceller);
getEditLog().logCancelDelegationToken(id);
@@ -5386,7 +5368,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
*/
private AuthenticationMethod getConnectionAuthenticationMethod()
throws IOException {
- UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+ UserGroupInformation ugi = getRemoteUser();
AuthenticationMethod authMethod = ugi.getAuthenticationMethod();
if (authMethod == AuthenticationMethod.PROXY) {
authMethod = ugi.getRealUser().getAuthenticationMethod();
@@ -5410,12 +5392,22 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
return NamenodeWebHdfsMethods.getRemoteIp();
}
+ // optimize ugi lookup for RPC operations to avoid a trip through
+ // UGI.getCurrentUser which is synch'ed
+ private static UserGroupInformation getRemoteUser() throws IOException {
+ UserGroupInformation ugi = null;
+ if (Server.isRpcInvocation()) {
+ ugi = Server.getRemoteUser();
+ }
+ return (ugi != null) ? ugi : UserGroupInformation.getCurrentUser();
+ }
+
/**
* Log fsck event in the audit log
*/
void logFsckEvent(String src, InetAddress remoteAddress) throws IOException {
if (isAuditEnabled()) {
- logAuditEvent(UserGroupInformation.getCurrentUser(),
+ logAuditEvent(true, getRemoteUser(),
remoteAddress,
"fsck", src, null, null);
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
index 91ebc968a04..d88bfd87960 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode;
import java.io.IOException;
import java.util.Arrays;
+import java.util.Collections;
import java.util.HashSet;
import java.util.Set;
import java.util.Stack;
@@ -31,14 +32,20 @@ import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.UserGroupInformation;
-/** Perform permission checking in {@link FSNamesystem}. */
+/**
+ * Class that helps in checking file system permission.
+ * The state of this class need not be synchronized as it has data structures that
+ * are read-only.
+ *
+ * Some of the helper methods are gaurded by {@link FSNamesystem#readLock()}.
+ */
class FSPermissionChecker {
static final Log LOG = LogFactory.getLog(UserGroupInformation.class);
-
private final UserGroupInformation ugi;
- public final String user;
- private final Set groups = new HashSet();
- public final boolean isSuper;
+ private final String user;
+ /** A set with group namess. Not synchronized since it is unmodifiable */
+ private final Set groups;
+ private final boolean isSuper;
FSPermissionChecker(String fsOwner, String supergroup
) throws AccessControlException{
@@ -47,10 +54,9 @@ class FSPermissionChecker {
} catch (IOException e) {
throw new AccessControlException(e);
}
-
- groups.addAll(Arrays.asList(ugi.getGroupNames()));
+ HashSet s = new HashSet(Arrays.asList(ugi.getGroupNames()));
+ groups = Collections.unmodifiableSet(s);
user = ugi.getShortUserName();
-
isSuper = user.equals(fsOwner) || groups.contains(supergroup);
}
@@ -60,20 +66,23 @@ class FSPermissionChecker {
*/
public boolean containsGroup(String group) {return groups.contains(group);}
+ public String getUser() {
+ return user;
+ }
+
+ public boolean isSuperUser() {
+ return isSuper;
+ }
+
/**
* Verify if the caller has the required permission. This will result into
* an exception if the caller is not allowed to access the resource.
- * @param owner owner of the system
- * @param supergroup supergroup of the system
*/
- public static void checkSuperuserPrivilege(UserGroupInformation owner,
- String supergroup)
- throws AccessControlException {
- FSPermissionChecker checker =
- new FSPermissionChecker(owner.getShortUserName(), supergroup);
- if (!checker.isSuper) {
+ public void checkSuperuserPrivilege()
+ throws AccessControlException {
+ if (!isSuper) {
throw new AccessControlException("Access denied for user "
- + checker.user + ". Superuser privilege is required");
+ + user + ". Superuser privilege is required");
}
}
@@ -103,9 +112,11 @@ class FSPermissionChecker {
* @param subAccess If path is a directory,
* it is the access required of the path and all the sub-directories.
* If path is not a directory, there is no effect.
- * @return a PermissionChecker object which caches data for later use.
* @throws AccessControlException
* @throws UnresolvedLinkException
+ *
+ * Guarded by {@link FSNamesystem#readLock()}
+ * Caller of this method must hold that lock.
*/
void checkPermission(String path, INodeDirectory root, boolean doCheckOwner,
FsAction ancestorAccess, FsAction parentAccess, FsAction access,
@@ -148,6 +159,7 @@ class FSPermissionChecker {
}
}
+ /** Guarded by {@link FSNamesystem#readLock()} */
private void checkOwner(INode inode) throws AccessControlException {
if (inode != null && user.equals(inode.getUserName())) {
return;
@@ -155,6 +167,7 @@ class FSPermissionChecker {
throw new AccessControlException("Permission denied");
}
+ /** Guarded by {@link FSNamesystem#readLock()} */
private void checkTraverse(INode[] inodes, int last
) throws AccessControlException {
for(int j = 0; j <= last; j++) {
@@ -162,6 +175,7 @@ class FSPermissionChecker {
}
}
+ /** Guarded by {@link FSNamesystem#readLock()} */
private void checkSubAccess(INode inode, FsAction access
) throws AccessControlException {
if (inode == null || !inode.isDirectory()) {
@@ -181,11 +195,13 @@ class FSPermissionChecker {
}
}
+ /** Guarded by {@link FSNamesystem#readLock()} */
private void check(INode[] inodes, int i, FsAction access
) throws AccessControlException {
check(i >= 0? inodes[i]: null, access);
}
+ /** Guarded by {@link FSNamesystem#readLock()} */
private void check(INode inode, FsAction access
) throws AccessControlException {
if (inode == null) {
@@ -206,7 +222,9 @@ class FSPermissionChecker {
+ ", access=" + access + ", inode=" + inode);
}
- private void checkStickyBit(INode parent, INode inode) throws AccessControlException {
+ /** Guarded by {@link FSNamesystem#readLock()} */
+ private void checkStickyBit(INode parent, INode inode)
+ throws AccessControlException {
if(!parent.getFsPermission().getStickyBit()) {
return;
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
index 91ff0891c96..b407a62da97 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
@@ -282,7 +282,11 @@ abstract class INode implements Comparable {
String getLocalParentDir() {
INode inode = isRoot() ? this : getParent();
- return (inode != null) ? inode.getFullPathName() : "";
+ String parentDir = "";
+ if (inode != null) {
+ parentDir = inode.getFullPathName();
+ }
+ return (parentDir != null) ? parentDir : "";
}
/**
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java
index 3488f074103..32e9fd15f05 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java
@@ -25,10 +25,10 @@ import java.util.Map;
import javax.servlet.ServletContext;
-import org.apache.commons.logging.Log;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.server.common.JspHelper;
import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
import org.apache.hadoop.hdfs.web.AuthFilter;
@@ -77,7 +77,8 @@ public class NameNodeHttpServer {
if (UserGroupInformation.isSecurityEnabled()) {
initSpnego(conf,
DFSConfigKeys.DFS_NAMENODE_INTERNAL_SPNEGO_USER_NAME_KEY,
- DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY);
+ DFSUtil.getSpnegoKeytabKey(conf,
+ DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY));
}
if (WebHdfsFileSystem.isEnabled(conf, LOG)) {
//add SPNEGO authentication filter for webhdfs
@@ -112,11 +113,8 @@ public class NameNodeHttpServer {
DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY +
"' is not set.");
}
- String httpKeytab = conf.get(
- DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY);
- if (httpKeytab == null) {
- httpKeytab = conf.get(DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY);
- }
+ String httpKeytab = conf.get(DFSUtil.getSpnegoKeytabKey(conf,
+ DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY));
if (httpKeytab != null && !httpKeytab.isEmpty()) {
params.put(
DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY,
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 90e79363b17..275b9bdbe61 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -337,7 +337,6 @@ class NameNodeRpcServer implements NamenodeProtocols {
throw new IllegalArgumentException(
"Unexpected not positive size: "+size);
}
- namesystem.checkOperation(OperationCategory.READ);
namesystem.checkSuperuserPrivilege();
return namesystem.getBlockManager().getBlocks(datanode, size);
}
@@ -707,7 +706,6 @@ class NameNodeRpcServer implements NamenodeProtocols {
@Override // ClientProtocol
public DatanodeInfo[] getDatanodeReport(DatanodeReportType type)
throws IOException {
- namesystem.checkOperation(OperationCategory.UNCHECKED);
DatanodeInfo results[] = namesystem.datanodeReport(type);
if (results == null ) {
throw new IOException("Cannot find datanode report");
@@ -732,19 +730,16 @@ class NameNodeRpcServer implements NamenodeProtocols {
@Override // ClientProtocol
public boolean restoreFailedStorage(String arg) throws IOException {
- namesystem.checkOperation(OperationCategory.UNCHECKED);
return namesystem.restoreFailedStorage(arg);
}
@Override // ClientProtocol
public void saveNamespace() throws IOException {
- namesystem.checkOperation(OperationCategory.UNCHECKED);
namesystem.saveNamespace();
}
@Override // ClientProtocol
public long rollEdits() throws AccessControlException, IOException {
- namesystem.checkOperation(OperationCategory.JOURNAL);
CheckpointSignature sig = namesystem.rollEditLog();
return sig.getCurSegmentTxId();
}
@@ -789,7 +784,6 @@ class NameNodeRpcServer implements NamenodeProtocols {
@Override // ClientProtocol
public void metaSave(String filename) throws IOException {
- namesystem.checkOperation(OperationCategory.UNCHECKED);
namesystem.metaSave(filename);
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
index c453db561eb..c82e9155ed6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
@@ -18,7 +18,9 @@
package org.apache.hadoop.hdfs.server.namenode;
import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
import org.apache.hadoop.hdfs.util.RwLock;
+import org.apache.hadoop.ipc.StandbyException;
import org.apache.hadoop.security.AccessControlException;
/** Namesystem operations. */
@@ -38,4 +40,6 @@ public interface Namesystem extends RwLock, SafeMode {
public boolean isGenStampInFuture(long generationStamp);
public void adjustSafeModeBlockTotals(int deltaSafe, int deltaTotal);
+
+ public void checkOperation(OperationCategory read) throws StandbyException;
}
\ No newline at end of file
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/RedundantEditLogInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/RedundantEditLogInputStream.java
index eb6a8ea1c5f..7d8135669d0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/RedundantEditLogInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/RedundantEditLogInputStream.java
@@ -267,4 +267,11 @@ class RedundantEditLogInputStream extends EditLogInputStream {
super(msg);
}
}
+
+ @Override
+ public void setMaxOpSize(int maxOpSize) {
+ for (EditLogInputStream elis : streams) {
+ elis.setMaxOpSize(maxOpSize);
+ }
+ }
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
index 763c7089abd..16c12608909 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
@@ -144,6 +144,11 @@ public class SecondaryNameNode implements Runnable {
return checkpointImage;
}
+ @VisibleForTesting
+ int getMergeErrorCount() {
+ return checkpointImage.getMergeErrorCount();
+ }
+
@VisibleForTesting
FSNamesystem getFSNamesystem() {
return namesystem;
@@ -250,15 +255,11 @@ public class SecondaryNameNode implements Runnable {
new AccessControlList(conf.get(DFS_ADMIN, " "))) {
{
if (UserGroupInformation.isSecurityEnabled()) {
- String httpKeytabKey = DFSConfigKeys.
- DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY;
- if (null == conf.get(httpKeytabKey)) {
- httpKeytabKey = DFSConfigKeys.DFS_SECONDARY_NAMENODE_KEYTAB_FILE_KEY;
- }
initSpnego(
conf,
DFSConfigKeys.DFS_SECONDARY_NAMENODE_INTERNAL_SPNEGO_USER_NAME_KEY,
- httpKeytabKey);
+ DFSUtil.getSpnegoKeytabKey(conf,
+ DFSConfigKeys.DFS_SECONDARY_NAMENODE_KEYTAB_FILE_KEY));
}
}
};
@@ -339,6 +340,7 @@ public class SecondaryNameNode implements Runnable {
// number of transactions in the edit log that haven't yet been checkpointed.
//
long period = checkpointConf.getCheckPeriod();
+ int maxRetries = checkpointConf.getMaxRetriesOnMergeError();
while (shouldRun) {
try {
@@ -364,6 +366,13 @@ public class SecondaryNameNode implements Runnable {
} catch (IOException e) {
LOG.error("Exception in doCheckpoint", e);
e.printStackTrace();
+ // Prevent a huge number of edits from being created due to
+ // unrecoverable conditions and endless retries.
+ if (checkpointImage.getMergeErrorCount() > maxRetries) {
+ LOG.fatal("Merging failed " +
+ checkpointImage.getMergeErrorCount() + " times.");
+ terminate(1);
+ }
} catch (Throwable e) {
LOG.fatal("Throwable Exception in doCheckpoint", e);
e.printStackTrace();
@@ -498,9 +507,21 @@ public class SecondaryNameNode implements Runnable {
RemoteEditLogManifest manifest =
namenode.getEditLogManifest(sig.mostRecentCheckpointTxId + 1);
+ // Fetch fsimage and edits. Reload the image if previous merge failed.
loadImage |= downloadCheckpointFiles(
- fsName, checkpointImage, sig, manifest); // Fetch fsimage and edits
- doMerge(sig, manifest, loadImage, checkpointImage, namesystem);
+ fsName, checkpointImage, sig, manifest) |
+ checkpointImage.hasMergeError();
+ try {
+ doMerge(sig, manifest, loadImage, checkpointImage, namesystem);
+ } catch (IOException ioe) {
+ // A merge error occurred. The in-memory file system state may be
+ // inconsistent, so the image and edits need to be reloaded.
+ checkpointImage.setMergeError();
+ throw ioe;
+ }
+ // Clear any error since merge was successful.
+ checkpointImage.clearMergeError();
+
//
// Upload the new image into the NameNode. Then tell the Namenode
@@ -754,6 +775,7 @@ public class SecondaryNameNode implements Runnable {
static class CheckpointStorage extends FSImage {
+ private int mergeErrorCount;
private static class CheckpointLogPurger implements LogsPurgeable {
private NNStorage storage;
@@ -815,6 +837,7 @@ public class SecondaryNameNode implements Runnable {
// we shouldn't have any editLog instance. Setting to null
// makes sure we don't accidentally depend on it.
editLog = null;
+ mergeErrorCount = 0;
// Replace the archival manager with one that can actually work on the
// 2NN's edits storage.
@@ -881,7 +904,24 @@ public class SecondaryNameNode implements Runnable {
}
}
}
-
+
+
+ boolean hasMergeError() {
+ return (mergeErrorCount > 0);
+ }
+
+ int getMergeErrorCount() {
+ return mergeErrorCount;
+ }
+
+ void setMergeError() {
+ mergeErrorCount++;
+ }
+
+ void clearMergeError() {
+ mergeErrorCount = 0;
+ }
+
/**
* Ensure that the current/ directory exists in all storage
* directories
@@ -915,7 +955,9 @@ public class SecondaryNameNode implements Runnable {
dstImage.reloadFromImageFile(file, dstNamesystem);
dstNamesystem.dir.imageLoadComplete();
}
-
+ // error simulation code for junit test
+ CheckpointFaultInjector.getInstance().duringMerge();
+
Checkpointer.rollForwardByApplyingLogs(manifest, dstImage, dstNamesystem);
// The following has the side effect of purging old fsimages/edit logs.
dstImage.saveFSImageInAllDirs(dstNamesystem, dstImage.getLastAppliedTxId());
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
index 507c8ec7648..60e703b4e02 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
@@ -229,7 +229,6 @@ public class TransferFsImage {
SecurityUtil.openSecureHttpConnection(url);
if (timeout <= 0) {
- // Set the ping interval as timeout
Configuration conf = new HdfsConfiguration();
timeout = conf.getInt(DFSConfigKeys.DFS_IMAGE_TRANSFER_TIMEOUT_KEY,
DFSConfigKeys.DFS_IMAGE_TRANSFER_TIMEOUT_DEFAULT);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/HAContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/HAContext.java
index b052e4ea9e3..823738798d6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/HAContext.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/HAContext.java
@@ -64,9 +64,17 @@ public interface HAContext {
void writeUnlock();
/**
- * Verify that the given operation category is allowed in the
- * current state. This is to allow NN implementations (eg BackupNode)
- * to override it with node-specific handling.
+ * Verify that the given operation category is allowed in the current state.
+ * This is to allow NN implementations (eg BackupNode) to override it with
+ * node-specific handling.
+ *
+ * If the operation which is being checked will be taking the FSNS lock, it's
+ * advisable to check the operation category both immediately before and after
+ * taking the lock. This is because clients rely on the StandbyException
+ * thrown by this method in order to trigger client failover, and if a client
+ * first tries to contact the Standby NN, it could block for a long time if
+ * the Standby is holding the lock for a while, e.g. when performing a
+ * checkpoint. See HDFS-4591 for more details.
*/
void checkOperation(OperationCategory op) throws StandbyException;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
index 0cb4b127115..f88f085b3ed 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
@@ -99,7 +99,6 @@ import org.apache.hadoop.hdfs.web.resources.UserParam;
import org.apache.hadoop.ipc.Server;
import org.apache.hadoop.net.NodeBase;
import org.apache.hadoop.security.Credentials;
-import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
@@ -212,7 +211,6 @@ public class NamenodeWebHdfsMethods {
namenode, ugi, renewer != null? renewer: ugi.getShortUserName());
final Token extends TokenIdentifier> t = c.getAllTokens().iterator().next();
t.setKind(WebHdfsFileSystem.TOKEN_KIND);
- SecurityUtil.setTokenService(t, namenode.getHttpAddress());
return t;
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsXmlLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsXmlLoader.java
index 95cc3b89120..cf761ccedd4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsXmlLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsXmlLoader.java
@@ -26,6 +26,7 @@ import java.util.Stack;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.util.XMLUtils;
import org.apache.hadoop.hdfs.util.XMLUtils.InvalidXmlException;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes;
@@ -176,7 +177,7 @@ class OfflineEditsXmlLoader
@Override
public void endElement (String uri, String name, String qName) {
- String str = cbuf.toString().trim();
+ String str = XMLUtils.unmangleXmlString(cbuf.toString()).trim();
cbuf = new StringBuffer();
switch (state) {
case EXPECT_EDITS_TAG:
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/LightWeightGSet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/LightWeightGSet.java
index 32d80c25516..5ab9a8cb355 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/LightWeightGSet.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/LightWeightGSet.java
@@ -24,8 +24,11 @@ import java.util.Iterator;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.HadoopIllegalArgumentException;
+import com.google.common.annotations.VisibleForTesting;
+
/**
* A low memory footprint {@link GSet} implementation,
* which uses an array for storing the elements
@@ -72,7 +75,7 @@ public class LightWeightGSet implements GSet {
/** Modification version for fail-fast.
* @see ConcurrentModificationException
*/
- private volatile int modification = 0;
+ private int modification = 0;
/**
* @param recommended_length Recommended size of the internal array.
@@ -285,4 +288,54 @@ public class LightWeightGSet implements GSet {
throw new UnsupportedOperationException("Remove is not supported.");
}
}
+
+ /**
+ * Let t = percentage of max memory.
+ * Let e = round(log_2 t).
+ * Then, we choose capacity = 2^e/(size of reference),
+ * unless it is outside the close interval [1, 2^30].
+ */
+ public static int computeCapacity(double percentage, String mapName) {
+ return computeCapacity(Runtime.getRuntime().maxMemory(), percentage,
+ mapName);
+ }
+
+ @VisibleForTesting
+ static int computeCapacity(long maxMemory, double percentage,
+ String mapName) {
+ if (percentage > 100.0 || percentage < 0.0) {
+ throw new HadoopIllegalArgumentException("Percentage " + percentage
+ + " must be greater than or equal to 0 "
+ + " and less than or equal to 100");
+ }
+ if (maxMemory < 0) {
+ throw new HadoopIllegalArgumentException("Memory " + maxMemory
+ + " must be greater than or equal to 0");
+ }
+ if (percentage == 0.0 || maxMemory == 0) {
+ return 0;
+ }
+ //VM detection
+ //See http://java.sun.com/docs/hotspot/HotSpotFAQ.html#64bit_detection
+ final String vmBit = System.getProperty("sun.arch.data.model");
+
+ //Percentage of max memory
+ final double percentDivisor = 100.0/percentage;
+ final double percentMemory = maxMemory/percentDivisor;
+
+ //compute capacity
+ final int e1 = (int)(Math.log(percentMemory)/Math.log(2.0) + 0.5);
+ final int e2 = e1 - ("32".equals(vmBit)? 2: 3);
+ final int exponent = e2 < 0? 0: e2 > 30? 30: e2;
+ final int c = 1 << exponent;
+
+ if (LightWeightGSet.LOG.isDebugEnabled()) {
+ LOG.debug("Computing capacity for map " + mapName);
+ LOG.debug("VM type = " + vmBit + "-bit");
+ LOG.debug(percentage + "% max memory = "
+ + StringUtils.TraditionalBinaryPrefix.long2String(maxMemory, "B", 1));
+ LOG.debug("capacity = 2^" + exponent + " = " + c + " entries");
+ }
+ return c;
+ }
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/XMLUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/XMLUtils.java
index a023b878558..d036b1e24f2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/XMLUtils.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/XMLUtils.java
@@ -46,6 +46,140 @@ public class XMLUtils {
}
}
+ /**
+ * Exception that reflects a string that cannot be unmangled.
+ */
+ public static class UnmanglingError extends RuntimeException {
+ private static final long serialVersionUID = 1L;
+
+ public UnmanglingError(String str, Exception e) {
+ super(str, e);
+ }
+
+ public UnmanglingError(String str) {
+ super(str);
+ }
+ }
+
+
+ /**
+ * Given a code point, determine if it should be mangled before being
+ * represented in an XML document.
+ *
+ * Any code point that isn't valid in XML must be mangled.
+ * See http://en.wikipedia.org/wiki/Valid_characters_in_XML for a
+ * quick reference, or the w3 standard for the authoritative reference.
+ *
+ * @param cp The code point
+ * @return True if the code point should be mangled
+ */
+ private static boolean codePointMustBeMangled(int cp) {
+ if (cp < 0x20) {
+ return ((cp != 0x9) && (cp != 0xa) && (cp != 0xd));
+ } else if ((0xd7ff < cp) && (cp < 0xe000)) {
+ return true;
+ } else if ((cp == 0xfffe) || (cp == 0xffff)) {
+ return true;
+ } else if (cp == 0x5c) {
+ // we mangle backslash to simplify decoding... it's
+ // easier if backslashes always begin mangled sequences.
+ return true;
+ }
+ return false;
+ }
+
+ private static int NUM_SLASH_POSITIONS = 4;
+
+ private static String mangleCodePoint(int cp) {
+ return String.format("\\%0" + NUM_SLASH_POSITIONS + "x;", cp);
+ }
+
+ /**
+ * Mangle a string so that it can be represented in an XML document.
+ *
+ * There are three kinds of code points in XML:
+ * - Those that can be represented normally,
+ * - Those that have to be escaped (for example, & must be represented
+ * as &)
+ * - Those that cannot be represented at all in XML.
+ *
+ * The built-in SAX functions will handle the first two types for us just
+ * fine. However, sometimes we come across a code point of the third type.
+ * In this case, we have to mangle the string in order to represent it at
+ * all. We also mangle backslash to avoid confusing a backslash in the
+ * string with part our escape sequence.
+ *
+ * The encoding used here is as follows: an illegal code point is
+ * represented as '\ABCD;', where ABCD is the hexadecimal value of
+ * the code point.
+ *
+ * @param str The input string.
+ *
+ * @return The mangled string.
+ */
+ public static String mangleXmlString(String str) {
+ final StringBuilder bld = new StringBuilder();
+ final int length = str.length();
+ for (int offset = 0; offset < length; ) {
+ final int cp = str.codePointAt(offset);
+ final int len = Character.charCount(cp);
+ if (codePointMustBeMangled(cp)) {
+ bld.append(mangleCodePoint(cp));
+ } else {
+ for (int i = 0; i < len; i++) {
+ bld.append(str.charAt(offset + i));
+ }
+ }
+ offset += len;
+ }
+ return bld.toString();
+ }
+
+ /**
+ * Demangle a string from an XML document.
+ * See {@link #mangleXmlString(String)} for a description of the mangling
+ * format.
+ *
+ * @param str The string to be demangled.
+ *
+ * @return The unmangled string
+ * @throws UnmanglingError if the input is malformed.
+ */
+ public static String unmangleXmlString(String str)
+ throws UnmanglingError {
+ int slashPosition = -1;
+ String escapedCp = "";
+ StringBuilder bld = new StringBuilder();
+ for (int i = 0; i < str.length(); i++) {
+ char ch = str.charAt(i);
+ if ((slashPosition >= 0) && (slashPosition < NUM_SLASH_POSITIONS)) {
+ escapedCp += ch;
+ ++slashPosition;
+ } else if (slashPosition == NUM_SLASH_POSITIONS) {
+ if (ch != ';') {
+ throw new UnmanglingError("unterminated code point escape: " +
+ "expected semicolon at end.");
+ }
+ try {
+ bld.appendCodePoint(Integer.parseInt(escapedCp, 16));
+ } catch (NumberFormatException e) {
+ throw new UnmanglingError("error parsing unmangling escape code", e);
+ }
+ escapedCp = "";
+ slashPosition = -1;
+ } else if (ch == '\\') {
+ slashPosition = 0;
+ } else {
+ bld.append(ch);
+ }
+ }
+ if (slashPosition != -1) {
+ throw new UnmanglingError("unterminated code point escape: string " +
+ "broke off in the middle");
+ }
+ return bld.toString();
+ }
+
/**
* Add a SAX tag with a string inside.
*
@@ -56,7 +190,7 @@ public class XMLUtils {
public static void addSaxString(ContentHandler contentHandler,
String tag, String val) throws SAXException {
contentHandler.startElement("", "", tag, new AttributesImpl());
- char c[] = val.toString().toCharArray();
+ char c[] = mangleXmlString(val).toCharArray();
contentHandler.characters(c, 0, c.length);
contentHandler.endElement("", "", tag);
}
@@ -67,6 +201,8 @@ public class XMLUtils {
*/
static public class Stanza {
private TreeMap > subtrees;
+
+ /** The unmangled value of this stanza. */
private String value;
public Stanza() {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
index e04fb694bdb..5d1d33f9102 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.server.namenode.INodeId;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
@@ -244,7 +245,8 @@ public class JsonUtil {
final long mTime = (Long) m.get("modificationTime");
final long blockSize = (Long) m.get("blockSize");
final short replication = (short) (long) (Long) m.get("replication");
- final long fileId = (Long) m.get("fileId");
+ final long fileId = m.containsKey("fileId") ? (Long) m.get("fileId")
+ : INodeId.GRANDFATHER_INODE_ID;
return new HdfsFileStatus(len, type == PathType.DIRECTORY, replication,
blockSize, mTime, aTime, permission, owner, group,
symlink, DFSUtil.string2Bytes(localName), fileId);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
index 02c147ab1a6..6f33827fa74 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
@@ -29,7 +29,7 @@ import java.net.MalformedURLException;
import java.net.URI;
import java.net.URISyntaxException;
import java.net.URL;
-import java.util.ArrayList;
+import java.security.PrivilegedExceptionAction;
import java.util.Collection;
import java.util.List;
import java.util.Map;
@@ -62,15 +62,16 @@ import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
-import org.apache.hadoop.hdfs.server.common.JspHelper;
import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
import org.apache.hadoop.hdfs.web.resources.AccessTimeParam;
import org.apache.hadoop.hdfs.web.resources.BlockSizeParam;
import org.apache.hadoop.hdfs.web.resources.BufferSizeParam;
import org.apache.hadoop.hdfs.web.resources.ConcatSourcesParam;
import org.apache.hadoop.hdfs.web.resources.CreateParentParam;
+import org.apache.hadoop.hdfs.web.resources.DelegationParam;
import org.apache.hadoop.hdfs.web.resources.DeleteOpParam;
import org.apache.hadoop.hdfs.web.resources.DestinationParam;
+import org.apache.hadoop.hdfs.web.resources.DoAsParam;
import org.apache.hadoop.hdfs.web.resources.GetOpParam;
import org.apache.hadoop.hdfs.web.resources.GroupParam;
import org.apache.hadoop.hdfs.web.resources.HttpOpParam;
@@ -106,10 +107,11 @@ import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.security.token.TokenRenewer;
import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSelector;
import org.apache.hadoop.util.Progressable;
-import org.apache.hadoop.util.StringUtils;
import org.mortbay.util.ajax.JSON;
+import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Charsets;
+import com.google.common.collect.Lists;
/** A FileSystem for HDFS over the web. */
public class WebHdfsFileSystem extends FileSystem
@@ -132,7 +134,8 @@ public class WebHdfsFileSystem extends FileSystem
private DelegationTokenRenewer dtRenewer = null;
- private synchronized void addRenewAction(final WebHdfsFileSystem webhdfs) {
+ @VisibleForTesting
+ protected synchronized void addRenewAction(final WebHdfsFileSystem webhdfs) {
if (dtRenewer == null) {
dtRenewer = DelegationTokenRenewer.getInstance();
}
@@ -148,22 +151,15 @@ public class WebHdfsFileSystem extends FileSystem
return b;
}
- private final UserGroupInformation ugi;
+ private UserGroupInformation ugi;
private InetSocketAddress nnAddr;
private URI uri;
+ private boolean hasInitedToken;
private Token> delegationToken;
private final AuthenticatedURL.Token authToken = new AuthenticatedURL.Token();
private RetryPolicy retryPolicy = null;
private Path workingDir;
- {
- try {
- ugi = UserGroupInformation.getCurrentUser();
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- }
-
/**
* Return the protocol scheme for the FileSystem.
*
@@ -180,6 +176,7 @@ public class WebHdfsFileSystem extends FileSystem
) throws IOException {
super.initialize(uri, conf);
setConf(conf);
+ ugi = UserGroupInformation.getCurrentUser();
try {
this.uri = new URI(uri.getScheme(), uri.getAuthority(), null, null, null);
} catch (URISyntaxException e) {
@@ -204,24 +201,26 @@ public class WebHdfsFileSystem extends FileSystem
protected void initDelegationToken() throws IOException {
// look for webhdfs token, then try hdfs
Token> token = selectDelegationToken(ugi);
-
- //since we don't already have a token, go get one
- boolean createdToken = false;
- if (token == null) {
- token = getDelegationToken(null);
- createdToken = (token != null);
- }
-
- // security might be disabled
if (token != null) {
+ LOG.debug("Found existing DT for " + token.getService());
setDelegationToken(token);
- if (createdToken) {
+ hasInitedToken = true;
+ }
+ }
+
+ protected synchronized Token> getDelegationToken() throws IOException {
+ if (!hasInitedToken) {
+ //since we don't already have a token, go get one
+ Token> token = getDelegationToken(null);
+ // security might be disabled
+ if (token != null) {
+ setDelegationToken(token);
addRenewAction(this);
LOG.debug("Created new DT for " + token.getService());
- } else {
- LOG.debug("Found existing DT for " + token.getService());
}
+ hasInitedToken = true;
}
+ return delegationToken;
}
protected Token selectDelegationToken(
@@ -239,6 +238,11 @@ public class WebHdfsFileSystem extends FileSystem
public URI getUri() {
return this.uri;
}
+
+ @Override
+ protected URI canonicalizeUri(URI uri) {
+ return NetUtils.getCanonicalUri(uri, getDefaultPort());
+ }
/** @return the home directory. */
public static String getHomeDirectoryString(final UserGroupInformation ugi) {
@@ -365,16 +369,26 @@ public class WebHdfsFileSystem extends FileSystem
return url;
}
- private String addDt2Query(String query) throws IOException {
- if (UserGroupInformation.isSecurityEnabled()) {
- synchronized (this) {
- if (delegationToken != null) {
- final String encoded = delegationToken.encodeToUrlString();
- return query + JspHelper.getDelegationTokenUrlParam(encoded);
- } // else we are talking to an insecure cluster
- }
+ Param,?>[] getAuthParameters(final HttpOpParam.Op op) throws IOException {
+ List> authParams = Lists.newArrayList();
+ // Skip adding delegation token for token operations because these
+ // operations require authentication.
+ Token> token = null;
+ if (UserGroupInformation.isSecurityEnabled() && !op.getRequireAuth()) {
+ token = getDelegationToken();
}
- return query;
+ if (token != null) {
+ authParams.add(new DelegationParam(token.encodeToUrlString()));
+ } else {
+ UserGroupInformation userUgi = ugi;
+ UserGroupInformation realUgi = userUgi.getRealUser();
+ if (realUgi != null) { // proxy user
+ authParams.add(new DoAsParam(userUgi.getShortUserName()));
+ userUgi = realUgi;
+ }
+ authParams.add(new UserParam(userUgi.getShortUserName()));
+ }
+ return authParams.toArray(new Param,?>[0]);
}
URL toUrl(final HttpOpParam.Op op, final Path fspath,
@@ -383,34 +397,15 @@ public class WebHdfsFileSystem extends FileSystem
final String path = PATH_PREFIX
+ (fspath == null? "/": makeQualified(fspath).toUri().getPath());
final String query = op.toQueryString()
- + '&' + new UserParam(ugi)
+ + Param.toSortedString("&", getAuthParameters(op))
+ Param.toSortedString("&", parameters);
- final URL url;
- if (op == PutOpParam.Op.RENEWDELEGATIONTOKEN
- || op == GetOpParam.Op.GETDELEGATIONTOKEN) {
- // Skip adding delegation token for getting or renewing delegation token,
- // because these operations require kerberos authentication.
- url = getNamenodeURL(path, query);
- } else {
- url = getNamenodeURL(path, addDt2Query(query));
- }
+ final URL url = getNamenodeURL(path, query);
if (LOG.isTraceEnabled()) {
LOG.trace("url=" + url);
}
return url;
}
- private HttpURLConnection getHttpUrlConnection(URL url)
- throws IOException, AuthenticationException {
- final HttpURLConnection conn;
- if (ugi.hasKerberosCredentials()) {
- conn = new AuthenticatedURL(AUTH).openConnection(url, authToken);
- } else {
- conn = (HttpURLConnection)url.openConnection();
- }
- return conn;
- }
-
/**
* Run a http operation.
* Connect to the http server, validate response, and obtain the JSON output.
@@ -455,6 +450,48 @@ public class WebHdfsFileSystem extends FileSystem
this.conn = conn;
}
+ private HttpURLConnection getHttpUrlConnection(final URL url)
+ throws IOException, AuthenticationException {
+ UserGroupInformation connectUgi = ugi.getRealUser();
+ if (connectUgi == null) {
+ connectUgi = ugi;
+ }
+ try {
+ return connectUgi.doAs(
+ new PrivilegedExceptionAction() {
+ @Override
+ public HttpURLConnection run() throws IOException {
+ return openHttpUrlConnection(url);
+ }
+ });
+ } catch (IOException ioe) {
+ Throwable cause = ioe.getCause();
+ if (cause != null && cause instanceof AuthenticationException) {
+ throw (AuthenticationException)cause;
+ }
+ throw ioe;
+ } catch (InterruptedException e) {
+ throw new IOException(e);
+ }
+ }
+
+ private HttpURLConnection openHttpUrlConnection(final URL url)
+ throws IOException {
+ final HttpURLConnection conn;
+ try {
+ if (op.getRequireAuth()) {
+ LOG.debug("open AuthenticatedURL connection");
+ conn = new AuthenticatedURL(AUTH).openConnection(url, authToken);
+ } else {
+ LOG.debug("open URL connection");
+ conn = (HttpURLConnection)url.openConnection();
+ }
+ } catch (AuthenticationException e) {
+ throw new IOException(e);
+ }
+ return conn;
+ }
+
private void init() throws IOException {
checkRetry = !redirected;
try {
@@ -721,17 +758,10 @@ public class WebHdfsFileSystem extends FileSystem
}
@Override
- public void concat(final Path trg, final Path [] psrcs) throws IOException {
+ public void concat(final Path trg, final Path [] srcs) throws IOException {
statistics.incrementWriteOps(1);
final HttpOpParam.Op op = PostOpParam.Op.CONCAT;
- List strPaths = new ArrayList(psrcs.length);
- for(Path psrc : psrcs) {
- strPaths.add(psrc.toUri().getPath());
- }
-
- String srcs = StringUtils.join(",", strPaths);
-
ConcatSourcesParam param = new ConcatSourcesParam(srcs);
run(op, trg, param);
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ConcatSourcesParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ConcatSourcesParam.java
index e6afbe3e4ec..b68c5f5b58e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ConcatSourcesParam.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/ConcatSourcesParam.java
@@ -18,15 +18,28 @@
package org.apache.hadoop.hdfs.web.resources;
+import org.apache.hadoop.fs.Path;
+
/** The concat source paths parameter. */
public class ConcatSourcesParam extends StringParam {
/** Parameter name. */
public static final String NAME = "sources";
- public static final String DEFAULT = NULL;
+ public static final String DEFAULT = "";
private static final Domain DOMAIN = new Domain(NAME, null);
+ private static String paths2String(Path[] paths) {
+ if (paths == null || paths.length == 0) {
+ return "";
+ }
+ final StringBuilder b = new StringBuilder(paths[0].toUri().getPath());
+ for(int i = 1; i < paths.length; i++) {
+ b.append(',').append(paths[i].toUri().getPath());
+ }
+ return b.toString();
+ }
+
/**
* Constructor.
* @param str a string representation of the parameter value.
@@ -35,6 +48,10 @@ public class ConcatSourcesParam extends StringParam {
super(DOMAIN, str);
}
+ public ConcatSourcesParam(Path[] paths) {
+ this(paths2String(paths));
+ }
+
@Override
public String getName() {
return NAME;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DeleteOpParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DeleteOpParam.java
index a82b8a72c8e..710e2e8992e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DeleteOpParam.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/DeleteOpParam.java
@@ -38,6 +38,11 @@ public class DeleteOpParam extends HttpOpParam {
return HttpOpParam.Type.DELETE;
}
+ @Override
+ public boolean getRequireAuth() {
+ return false;
+ }
+
@Override
public boolean getDoOutput() {
return false;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
index eaf23431450..916fe553ac4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
@@ -31,7 +31,7 @@ public class GetOpParam extends HttpOpParam {
GETFILECHECKSUM(true, HttpURLConnection.HTTP_OK),
GETHOMEDIRECTORY(false, HttpURLConnection.HTTP_OK),
- GETDELEGATIONTOKEN(false, HttpURLConnection.HTTP_OK),
+ GETDELEGATIONTOKEN(false, HttpURLConnection.HTTP_OK, true),
/** GET_BLOCK_LOCATIONS is a private unstable op. */
GET_BLOCK_LOCATIONS(false, HttpURLConnection.HTTP_OK),
@@ -40,16 +40,28 @@ public class GetOpParam extends HttpOpParam {
final boolean redirect;
final int expectedHttpResponseCode;
+ final boolean requireAuth;
Op(final boolean redirect, final int expectedHttpResponseCode) {
+ this(redirect, expectedHttpResponseCode, false);
+ }
+
+ Op(final boolean redirect, final int expectedHttpResponseCode,
+ final boolean requireAuth) {
this.redirect = redirect;
this.expectedHttpResponseCode = expectedHttpResponseCode;
+ this.requireAuth = requireAuth;
}
@Override
public HttpOpParam.Type getType() {
return HttpOpParam.Type.GET;
}
+
+ @Override
+ public boolean getRequireAuth() {
+ return requireAuth;
+ }
@Override
public boolean getDoOutput() {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/HttpOpParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/HttpOpParam.java
index 1d029ec65cd..2237fb64813 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/HttpOpParam.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/HttpOpParam.java
@@ -43,6 +43,9 @@ public abstract class HttpOpParam & HttpOpParam.Op>
/** @return the Http operation type. */
public Type getType();
+ /** @return true if the operation cannot use a token */
+ public boolean getRequireAuth();
+
/** @return true if the operation will do output. */
public boolean getDoOutput();
@@ -92,6 +95,11 @@ public abstract class HttpOpParam & HttpOpParam.Op>
return op.getType();
}
+ @Override
+ public boolean getRequireAuth() {
+ return op.getRequireAuth();
+ }
+
@Override
public boolean getDoOutput() {
return op.getDoOutput();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/PostOpParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/PostOpParam.java
index 4bb5673ab10..54034f0e818 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/PostOpParam.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/PostOpParam.java
@@ -41,6 +41,11 @@ public class PostOpParam extends HttpOpParam {
public Type getType() {
return Type.POST;
}
+
+ @Override
+ public boolean getRequireAuth() {
+ return false;
+ }
@Override
public boolean getDoOutput() {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/PutOpParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/PutOpParam.java
index 77bad214225..6ee84c4ccf4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/PutOpParam.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/PutOpParam.java
@@ -34,23 +34,35 @@ public class PutOpParam extends HttpOpParam {
SETPERMISSION(false, HttpURLConnection.HTTP_OK),
SETTIMES(false, HttpURLConnection.HTTP_OK),
- RENEWDELEGATIONTOKEN(false, HttpURLConnection.HTTP_OK),
- CANCELDELEGATIONTOKEN(false, HttpURLConnection.HTTP_OK),
+ RENEWDELEGATIONTOKEN(false, HttpURLConnection.HTTP_OK, true),
+ CANCELDELEGATIONTOKEN(false, HttpURLConnection.HTTP_OK, true),
NULL(false, HttpURLConnection.HTTP_NOT_IMPLEMENTED);
final boolean doOutputAndRedirect;
final int expectedHttpResponseCode;
+ final boolean requireAuth;
Op(final boolean doOutputAndRedirect, final int expectedHttpResponseCode) {
+ this(doOutputAndRedirect, expectedHttpResponseCode, false);
+ }
+
+ Op(final boolean doOutputAndRedirect, final int expectedHttpResponseCode,
+ final boolean requireAuth) {
this.doOutputAndRedirect = doOutputAndRedirect;
this.expectedHttpResponseCode = expectedHttpResponseCode;
+ this.requireAuth = requireAuth;
}
@Override
public HttpOpParam.Type getType() {
return HttpOpParam.Type.PUT;
}
+
+ @Override
+ public boolean getRequireAuth() {
+ return requireAuth;
+ }
@Override
public boolean getDoOutput() {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/overview.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/overview.html
index c0cafc64082..759c093aa59 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/overview.html
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/overview.html
@@ -60,9 +60,7 @@ that process vast amounts of data. Here's what makes Hadoop especially useful:
Hadoop was been demonstrated on GNU/Linux clusters with 2000 nodes.
- Win32 is supported as a development platform. Distributed operation
- has not been well tested on Win32, so this is not a production
- platform.
+ Windows is also a supported platform.
@@ -84,15 +82,6 @@ that process vast amounts of data. Here's what makes Hadoop especially useful:
-Additional requirements for Windows
-
-
- -
- Cygwin - Required for shell support in
- addition to the required software above.
-
-
-
Installing Required Software
If your platform does not have the required software listed above, you
@@ -104,13 +93,6 @@ $ sudo apt-get install ssh
$ sudo apt-get install rsync
-On Windows, if you did not install the required software when you
-installed cygwin, start the cygwin installer and select the packages:
-
- - openssh - the "Net" category
- - rsync - the "Net" category
-
-
Getting Started
First, you need to get a copy of the Hadoop code.
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/native/tests/test-libhdfs.sh b/hadoop-hdfs-project/hadoop-hdfs/src/main/native/tests/test-libhdfs.sh
index 51bb15f45dc..3407e9cf8e2 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/native/tests/test-libhdfs.sh
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/native/tests/test-libhdfs.sh
@@ -82,7 +82,7 @@ unset IFS
findlibjvm () {
javabasedir=$JAVA_HOME
case $OS_NAME in
- cygwin* | mingw* | pw23* )
+ mingw* | pw23* )
lib_jvm_dir=`find $javabasedir -follow \( \
\( -name client -type d -prune \) -o \
\( -name "jvm.dll" -exec dirname {} \; \) \) 2> /dev/null | tr "\n" " "`
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 5380c060028..79e8476ac91 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -47,7 +47,7 @@
RPC address that handles all clients requests. In the case of HA/Federation where multiple namenodes exist,
the name service id is added to the name e.g. dfs.namenode.rpc-address.ns1
dfs.namenode.rpc-address.EXAMPLENAMESERVICE
- The value of this property will take the form of hdfs://nn-host1:rpc-port.
+ The value of this property will take the form of nn-host1:rpc-port.
@@ -59,7 +59,7 @@
connecting to this address if it is configured. In the case of HA/Federation where multiple namenodes exist,
the name service id is added to the name e.g. dfs.namenode.servicerpc-address.ns1
dfs.namenode.rpc-address.EXAMPLENAMESERVICE
- The value of this property will take the form of hdfs://nn-host1:rpc-port.
+ The value of this property will take the form of nn-host1:rpc-port.
If the value of this property is unset the value of dfs.namenode.rpc-address will be used as the default.
@@ -594,6 +594,17 @@
Packet size for clients to write
+
+ dfs.client.write.exclude.nodes.cache.expiry.interval.millis
+ 600000
+ The maximum period to keep a DN in the excluded nodes list
+ at a client. After this period, in milliseconds, the previously excluded node(s) will
+ be removed automatically from the cache and will be considered good for block allocations
+ again. Useful to lower or raise in situations where you keep a file open for very long
+ periods (such as a Write-Ahead-Log (WAL) file) to make the writer tolerant to cluster maintenance
+ restarts. Defaults to 10 minutes.
+
+
dfs.namenode.checkpoint.dir
file://${hadoop.tmp.dir}/dfs/namesecondary
@@ -624,7 +635,7 @@
dfs.namenode.checkpoint.txns
- 40000
+ 1000000
The Secondary NameNode or CheckpointNode will create a checkpoint
of the namespace every 'dfs.namenode.checkpoint.txns' transactions, regardless
of whether 'dfs.namenode.checkpoint.period' has expired.
@@ -640,6 +651,15 @@
+
+ dfs.namenode.checkpoint.max-retries
+ 3
+ The SecondaryNameNode retries failed checkpointing. If the
+ failure occurs while loading fsimage or replaying edits, the number of
+ retries is limited by this variable.
+
+
+
dfs.namenode.num.checkpoints.retained
2
@@ -724,12 +744,29 @@
+
+ dfs.image.transfer.timeout
+ 600000
+
+ Timeout for image transfer in milliseconds. This timeout and the related
+ dfs.image.transfer.bandwidthPerSec parameter should be configured such
+ that normal image transfer can complete within the timeout.
+ This timeout prevents client hangs when the sender fails during
+ image transfer, which is particularly important during checkpointing.
+ Note that this timeout applies to the entirety of image transfer, and
+ is not a socket timeout.
+
+
+
dfs.image.transfer.bandwidthPerSec
0
- Specifies the maximum amount of bandwidth that can be utilized for image
- transfer in term of the number of bytes per second.
+ Maximum bandwidth used for image transfer in bytes per second.
+ This can help keep normal namenode operations responsive during
+ checkpointing. The maximum bandwidth and timeout in
+ dfs.image.transfer.timeout should be set such that normal image
+ transfers can complete successfully.
A default value of 0 indicates that throttling is disabled.
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HdfsUserGuide.apt.vm b/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HdfsUserGuide.apt.vm
index 82f1046eb47..b9d1c637a32 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HdfsUserGuide.apt.vm
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HdfsUserGuide.apt.vm
@@ -193,7 +193,7 @@ HDFS Users Guide
* <<>>, set to 1 hour by default, specifies
the maximum delay between two consecutive checkpoints, and
- * <<>>, set to 40000 default, defines the
+ * <<>>, set to 1 million by default, defines the
number of uncheckpointed transactions on the NameNode which will
force an urgent checkpoint, even if the checkpoint period has not
been reached.
@@ -232,7 +232,7 @@ HDFS Users Guide
* <<>>, set to 1 hour by default, specifies
the maximum delay between two consecutive checkpoints
- * <<>>, set to 40000 default, defines the
+ * <<>>, set to 1 million by default, defines the
number of uncheckpointed transactions on the NameNode which will
force an urgent checkpoint, even if the checkpoint period has not
been reached.
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/WebHDFS.apt.vm b/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/WebHDFS.apt.vm
index 90f8dabce71..7735f8dafed 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/WebHDFS.apt.vm
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/WebHDFS.apt.vm
@@ -109,7 +109,7 @@ WebHDFS REST API
* {{{Append to a File}<<>>}}
(see {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.append)
- * {{{Concat File(s)}<<>>}}
+ * {{{Concatenate Files}<<>>}}
(see {{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.concat)
* HTTP DELETE
@@ -307,7 +307,7 @@ Content-Length: 0
* Submit a HTTP POST request.
+---------------------------------
-curl -i -X POST "http://:/webhdfs/v1/?op=CONCAT&sources="
+curl -i -X POST "http://:/webhdfs/v1/?op=CONCAT&sources="
+---------------------------------
The client receives a response with zero content length:
@@ -319,10 +319,6 @@ Content-Length: 0
[]
- This REST API call is available as of Hadoop version 2.0.3.
- Please note that is a comma seperated list of absolute paths.
- (Example: sources=/test/file1,/test/file2,/test/file3)
-
See also:
{{{Sources}<<>>}},
{{{../../api/org/apache/hadoop/fs/FileSystem.html}FileSystem}}.concat
@@ -1761,7 +1757,7 @@ var tokenProperties =
*----------------+-------------------------------------------------------------------+
|| Name | <<>> |
*----------------+-------------------------------------------------------------------+
-|| Description | The comma seperated absolute paths used for concatenation. |
+|| Description | A list of source paths. |
*----------------+-------------------------------------------------------------------+
|| Type | String |
*----------------+-------------------------------------------------------------------+
@@ -1769,12 +1765,9 @@ var tokenProperties =
*----------------+-------------------------------------------------------------------+
|| Valid Values | A list of comma seperated absolute FileSystem paths without scheme and authority. |
*----------------+-------------------------------------------------------------------+
-|| Syntax | See the note in {{Delegation}}. |
+|| Syntax | Any string. |
*----------------+-------------------------------------------------------------------+
- <> that sources are absolute FileSystem paths.
-
-
See also:
{{{Concat File(s)}<<>>}}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/LogVerificationAppender.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/LogVerificationAppender.java
new file mode 100644
index 00000000000..d6698b88c4b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/LogVerificationAppender.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.log4j.AppenderSkeleton;
+import org.apache.log4j.spi.LoggingEvent;
+import org.apache.log4j.spi.ThrowableInformation;
+
+/**
+ * Used to verify that certain exceptions or messages are present in log output.
+ */
+public class LogVerificationAppender extends AppenderSkeleton {
+ private final List log = new ArrayList();
+
+ @Override
+ public boolean requiresLayout() {
+ return false;
+ }
+
+ @Override
+ protected void append(final LoggingEvent loggingEvent) {
+ log.add(loggingEvent);
+ }
+
+ @Override
+ public void close() {
+ }
+
+ public List getLog() {
+ return new ArrayList(log);
+ }
+
+ public int countExceptionsWithMessage(final String text) {
+ int count = 0;
+ for (LoggingEvent e: getLog()) {
+ ThrowableInformation t = e.getThrowableInformation();
+ if (t != null) {
+ String m = t.getThrowable().getMessage();
+ if (m.contains(text)) {
+ count++;
+ }
+ }
+ }
+ return count;
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientExcludedNodes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientExcludedNodes.java
index dccc82f1a42..3574323d545 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientExcludedNodes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientExcludedNodes.java
@@ -21,36 +21,134 @@ import static org.junit.Assert.fail;
import java.io.IOException;
import java.io.OutputStream;
+import java.util.List;
+
+import junit.framework.Assert;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
+import org.apache.hadoop.util.ThreadUtil;
+
+import org.junit.After;
+import org.junit.Before;
import org.junit.Test;
/**
- * These tests make sure that DFSClient retries fetching data from DFS
- * properly in case of errors.
+ * These tests make sure that DFSClient excludes writing data to
+ * a DN properly in case of errors.
*/
public class TestDFSClientExcludedNodes {
- @Test
+ private MiniDFSCluster cluster;
+ private Configuration conf;
+
+ @Before
+ public void setUp() {
+ cluster = null;
+ conf = new HdfsConfiguration();
+ }
+
+ @After
+ public void tearDown() {
+ if (cluster != null) {
+ cluster.shutdown();
+ }
+ }
+
+ @Test(timeout=60000)
public void testExcludedNodes() throws IOException {
- Configuration conf = new HdfsConfiguration();
- MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
+ cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
FileSystem fs = cluster.getFileSystem();
Path filePath = new Path("/testExcludedNodes");
// kill a datanode
cluster.stopDataNode(AppendTestUtil.nextInt(3));
- OutputStream out = fs.create(filePath, true, 4096);
+ OutputStream out = fs.create(
+ filePath,
+ true,
+ 4096,
+ (short) 3,
+ fs.getDefaultBlockSize(filePath)
+ );
out.write(20);
try {
out.close();
} catch (Exception e) {
- fail("DataNode failure should not result in a block abort: \n" + e.getMessage());
+ fail("Single DN failure should not result in a block abort: \n" +
+ e.getMessage());
+ }
+ }
+
+ @Test(timeout=60000)
+ public void testExcludedNodesForgiveness() throws IOException {
+ // Forgive nodes in under 2.5s for this test case.
+ conf.setLong(
+ DFSConfigKeys.DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL,
+ 2500);
+ // We'll be using a 512 bytes block size just for tests
+ // so making sure the checksum bytes too match it.
+ conf.setInt("io.bytes.per.checksum", 512);
+ cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
+ List props = cluster.dataNodes;
+ FileSystem fs = cluster.getFileSystem();
+ Path filePath = new Path("/testForgivingExcludedNodes");
+
+ // 256 bytes data chunk for writes
+ byte[] bytes = new byte[256];
+ for (int index=0; index nsList = (List) f.get(null);
+
+ NameService ns = nsList.get(0);
+ Log log = LogFactory.getLog("NameServiceSpy");
+
+ ns = Mockito.mock(NameService.class,
+ new GenericTestUtils.DelegateAnswer(log, ns));
+ nsList.set(0, ns);
+ return ns;
+ } catch (Throwable t) {
+ LOG.info("Unable to spy on DNS. Skipping test.", t);
+ // In case the JDK we're testing on doesn't work like Sun's, just
+ // skip the test.
+ Assume.assumeNoException(t);
+ throw new RuntimeException(t);
+ }
+ }
+
+ /**
+ * Test that the client doesn't ever try to DNS-resolve the logical URI.
+ * Regression test for HADOOP-9150.
+ */
+ @Test
+ public void testDoesntDnsResolveLogicalURI() throws Exception {
+ NameService spyNS = spyOnNameService();
+
+ FileSystem fs = HATestUtil.configureFailoverFs(cluster, conf);
+ String logicalHost = fs.getUri().getHost();
+ Path qualifiedRoot = fs.makeQualified(new Path("/"));
+
+ // Make a few calls against the filesystem.
+ fs.getCanonicalServiceName();
+ fs.listStatus(qualifiedRoot);
+
+ // Ensure that the logical hostname was never resolved.
+ Mockito.verify(spyNS, Mockito.never()).lookupAllHostAddr(Mockito.eq(logicalHost));
+ }
+
+ /**
+ * Same test as above, but for FileContext.
+ */
+ @Test
+ public void testFileContextDoesntDnsResolveLogicalURI() throws Exception {
+ NameService spyNS = spyOnNameService();
+ FileSystem fs = HATestUtil.configureFailoverFs(cluster, conf);
+ String logicalHost = fs.getUri().getHost();
+ Configuration haClientConf = fs.getConf();
+
+ FileContext fc = FileContext.getFileContext(haClientConf);
+ Path root = new Path("/");
+ fc.listStatus(root);
+ fc.listStatus(fc.makeQualified(root));
+ fc.getDefaultFileSystem().getCanonicalServiceName();
+
+ // Ensure that the logical hostname was never resolved.
+ Mockito.verify(spyNS, Mockito.never()).lookupAllHostAddr(Mockito.eq(logicalHost));
+ }
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
index 4b26e77d809..cf62cd5257c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
@@ -35,6 +35,7 @@ import java.util.Arrays;
import java.util.List;
import java.util.Random;
import java.util.Scanner;
+import java.util.concurrent.atomic.AtomicInteger;
import java.util.zip.DeflaterOutputStream;
import java.util.zip.GZIPOutputStream;
@@ -68,7 +69,8 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERV
*/
public class TestDFSShell {
private static final Log LOG = LogFactory.getLog(TestDFSShell.class);
-
+ private static AtomicInteger counter = new AtomicInteger();
+
static final String TEST_ROOT_DIR =
new Path(System.getProperty("test.build.data","/tmp"))
.toString().replace(' ', '+');
@@ -103,7 +105,7 @@ public class TestDFSShell {
System.out.println(Thread.currentThread().getStackTrace()[2] + " " + s);
}
- @Test
+ @Test (timeout = 30000)
public void testZeroSizeFile() throws IOException {
Configuration conf = new HdfsConfiguration();
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
@@ -146,7 +148,7 @@ public class TestDFSShell {
}
}
- @Test
+ @Test (timeout = 30000)
public void testRecrusiveRm() throws IOException {
Configuration conf = new HdfsConfiguration();
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
@@ -172,7 +174,7 @@ public class TestDFSShell {
}
}
- @Test
+ @Test (timeout = 30000)
public void testDu() throws IOException {
Configuration conf = new HdfsConfiguration();
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
@@ -222,7 +224,8 @@ public class TestDFSShell {
}
}
- @Test
+
+ @Test (timeout = 30000)
public void testPut() throws IOException {
Configuration conf = new HdfsConfiguration();
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
@@ -321,7 +324,7 @@ public class TestDFSShell {
/** check command error outputs and exit statuses. */
- @Test
+ @Test (timeout = 30000)
public void testErrOutPut() throws Exception {
Configuration conf = new HdfsConfiguration();
MiniDFSCluster cluster = null;
@@ -471,7 +474,7 @@ public class TestDFSShell {
}
}
- @Test
+ @Test (timeout = 30000)
public void testURIPaths() throws Exception {
Configuration srcConf = new HdfsConfiguration();
Configuration dstConf = new HdfsConfiguration();
@@ -511,7 +514,7 @@ public class TestDFSShell {
createLocalFile(furi);
argv = new String[3];
argv[0] = "-put";
- argv[1] = furi.toString();
+ argv[1] = furi.toURI().toString();
argv[2] = dstFs.getUri().toString() + "/furi";
ret = ToolRunner.run(shell, argv);
assertEquals(" put is working ", 0, ret);
@@ -564,7 +567,7 @@ public class TestDFSShell {
}
}
- @Test
+ @Test (timeout = 30000)
public void testText() throws Exception {
Configuration conf = new HdfsConfiguration();
MiniDFSCluster cluster = null;
@@ -680,7 +683,7 @@ public class TestDFSShell {
}
}
- @Test
+ @Test (timeout = 30000)
public void testCopyToLocal() throws IOException {
Configuration conf = new HdfsConfiguration();
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
@@ -778,7 +781,7 @@ public class TestDFSShell {
return path;
}
- @Test
+ @Test (timeout = 30000)
public void testCount() throws Exception {
Configuration conf = new HdfsConfiguration();
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
@@ -866,52 +869,59 @@ public class TestDFSShell {
shell.setConf(conf);
try {
- //first make dir
- Path dir = new Path(chmodDir);
- fs.delete(dir, true);
- fs.mkdirs(dir);
+ //first make dir
+ Path dir = new Path(chmodDir);
+ fs.delete(dir, true);
+ fs.mkdirs(dir);
- confirmPermissionChange(/* Setting */ "u+rwx,g=rw,o-rwx",
+ confirmPermissionChange(/* Setting */ "u+rwx,g=rw,o-rwx",
/* Should give */ "rwxrw----", fs, shell, dir);
-
- //create an empty file
- Path file = new Path(chmodDir, "file");
- TestDFSShell.writeFile(fs, file);
- //test octal mode
- confirmPermissionChange( "644", "rw-r--r--", fs, shell, file);
+ //create an empty file
+ Path file = new Path(chmodDir, "file");
+ TestDFSShell.writeFile(fs, file);
- //test recursive
- runCmd(shell, "-chmod", "-R", "a+rwX", chmodDir);
- assertEquals("rwxrwxrwx",
- fs.getFileStatus(dir).getPermission().toString());
- assertEquals("rw-rw-rw-",
- fs.getFileStatus(file).getPermission().toString());
+ //test octal mode
+ confirmPermissionChange("644", "rw-r--r--", fs, shell, file);
- // test sticky bit on directories
- Path dir2 = new Path(dir, "stickybit" );
- fs.mkdirs(dir2 );
- LOG.info("Testing sticky bit on: " + dir2);
- LOG.info("Sticky bit directory initial mode: " +
- fs.getFileStatus(dir2).getPermission());
-
- confirmPermissionChange("u=rwx,g=rx,o=rx", "rwxr-xr-x", fs, shell, dir2);
-
- confirmPermissionChange("+t", "rwxr-xr-t", fs, shell, dir2);
+ //test recursive
+ runCmd(shell, "-chmod", "-R", "a+rwX", chmodDir);
+ assertEquals("rwxrwxrwx",
+ fs.getFileStatus(dir).getPermission().toString());
+ assertEquals("rw-rw-rw-",
+ fs.getFileStatus(file).getPermission().toString());
- confirmPermissionChange("-t", "rwxr-xr-x", fs, shell, dir2);
+ // Skip "sticky bit" tests on Windows.
+ //
+ if (!Path.WINDOWS) {
+ // test sticky bit on directories
+ Path dir2 = new Path(dir, "stickybit");
+ fs.mkdirs(dir2);
+ LOG.info("Testing sticky bit on: " + dir2);
+ LOG.info("Sticky bit directory initial mode: " +
+ fs.getFileStatus(dir2).getPermission());
- confirmPermissionChange("=t", "--------T", fs, shell, dir2);
+ confirmPermissionChange("u=rwx,g=rx,o=rx", "rwxr-xr-x", fs, shell, dir2);
- confirmPermissionChange("0000", "---------", fs, shell, dir2);
+ confirmPermissionChange("+t", "rwxr-xr-t", fs, shell, dir2);
- confirmPermissionChange("1666", "rw-rw-rwT", fs, shell, dir2);
+ confirmPermissionChange("-t", "rwxr-xr-x", fs, shell, dir2);
+
+ confirmPermissionChange("=t", "--------T", fs, shell, dir2);
+
+ confirmPermissionChange("0000", "---------", fs, shell, dir2);
+
+ confirmPermissionChange("1666", "rw-rw-rwT", fs, shell, dir2);
+
+ confirmPermissionChange("777", "rwxrwxrwt", fs, shell, dir2);
+
+ fs.delete(dir2, true);
+ } else {
+ LOG.info("Skipped sticky bit tests on Windows");
+ }
+
+ fs.delete(dir, true);
- confirmPermissionChange("777", "rwxrwxrwt", fs, shell, dir2);
-
- fs.delete(dir2, true);
- fs.delete(dir, true);
-
} finally {
try {
fs.close();
@@ -945,7 +955,7 @@ public class TestDFSShell {
}
}
- @Test
+ @Test (timeout = 30000)
public void testFilePermissions() throws IOException {
Configuration conf = new HdfsConfiguration();
@@ -1011,7 +1021,7 @@ public class TestDFSShell {
/**
* Tests various options of DFSShell.
*/
- @Test
+ @Test (timeout = 120000)
public void testDFSShell() throws IOException {
Configuration conf = new HdfsConfiguration();
/* This tests some properties of ChecksumFileSystem as well.
@@ -1391,7 +1401,7 @@ public class TestDFSShell {
String run(int exitcode, String... options) throws IOException;
}
- @Test
+ @Test (timeout = 30000)
public void testRemoteException() throws Exception {
UserGroupInformation tmpUGI =
UserGroupInformation.createUserForTesting("tmpname", new String[] {"mygroup"});
@@ -1435,73 +1445,96 @@ public class TestDFSShell {
}
}
- @Test
+ @Test (timeout = 30000)
public void testGet() throws IOException {
DFSTestUtil.setLogLevel2All(FSInputChecker.LOG);
+
+ final String fname = "testGet.txt";
+ Path root = new Path("/test/get");
+ final Path remotef = new Path(root, fname);
final Configuration conf = new HdfsConfiguration();
- // Race can happen here: block scanner is reading the file when test tries
- // to corrupt the test file, which will fail the test on Windows platform.
- // Disable block scanner to avoid this race.
- conf.setInt(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1);
-
- MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
- DistributedFileSystem dfs = (DistributedFileSystem)cluster.getFileSystem();
+
+ TestGetRunner runner = new TestGetRunner() {
+ private int count = 0;
+ private FsShell shell = new FsShell(conf);
+
+ public String run(int exitcode, String... options) throws IOException {
+ String dst = TEST_ROOT_DIR + "/" + fname+ ++count;
+ String[] args = new String[options.length + 3];
+ args[0] = "-get";
+ args[args.length - 2] = remotef.toString();
+ args[args.length - 1] = dst;
+ for(int i = 0; i < options.length; i++) {
+ args[i + 1] = options[i];
+ }
+ show("args=" + Arrays.asList(args));
+
+ try {
+ assertEquals(exitcode, shell.run(args));
+ } catch (Exception e) {
+ assertTrue(StringUtils.stringifyException(e), false);
+ }
+ return exitcode == 0? DFSTestUtil.readFile(new File(dst)): null;
+ }
+ };
+
+ File localf = createLocalFile(new File(TEST_ROOT_DIR, fname));
+ MiniDFSCluster cluster = null;
+ DistributedFileSystem dfs = null;
try {
- final String fname = "testGet.txt";
- final File localf = createLocalFile(new File(TEST_ROOT_DIR, fname));
- final String localfcontent = DFSTestUtil.readFile(localf);
- final Path root = mkdir(dfs, new Path("/test/get"));
- final Path remotef = new Path(root, fname);
+ cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).format(true)
+ .build();
+ dfs = (DistributedFileSystem)cluster.getFileSystem();
+
+ mkdir(dfs, root);
dfs.copyFromLocalFile(false, false, new Path(localf.getPath()), remotef);
-
- final FsShell shell = new FsShell();
- shell.setConf(conf);
- TestGetRunner runner = new TestGetRunner() {
- private int count = 0;
-
- @Override
- public String run(int exitcode, String... options) throws IOException {
- String dst = TEST_ROOT_DIR + "/" + fname+ ++count;
- String[] args = new String[options.length + 3];
- args[0] = "-get";
- args[args.length - 2] = remotef.toString();
- args[args.length - 1] = dst;
- for(int i = 0; i < options.length; i++) {
- args[i + 1] = options[i];
- }
- show("args=" + Arrays.asList(args));
-
- try {
- assertEquals(exitcode, shell.run(args));
- } catch (Exception e) {
- assertTrue(StringUtils.stringifyException(e), false);
- }
- return exitcode == 0? DFSTestUtil.readFile(new File(dst)): null;
- }
- };
+ String localfcontent = DFSTestUtil.readFile(localf);
assertEquals(localfcontent, runner.run(0));
assertEquals(localfcontent, runner.run(0, "-ignoreCrc"));
- //find and modify the block files
+ // find block files to modify later
List files = getBlockFiles(cluster);
+
+ // Shut down cluster and then corrupt the block files by overwriting a
+ // portion with junk data. We must shut down the cluster so that threads
+ // in the data node do not hold locks on the block files while we try to
+ // write into them. Particularly on Windows, the data node's use of the
+ // FileChannel.transferTo method can cause block files to be memory mapped
+ // in read-only mode during the transfer to a client, and this causes a
+ // locking conflict. The call to shutdown the cluster blocks until all
+ // DataXceiver threads exit, preventing this problem.
+ dfs.close();
+ cluster.shutdown();
+
show("files=" + files);
corrupt(files);
+ // Start the cluster again, but do not reformat, so prior files remain.
+ cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).format(false)
+ .build();
+ dfs = (DistributedFileSystem)cluster.getFileSystem();
+
assertEquals(null, runner.run(1));
String corruptedcontent = runner.run(0, "-ignoreCrc");
assertEquals(localfcontent.substring(1), corruptedcontent.substring(1));
assertEquals(localfcontent.charAt(0)+1, corruptedcontent.charAt(0));
-
- localf.delete();
} finally {
- try {dfs.close();} catch (Exception e) {}
- cluster.shutdown();
+ if (null != dfs) {
+ try {
+ dfs.close();
+ } catch (Exception e) {
+ }
+ }
+ if (null != cluster) {
+ cluster.shutdown();
+ }
+ localf.delete();
}
}
- @Test
+ @Test (timeout = 30000)
public void testLsr() throws Exception {
final Configuration conf = new HdfsConfiguration();
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
@@ -1559,7 +1592,7 @@ public class TestDFSShell {
* and return -1 exit code.
* @throws Exception
*/
- @Test
+ @Test (timeout = 30000)
public void testInvalidShell() throws Exception {
Configuration conf = new Configuration(); // default FS (non-DFS)
DFSAdmin admin = new DFSAdmin();
@@ -1569,29 +1602,31 @@ public class TestDFSShell {
}
// force Copy Option is -f
- @Test
+ @Test (timeout = 30000)
public void testCopyCommandsWithForceOption() throws Exception {
+ final int SUCCESS = 0;
+ final int ERROR = 1;
+
Configuration conf = new Configuration();
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1)
.format(true).build();
FsShell shell = null;
FileSystem fs = null;
final File localFile = new File(TEST_ROOT_DIR, "testFileForPut");
- final String localfilepath = localFile.getAbsolutePath();
- final String testdir = TEST_ROOT_DIR + "/ForceTestDir";
+ final String localfilepath = new Path(localFile.getAbsolutePath()).toUri().toString();
+ final String testdir = "/tmp/TestDFSShell-testCopyCommandsWithForceOption-"
+ + counter.getAndIncrement();
final Path hdfsTestDir = new Path(testdir);
try {
fs = cluster.getFileSystem();
fs.mkdirs(hdfsTestDir);
localFile.createNewFile();
- writeFile(fs, new Path(TEST_ROOT_DIR, "testFileForPut"));
+ writeFile(fs, new Path(testdir, "testFileForPut"));
shell = new FsShell();
// Tests for put
String[] argv = new String[] { "-put", "-f", localfilepath, testdir };
int res = ToolRunner.run(shell, argv);
- int SUCCESS = 0;
- int ERROR = 1;
assertEquals("put -f is not working", SUCCESS, res);
argv = new String[] { "-put", localfilepath, testdir };
@@ -1663,8 +1698,13 @@ public class TestDFSShell {
try {
// Create and delete a file
fs = cluster.getFileSystem();
- writeFile(fs, new Path(TEST_ROOT_DIR, "foo"));
- final String testFile = TEST_ROOT_DIR + "/foo";
+
+ // Use a separate tmp dir for each invocation.
+ final String testdir = "/tmp/TestDFSShell-deleteFileUsingTrash-" +
+ counter.getAndIncrement();
+
+ writeFile(fs, new Path(testdir, "foo"));
+ final String testFile = testdir + "/foo";
final String trashFile = shell.getCurrentTrashDir() + "/" + testFile;
String[] argv = new String[] { "-rm", testFile };
int res = ToolRunner.run(shell, argv);
@@ -1696,7 +1736,7 @@ public class TestDFSShell {
* Test that the server trash configuration is respected when
* the client configuration is not set.
*/
- @Test
+ @Test (timeout = 30000)
public void testServerConfigRespected() throws Exception {
deleteFileUsingTrash(true, false);
}
@@ -1705,7 +1745,7 @@ public class TestDFSShell {
* Test that server trash configuration is respected even when the
* client configuration is set.
*/
- @Test
+ @Test (timeout = 30000)
public void testServerConfigRespectedWithClient() throws Exception {
deleteFileUsingTrash(true, true);
}
@@ -1714,7 +1754,7 @@ public class TestDFSShell {
* Test that the client trash configuration is respected when
* the server configuration is not set.
*/
- @Test
+ @Test (timeout = 30000)
public void testClientConfigRespected() throws Exception {
deleteFileUsingTrash(false, true);
}
@@ -1722,7 +1762,7 @@ public class TestDFSShell {
/**
* Test that trash is disabled by default.
*/
- @Test
+ @Test (timeout = 30000)
public void testNoTrashConfig() throws Exception {
deleteFileUsingTrash(false, false);
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
index 8d71791fd92..75e9329298a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
@@ -45,6 +45,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
import org.apache.hadoop.util.StringUtils;
+import org.apache.log4j.Logger;
import org.junit.Test;
/**
@@ -293,6 +294,11 @@ public class TestDFSUpgradeFromImage {
new File(baseDir, "name2/current/VERSION"),
"imageMD5Digest", "22222222222222222222222222222222");
+ // Attach our own log appender so we can verify output
+ final LogVerificationAppender appender = new LogVerificationAppender();
+ final Logger logger = Logger.getRootLogger();
+ logger.addAppender(appender);
+
// Upgrade should now fail
try {
upgradeAndVerify(new MiniDFSCluster.Builder(upgradeConf).
@@ -300,9 +306,12 @@ public class TestDFSUpgradeFromImage {
fail("Upgrade did not fail with bad MD5");
} catch (IOException ioe) {
String msg = StringUtils.stringifyException(ioe);
- if (!msg.contains("is corrupt with MD5 checksum")) {
+ if (!msg.contains("Failed to load an FSImage file")) {
throw ioe;
}
+ int md5failures = appender.countExceptionsWithMessage(
+ " is corrupt with MD5 checksum of ");
+ assertEquals("Upgrade did not fail with bad MD5", 1, md5failures);
}
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
index 8482f81ddb4..91a017a2363 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
@@ -641,4 +641,24 @@ public class TestDFSUtil {
assertFalse(DFSUtil.isValidName("/foo/:/bar"));
assertFalse(DFSUtil.isValidName("/foo:bar"));
}
+
+ @Test(timeout=5000)
+ public void testGetSpnegoKeytabKey() {
+ HdfsConfiguration conf = new HdfsConfiguration();
+ String defaultKey = "default.spengo.key";
+ conf.unset(DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY);
+ assertEquals("Test spnego key in config is null", defaultKey,
+ DFSUtil.getSpnegoKeytabKey(conf, defaultKey));
+
+ conf.set(DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY, "");
+ assertEquals("Test spnego key is empty", defaultKey,
+ DFSUtil.getSpnegoKeytabKey(conf, defaultKey));
+
+ String spengoKey = "spengo.key";
+ conf.set(DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY,
+ spengoKey);
+ assertEquals("Test spnego key is NOT null",
+ DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY,
+ DFSUtil.getSpnegoKeytabKey(conf, defaultKey));
+ }
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferKeepalive.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferKeepalive.java
index 4e785110476..3c9ee25b111 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferKeepalive.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferKeepalive.java
@@ -71,6 +71,7 @@ public class TestDataTransferKeepalive {
.numDataNodes(1).build();
fs = cluster.getFileSystem();
dfsClient = ((DistributedFileSystem)fs).dfs;
+ dfsClient.peerCache.clear();
String poolId = cluster.getNamesystem().getBlockPoolId();
dn = cluster.getDataNodes().get(0);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileConcurrentReader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileConcurrentReader.java
index 97659eeab3e..c1aa9d1f091 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileConcurrentReader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileConcurrentReader.java
@@ -151,7 +151,7 @@ public class TestFileConcurrentReader {
/**
* Test that that writes to an incomplete block are available to a reader
*/
- @Test
+ @Test (timeout = 30000)
public void testUnfinishedBlockRead()
throws IOException {
// create a new file in the root, write data, do no close
@@ -174,7 +174,7 @@ public class TestFileConcurrentReader {
* would result in too small a buffer to do the buffer-copy needed
* for partial chunks.
*/
- @Test
+ @Test (timeout = 30000)
public void testUnfinishedBlockPacketBufferOverrun() throws IOException {
// check that / exists
Path path = new Path("/");
@@ -200,7 +200,7 @@ public class TestFileConcurrentReader {
// use a small block size and a large write so that DN is busy creating
// new blocks. This makes it almost 100% sure we can reproduce
// case of client getting a DN that hasn't yet created the blocks
- @Test
+ @Test (timeout = 30000)
public void testImmediateReadOfNewFile()
throws IOException {
final int blockSize = 64 * 1024;
@@ -277,12 +277,12 @@ public class TestFileConcurrentReader {
// for some reason, using tranferTo evokes the race condition more often
// so test separately
- @Test
+ @Test (timeout = 30000)
public void testUnfinishedBlockCRCErrorTransferTo() throws IOException {
runTestUnfinishedBlockCRCError(true, SyncType.SYNC, DEFAULT_WRITE_SIZE);
}
- @Test
+ @Test (timeout = 30000)
public void testUnfinishedBlockCRCErrorTransferToVerySmallWrite()
throws IOException {
runTestUnfinishedBlockCRCError(true, SyncType.SYNC, SMALL_WRITE_SIZE);
@@ -290,18 +290,17 @@ public class TestFileConcurrentReader {
// fails due to issue w/append, disable
@Ignore
- @Test
public void _testUnfinishedBlockCRCErrorTransferToAppend()
throws IOException {
runTestUnfinishedBlockCRCError(true, SyncType.APPEND, DEFAULT_WRITE_SIZE);
}
- @Test
+ @Test (timeout = 30000)
public void testUnfinishedBlockCRCErrorNormalTransfer() throws IOException {
runTestUnfinishedBlockCRCError(false, SyncType.SYNC, DEFAULT_WRITE_SIZE);
}
- @Test
+ @Test (timeout = 30000)
public void testUnfinishedBlockCRCErrorNormalTransferVerySmallWrite()
throws IOException {
runTestUnfinishedBlockCRCError(false, SyncType.SYNC, SMALL_WRITE_SIZE);
@@ -309,7 +308,6 @@ public class TestFileConcurrentReader {
// fails due to issue w/append, disable
@Ignore
- @Test
public void _testUnfinishedBlockCRCErrorNormalTransferAppend()
throws IOException {
runTestUnfinishedBlockCRCError(false, SyncType.APPEND, DEFAULT_WRITE_SIZE);
@@ -338,33 +336,33 @@ public class TestFileConcurrentReader {
final AtomicBoolean writerDone = new AtomicBoolean(false);
final AtomicBoolean writerStarted = new AtomicBoolean(false);
final AtomicBoolean error = new AtomicBoolean(false);
- final FSDataOutputStream initialOutputStream = fileSystem.create(file);
- final Thread writer = new Thread(new Runnable() {
- private FSDataOutputStream outputStream = initialOutputStream;
+ final Thread writer = new Thread(new Runnable() {
@Override
public void run() {
try {
- for (int i = 0; !error.get() && i < numWrites; i++) {
- try {
+ FSDataOutputStream outputStream = fileSystem.create(file);
+ if (syncType == SyncType.APPEND) {
+ outputStream.close();
+ outputStream = fileSystem.append(file);
+ }
+ try {
+ for (int i = 0; !error.get() && i < numWrites; i++) {
final byte[] writeBuf =
- DFSTestUtil.generateSequentialBytes(i * writeSize, writeSize);
+ DFSTestUtil.generateSequentialBytes(i * writeSize, writeSize);
outputStream.write(writeBuf);
if (syncType == SyncType.SYNC) {
outputStream.hflush();
- } else { // append
- outputStream.close();
- outputStream = fileSystem.append(file);
}
writerStarted.set(true);
- } catch (IOException e) {
- error.set(true);
- LOG.error("error writing to file", e);
}
+ } catch (IOException e) {
+ error.set(true);
+ LOG.error("error writing to file", e);
+ } finally {
+ outputStream.close();
}
-
writerDone.set(true);
- outputStream.close();
} catch (Exception e) {
LOG.error("error in writer", e);
@@ -415,7 +413,6 @@ public class TestFileConcurrentReader {
Thread.currentThread().interrupt();
}
- initialOutputStream.close();
}
private boolean validateSequentialBytes(byte[] buf, int startPos, int len) {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHftpURLTimeouts.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHftpURLTimeouts.java
index 345c150a74a..d9a22c10111 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHftpURLTimeouts.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHftpURLTimeouts.java
@@ -19,6 +19,7 @@
package org.apache.hadoop.hdfs;
import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
@@ -39,7 +40,7 @@ import org.junit.Test;
public class TestHftpURLTimeouts {
@BeforeClass
public static void setup() {
- URLUtils.SOCKET_TIMEOUT = 1;
+ URLUtils.SOCKET_TIMEOUT = 5;
}
@Test
@@ -116,6 +117,7 @@ public class TestHftpURLTimeouts {
conns.add(fs.openConnection("/", ""));
} catch (SocketTimeoutException ste) {
String message = ste.getMessage();
+ assertNotNull(message);
// https will get a read timeout due to SSL negotiation, but
// a normal http will not, so need to ignore SSL read timeouts
// until a connect timeout occurs
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java
index 99f30dd73ec..1400f07e062 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java
@@ -65,7 +65,7 @@ public class TestMiniDFSCluster {
*
* @throws Throwable on a failure
*/
- @Test
+ @Test(timeout=100000)
public void testClusterWithoutSystemProperties() throws Throwable {
System.clearProperty(MiniDFSCluster.PROP_TEST_BUILD_DATA);
Configuration conf = new HdfsConfiguration();
@@ -74,7 +74,8 @@ public class TestMiniDFSCluster {
conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, c1Path);
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
try {
- assertEquals(c1Path+"/data", cluster.getDataDirectory());
+ assertEquals(new File(c1Path + "/data"),
+ new File(cluster.getDataDirectory()));
} finally {
cluster.shutdown();
}
@@ -84,7 +85,7 @@ public class TestMiniDFSCluster {
* Bring up two clusters and assert that they are in different directories.
* @throws Throwable on a failure
*/
- @Test
+ @Test(timeout=100000)
public void testDualClusters() throws Throwable {
File testDataCluster2 = new File(testDataPath, CLUSTER_2);
File testDataCluster3 = new File(testDataPath, CLUSTER_3);
@@ -95,7 +96,7 @@ public class TestMiniDFSCluster {
MiniDFSCluster cluster3 = null;
try {
String dataDir2 = cluster2.getDataDirectory();
- assertEquals(c2Path + "/data", dataDir2);
+ assertEquals(new File(c2Path + "/data"), new File(dataDir2));
//change the data dir
conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR,
testDataCluster3.getAbsolutePath());
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestShortCircuitLocalRead.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestShortCircuitLocalRead.java
index 94931ca4806..f15da5c30e4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestShortCircuitLocalRead.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestShortCircuitLocalRead.java
@@ -18,12 +18,15 @@
package org.apache.hadoop.hdfs;
import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
import java.io.EOFException;
import java.io.File;
import java.io.IOException;
import java.io.RandomAccessFile;
+import java.net.URI;
import java.nio.ByteBuffer;
+import java.security.PrivilegedExceptionAction;
import java.util.concurrent.TimeoutException;
import org.apache.hadoop.conf.Configuration;
@@ -33,6 +36,7 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -42,6 +46,7 @@ import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.net.unix.DomainSocket;
import org.apache.hadoop.net.unix.TemporarySocketDirectory;
+import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Time;
@@ -108,9 +113,22 @@ public class TestShortCircuitLocalRead {
}
}
}
+
+ private static String getCurrentUser() throws IOException {
+ return UserGroupInformation.getCurrentUser().getShortUserName();
+ }
- static void checkFileContent(FileSystem fs, Path name, byte[] expected,
- int readOffset) throws IOException {
+ /** Check file content, reading as user {@code readingUser} */
+ static void checkFileContent(URI uri, Path name, byte[] expected,
+ int readOffset, String readingUser, Configuration conf,
+ boolean legacyShortCircuitFails)
+ throws IOException, InterruptedException {
+ // Ensure short circuit is enabled
+ DistributedFileSystem fs = getFileSystem(readingUser, uri, conf);
+ if (legacyShortCircuitFails) {
+ assertTrue(fs.getClient().useLegacyBlockReaderLocal());
+ }
+
FSDataInputStream stm = fs.open(name);
byte[] actual = new byte[expected.length-readOffset];
stm.readFully(readOffset, actual);
@@ -135,6 +153,10 @@ public class TestShortCircuitLocalRead {
nread += nbytes;
}
checkData(actual, readOffset, expected, "Read 3");
+
+ if (legacyShortCircuitFails) {
+ assertFalse(fs.getClient().useLegacyBlockReaderLocal());
+ }
stm.close();
}
@@ -146,11 +168,17 @@ public class TestShortCircuitLocalRead {
return arr;
}
- /**
- * Verifies that reading a file with the direct read(ByteBuffer) api gives the expected set of bytes.
- */
- static void checkFileContentDirect(FileSystem fs, Path name, byte[] expected,
- int readOffset) throws IOException {
+ /** Check the file content, reading as user {@code readingUser} */
+ static void checkFileContentDirect(URI uri, Path name, byte[] expected,
+ int readOffset, String readingUser, Configuration conf,
+ boolean legacyShortCircuitFails)
+ throws IOException, InterruptedException {
+ // Ensure short circuit is enabled
+ DistributedFileSystem fs = getFileSystem(readingUser, uri, conf);
+ if (legacyShortCircuitFails) {
+ assertTrue(fs.getClient().useLegacyBlockReaderLocal());
+ }
+
HdfsDataInputStream stm = (HdfsDataInputStream)fs.open(name);
ByteBuffer actual = ByteBuffer.allocateDirect(expected.length - readOffset);
@@ -180,15 +208,33 @@ public class TestShortCircuitLocalRead {
nread += nbytes;
}
checkData(arrayFromByteBuffer(actual), readOffset, expected, "Read 3");
+ if (legacyShortCircuitFails) {
+ assertFalse(fs.getClient().useLegacyBlockReaderLocal());
+ }
stm.close();
}
+ public void doTestShortCircuitReadLegacy(boolean ignoreChecksum, int size,
+ int readOffset, String shortCircuitUser, String readingUser,
+ boolean legacyShortCircuitFails) throws IOException, InterruptedException {
+ doTestShortCircuitReadImpl(ignoreChecksum, size, readOffset,
+ shortCircuitUser, readingUser, legacyShortCircuitFails);
+ }
+
+ public void doTestShortCircuitRead(boolean ignoreChecksum, int size,
+ int readOffset) throws IOException, InterruptedException {
+ String shortCircuitUser = getCurrentUser();
+ doTestShortCircuitReadImpl(ignoreChecksum, size, readOffset,
+ null, getCurrentUser(), false);
+ }
+
/**
* Test that file data can be read by reading the block file
* directly from the local store.
*/
- public void doTestShortCircuitRead(boolean ignoreChecksum, int size,
- int readOffset) throws IOException {
+ public void doTestShortCircuitReadImpl(boolean ignoreChecksum, int size,
+ int readOffset, String shortCircuitUser, String readingUser,
+ boolean legacyShortCircuitFails) throws IOException, InterruptedException {
Configuration conf = new Configuration();
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, true);
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
@@ -196,6 +242,11 @@ public class TestShortCircuitLocalRead {
conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
new File(sockDir.getDir(),
"TestShortCircuitLocalRead._PORT.sock").getAbsolutePath());
+ if (shortCircuitUser != null) {
+ conf.set(DFSConfigKeys.DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY,
+ shortCircuitUser);
+ conf.setBoolean(DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL, true);
+ }
if (simulatedStorage) {
SimulatedFSDataset.setFactory(conf);
}
@@ -208,53 +259,94 @@ public class TestShortCircuitLocalRead {
assertTrue("/ should be a directory", fs.getFileStatus(path)
.isDirectory() == true);
- byte[] fileData = AppendTestUtil.randomBytes(seed, size);
// create a new file in home directory. Do not close it.
- Path file1 = new Path("filelocal.dat");
+ byte[] fileData = AppendTestUtil.randomBytes(seed, size);
+ Path file1 = fs.makeQualified(new Path("filelocal.dat"));
FSDataOutputStream stm = createFile(fs, file1, 1);
-
- // write to file
stm.write(fileData);
stm.close();
- checkFileContent(fs, file1, fileData, readOffset);
- checkFileContentDirect(fs, file1, fileData, readOffset);
+
+ URI uri = cluster.getURI();
+ checkFileContent(uri, file1, fileData, readOffset, readingUser, conf,
+ legacyShortCircuitFails);
+ checkFileContentDirect(uri, file1, fileData, readOffset, readingUser,
+ conf, legacyShortCircuitFails);
} finally {
fs.close();
cluster.shutdown();
}
}
- @Test
- public void testFileLocalReadNoChecksum() throws IOException {
+ @Test(timeout=10000)
+ public void testFileLocalReadNoChecksum() throws Exception {
doTestShortCircuitRead(true, 3*blockSize+100, 0);
}
- @Test
- public void testFileLocalReadChecksum() throws IOException {
+ @Test(timeout=10000)
+ public void testFileLocalReadChecksum() throws Exception {
doTestShortCircuitRead(false, 3*blockSize+100, 0);
}
- @Test
- public void testSmallFileLocalRead() throws IOException {
+ @Test(timeout=10000)
+ public void testSmallFileLocalRead() throws Exception {
doTestShortCircuitRead(false, 13, 0);
doTestShortCircuitRead(false, 13, 5);
doTestShortCircuitRead(true, 13, 0);
doTestShortCircuitRead(true, 13, 5);
}
- @Test
- public void testReadFromAnOffset() throws IOException {
+ @Test(timeout=10000)
+ public void testLocalReadLegacy() throws Exception {
+ doTestShortCircuitReadLegacy(true, 13, 0, getCurrentUser(),
+ getCurrentUser(), false);
+ }
+
+ /**
+ * Try a short circuit from a reader that is not allowed to
+ * to use short circuit. The test ensures reader falls back to non
+ * shortcircuit reads when shortcircuit is disallowed.
+ */
+ @Test(timeout=10000)
+ public void testLocalReadFallback() throws Exception {
+ doTestShortCircuitReadLegacy(true, 13, 0, getCurrentUser(), "notallowed", true);
+ }
+
+ @Test(timeout=10000)
+ public void testReadFromAnOffset() throws Exception {
doTestShortCircuitRead(false, 3*blockSize+100, 777);
doTestShortCircuitRead(true, 3*blockSize+100, 777);
}
- @Test
- public void testLongFile() throws IOException {
+ @Test(timeout=10000)
+ public void testLongFile() throws Exception {
doTestShortCircuitRead(false, 10*blockSize+100, 777);
doTestShortCircuitRead(true, 10*blockSize+100, 777);
}
- @Test
+ private ClientDatanodeProtocol getProxy(UserGroupInformation ugi,
+ final DatanodeID dnInfo, final Configuration conf) throws IOException,
+ InterruptedException {
+ return ugi.doAs(new PrivilegedExceptionAction() {
+ @Override
+ public ClientDatanodeProtocol run() throws Exception {
+ return DFSUtil.createClientDatanodeProtocolProxy(dnInfo, conf, 60000,
+ false);
+ }
+ });
+ }
+
+ private static DistributedFileSystem getFileSystem(String user, final URI uri,
+ final Configuration conf) throws InterruptedException, IOException {
+ UserGroupInformation ugi = UserGroupInformation.createRemoteUser(user);
+ return ugi.doAs(new PrivilegedExceptionAction() {
+ @Override
+ public DistributedFileSystem run() throws Exception {
+ return (DistributedFileSystem)FileSystem.get(uri, conf);
+ }
+ });
+ }
+
+ @Test(timeout=10000)
public void testDeprecatedGetBlockLocalPathInfoRpc()
throws IOException, InterruptedException {
final Configuration conf = new Configuration();
@@ -287,7 +379,7 @@ public class TestShortCircuitLocalRead {
}
}
- @Test
+ @Test(timeout=10000)
public void testSkipWithVerifyChecksum() throws IOException {
int size = blockSize;
Configuration conf = new Configuration();
@@ -417,7 +509,7 @@ public class TestShortCircuitLocalRead {
}
/**
- * Test to run benchmarks between shortcircuit read vs regular read with
+ * Test to run benchmarks between short circuit read vs regular read with
* specified number of threads simultaneously reading.
*
* Run this using the following command:
@@ -435,7 +527,7 @@ public class TestShortCircuitLocalRead {
int threadCount = Integer.valueOf(args[2]);
// Setup create a file
- Configuration conf = new Configuration();
+ final Configuration conf = new Configuration();
conf.setBoolean(DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY, shortcircuit);
conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
"/tmp/TestShortCircuitLocalRead._PORT");
@@ -463,9 +555,13 @@ public class TestShortCircuitLocalRead {
public void run() {
for (int i = 0; i < iteration; i++) {
try {
- checkFileContent(fs, file1, dataToWrite, 0);
+ String user = getCurrentUser();
+ checkFileContent(fs.getUri(), file1, dataToWrite, 0, user, conf,
+ true);
} catch (IOException e) {
e.printStackTrace();
+ } catch (InterruptedException e) {
+ e.printStackTrace();
}
}
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestNNWithQJM.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestNNWithQJM.java
index a5463d03b39..9ce2ae3c7ce 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestNNWithQJM.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestNNWithQJM.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.hdfs.qjournal;
import static org.junit.Assert.*;
+import static org.junit.Assume.*;
import java.io.File;
import java.io.IOException;
@@ -43,7 +44,7 @@ import org.junit.Test;
public class TestNNWithQJM {
Configuration conf = new HdfsConfiguration();
- private MiniJournalCluster mjc;
+ private MiniJournalCluster mjc = null;
private Path TEST_PATH = new Path("/test-dir");
private Path TEST_PATH_2 = new Path("/test-dir");
@@ -61,10 +62,11 @@ public class TestNNWithQJM {
public void stopJNs() throws Exception {
if (mjc != null) {
mjc.shutdown();
+ mjc = null;
}
}
- @Test
+ @Test (timeout = 30000)
public void testLogAndRestart() throws IOException {
conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
MiniDFSCluster.getBaseDirectory() + "/TestNNWithQJM/image");
@@ -93,9 +95,12 @@ public class TestNNWithQJM {
cluster.shutdown();
}
}
-
- @Test
+
+ @Test (timeout = 30000)
public void testNewNamenodeTakesOverWriter() throws Exception {
+ // Skip the test on Windows. See HDFS-4584.
+ assumeTrue(!Path.WINDOWS);
+
File nn1Dir = new File(
MiniDFSCluster.getBaseDirectory() + "/TestNNWithQJM/image-nn1");
File nn2Dir = new File(
@@ -154,7 +159,7 @@ public class TestNNWithQJM {
}
}
- @Test
+ @Test (timeout = 30000)
public void testMismatchedNNIsRejected() throws Exception {
conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
MiniDFSCluster.getBaseDirectory() + "/TestNNWithQJM/image");
@@ -188,8 +193,8 @@ public class TestNNWithQJM {
"Unable to start log segment 1: too few journals", ioe);
}
}
-
- @Test
+
+ @Test (timeout = 30000)
public void testWebPageHasQjmInfo() throws Exception {
conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
MiniDFSCluster.getBaseDirectory() + "/TestNNWithQJM/image");
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournal.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournal.java
index c9db35faca1..3c4d575db87 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournal.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournal.java
@@ -36,10 +36,7 @@ import org.apache.hadoop.hdfs.server.common.StorageErrorReporter;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.After;
-import org.junit.Assume;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.*;
import org.mockito.Mockito;
public class TestJournal {
@@ -77,7 +74,7 @@ public class TestJournal {
IOUtils.closeStream(journal);
}
- @Test
+ @Test (timeout = 10000)
public void testEpochHandling() throws Exception {
assertEquals(0, journal.getLastPromisedEpoch());
NewEpochResponseProto newEpoch =
@@ -110,7 +107,7 @@ public class TestJournal {
}
}
- @Test
+ @Test (timeout = 10000)
public void testMaintainCommittedTxId() throws Exception {
journal.newEpoch(FAKE_NSINFO, 1);
journal.startLogSegment(makeRI(1), 1);
@@ -125,7 +122,7 @@ public class TestJournal {
assertEquals(3, journal.getCommittedTxnIdForTests());
}
- @Test
+ @Test (timeout = 10000)
public void testRestartJournal() throws Exception {
journal.newEpoch(FAKE_NSINFO, 1);
journal.startLogSegment(makeRI(1), 1);
@@ -149,7 +146,7 @@ public class TestJournal {
assertEquals(1, newEpoch.getLastSegmentTxId());
}
- @Test
+ @Test (timeout = 10000)
public void testFormatResetsCachedValues() throws Exception {
journal.newEpoch(FAKE_NSINFO, 12345L);
journal.startLogSegment(new RequestInfo(JID, 12345L, 1L, 0L), 1L);
@@ -158,6 +155,8 @@ public class TestJournal {
assertEquals(12345L, journal.getLastWriterEpoch());
assertTrue(journal.isFormatted());
+ // Close the journal in preparation for reformatting it.
+ journal.close();
journal.format(FAKE_NSINFO_2);
assertEquals(0, journal.getLastPromisedEpoch());
@@ -170,7 +169,7 @@ public class TestJournal {
* before any transactions are written, that the next newEpoch() call
* returns the prior segment txid as its most recent segment.
*/
- @Test
+ @Test (timeout = 10000)
public void testNewEpochAtBeginningOfSegment() throws Exception {
journal.newEpoch(FAKE_NSINFO, 1);
journal.startLogSegment(makeRI(1), 1);
@@ -182,7 +181,7 @@ public class TestJournal {
assertEquals(1, resp.getLastSegmentTxId());
}
- @Test
+ @Test (timeout = 10000)
public void testJournalLocking() throws Exception {
Assume.assumeTrue(journal.getStorage().getStorageDir(0).isLockSupported());
StorageDirectory sd = journal.getStorage().getStorageDir(0);
@@ -206,13 +205,14 @@ public class TestJournal {
// Hence, should be able to create a new Journal in the same dir.
Journal journal2 = new Journal(TEST_LOG_DIR, JID, mockErrorReporter);
journal2.newEpoch(FAKE_NSINFO, 2);
+ journal2.close();
}
/**
* Test finalizing a segment after some batch of edits were missed.
* This should fail, since we validate the log before finalization.
*/
- @Test
+ @Test (timeout = 10000)
public void testFinalizeWhenEditsAreMissed() throws Exception {
journal.newEpoch(FAKE_NSINFO, 1);
journal.startLogSegment(makeRI(1), 1);
@@ -246,7 +246,7 @@ public class TestJournal {
* Ensure that finalizing a segment which doesn't exist throws the
* appropriate exception.
*/
- @Test
+ @Test (timeout = 10000)
public void testFinalizeMissingSegment() throws Exception {
journal.newEpoch(FAKE_NSINFO, 1);
try {
@@ -267,7 +267,7 @@ public class TestJournal {
* Eventually, the connection comes back, and the NN tries to start a new
* segment at a higher txid. This should abort the old one and succeed.
*/
- @Test
+ @Test (timeout = 10000)
public void testAbortOldSegmentIfFinalizeIsMissed() throws Exception {
journal.newEpoch(FAKE_NSINFO, 1);
@@ -296,7 +296,7 @@ public class TestJournal {
* Test behavior of startLogSegment() when a segment with the
* same transaction ID already exists.
*/
- @Test
+ @Test (timeout = 10000)
public void testStartLogSegmentWhenAlreadyExists() throws Exception {
journal.newEpoch(FAKE_NSINFO, 1);
@@ -345,7 +345,7 @@ public class TestJournal {
return new RequestInfo(JID, 1, serial, 0);
}
- @Test
+ @Test (timeout = 10000)
public void testNamespaceVerification() throws Exception {
journal.newEpoch(FAKE_NSINFO, 1);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java
index f57f7e66b40..e6e140443bf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.MetricsAsserts;
+import org.apache.hadoop.util.Shell;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@@ -65,6 +66,8 @@ public class TestJournalNode {
private Configuration conf = new Configuration();
private IPCLoggerChannel ch;
private String journalId;
+ private File TEST_BUILD_DATA =
+ new File(System.getProperty("test.build.data", "build/test/data"));
static {
// Avoid an error when we double-initialize JvmMetrics
@@ -96,7 +99,7 @@ public class TestJournalNode {
jn.stop(0);
}
- @Test
+ @Test(timeout=100000)
public void testJournal() throws Exception {
MetricsRecordBuilder metrics = MetricsAsserts.getMetrics(
journal.getMetricsForTests().getName());
@@ -129,7 +132,7 @@ public class TestJournalNode {
}
- @Test
+ @Test(timeout=100000)
public void testReturnsSegmentInfoAtEpochTransition() throws Exception {
ch.newEpoch(1).get();
ch.setEpoch(1);
@@ -157,7 +160,7 @@ public class TestJournalNode {
assertEquals(1, response.getLastSegmentTxId());
}
- @Test
+ @Test(timeout=100000)
public void testHttpServer() throws Exception {
InetSocketAddress addr = jn.getBoundHttpAddress();
assertTrue(addr.getPort() > 0);
@@ -210,7 +213,7 @@ public class TestJournalNode {
* Test that the JournalNode performs correctly as a Paxos
* Acceptor process.
*/
- @Test
+ @Test(timeout=100000)
public void testAcceptRecoveryBehavior() throws Exception {
// We need to run newEpoch() first, or else we have no way to distinguish
// different proposals for the same decision.
@@ -270,20 +273,27 @@ public class TestJournalNode {
}
}
- @Test
+ @Test(timeout=100000)
public void testFailToStartWithBadConfig() throws Exception {
Configuration conf = new Configuration();
conf.set(DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_KEY, "non-absolute-path");
assertJNFailsToStart(conf, "should be an absolute path");
// Existing file which is not a directory
- conf.set(DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_KEY, "/dev/null");
- assertJNFailsToStart(conf, "is not a directory");
+ File existingFile = new File(TEST_BUILD_DATA, "testjournalnodefile");
+ assertTrue(existingFile.createNewFile());
+ try {
+ conf.set(DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_KEY,
+ existingFile.getAbsolutePath());
+ assertJNFailsToStart(conf, "Not a directory");
+ } finally {
+ existingFile.delete();
+ }
// Directory which cannot be created
- conf.set(DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_KEY, "/proc/does-not-exist");
- assertJNFailsToStart(conf, "Could not create");
-
+ conf.set(DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_KEY,
+ Shell.WINDOWS ? "\\\\cannotBeCreated" : "/proc/does-not-exist");
+ assertJNFailsToStart(conf, "Can not create directory");
}
private static void assertJNFailsToStart(Configuration conf,
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java
index 80c8eb50083..86994bed3ac 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java
@@ -104,7 +104,7 @@ public class TestNodeCount {
while (iter.hasNext()) {
DatanodeDescriptor dn = iter.next();
Collection blocks = bm.excessReplicateMap.get(dn.getStorageID());
- if (blocks == null || !blocks.contains(block) ) {
+ if (blocks == null || !blocks.contains(block.getLocalBlock()) ) {
nonExcessDN = dn;
break;
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
index 38a5f0df3bb..0322eefad78 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.LogVerificationAppender;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -45,7 +46,6 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.net.NetworkTopology;
import org.apache.hadoop.net.Node;
import org.apache.hadoop.util.Time;
-import org.apache.log4j.AppenderSkeleton;
import org.apache.log4j.Level;
import org.apache.log4j.Logger;
import org.apache.log4j.spi.LoggingEvent;
@@ -419,7 +419,7 @@ public class TestReplicationPolicy {
(HdfsConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0, 0);
}
- final TestAppender appender = new TestAppender();
+ final LogVerificationAppender appender = new LogVerificationAppender();
final Logger logger = Logger.getRootLogger();
logger.addAppender(appender);
@@ -446,28 +446,6 @@ public class TestReplicationPolicy {
HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
}
}
-
- class TestAppender extends AppenderSkeleton {
- private final List log = new ArrayList();
-
- @Override
- public boolean requiresLayout() {
- return false;
- }
-
- @Override
- protected void append(final LoggingEvent loggingEvent) {
- log.add(loggingEvent);
- }
-
- @Override
- public void close() {
- }
-
- public List getLog() {
- return new ArrayList(log);
- }
- }
private boolean containsWithinRange(DatanodeDescriptor target,
DatanodeDescriptor[] nodes, int startIndex, int endIndex) {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataDirs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataDirs.java
index 9226beaf297..fc7a3ff01d1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataDirs.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataDirs.java
@@ -27,33 +27,26 @@ import java.util.List;
import org.junit.Test;
import static org.junit.Assert.*;
import static org.mockito.Mockito.*;
-import static org.apache.hadoop.test.MockitoMaker.*;
-import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.server.datanode.DataNode.DataNodeDiskChecker;
public class TestDataDirs {
- @Test public void testGetDataDirsFromURIs() throws Throwable {
- File localDir = make(stub(File.class).returning(true).from.exists());
- when(localDir.mkdir()).thenReturn(true);
- FsPermission normalPerm = new FsPermission("700");
- FsPermission badPerm = new FsPermission("000");
- FileStatus stat = make(stub(FileStatus.class)
- .returning(normalPerm, normalPerm, badPerm).from.getPermission());
- when(stat.isDirectory()).thenReturn(true);
- LocalFileSystem fs = make(stub(LocalFileSystem.class)
- .returning(stat).from.getFileStatus(any(Path.class)));
- when(fs.pathToFile(any(Path.class))).thenReturn(localDir);
+ @Test (timeout = 10000)
+ public void testGetDataDirsFromURIs() throws Throwable {
+
+ DataNodeDiskChecker diskChecker = mock(DataNodeDiskChecker.class);
+ doThrow(new IOException()).doThrow(new IOException()).doNothing()
+ .when(diskChecker).checkDir(any(LocalFileSystem.class), any(Path.class));
+ LocalFileSystem fs = mock(LocalFileSystem.class);
Collection uris = Arrays.asList(new URI("file:/p1/"),
new URI("file:/p2/"), new URI("file:/p3/"));
- List dirs = DataNode.getDataDirsFromURIs(uris, fs, normalPerm);
-
- verify(fs, times(2)).setPermission(any(Path.class), eq(normalPerm));
- verify(fs, times(6)).getFileStatus(any(Path.class));
- assertEquals("number of valid data dirs", dirs.size(), 1);
+ List dirs = DataNode.getDataDirsFromURIs(uris, fs, diskChecker);
+ assertEquals("number of valid data dirs", 1, dirs.size());
+ String validDir = dirs.iterator().next().getPath();
+ assertEquals("p3 should be valid", new File("/p3").getPath(), validDir);
}
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java
index f4dc62bf45e..2ffaaafca3a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java
@@ -272,15 +272,15 @@ public abstract class FSImageTestUtil {
for (File dir : dirs) {
FSImageTransactionalStorageInspector inspector =
inspectStorageDirectory(dir, NameNodeDirType.IMAGE);
- FSImageFile latestImage = inspector.getLatestImage();
- assertNotNull("No image in " + dir, latestImage);
- long thisTxId = latestImage.getCheckpointTxId();
+ List latestImages = inspector.getLatestImages();
+ assert(!latestImages.isEmpty());
+ long thisTxId = latestImages.get(0).getCheckpointTxId();
if (imageTxId != -1 && thisTxId != imageTxId) {
fail("Storage directory " + dir + " does not have the same " +
"last image index " + imageTxId + " as another");
}
imageTxId = thisTxId;
- imageFiles.add(inspector.getLatestImage().getFile());
+ imageFiles.add(inspector.getLatestImages().get(0).getFile());
}
assertFileContentsSame(imageFiles.toArray(new File[0]));
@@ -424,7 +424,7 @@ public abstract class FSImageTestUtil {
new FSImageTransactionalStorageInspector();
inspector.inspectDirectory(sd);
- return inspector.getLatestImage().getFile();
+ return inspector.getLatestImages().get(0).getFile();
}
/**
@@ -439,8 +439,8 @@ public abstract class FSImageTestUtil {
new FSImageTransactionalStorageInspector();
inspector.inspectDirectory(sd);
- FSImageFile latestImage = inspector.getLatestImage();
- return (latestImage == null) ? null : latestImage.getFile();
+ List latestImages = inspector.getLatestImages();
+ return (latestImages.isEmpty()) ? null : latestImages.get(0).getFile();
}
/**
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java
index b29f5e041f3..51ca00921c7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java
@@ -143,7 +143,7 @@ public class OfflineEditsViewerHelper {
(DistributedFileSystem)cluster.getFileSystem();
FileContext fc = FileContext.getFileContext(cluster.getURI(0), config);
// OP_ADD 0, OP_SET_GENSTAMP 10
- Path pathFileCreate = new Path("/file_create");
+ Path pathFileCreate = new Path("/file_create_u\1F431");
FSDataOutputStream s = dfs.create(pathFileCreate);
// OP_CLOSE 9
s.close();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogs.java
index 0fe6255fc9a..bbf1dac25dd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogs.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogs.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.log4j.Level;
+import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.log4j.PatternLayout;
import org.apache.log4j.RollingFileAppender;
@@ -233,9 +234,15 @@ public class TestAuditLogs {
/** Sets up log4j logger for auditlogs */
private void setupAuditLogs() throws IOException {
+ // Shutdown the LogManager to release all logger open file handles.
+ // Unfortunately, Apache commons logging library does not provide
+ // means to release underlying loggers. For additional info look up
+ // commons library FAQ.
+ LogManager.shutdown();
+
File file = new File(auditLogFile);
if (file.exists()) {
- file.delete();
+ assertTrue(file.delete());
}
Logger logger = ((Log4JLogger) FSNamesystem.auditLog).getLogger();
logger.setLevel(Level.INFO);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
index aa831d2bc09..de0cf8a5ef5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
@@ -74,6 +74,8 @@ import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.GenericTestUtils.DelayAnswer;
import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
+import org.apache.hadoop.util.ExitUtil;
+import org.apache.hadoop.util.ExitUtil.ExitException;
import org.apache.hadoop.util.StringUtils;
import org.apache.log4j.Level;
import org.junit.After;
@@ -226,6 +228,111 @@ public class TestCheckpoint {
toString().indexOf("storageDirToCheck") != -1);
}
+ /*
+ * Simulate exception during edit replay.
+ */
+ @Test(timeout=30000)
+ public void testReloadOnEditReplayFailure () throws IOException {
+ Configuration conf = new HdfsConfiguration();
+ FSDataOutputStream fos = null;
+ SecondaryNameNode secondary = null;
+ MiniDFSCluster cluster = null;
+ FileSystem fs = null;
+
+ try {
+ cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDatanodes)
+ .build();
+ cluster.waitActive();
+ fs = cluster.getFileSystem();
+ secondary = startSecondaryNameNode(conf);
+ fos = fs.create(new Path("tmpfile0"));
+ fos.write(new byte[] { 0, 1, 2, 3 });
+ secondary.doCheckpoint();
+ fos.write(new byte[] { 0, 1, 2, 3 });
+ fos.hsync();
+
+ // Cause merge to fail in next checkpoint.
+ Mockito.doThrow(new IOException(
+ "Injecting failure during merge"))
+ .when(faultInjector).duringMerge();
+
+ try {
+ secondary.doCheckpoint();
+ fail("Fault injection failed.");
+ } catch (IOException ioe) {
+ // This is expected.
+ }
+ Mockito.reset(faultInjector);
+
+ // The error must be recorded, so next checkpoint will reload image.
+ fos.write(new byte[] { 0, 1, 2, 3 });
+ fos.hsync();
+
+ assertTrue("Another checkpoint should have reloaded image",
+ secondary.doCheckpoint());
+ } finally {
+ if (secondary != null) {
+ secondary.shutdown();
+ }
+ if (fs != null) {
+ fs.close();
+ }
+ if (cluster != null) {
+ cluster.shutdown();
+ }
+ Mockito.reset(faultInjector);
+ }
+ }
+
+ /*
+ * Simulate 2NN exit due to too many merge failures.
+ */
+ @Test(timeout=10000)
+ public void testTooManyEditReplayFailures() throws IOException {
+ Configuration conf = new HdfsConfiguration();
+ conf.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_MAX_RETRIES_KEY, "1");
+ conf.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_KEY, "1");
+
+ FSDataOutputStream fos = null;
+ SecondaryNameNode secondary = null;
+ MiniDFSCluster cluster = null;
+ FileSystem fs = null;
+
+ try {
+ cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDatanodes)
+ .checkExitOnShutdown(false).build();
+ cluster.waitActive();
+ fs = cluster.getFileSystem();
+ fos = fs.create(new Path("tmpfile0"));
+ fos.write(new byte[] { 0, 1, 2, 3 });
+
+ // Cause merge to fail in next checkpoint.
+ Mockito.doThrow(new IOException(
+ "Injecting failure during merge"))
+ .when(faultInjector).duringMerge();
+
+ secondary = startSecondaryNameNode(conf);
+ secondary.doWork();
+ // Fail if we get here.
+ fail("2NN did not exit.");
+ } catch (ExitException ee) {
+ // ignore
+ ExitUtil.resetFirstExitException();
+ assertEquals("Max retries", 1, secondary.getMergeErrorCount() - 1);
+ } finally {
+ if (secondary != null) {
+ secondary.shutdown();
+ }
+ if (fs != null) {
+ fs.close();
+ }
+ if (cluster != null) {
+ cluster.shutdown();
+ }
+ Mockito.reset(faultInjector);
+ }
+ }
+
/*
* Simulate namenode crashing after rolling edit log.
*/
@@ -1304,6 +1411,60 @@ public class TestCheckpoint {
}
}
+ /**
+ * Test NN restart if a failure happens in between creating the fsimage
+ * MD5 file and renaming the fsimage.
+ */
+ @Test(timeout=30000)
+ public void testFailureBeforeRename () throws IOException {
+ Configuration conf = new HdfsConfiguration();
+ FSDataOutputStream fos = null;
+ SecondaryNameNode secondary = null;
+ MiniDFSCluster cluster = null;
+ FileSystem fs = null;
+ NameNode namenode = null;
+
+ try {
+ cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDatanodes)
+ .build();
+ cluster.waitActive();
+ namenode = cluster.getNameNode();
+ fs = cluster.getFileSystem();
+ secondary = startSecondaryNameNode(conf);
+ fos = fs.create(new Path("tmpfile0"));
+ fos.write(new byte[] { 0, 1, 2, 3 });
+ secondary.doCheckpoint();
+ fos.write(new byte[] { 0, 1, 2, 3 });
+ fos.hsync();
+
+ // Cause merge to fail in next checkpoint.
+ Mockito.doThrow(new IOException(
+ "Injecting failure after MD5Rename"))
+ .when(faultInjector).afterMD5Rename();
+
+ try {
+ secondary.doCheckpoint();
+ fail("Fault injection failed.");
+ } catch (IOException ioe) {
+ // This is expected.
+ }
+ Mockito.reset(faultInjector);
+ // Namenode should still restart successfully
+ cluster.restartNameNode();
+ } finally {
+ if (secondary != null) {
+ secondary.shutdown();
+ }
+ if (fs != null) {
+ fs.close();
+ }
+ if (cluster != null) {
+ cluster.shutdown();
+ }
+ Mockito.reset(faultInjector);
+ }
+ }
+
/**
* Test case where two secondary namenodes are checkpointing the same
* NameNode. This differs from {@link #testMultipleSecondaryNamenodes()}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
index 3cf3ca6c7bc..26c6fdba652 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
@@ -861,6 +861,11 @@ public class TestEditLog {
public boolean isInProgress() {
return true;
}
+
+ @Override
+ public void setMaxOpSize(int maxOpSize) {
+ reader.setMaxOpSize(maxOpSize);
+ }
}
@Test
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageStorageInspector.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageStorageInspector.java
index 01d54b814db..5e3ac4b7a2b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageStorageInspector.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageStorageInspector.java
@@ -57,7 +57,7 @@ public class TestFSImageStorageInspector {
inspector.inspectDirectory(mockDir);
assertEquals(2, inspector.foundImages.size());
- FSImageFile latestImage = inspector.getLatestImage();
+ FSImageFile latestImage = inspector.getLatestImages().get(0);
assertEquals(456, latestImage.txId);
assertSame(mockDir, latestImage.sd);
assertTrue(inspector.isUpgradeFinalized());
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestHostsFiles.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestHostsFiles.java
index 2a0ab2003b8..ea9175b214a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestHostsFiles.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestHostsFiles.java
@@ -120,12 +120,13 @@ public class TestHostsFiles {
InetSocketAddress nnHttpAddress = cluster.getNameNode().getHttpAddress();
LOG.info("nnaddr = '" + nnHttpAddress + "'");
- URL nnjsp = new URL("http://" + nnHttpAddress.getHostName() + ":" + nnHttpAddress.getPort() + "/dfshealth.jsp");
+ String nnHostName = nnHttpAddress.getHostName();
+ URL nnjsp = new URL("http://" + nnHostName + ":" + nnHttpAddress.getPort() + "/dfshealth.jsp");
LOG.info("fetching " + nnjsp);
String dfshealthPage = StringEscapeUtils.unescapeHtml(DFSTestUtil.urlGet(nnjsp));
LOG.info("got " + dfshealthPage);
- assertTrue("dfshealth should contain localhost, got:" + dfshealthPage,
- dfshealthPage.contains("localhost"));
+ assertTrue("dfshealth should contain " + nnHostName + ", got:" + dfshealthPage,
+ dfshealthPage.contains(nnHostName));
} finally {
cluster.shutdown();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
index 9d5efb33498..37b22714c20 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
@@ -184,34 +184,41 @@ public class TestINodeFile {
long fileLen = 1024;
replication = 3;
Configuration conf = new Configuration();
- MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(
- replication).build();
- cluster.waitActive();
- FSNamesystem fsn = cluster.getNamesystem();
- FSDirectory fsdir = fsn.getFSDirectory();
- DistributedFileSystem dfs = cluster.getFileSystem();
-
- // Create a file for test
- final Path dir = new Path("/dir");
- final Path file = new Path(dir, "file");
- DFSTestUtil.createFile(dfs, file, fileLen, replication, 0L);
-
- // Check the full path name of the INode associating with the file
- INode fnode = fsdir.getINode(file.toString());
- assertEquals(file.toString(), fnode.getFullPathName());
-
- // Call FSDirectory#unprotectedSetQuota which calls
- // INodeDirectory#replaceChild
- dfs.setQuota(dir, Long.MAX_VALUE - 1, replication * fileLen * 10);
- final Path newDir = new Path("/newdir");
- final Path newFile = new Path(newDir, "file");
- // Also rename dir
- dfs.rename(dir, newDir, Options.Rename.OVERWRITE);
- // /dir/file now should be renamed to /newdir/file
- fnode = fsdir.getINode(newFile.toString());
- // getFullPathName can return correct result only if the parent field of
- // child node is set correctly
- assertEquals(newFile.toString(), fnode.getFullPathName());
+ MiniDFSCluster cluster = null;
+ try {
+ cluster =
+ new MiniDFSCluster.Builder(conf).numDataNodes(replication).build();
+ cluster.waitActive();
+ FSNamesystem fsn = cluster.getNamesystem();
+ FSDirectory fsdir = fsn.getFSDirectory();
+ DistributedFileSystem dfs = cluster.getFileSystem();
+
+ // Create a file for test
+ final Path dir = new Path("/dir");
+ final Path file = new Path(dir, "file");
+ DFSTestUtil.createFile(dfs, file, fileLen, replication, 0L);
+
+ // Check the full path name of the INode associating with the file
+ INode fnode = fsdir.getINode(file.toString());
+ assertEquals(file.toString(), fnode.getFullPathName());
+
+ // Call FSDirectory#unprotectedSetQuota which calls
+ // INodeDirectory#replaceChild
+ dfs.setQuota(dir, Long.MAX_VALUE - 1, replication * fileLen * 10);
+ final Path newDir = new Path("/newdir");
+ final Path newFile = new Path(newDir, "file");
+ // Also rename dir
+ dfs.rename(dir, newDir, Options.Rename.OVERWRITE);
+ // /dir/file now should be renamed to /newdir/file
+ fnode = fsdir.getINode(newFile.toString());
+ // getFullPathName can return correct result only if the parent field of
+ // child node is set correctly
+ assertEquals(newFile.toString(), fnode.getFullPathName());
+ } finally {
+ if (cluster != null) {
+ cluster.shutdown();
+ }
+ }
}
@Test
@@ -385,41 +392,47 @@ public class TestINodeFile {
Configuration conf = new Configuration();
conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY,
DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT);
- MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1)
- .build();
- cluster.waitActive();
-
- FSNamesystem fsn = cluster.getNamesystem();
- long lastId = fsn.getLastInodeId();
+ MiniDFSCluster cluster = null;
+ try {
+ cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+ cluster.waitActive();
- assertTrue(lastId == 1001);
+ FSNamesystem fsn = cluster.getNamesystem();
+ long lastId = fsn.getLastInodeId();
- // Create one directory and the last inode id should increase to 1002
- FileSystem fs = cluster.getFileSystem();
- Path path = new Path("/test1");
- assertTrue(fs.mkdirs(path));
- assertTrue(fsn.getLastInodeId() == 1002);
+ assertTrue(lastId == 1001);
- // Use namenode rpc to create a file
- NamenodeProtocols nnrpc = cluster.getNameNodeRpc();
- HdfsFileStatus fileStatus = nnrpc.create("/test1/file", new FsPermission(
- (short) 0755), "client",
- new EnumSetWritable(EnumSet.of(CreateFlag.CREATE)), true,
- (short) 1, 128 * 1024 * 1024L);
- assertTrue(fsn.getLastInodeId() == 1003);
- assertTrue(fileStatus.getFileId() == 1003);
+ // Create one directory and the last inode id should increase to 1002
+ FileSystem fs = cluster.getFileSystem();
+ Path path = new Path("/test1");
+ assertTrue(fs.mkdirs(path));
+ assertTrue(fsn.getLastInodeId() == 1002);
- // Rename doesn't increase inode id
- Path renamedPath = new Path("/test2");
- fs.rename(path, renamedPath);
- assertTrue(fsn.getLastInodeId() == 1003);
+ // Use namenode rpc to create a file
+ NamenodeProtocols nnrpc = cluster.getNameNodeRpc();
+ HdfsFileStatus fileStatus = nnrpc.create("/test1/file", new FsPermission(
+ (short) 0755), "client",
+ new EnumSetWritable(EnumSet.of(CreateFlag.CREATE)), true,
+ (short) 1, 128 * 1024 * 1024L);
+ assertTrue(fsn.getLastInodeId() == 1003);
+ assertTrue(fileStatus.getFileId() == 1003);
- cluster.restartNameNode();
- cluster.waitActive();
- // Make sure empty editlog can be handled
- cluster.restartNameNode();
- cluster.waitActive();
- assertTrue(fsn.getLastInodeId() == 1003);
+ // Rename doesn't increase inode id
+ Path renamedPath = new Path("/test2");
+ fs.rename(path, renamedPath);
+ assertTrue(fsn.getLastInodeId() == 1003);
+
+ cluster.restartNameNode();
+ cluster.waitActive();
+ // Make sure empty editlog can be handled
+ cluster.restartNameNode();
+ cluster.waitActive();
+ assertTrue(fsn.getLastInodeId() == 1003);
+ } finally {
+ if (cluster != null) {
+ cluster.shutdown();
+ }
+ }
}
@Test
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRecovery.java
index 93b588d0c8c..18a45bcb18a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRecovery.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRecovery.java
@@ -83,6 +83,7 @@ public class TestNameNodeRecovery {
elfos.close();
elfos = null;
elfis = new EditLogFileInputStream(TEST_LOG_NAME);
+ elfis.setMaxOpSize(elts.getMaxOpSize());
// reading through normally will get you an exception
Set validTxIds = elts.getValidTxIds();
@@ -143,7 +144,7 @@ public class TestNameNodeRecovery {
/**
* A test scenario for the edit log
*/
- private interface EditLogTestSetup {
+ private static abstract class EditLogTestSetup {
/**
* Set up the edit log.
*/
@@ -162,6 +163,13 @@ public class TestNameNodeRecovery {
* edit log.
**/
abstract public Set getValidTxIds();
+
+ /**
+ * Return the maximum opcode size we will use for input.
+ */
+ public int getMaxOpSize() {
+ return DFSConfigKeys.DFS_NAMENODE_MAX_OP_SIZE_DEFAULT;
+ }
}
static void padEditLog(EditLogOutputStream elos, int paddingLength)
@@ -182,10 +190,10 @@ public class TestNameNodeRecovery {
}
static void addDeleteOpcode(EditLogOutputStream elos,
- OpInstanceCache cache) throws IOException {
+ OpInstanceCache cache, long txId, String path) throws IOException {
DeleteOp op = DeleteOp.getInstance(cache);
- op.setTransactionId(0x0);
- op.setPath("/foo");
+ op.setTransactionId(txId);
+ op.setPath(path);
op.setTimestamp(0);
elos.write(op);
}
@@ -198,7 +206,7 @@ public class TestNameNodeRecovery {
* able to handle any amount of padding (including no padding) without
* throwing an exception.
*/
- private static class EltsTestEmptyLog implements EditLogTestSetup {
+ private static class EltsTestEmptyLog extends EditLogTestSetup {
private int paddingLength;
public EltsTestEmptyLog(int paddingLength) {
@@ -242,6 +250,42 @@ public class TestNameNodeRecovery {
3 * EditLogFileOutputStream.MIN_PREALLOCATION_LENGTH));
}
+ /**
+ * Test using a non-default maximum opcode length.
+ */
+ private static class EltsTestNonDefaultMaxOpSize extends EditLogTestSetup {
+ public EltsTestNonDefaultMaxOpSize() {
+ }
+
+ @Override
+ public void addTransactionsToLog(EditLogOutputStream elos,
+ OpInstanceCache cache) throws IOException {
+ addDeleteOpcode(elos, cache, 0, "/foo");
+ addDeleteOpcode(elos, cache, 1,
+ "/supercalifragalisticexpialadocius.supercalifragalisticexpialadocius");
+ }
+
+ @Override
+ public long getLastValidTxId() {
+ return 0;
+ }
+
+ @Override
+ public Set getValidTxIds() {
+ return Sets.newHashSet(0L);
+ }
+
+ public int getMaxOpSize() {
+ return 30;
+ }
+ }
+
+ /** Test an empty edit log with extra-long padding */
+ @Test(timeout=180000)
+ public void testNonDefaultMaxOpSize() throws IOException {
+ runEditLogTest(new EltsTestNonDefaultMaxOpSize());
+ }
+
/**
* Test the scenario where an edit log contains some padding (0xff) bytes
* followed by valid opcode data.
@@ -249,7 +293,7 @@ public class TestNameNodeRecovery {
* These edit logs are corrupt, but all the opcodes should be recoverable
* with recovery mode.
*/
- private static class EltsTestOpcodesAfterPadding implements EditLogTestSetup {
+ private static class EltsTestOpcodesAfterPadding extends EditLogTestSetup {
private int paddingLength;
public EltsTestOpcodesAfterPadding(int paddingLength) {
@@ -260,7 +304,7 @@ public class TestNameNodeRecovery {
public void addTransactionsToLog(EditLogOutputStream elos,
OpInstanceCache cache) throws IOException {
padEditLog(elos, paddingLength);
- addDeleteOpcode(elos, cache);
+ addDeleteOpcode(elos, cache, 0, "/foo");
}
@Override
@@ -286,7 +330,7 @@ public class TestNameNodeRecovery {
3 * EditLogFileOutputStream.MIN_PREALLOCATION_LENGTH));
}
- private static class EltsTestGarbageInEditLog implements EditLogTestSetup {
+ private static class EltsTestGarbageInEditLog extends EditLogTestSetup {
final private long BAD_TXID = 4;
final private long MAX_TXID = 10;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProcessCorruptBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProcessCorruptBlocks.java
index d6d849da0ab..abb2337a5dd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProcessCorruptBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProcessCorruptBlocks.java
@@ -158,7 +158,7 @@ public class TestProcessCorruptBlocks {
* (corrupt replica should be removed since number of good
* replicas (1) is equal to replication factor (1))
*/
- @Test
+ @Test(timeout=20000)
public void testWithReplicationFactorAsOne() throws Exception {
Configuration conf = new HdfsConfiguration();
conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 1000L);
@@ -183,9 +183,14 @@ public class TestProcessCorruptBlocks {
namesystem.setReplication(fileName.toString(), (short) 1);
// wait for 3 seconds so that all block reports are processed.
- try {
- Thread.sleep(3000);
- } catch (InterruptedException ignored) {
+ for (int i = 0; i < 10; i++) {
+ try {
+ Thread.sleep(1000);
+ } catch (InterruptedException ignored) {
+ }
+ if (countReplicas(namesystem, block).corruptReplicas() == 0) {
+ break;
+ }
}
assertEquals(1, countReplicas(namesystem, block).liveReplicas());
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
index a852a690408..ff88711588c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
@@ -41,6 +41,7 @@ import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -219,7 +220,7 @@ public class TestSaveNamespace {
* Verify that a saveNamespace command brings faulty directories
* in fs.name.dir and fs.edit.dir back online.
*/
- @Test
+ @Test (timeout=30000)
public void testReinsertnamedirsInSavenamespace() throws Exception {
// create a configuration with the key to restore error
// directories in fs.name.dir
@@ -237,10 +238,13 @@ public class TestSaveNamespace {
FSImage spyImage = spy(originalImage);
fsn.dir.fsImage = spyImage;
+ FileSystem fs = FileSystem.getLocal(conf);
File rootDir = storage.getStorageDir(0).getRoot();
- rootDir.setExecutable(false);
- rootDir.setWritable(false);
- rootDir.setReadable(false);
+ Path rootPath = new Path(rootDir.getPath(), "current");
+ final FsPermission permissionNone = new FsPermission((short) 0);
+ final FsPermission permissionAll = new FsPermission(
+ FsAction.ALL, FsAction.READ_EXECUTE, FsAction.READ_EXECUTE);
+ fs.setPermission(rootPath, permissionNone);
try {
doAnEdit(fsn, 1);
@@ -257,9 +261,7 @@ public class TestSaveNamespace {
" bad directories.",
storage.getRemovedStorageDirs().size() == 1);
- rootDir.setExecutable(true);
- rootDir.setWritable(true);
- rootDir.setReadable(true);
+ fs.setPermission(rootPath, permissionAll);
// The next call to savenamespace should try inserting the
// erroneous directory back to fs.name.dir. This command should
@@ -290,9 +292,7 @@ public class TestSaveNamespace {
LOG.info("Reloaded image is good.");
} finally {
if (rootDir.exists()) {
- rootDir.setExecutable(true);
- rootDir.setWritable(true);
- rootDir.setReadable(true);
+ fs.setPermission(rootPath, permissionAll);
}
if (fsn != null) {
@@ -305,27 +305,27 @@ public class TestSaveNamespace {
}
}
- @Test
+ @Test (timeout=30000)
public void testRTEWhileSavingSecondImage() throws Exception {
saveNamespaceWithInjectedFault(Fault.SAVE_SECOND_FSIMAGE_RTE);
}
- @Test
+ @Test (timeout=30000)
public void testIOEWhileSavingSecondImage() throws Exception {
saveNamespaceWithInjectedFault(Fault.SAVE_SECOND_FSIMAGE_IOE);
}
- @Test
+ @Test (timeout=30000)
public void testCrashInAllImageDirs() throws Exception {
saveNamespaceWithInjectedFault(Fault.SAVE_ALL_FSIMAGES);
}
- @Test
+ @Test (timeout=30000)
public void testCrashWhenWritingVersionFiles() throws Exception {
saveNamespaceWithInjectedFault(Fault.WRITE_STORAGE_ALL);
}
- @Test
+ @Test (timeout=30000)
public void testCrashWhenWritingVersionFileInOneDir() throws Exception {
saveNamespaceWithInjectedFault(Fault.WRITE_STORAGE_ONE);
}
@@ -337,7 +337,7 @@ public class TestSaveNamespace {
* failed checkpoint since it only affected ".ckpt" files, not
* valid image files
*/
- @Test
+ @Test (timeout=30000)
public void testFailedSaveNamespace() throws Exception {
doTestFailedSaveNamespace(false);
}
@@ -347,7 +347,7 @@ public class TestSaveNamespace {
* the operator restores the directories and calls it again.
* This should leave the NN in a clean state for next start.
*/
- @Test
+ @Test (timeout=30000)
public void testFailedSaveNamespaceWithRecovery() throws Exception {
doTestFailedSaveNamespace(true);
}
@@ -421,7 +421,7 @@ public class TestSaveNamespace {
}
}
- @Test
+ @Test (timeout=30000)
public void testSaveWhileEditsRolled() throws Exception {
Configuration conf = getConf();
NameNode.initMetrics(conf, NamenodeRole.NAMENODE);
@@ -457,7 +457,7 @@ public class TestSaveNamespace {
}
}
- @Test
+ @Test (timeout=30000)
public void testTxIdPersistence() throws Exception {
Configuration conf = getConf();
NameNode.initMetrics(conf, NamenodeRole.NAMENODE);
@@ -580,7 +580,7 @@ public class TestSaveNamespace {
* open lease and destination directory exist.
* This test is a regression for HDFS-2827
*/
- @Test
+ @Test (timeout=30000)
public void testSaveNamespaceWithRenamedLease() throws Exception {
MiniDFSCluster cluster = new MiniDFSCluster.Builder(new Configuration())
.numDataNodes(1).build();
@@ -603,7 +603,7 @@ public class TestSaveNamespace {
}
}
- @Test
+ @Test (timeout=30000)
public void testSaveNamespaceWithDanglingLease() throws Exception {
MiniDFSCluster cluster = new MiniDFSCluster.Builder(new Configuration())
.numDataNodes(1).build();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
index fbb88ffe1f7..de111053574 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
@@ -31,12 +31,10 @@ import java.net.URI;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
-import java.util.Random;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
@@ -46,17 +44,21 @@ import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.LogVerificationAppender;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
import org.apache.hadoop.hdfs.util.MD5FileUtils;
import org.apache.hadoop.io.MD5Hash;
+import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.StringUtils;
+import org.apache.log4j.Logger;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@@ -111,11 +113,12 @@ public class TestStartup {
}
}
- /**
- * start MiniDFScluster, create a file (to create edits) and do a checkpoint
+ /**
+ * Create a number of fsimage checkpoints
+ * @param count number of checkpoints to create
* @throws IOException
*/
- public void createCheckPoint() throws IOException {
+ public void createCheckPoint(int count) throws IOException {
LOG.info("--starting mini cluster");
// manage dirs parameter set to false
MiniDFSCluster cluster = null;
@@ -133,15 +136,18 @@ public class TestStartup {
sn = new SecondaryNameNode(config);
assertNotNull(sn);
- // create a file
- FileSystem fileSys = cluster.getFileSystem();
- Path file1 = new Path("t1");
- DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize,
- (short) 1, seed);
-
- LOG.info("--doing checkpoint");
- sn.doCheckpoint(); // this shouldn't fail
- LOG.info("--done checkpoint");
+ // Create count new files and checkpoints
+ for (int i=0; i nameDirs = (List)FSNamesystem.getNamespaceDirs(config);
+ // Corrupt the md5 files in all the namedirs
+ for (URI uri: nameDirs) {
+ // Directory layout looks like:
+ // test/data/dfs/nameN/current/{fsimage,edits,...}
+ File nameDir = new File(uri.getPath());
+ File dfsDir = nameDir.getParentFile();
+ assertEquals(dfsDir.getName(), "dfs"); // make sure we got right dir
+ // Set the md5 file to all zeros
+ File imageFile = new File(nameDir,
+ Storage.STORAGE_DIR_CURRENT + "/"
+ + NNStorage.getImageFileName(0));
+ MD5FileUtils.saveMD5File(imageFile, new MD5Hash(new byte[16]));
+ // Only need to corrupt one if !corruptAll
+ if (!corruptAll) {
+ break;
+ }
}
}
@@ -165,7 +200,7 @@ public class TestStartup {
// get name dir and its length, then delete and recreate the directory
File dir = new File(nameDirs.get(0).getPath()); // has only one
- this.fsimageLength = new File(new File(dir, "current"),
+ this.fsimageLength = new File(new File(dir, Storage.STORAGE_DIR_CURRENT),
NameNodeFile.IMAGE.getName()).length();
if(dir.exists() && !(FileUtil.fullyDelete(dir)))
@@ -178,7 +213,7 @@ public class TestStartup {
dir = new File( nameEditsDirs.get(0).getPath()); //has only one
- this.editsLength = new File(new File(dir, "current"),
+ this.editsLength = new File(new File(dir, Storage.STORAGE_DIR_CURRENT),
NameNodeFile.EDITS.getName()).length();
if(dir.exists() && !(FileUtil.fullyDelete(dir)))
@@ -262,7 +297,7 @@ public class TestStartup {
config.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_DIR_KEY,
fileAsURI(new File(hdfsDir, "chkpt")).toString());
- createCheckPoint();
+ createCheckPoint(1);
corruptNameNodeFiles();
checkNameNodeFiles();
@@ -289,7 +324,7 @@ public class TestStartup {
config.set(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_DIR_KEY,
fileAsURI(new File(hdfsDir, "chkpt")).toString());
- createCheckPoint();
+ createCheckPoint(1);
corruptNameNodeFiles();
checkNameNodeFiles();
}
@@ -447,20 +482,18 @@ public class TestStartup {
FileSystem fs = cluster.getFileSystem();
fs.mkdirs(new Path("/test"));
- // Directory layout looks like:
- // test/data/dfs/nameN/current/{fsimage,edits,...}
- File nameDir = new File(cluster.getNameDirs(0).iterator().next().getPath());
- File dfsDir = nameDir.getParentFile();
- assertEquals(dfsDir.getName(), "dfs"); // make sure we got right dir
-
LOG.info("Shutting down cluster #1");
cluster.shutdown();
cluster = null;
- // Corrupt the md5 file to all 0s
- File imageFile = new File(nameDir, "current/" + NNStorage.getImageFileName(0));
- MD5FileUtils.saveMD5File(imageFile, new MD5Hash(new byte[16]));
-
+ // Corrupt the md5 files in all the namedirs
+ corruptFSImageMD5(true);
+
+ // Attach our own log appender so we can verify output
+ final LogVerificationAppender appender = new LogVerificationAppender();
+ final Logger logger = Logger.getRootLogger();
+ logger.addAppender(appender);
+
// Try to start a new cluster
LOG.info("\n===========================================\n" +
"Starting same cluster after simulated crash");
@@ -471,9 +504,12 @@ public class TestStartup {
.build();
fail("Should not have successfully started with corrupt image");
} catch (IOException ioe) {
- if (!ioe.getCause().getMessage().contains("is corrupt with MD5")) {
- throw ioe;
- }
+ GenericTestUtils.assertExceptionContains(
+ "Failed to load an FSImage file!", ioe);
+ int md5failures = appender.countExceptionsWithMessage(
+ " is corrupt with MD5 checksum of ");
+ // Two namedirs, so should have seen two failures
+ assertEquals(2, md5failures);
}
} finally {
if (cluster != null) {
@@ -482,6 +518,21 @@ public class TestStartup {
}
}
+ @Test(timeout=30000)
+ public void testCorruptImageFallback() throws IOException {
+ // Create two checkpoints
+ createCheckPoint(2);
+ // Delete a single md5sum
+ corruptFSImageMD5(false);
+ // Should still be able to start
+ MiniDFSCluster cluster = new MiniDFSCluster.Builder(config)
+ .format(false)
+ .manageDataDfsDirs(false)
+ .manageNameDfsDirs(false)
+ .build();
+ cluster.waitActive();
+}
+
/**
* This test tests hosts include list contains host names. After namenode
* restarts, the still alive datanodes should not have any trouble in getting
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
index 7eb347e74e0..2f93cca5dd8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode.ha;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
import java.io.File;
import java.io.IOException;
@@ -26,6 +27,8 @@ import java.io.OutputStream;
import java.net.URI;
import java.util.List;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -43,7 +46,10 @@ import org.apache.hadoop.hdfs.util.Canceler;
import org.apache.hadoop.io.compress.CompressionCodecFactory;
import org.apache.hadoop.io.compress.CompressionOutputStream;
import org.apache.hadoop.io.compress.GzipCodec;
+import org.apache.hadoop.ipc.StandbyException;
import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.GenericTestUtils.DelayAnswer;
+import org.apache.hadoop.util.ThreadUtil;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@@ -59,6 +65,8 @@ public class TestStandbyCheckpoints {
protected MiniDFSCluster cluster;
protected NameNode nn0, nn1;
protected FileSystem fs;
+
+ private static final Log LOG = LogFactory.getLog(TestStandbyCheckpoints.class);
@SuppressWarnings("rawtypes")
@Before
@@ -231,6 +239,49 @@ public class TestStandbyCheckpoints {
assertTrue(canceledOne);
}
+
+ /**
+ * Make sure that clients will receive StandbyExceptions even when a
+ * checkpoint is in progress on the SBN, and therefore the StandbyCheckpointer
+ * thread will have FSNS lock. Regression test for HDFS-4591.
+ */
+ @Test(timeout=120000)
+ public void testStandbyExceptionThrownDuringCheckpoint() throws Exception {
+
+ // Set it up so that we know when the SBN checkpoint starts and ends.
+ FSImage spyImage1 = NameNodeAdapter.spyOnFsImage(nn1);
+ DelayAnswer answerer = new DelayAnswer(LOG);
+ Mockito.doAnswer(answerer).when(spyImage1)
+ .saveNamespace(Mockito.any(FSNamesystem.class),
+ Mockito.any(Canceler.class));
+
+ // Perform some edits and wait for a checkpoint to start on the SBN.
+ doEdits(0, 2000);
+ nn0.getRpcServer().rollEditLog();
+ answerer.waitForCall();
+ answerer.proceed();
+ assertTrue("SBN is not performing checkpoint but it should be.",
+ answerer.getFireCount() == 1 && answerer.getResultCount() == 0);
+
+ // Make sure that the lock has actually been taken by the checkpointing
+ // thread.
+ ThreadUtil.sleepAtLeastIgnoreInterrupts(1000);
+ try {
+ // Perform an RPC to the SBN and make sure it throws a StandbyException.
+ nn1.getRpcServer().getFileInfo("/");
+ fail("Should have thrown StandbyException, but instead succeeded.");
+ } catch (StandbyException se) {
+ GenericTestUtils.assertExceptionContains("is not supported", se);
+ }
+
+ // Make sure that the checkpoint is still going on, implying that the client
+ // RPC to the SBN happened during the checkpoint.
+ assertTrue("SBN should have still been checkpointing.",
+ answerer.getFireCount() == 1 && answerer.getResultCount() == 0);
+ answerer.waitForResult();
+ assertTrue("SBN should have finished checkpointing.",
+ answerer.getFireCount() == 1 && answerer.getResultCount() == 1);
+ }
private void doEdits(int start, int stop) throws IOException {
for (int i = start; i < stop; i++) {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyIsHot.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyIsHot.java
index 041d754e154..ecd52437b0f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyIsHot.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyIsHot.java
@@ -143,6 +143,7 @@ public class TestStandbyIsHot {
conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 1024);
// We read from the standby to watch block locations
HAUtil.setAllowStandbyReads(conf, true);
+ conf.setLong(DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY, 0);
conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
.nnTopology(MiniDFSNNTopology.simpleHATopology())
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
index 56bf4a8ecae..007fda5fb5e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
@@ -123,7 +123,25 @@ public class TestNameNodeMetrics {
stm.read(buffer,0,4);
stm.close();
}
-
+
+ /**
+ * Test that capacity metrics are exported and pass
+ * basic sanity tests.
+ */
+ @Test (timeout = 1800)
+ public void testCapacityMetrics() throws Exception {
+ MetricsRecordBuilder rb = getMetrics(NS_METRICS);
+ long capacityTotal = MetricsAsserts.getLongGauge("CapacityTotal", rb);
+ assert(capacityTotal != 0);
+ long capacityUsed = MetricsAsserts.getLongGauge("CapacityUsed", rb);
+ long capacityRemaining =
+ MetricsAsserts.getLongGauge("CapacityRemaining", rb);
+ long capacityUsedNonDFS =
+ MetricsAsserts.getLongGauge("CapacityUsedNonDFS", rb);
+ assert(capacityUsed + capacityRemaining + capacityUsedNonDFS ==
+ capacityTotal);
+ }
+
/** Test metrics indicating the number of stale DataNodes */
@Test
public void testStaleNodes() throws Exception {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestGetConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestGetConf.java
index 652979eab48..f6730c1d4c1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestGetConf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestGetConf.java
@@ -224,7 +224,7 @@ public class TestGetConf {
/**
* Test empty configuration
*/
- @Test
+ @Test(timeout=10000)
public void testEmptyConf() throws Exception {
HdfsConfiguration conf = new HdfsConfiguration(false);
// Verify getting addresses fails
@@ -247,7 +247,7 @@ public class TestGetConf {
/**
* Test invalid argument to the tool
*/
- @Test
+ @Test(timeout=10000)
public void testInvalidArgument() throws Exception {
HdfsConfiguration conf = new HdfsConfiguration();
String[] args = {"-invalidArgument"};
@@ -259,7 +259,7 @@ public class TestGetConf {
* Tests to make sure the returned addresses are correct in case of default
* configuration with no federation
*/
- @Test
+ @Test(timeout=10000)
public void testNonFederation() throws Exception {
HdfsConfiguration conf = new HdfsConfiguration(false);
@@ -294,7 +294,7 @@ public class TestGetConf {
* Tests to make sure the returned addresses are correct in case of federation
* of setup.
*/
- @Test
+ @Test(timeout=10000)
public void testFederation() throws Exception {
final int nsCount = 10;
HdfsConfiguration conf = new HdfsConfiguration(false);
@@ -333,15 +333,16 @@ public class TestGetConf {
verifyAddresses(conf, TestType.NNRPCADDRESSES, true, nnAddresses);
}
- @Test
+ @Test(timeout=10000)
public void testGetSpecificKey() throws Exception {
HdfsConfiguration conf = new HdfsConfiguration();
conf.set("mykey", " myval ");
String[] args = {"-confKey", "mykey"};
- assertTrue(runTool(conf, args, true).equals("myval\n"));
+ String toolResult = runTool(conf, args, true);
+ assertEquals(String.format("myval%n"), toolResult);
}
- @Test
+ @Test(timeout=10000)
public void testExtraArgsThrowsError() throws Exception {
HdfsConfiguration conf = new HdfsConfiguration();
conf.set("mykey", "myval");
@@ -354,7 +355,7 @@ public class TestGetConf {
* Tests commands other than {@link Command#NAMENODE}, {@link Command#BACKUP},
* {@link Command#SECONDARY} and {@link Command#NNRPCADDRESSES}
*/
- @Test
+ @Test(timeout=10000)
public void testTool() throws Exception {
HdfsConfiguration conf = new HdfsConfiguration(false);
for (Command cmd : Command.values()) {
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestGSet.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestGSet.java
index ed2007775fe..971d538b272 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestGSet.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestGSet.java
@@ -21,6 +21,7 @@ import java.util.ConcurrentModificationException;
import java.util.Iterator;
import java.util.Random;
+import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.util.Time;
import org.junit.Assert;
import org.junit.Test;
@@ -452,4 +453,81 @@ public class TestGSet {
next = e;
}
}
+
+ /**
+ * Test for {@link LightWeightGSet#computeCapacity(double, String)}
+ * with invalid percent less than 0.
+ */
+ @Test(expected=HadoopIllegalArgumentException.class)
+ public void testComputeCapacityNegativePercent() {
+ LightWeightGSet.computeCapacity(1024, -1.0, "testMap");
+ }
+
+ /**
+ * Test for {@link LightWeightGSet#computeCapacity(double, String)}
+ * with invalid percent greater than 100.
+ */
+ @Test(expected=HadoopIllegalArgumentException.class)
+ public void testComputeCapacityInvalidPercent() {
+ LightWeightGSet.computeCapacity(1024, 101.0, "testMap");
+ }
+
+ /**
+ * Test for {@link LightWeightGSet#computeCapacity(double, String)}
+ * with invalid negative max memory
+ */
+ @Test(expected=HadoopIllegalArgumentException.class)
+ public void testComputeCapacityInvalidMemory() {
+ LightWeightGSet.computeCapacity(-1, 50.0, "testMap");
+ }
+
+ private static boolean isPowerOfTwo(int num) {
+ return num == 0 || (num > 0 && Integer.bitCount(num) == 1);
+ }
+
+ /** Return capacity as percentage of total memory */
+ private static int getPercent(long total, int capacity) {
+ // Reference size in bytes
+ double referenceSize =
+ System.getProperty("sun.arch.data.model").equals("32") ? 4.0 : 8.0;
+ return (int)(((capacity * referenceSize)/total) * 100.0);
+ }
+
+ /** Return capacity as percentage of total memory */
+ private static void testCapacity(long maxMemory, double percent) {
+ int capacity = LightWeightGSet.computeCapacity(maxMemory, percent, "map");
+ LightWeightGSet.LOG.info("Validating - total memory " + maxMemory + " percent "
+ + percent + " returned capacity " + capacity);
+ // Returned capacity is zero or power of two
+ Assert.assertTrue(isPowerOfTwo(capacity));
+
+ // Ensure the capacity returned is the nearest to the asked perecentage
+ int capacityPercent = getPercent(maxMemory, capacity);
+ if (capacityPercent == percent) {
+ return;
+ } else if (capacityPercent > percent) {
+ Assert.assertTrue(getPercent(maxMemory, capacity * 2) > percent);
+ } else {
+ Assert.assertTrue(getPercent(maxMemory, capacity / 2) < percent);
+ }
+ }
+
+ /**
+ * Test for {@link LightWeightGSet#computeCapacity(double, String)}
+ */
+ @Test
+ public void testComputeCapacity() {
+ // Tests for boundary conditions where percent or memory are zero
+ testCapacity(0, 0.0);
+ testCapacity(100, 0.0);
+ testCapacity(0, 100.0);
+
+ // Compute capacity for some 100 random max memory and percentage
+ Random r = new Random();
+ for (int i = 0; i < 100; i++) {
+ long maxMemory = r.nextInt(Integer.MAX_VALUE);
+ double percent = r.nextInt(101);
+ testCapacity(maxMemory, percent);
+ }
+ }
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestXMLUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestXMLUtils.java
new file mode 100644
index 00000000000..520107c0707
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestXMLUtils.java
@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.util;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.hdfs.util.XMLUtils.UnmanglingError;
+import org.junit.Test;
+
+public class TestXMLUtils {
+ private static void testRoundTrip(String str, String expectedMangled) {
+ String mangled = XMLUtils.mangleXmlString(str);
+ Assert.assertEquals(mangled, expectedMangled);
+ String unmangled = XMLUtils.unmangleXmlString(mangled);
+ Assert.assertEquals(unmangled, str);
+ }
+
+ @Test
+ public void testMangleEmptyString() throws Exception {
+ testRoundTrip("", "");
+ }
+
+ @Test
+ public void testMangleVanillaString() throws Exception {
+ testRoundTrip("abcdef", "abcdef");
+ }
+
+ @Test
+ public void testMangleStringWithBackSlash() throws Exception {
+ testRoundTrip("a\\bcdef", "a\\005c;bcdef");
+ testRoundTrip("\\\\", "\\005c;\\005c;");
+ }
+
+ @Test
+ public void testMangleStringWithForbiddenCodePoint() throws Exception {
+ testRoundTrip("a\u0001bcdef", "a\\0001;bcdef");
+ testRoundTrip("a\u0002\ud800bcdef", "a\\0002;\\d800;bcdef");
+ }
+
+ @Test
+ public void testInvalidSequence() throws Exception {
+ try {
+ XMLUtils.unmangleXmlString("\\000g;foo");
+ Assert.fail("expected an unmangling error");
+ } catch (UnmanglingError e) {
+ // pass through
+ }
+ try {
+ XMLUtils.unmangleXmlString("\\0");
+ Assert.fail("expected an unmangling error");
+ } catch (UnmanglingError e) {
+ // pass through
+ }
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTokens.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTokens.java
new file mode 100644
index 00000000000..9751a666cdb
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTokens.java
@@ -0,0 +1,202 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.web;
+
+import static org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod.KERBEROS;
+import static org.junit.Assert.*;
+import static org.mockito.Matchers.*;
+import static org.mockito.Mockito.*;
+
+import java.io.IOException;
+import java.net.URI;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.web.resources.DeleteOpParam;
+import org.apache.hadoop.hdfs.web.resources.GetOpParam;
+import org.apache.hadoop.hdfs.web.resources.HttpOpParam;
+import org.apache.hadoop.hdfs.web.resources.PostOpParam;
+import org.apache.hadoop.hdfs.web.resources.PutOpParam;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestWebHdfsTokens {
+ static Configuration conf;
+ static UserGroupInformation ugi;
+
+ @BeforeClass
+ public static void setup() throws IOException {
+ conf = new Configuration();
+ SecurityUtil.setAuthenticationMethod(KERBEROS, conf);
+ UserGroupInformation.setConfiguration(conf);
+ ugi = UserGroupInformation.getCurrentUser();
+ }
+
+ @SuppressWarnings("unchecked")
+ @Test(timeout=1000)
+ public void testInitWithNoToken() throws IOException {
+ WebHdfsFileSystem fs = spy(new WebHdfsFileSystem());
+ doReturn(null).when(fs).getDelegationToken(anyString());
+ doNothing().when(fs).addRenewAction(any(WebHdfsFileSystem.class));
+ fs.initialize(URI.create("webhdfs://127.0.0.1:0"), conf);
+
+ // when not in ugi, don't get one
+ verify(fs).initDelegationToken();
+ verify(fs).selectDelegationToken(ugi);
+ verify(fs, never()).setDelegationToken(any(Token.class));
+ verify(fs, never()).getDelegationToken();
+ verify(fs, never()).getDelegationToken(anyString());
+ }
+
+ @SuppressWarnings("unchecked")
+ @Test(timeout=1000)
+ public void testInitWithUGIToken() throws IOException {
+ WebHdfsFileSystem fs = spy(new WebHdfsFileSystem());
+ Token token = mock(Token.class);
+ doReturn(token).when(fs).selectDelegationToken(ugi);
+ doReturn(null).when(fs).getDelegationToken(anyString());
+ doNothing().when(fs).addRenewAction(any(WebHdfsFileSystem.class));
+ fs.initialize(URI.create("webhdfs://127.0.0.1:0"), conf);
+
+ // when in the ugi, store it but don't renew it
+ verify(fs).initDelegationToken();
+ verify(fs).selectDelegationToken(ugi);
+ verify(fs).setDelegationToken(token);
+ verify(fs, never()).getDelegationToken();
+ verify(fs, never()).getDelegationToken(anyString());
+ verify(fs, never()).addRenewAction(fs);
+ }
+
+ @SuppressWarnings("unchecked")
+ @Test(timeout=1000)
+ public void testInternalGetDelegationToken() throws IOException {
+ WebHdfsFileSystem fs = spy(new WebHdfsFileSystem());
+ Token token = mock(Token.class);
+ doReturn(null).when(fs).selectDelegationToken(ugi);
+ doReturn(token).when(fs).getDelegationToken(anyString());
+ doNothing().when(fs).addRenewAction(any(WebHdfsFileSystem.class));
+ fs.initialize(URI.create("webhdfs://127.0.0.1:0"), conf);
+
+ // get token, store it, and renew it
+ Token> token2 = fs.getDelegationToken();
+ assertEquals(token2, token);
+ verify(fs).getDelegationToken(null);
+ verify(fs).setDelegationToken(token);
+ verify(fs).addRenewAction(fs);
+ reset(fs);
+
+ // just return token, don't get/set/renew
+ token2 = fs.getDelegationToken();
+ assertEquals(token2, token);
+ verify(fs, never()).getDelegationToken(null);
+ verify(fs, never()).setDelegationToken(any(Token.class));
+ verify(fs, never()).addRenewAction(fs);
+ }
+
+ @SuppressWarnings("unchecked")
+ @Test(timeout=1000)
+ public void testTokenForNonTokenOp() throws IOException {
+ WebHdfsFileSystem fs = spy(new WebHdfsFileSystem());
+ Token token = mock(Token.class);
+ doReturn(null).when(fs).selectDelegationToken(ugi);
+ doReturn(token).when(fs).getDelegationToken(null);
+ doNothing().when(fs).addRenewAction(any(WebHdfsFileSystem.class));
+ fs.initialize(URI.create("webhdfs://127.0.0.1:0"), conf);
+
+ // should get/set/renew token
+ fs.toUrl(GetOpParam.Op.OPEN, null);
+ verify(fs).getDelegationToken();
+ verify(fs).getDelegationToken(null);
+ verify(fs).setDelegationToken(token);
+ verify(fs).addRenewAction(fs);
+ reset(fs);
+
+ // should return prior token
+ fs.toUrl(GetOpParam.Op.OPEN, null);
+ verify(fs).getDelegationToken();
+ verify(fs, never()).getDelegationToken(null);
+ verify(fs, never()).setDelegationToken(token);
+ verify(fs, never()).addRenewAction(fs);
+ }
+
+ @Test(timeout=1000)
+ public void testNoTokenForGetToken() throws IOException {
+ checkNoTokenForOperation(GetOpParam.Op.GETDELEGATIONTOKEN);
+ }
+
+ @Test(timeout=1000)
+ public void testNoTokenForCanclToken() throws IOException {
+ checkNoTokenForOperation(PutOpParam.Op.RENEWDELEGATIONTOKEN);
+ }
+
+ @Test(timeout=1000)
+ public void testNoTokenForCancelToken() throws IOException {
+ checkNoTokenForOperation(PutOpParam.Op.CANCELDELEGATIONTOKEN);
+ }
+
+ @SuppressWarnings("unchecked")
+ private void checkNoTokenForOperation(HttpOpParam.Op op) throws IOException {
+ WebHdfsFileSystem fs = spy(new WebHdfsFileSystem());
+ doReturn(null).when(fs).selectDelegationToken(ugi);
+ doReturn(null).when(fs).getDelegationToken(null);
+ doNothing().when(fs).addRenewAction(any(WebHdfsFileSystem.class));
+ fs.initialize(URI.create("webhdfs://127.0.0.1:0"), conf);
+
+ // do not get a token!
+ fs.toUrl(op, null);
+ verify(fs, never()).getDelegationToken();
+ verify(fs, never()).getDelegationToken(null);
+ verify(fs, never()).setDelegationToken(any(Token.class));
+ verify(fs, never()).addRenewAction(fs);
+ }
+
+ @Test(timeout=1000)
+ public void testGetOpRequireAuth() {
+ for (HttpOpParam.Op op : GetOpParam.Op.values()) {
+ boolean expect = (op == GetOpParam.Op.GETDELEGATIONTOKEN);
+ assertEquals(expect, op.getRequireAuth());
+ }
+ }
+
+ @Test(timeout=1000)
+ public void testPutOpRequireAuth() {
+ for (HttpOpParam.Op op : PutOpParam.Op.values()) {
+ boolean expect = (op == PutOpParam.Op.RENEWDELEGATIONTOKEN ||
+ op == PutOpParam.Op.CANCELDELEGATIONTOKEN);
+ assertEquals(expect, op.getRequireAuth());
+ }
+ }
+
+ @Test(timeout=1000)
+ public void testPostOpRequireAuth() {
+ for (HttpOpParam.Op op : PostOpParam.Op.values()) {
+ assertFalse(op.getRequireAuth());
+ }
+ }
+
+ @Test(timeout=1000)
+ public void testDeleteOpRequireAuth() {
+ for (HttpOpParam.Op op : DeleteOpParam.Op.values()) {
+ assertFalse(op.getRequireAuth());
+ }
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java
index aef467a0ef2..551a37b5c0e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.hdfs.web;
+import static org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod.KERBEROS;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
@@ -26,78 +27,270 @@ import static org.mockito.Mockito.mock;
import java.io.IOException;
import java.net.URI;
import java.net.URL;
+import java.util.Arrays;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import org.apache.hadoop.hdfs.web.resources.DelegationParam;
-import org.apache.hadoop.hdfs.web.resources.HttpOpParam;
-import org.apache.hadoop.hdfs.web.resources.PutOpParam;
-import org.apache.hadoop.hdfs.web.resources.TokenArgumentParam;
+import org.apache.hadoop.hdfs.web.resources.*;
import org.apache.hadoop.io.Text;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.SecurityUtilTestHelper;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.*;
public class TestWebHdfsUrl {
+ // NOTE: port is never used
+ final URI uri = URI.create(WebHdfsFileSystem.SCHEME + "://" + "127.0.0.1:0");
- @Test
- public void testDelegationTokenInUrl() throws IOException {
- Configuration conf = new Configuration();
- final String uri = WebHdfsFileSystem.SCHEME + "://" + "127.0.0.1:9071";
- // Turn on security
- conf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
- UserGroupInformation.setConfiguration(conf);
- UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
- DelegationTokenIdentifier dtId = new DelegationTokenIdentifier(new Text(
- ugi.getUserName()), null, null);
- FSNamesystem namesystem = mock(FSNamesystem.class);
- DelegationTokenSecretManager dtSecretManager = new DelegationTokenSecretManager(
- 86400000, 86400000, 86400000, 86400000, namesystem);
- dtSecretManager.startThreads();
- Token token = new Token(
- dtId, dtSecretManager);
- token.setService(new Text("127.0.0.1:9071"));
- token.setKind(WebHdfsFileSystem.TOKEN_KIND);
- ugi.addToken(token);
- final WebHdfsFileSystem webhdfs = (WebHdfsFileSystem) FileSystem.get(
- URI.create(uri), conf);
- String tokenString = token.encodeToUrlString();
- Path fsPath = new Path("/");
- URL renewTokenUrl = webhdfs.toUrl(PutOpParam.Op.RENEWDELEGATIONTOKEN,
- fsPath, new TokenArgumentParam(tokenString));
- URL cancelTokenUrl = webhdfs.toUrl(PutOpParam.Op.CANCELDELEGATIONTOKEN,
- fsPath, new TokenArgumentParam(tokenString));
- Assert.assertEquals(
- generateUrlQueryPrefix(PutOpParam.Op.RENEWDELEGATIONTOKEN,
- ugi.getUserName())
- + "&token=" + tokenString, renewTokenUrl.getQuery());
- Token delegationToken = new Token(
- token);
- delegationToken.setKind(WebHdfsFileSystem.TOKEN_KIND);
- Assert.assertEquals(
- generateUrlQueryPrefix(PutOpParam.Op.CANCELDELEGATIONTOKEN,
- ugi.getUserName())
- + "&token="
- + tokenString
- + "&"
- + DelegationParam.NAME
- + "="
- + delegationToken.encodeToUrlString(), cancelTokenUrl.getQuery());
- }
-
- private String generateUrlQueryPrefix(HttpOpParam.Op op, String username) {
- return "op=" + op.toString() + "&user.name=" + username;
+ @Before
+ public void resetUGI() {
+ UserGroupInformation.setConfiguration(new Configuration());
}
- @Test
+ @Test(timeout=4000)
+ public void testSimpleAuthParamsInUrl() throws IOException {
+ Configuration conf = new Configuration();
+
+ UserGroupInformation ugi =
+ UserGroupInformation.createRemoteUser("test-user");
+ UserGroupInformation.setLoginUser(ugi);
+
+ WebHdfsFileSystem webhdfs = getWebHdfsFileSystem(ugi, conf);
+ Path fsPath = new Path("/");
+
+ // send user+token
+ URL fileStatusUrl = webhdfs.toUrl(GetOpParam.Op.GETFILESTATUS, fsPath);
+ checkQueryParams(
+ new String[]{
+ GetOpParam.Op.GETFILESTATUS.toQueryString(),
+ new UserParam(ugi.getShortUserName()).toString()
+ },
+ fileStatusUrl);
+ }
+
+ @Test(timeout=4000)
+ public void testSimpleProxyAuthParamsInUrl() throws IOException {
+ Configuration conf = new Configuration();
+
+ UserGroupInformation ugi =
+ UserGroupInformation.createRemoteUser("test-user");
+ ugi = UserGroupInformation.createProxyUser("test-proxy-user", ugi);
+ UserGroupInformation.setLoginUser(ugi);
+
+ WebHdfsFileSystem webhdfs = getWebHdfsFileSystem(ugi, conf);
+ Path fsPath = new Path("/");
+
+ // send real+effective
+ URL fileStatusUrl = webhdfs.toUrl(GetOpParam.Op.GETFILESTATUS, fsPath);
+ checkQueryParams(
+ new String[]{
+ GetOpParam.Op.GETFILESTATUS.toQueryString(),
+ new UserParam(ugi.getRealUser().getShortUserName()).toString(),
+ new DoAsParam(ugi.getShortUserName()).toString()
+ },
+ fileStatusUrl);
+ }
+
+ @Test(timeout=4000)
+ public void testSecureAuthParamsInUrl() throws IOException {
+ Configuration conf = new Configuration();
+ // fake turning on security so api thinks it should use tokens
+ SecurityUtil.setAuthenticationMethod(KERBEROS, conf);
+ UserGroupInformation.setConfiguration(conf);
+
+ UserGroupInformation ugi =
+ UserGroupInformation.createRemoteUser("test-user");
+ ugi.setAuthenticationMethod(KERBEROS);
+ UserGroupInformation.setLoginUser(ugi);
+
+ WebHdfsFileSystem webhdfs = getWebHdfsFileSystem(ugi, conf);
+ Path fsPath = new Path("/");
+ String tokenString = webhdfs.getDelegationToken().encodeToUrlString();
+
+ // send user
+ URL getTokenUrl = webhdfs.toUrl(GetOpParam.Op.GETDELEGATIONTOKEN, fsPath);
+ checkQueryParams(
+ new String[]{
+ GetOpParam.Op.GETDELEGATIONTOKEN.toQueryString(),
+ new UserParam(ugi.getShortUserName()).toString()
+ },
+ getTokenUrl);
+
+ // send user
+ URL renewTokenUrl = webhdfs.toUrl(PutOpParam.Op.RENEWDELEGATIONTOKEN,
+ fsPath, new TokenArgumentParam(tokenString));
+ checkQueryParams(
+ new String[]{
+ PutOpParam.Op.RENEWDELEGATIONTOKEN.toQueryString(),
+ new UserParam(ugi.getShortUserName()).toString(),
+ new TokenArgumentParam(tokenString).toString(),
+ },
+ renewTokenUrl);
+
+ // send token
+ URL cancelTokenUrl = webhdfs.toUrl(PutOpParam.Op.CANCELDELEGATIONTOKEN,
+ fsPath, new TokenArgumentParam(tokenString));
+ checkQueryParams(
+ new String[]{
+ PutOpParam.Op.CANCELDELEGATIONTOKEN.toQueryString(),
+ new UserParam(ugi.getShortUserName()).toString(),
+ new TokenArgumentParam(tokenString).toString(),
+ },
+ cancelTokenUrl);
+
+ // send token
+ URL fileStatusUrl = webhdfs.toUrl(GetOpParam.Op.GETFILESTATUS, fsPath);
+ checkQueryParams(
+ new String[]{
+ GetOpParam.Op.GETFILESTATUS.toQueryString(),
+ new DelegationParam(tokenString).toString()
+ },
+ fileStatusUrl);
+
+ // wipe out internal token to simulate auth always required
+ webhdfs.setDelegationToken(null);
+
+ // send user
+ cancelTokenUrl = webhdfs.toUrl(PutOpParam.Op.CANCELDELEGATIONTOKEN,
+ fsPath, new TokenArgumentParam(tokenString));
+ checkQueryParams(
+ new String[]{
+ PutOpParam.Op.CANCELDELEGATIONTOKEN.toQueryString(),
+ new UserParam(ugi.getShortUserName()).toString(),
+ new TokenArgumentParam(tokenString).toString(),
+ },
+ cancelTokenUrl);
+
+ // send user
+ fileStatusUrl = webhdfs.toUrl(GetOpParam.Op.GETFILESTATUS, fsPath);
+ checkQueryParams(
+ new String[]{
+ GetOpParam.Op.GETFILESTATUS.toQueryString(),
+ new UserParam(ugi.getShortUserName()).toString()
+ },
+ fileStatusUrl);
+ }
+
+ @Test(timeout=4000)
+ public void testSecureProxyAuthParamsInUrl() throws IOException {
+ Configuration conf = new Configuration();
+ // fake turning on security so api thinks it should use tokens
+ SecurityUtil.setAuthenticationMethod(KERBEROS, conf);
+ UserGroupInformation.setConfiguration(conf);
+
+ UserGroupInformation ugi =
+ UserGroupInformation.createRemoteUser("test-user");
+ ugi.setAuthenticationMethod(KERBEROS);
+ ugi = UserGroupInformation.createProxyUser("test-proxy-user", ugi);
+ UserGroupInformation.setLoginUser(ugi);
+
+ WebHdfsFileSystem webhdfs = getWebHdfsFileSystem(ugi, conf);
+ Path fsPath = new Path("/");
+ String tokenString = webhdfs.getDelegationToken().encodeToUrlString();
+
+ // send real+effective
+ URL getTokenUrl = webhdfs.toUrl(GetOpParam.Op.GETDELEGATIONTOKEN, fsPath);
+ checkQueryParams(
+ new String[]{
+ GetOpParam.Op.GETDELEGATIONTOKEN.toQueryString(),
+ new UserParam(ugi.getRealUser().getShortUserName()).toString(),
+ new DoAsParam(ugi.getShortUserName()).toString()
+ },
+ getTokenUrl);
+
+ // send real+effective
+ URL renewTokenUrl = webhdfs.toUrl(PutOpParam.Op.RENEWDELEGATIONTOKEN,
+ fsPath, new TokenArgumentParam(tokenString));
+ checkQueryParams(
+ new String[]{
+ PutOpParam.Op.RENEWDELEGATIONTOKEN.toQueryString(),
+ new UserParam(ugi.getRealUser().getShortUserName()).toString(),
+ new DoAsParam(ugi.getShortUserName()).toString(),
+ new TokenArgumentParam(tokenString).toString(),
+ },
+ renewTokenUrl);
+
+ // send token
+ URL cancelTokenUrl = webhdfs.toUrl(PutOpParam.Op.CANCELDELEGATIONTOKEN,
+ fsPath, new TokenArgumentParam(tokenString));
+ checkQueryParams(
+ new String[]{
+ PutOpParam.Op.CANCELDELEGATIONTOKEN.toQueryString(),
+ new UserParam(ugi.getRealUser().getShortUserName()).toString(),
+ new DoAsParam(ugi.getShortUserName()).toString(),
+ new TokenArgumentParam(tokenString).toString(),
+ },
+ cancelTokenUrl);
+
+ // send token
+ URL fileStatusUrl = webhdfs.toUrl(GetOpParam.Op.GETFILESTATUS, fsPath);
+ checkQueryParams(
+ new String[]{
+ GetOpParam.Op.GETFILESTATUS.toQueryString(),
+ new DelegationParam(tokenString).toString()
+ },
+ fileStatusUrl);
+
+ // wipe out internal token to simulate auth always required
+ webhdfs.setDelegationToken(null);
+
+ // send real+effective
+ cancelTokenUrl = webhdfs.toUrl(PutOpParam.Op.CANCELDELEGATIONTOKEN,
+ fsPath, new TokenArgumentParam(tokenString));
+ checkQueryParams(
+ new String[]{
+ PutOpParam.Op.CANCELDELEGATIONTOKEN.toQueryString(),
+ new UserParam(ugi.getRealUser().getShortUserName()).toString(),
+ new DoAsParam(ugi.getShortUserName()).toString(),
+ new TokenArgumentParam(tokenString).toString()
+ },
+ cancelTokenUrl);
+
+ // send real+effective
+ fileStatusUrl = webhdfs.toUrl(GetOpParam.Op.GETFILESTATUS, fsPath);
+ checkQueryParams(
+ new String[]{
+ GetOpParam.Op.GETFILESTATUS.toQueryString(),
+ new UserParam(ugi.getRealUser().getShortUserName()).toString(),
+ new DoAsParam(ugi.getShortUserName()).toString()
+ },
+ fileStatusUrl);
+ }
+
+ private void checkQueryParams(String[] expected, URL url) {
+ Arrays.sort(expected);
+ String[] query = url.getQuery().split("&");
+ Arrays.sort(query);
+ assertEquals(Arrays.toString(expected), Arrays.toString(query));
+ }
+
+ private WebHdfsFileSystem getWebHdfsFileSystem(UserGroupInformation ugi,
+ Configuration conf) throws IOException {
+ if (UserGroupInformation.isSecurityEnabled()) {
+ DelegationTokenIdentifier dtId = new DelegationTokenIdentifier(new Text(
+ ugi.getUserName()), null, null);
+ FSNamesystem namesystem = mock(FSNamesystem.class);
+ DelegationTokenSecretManager dtSecretManager = new DelegationTokenSecretManager(
+ 86400000, 86400000, 86400000, 86400000, namesystem);
+ dtSecretManager.startThreads();
+ Token token = new Token(
+ dtId, dtSecretManager);
+ SecurityUtil.setTokenService(
+ token, NetUtils.createSocketAddr(uri.getAuthority()));
+ token.setKind(WebHdfsFileSystem.TOKEN_KIND);
+ ugi.addToken(token);
+ }
+ return (WebHdfsFileSystem) FileSystem.get(uri, conf);
+ }
+
+ @Test(timeout=4000)
public void testSelectHdfsDelegationToken() throws Exception {
SecurityUtilTestHelper.setTokenServiceUseIp(true);
@@ -186,8 +379,5 @@ public class TestWebHdfsUrl {
public int getDefaultPort() {
return super.getDefaultPort();
}
- // don't automatically get a token
- @Override
- protected void initDelegationToken() throws IOException {}
}
}
\ No newline at end of file
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java
index c228c1f2989..97223ab2fb4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/resources/TestParam.java
@@ -17,18 +17,22 @@
*/
package org.apache.hadoop.hdfs.web.resources;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+import java.util.Arrays;
+
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.util.StringUtils;
import org.junit.Assert;
import org.junit.Test;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-
public class TestParam {
public static final Log LOG = LogFactory.getLog(TestParam.class);
@@ -265,4 +269,20 @@ public class TestParam {
UserParam userParam = new UserParam("a$");
assertNotNull(userParam.getValue());
}
+
+ @Test
+ public void testConcatSourcesParam() {
+ final String[] strings = {"/", "/foo", "/bar"};
+ for(int n = 0; n < strings.length; n++) {
+ final String[] sub = new String[n];
+ final Path[] paths = new Path[n];
+ for(int i = 0; i < paths.length; i++) {
+ paths[i] = new Path(sub[i] = strings[i]);
+ }
+
+ final String expected = StringUtils.join(",", Arrays.asList(sub));
+ final ConcatSourcesParam computed = new ConcatSourcesParam(paths);
+ Assert.assertEquals(expected, computed.getValue());
+ }
+ }
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/net/TestNetworkTopology.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/net/TestNetworkTopology.java
index 2c4721b6204..83d5a88cb23 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/net/TestNetworkTopology.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/net/TestNetworkTopology.java
@@ -26,12 +26,23 @@ import static org.junit.Assert.fail;
import java.util.HashMap;
import java.util.Map;
+import junit.framework.Assert;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
import org.junit.Before;
import org.junit.Test;
public class TestNetworkTopology {
+ private static final Log LOG = LogFactory.getLog(TestNetworkTopology.class);
private final static NetworkTopology cluster = new NetworkTopology();
private DatanodeDescriptor dataNodes[];
@@ -213,4 +224,65 @@ public class TestNetworkTopology {
}
}
}
+
+ @Test(timeout=180000)
+ public void testInvalidNetworkTopologiesNotCachedInHdfs() throws Exception {
+ // start a cluster
+ Configuration conf = new HdfsConfiguration();
+ MiniDFSCluster cluster = null;
+ try {
+ // bad rack topology
+ String racks[] = { "/a/b", "/c" };
+ String hosts[] = { "foo1.example.com", "foo2.example.com" };
+ cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).
+ racks(racks).hosts(hosts).build();
+ cluster.waitActive();
+
+ NamenodeProtocols nn = cluster.getNameNodeRpc();
+ Assert.assertNotNull(nn);
+
+ // Wait for one DataNode to register.
+ // The other DataNode will not be able to register up because of the rack mismatch.
+ DatanodeInfo[] info;
+ while (true) {
+ info = nn.getDatanodeReport(DatanodeReportType.LIVE);
+ Assert.assertFalse(info.length == 2);
+ if (info.length == 1) {
+ break;
+ }
+ Thread.sleep(1000);
+ }
+ // Set the network topology of the other node to the match the network
+ // topology of the node that came up.
+ int validIdx = info[0].getHostName().equals(hosts[0]) ? 0 : 1;
+ int invalidIdx = validIdx == 1 ? 0 : 1;
+ StaticMapping.addNodeToRack(hosts[invalidIdx], racks[validIdx]);
+ LOG.info("datanode " + validIdx + " came up with network location " +
+ info[0].getNetworkLocation());
+
+ // Restart the DN with the invalid topology and wait for it to register.
+ cluster.restartDataNode(invalidIdx);
+ Thread.sleep(5000);
+ while (true) {
+ info = nn.getDatanodeReport(DatanodeReportType.LIVE);
+ if (info.length == 2) {
+ break;
+ }
+ if (info.length == 0) {
+ LOG.info("got no valid DNs");
+ } else if (info.length == 1) {
+ LOG.info("got one valid DN: " + info[0].getHostName() +
+ " (at " + info[0].getNetworkLocation() + ")");
+ }
+ Thread.sleep(1000);
+ }
+ Assert.assertEquals(info[0].getNetworkLocation(),
+ info[1].getNetworkLocation());
+ } finally {
+ if (cluster != null) {
+ cluster.shutdown();
+ }
+ }
+ }
+
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
index 2fb10837fcd..940c0f6f98e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
@@ -5457,6 +5457,39 @@
+
+
+ checksum: checksum of files(relative path) using globbing
+
+ -fs NAMENODE -mkdir -p dir0
+ -fs NAMENODE -put CLITEST_DATA/data15bytes dir0/data15bytes
+ -fs NAMENODE -put CLITEST_DATA/data30bytes dir0/data30bytes
+ -fs NAMENODE -put CLITEST_DATA/data60bytes dir0/data60bytes
+ -fs NAMENODE -put CLITEST_DATA/data120bytes dir0/data120bytes
+ -fs NAMENODE -checksum dir0/data*
+
+
+ -fs NAMENODE -rm -r /user
+
+
+
+ RegexpComparator
+ ^dir0/data120bytes\tMD5-of-0MD5-of-512CRC32C\t000002000000000000000000a58cdc3c0967fc8cddb7fed5960d06f2
+
+
+ RegexpComparator
+ ^dir0/data15bytes\tMD5-of-0MD5-of-512CRC32C\t0000020000000000000000007267e9528002723a30939aefc238d665
+
+
+ RegexpComparator
+ ^dir0/data30bytes\tMD5-of-0MD5-of-512CRC32C\t000002000000000000000000fc09371298117c4943cf089b4bd79c96
+
+
+ RegexpComparator
+ ^dir0/data60bytes\tMD5-of-0MD5-of-512CRC32C\t000002000000000000000000009476431d851dd7b0a8d057a404d7b9
+
+
+