Repository: hadoop
Updated Branches:
  refs/heads/branch-2.8 d02886bf8 -> 4c0b9c906


HDFS-9528. Cleanup namenode audit/log/exception messages. (szetszwo via 
umamahesh)

(cherry picked from commit 796a676d18bd7cd3ed4113d002e0e69cf261d6d1)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/4c0b9c90
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/4c0b9c90
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/4c0b9c90

Branch: refs/heads/branch-2.8
Commit: 4c0b9c9062dde6e36c2c367d5bd48bcf5a354099
Parents: d02886b
Author: Uma Mahesh <umamah...@apache.org>
Authored: Fri Dec 11 17:57:35 2015 -0800
Committer: Uma Mahesh <umamah...@apache.org>
Committed: Fri Dec 11 19:32:40 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   2 +
 .../server/blockmanagement/BlockManager.java    |  51 +++------
 .../blockmanagement/DecommissionManager.java    |  32 +++---
 .../hdfs/server/namenode/FSNamesystem.java      | 103 +++++++++----------
 .../hdfs/server/namenode/NamenodeFsck.java      |  10 +-
 .../org/apache/hadoop/hdfs/TestFileAppend4.java |  18 ++--
 .../apache/hadoop/hdfs/TestFileCreation.java    |   6 +-
 .../blockmanagement/BlockManagerTestUtil.java   |   2 +-
 8 files changed, 90 insertions(+), 134 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4c0b9c90/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt 
b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index d7c21b0..df5950b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -841,6 +841,8 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-9472. concat() API does not give proper exception messages on 
./reserved 
     relative path (Rakesh R via umamahesh)
+    
+    HDFS-9528. Cleanup namenode audit/log/exception messages. (szetszwo via 
umamahesh)
 
     HDFS-9532. Detailed exception info is lost in reportTo methods of
     ErrorReportAction and ReportBadBlockAction. (Yongjun Zhang)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4c0b9c90/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 908b72d..cbc4571 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -991,27 +991,17 @@ public class BlockManager implements BlockStatsMXBean {
    public void verifyReplication(String src,
                           short replication,
                           String clientName) throws IOException {
+    String err = null;
+    if (replication > maxReplication) {
+      err = " exceeds maximum of " + maxReplication;
+    } else if (replication < minReplication) {
+      err = " is less than the required minimum of " + minReplication;
+    }
 
-    if (replication < minReplication || replication > maxReplication) {
-      StringBuilder msg = new StringBuilder("Requested replication factor of 
");
-
-      msg.append(replication);
-
-      if (replication > maxReplication) {
-        msg.append(" exceeds maximum of ");
-        msg.append(maxReplication);
-      } else {
-        msg.append(" is less than the required minimum of ");
-        msg.append(minReplication);
-      }
-
-      msg.append(" for ").append(src);
-
-      if (clientName != null) {
-        msg.append(" from ").append(clientName);
-      }
-
-      throw new IOException(msg.toString());
+    if (err != null) {
+      throw new IOException("Requested replication factor of " + replication
+          + err + " for " + src
+          + (clientName == null? "": ", clientName=" + clientName));
     }
   }
 
@@ -1140,8 +1130,7 @@ public class BlockManager implements BlockStatsMXBean {
       datanodes.append(node).append(" ");
     }
     if (datanodes.length() != 0) {
-      blockLog.debug("BLOCK* addToInvalidates: {} {}", b,
-          datanodes.toString());
+      blockLog.debug("BLOCK* addToInvalidates: {} {}", b, datanodes);
     }
   }
 
@@ -2585,7 +2574,8 @@ public class BlockManager implements BlockStatsMXBean {
     if (result == AddBlockResult.ADDED) {
       curReplicaDelta = 1;
       if (logEveryBlock) {
-        logAddStoredBlock(storedBlock, node);
+        blockLog.debug("BLOCK* addStoredBlock: {} is added to {} (size={})",
+            node, storedBlock, storedBlock.getNumBytes());
       }
     } else if (result == AddBlockResult.REPLACED) {
       curReplicaDelta = 0;
@@ -2657,21 +2647,6 @@ public class BlockManager implements BlockStatsMXBean {
     return storedBlock;
   }
 
-  private void logAddStoredBlock(BlockInfo storedBlock,
-      DatanodeDescriptor node) {
-    if (!blockLog.isDebugEnabled()) {
-      return;
-    }
-
-    StringBuilder sb = new StringBuilder(500);
-    sb.append("BLOCK* addStoredBlock: blockMap updated: ")
-      .append(node)
-      .append(" is added to ");
-    storedBlock.appendStringTo(sb);
-    sb.append(" size " )
-      .append(storedBlock.getNumBytes());
-    blockLog.debug(sb.toString());
-  }
   /**
    * Invalidate corrupt replicas.
    * <p>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4c0b9c90/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
index 7b728d5..c895a45 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
@@ -17,6 +17,9 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.hadoop.util.Time.monotonicNow;
+
 import java.util.AbstractList;
 import java.util.Iterator;
 import java.util.LinkedList;
@@ -26,13 +29,9 @@ import java.util.Queue;
 import java.util.TreeMap;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -44,8 +43,9 @@ import org.apache.hadoop.util.ChunkedArrayList;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static com.google.common.base.Preconditions.checkArgument;
-import static org.apache.hadoop.util.Time.monotonicNow;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 /**
  * Manages datanode decommissioning. A background monitor thread 
@@ -464,17 +464,10 @@ public class DecommissionManager {
             LOG.debug("Node {} is sufficiently replicated and healthy, "
                 + "marked as decommissioned.", dn);
           } else {
-            if (LOG.isDebugEnabled()) {
-              StringBuilder b = new StringBuilder("Node {} ");
-              if (isHealthy) {
-                b.append("is ");
-              } else {
-                b.append("isn't ");
-              }
-              b.append("healthy and still needs to replicate {} more blocks," +
-                  " decommissioning is still in progress.");
-              LOG.debug(b.toString(), dn, blocks.size());
-            }
+            LOG.debug("Node {} {} healthy."
+                + " It needs to replicate {} more blocks."
+                + " Decommissioning is still in progress.",
+                dn, isHealthy? "is": "isn't", blocks.size());
           }
         } else {
           LOG.debug("Node {} still has {} blocks to replicate "
@@ -615,8 +608,7 @@ public class DecommissionManager {
   }
 
   @VisibleForTesting
-  void runMonitor() throws ExecutionException, InterruptedException {
-    Future f = executor.submit(monitor);
-    f.get();
+  void runMonitorForTest() throws ExecutionException, InterruptedException {
+    executor.submit(monitor).get();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4c0b9c90/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 08dcec1..384bbf6 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -312,19 +312,11 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
   NameNodeMXBean {
   public static final Log LOG = LogFactory.getLog(FSNamesystem.class);
 
-  private static final ThreadLocal<StringBuilder> auditBuffer =
-    new ThreadLocal<StringBuilder>() {
-      @Override
-      protected StringBuilder initialValue() {
-        return new StringBuilder();
-      }
-  };
-
   private final BlockIdManager blockIdManager;
 
-  @VisibleForTesting
-  public boolean isAuditEnabled() {
-    return !isDefaultAuditLogger || auditLog.isInfoEnabled();
+  boolean isAuditEnabled() {
+    return (!isDefaultAuditLogger || auditLog.isInfoEnabled())
+        && !auditLoggers.isEmpty();
   }
 
   private void logAuditEvent(boolean succeeded, String cmd, String src)
@@ -352,14 +344,14 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
           stat.getAccessTime(), stat.getPermission(), stat.getOwner(),
           stat.getGroup(), symlink, path);
     }
+    final String ugiStr = ugi.toString();
     for (AuditLogger logger : auditLoggers) {
       if (logger instanceof HdfsAuditLogger) {
         HdfsAuditLogger hdfsLogger = (HdfsAuditLogger) logger;
-        hdfsLogger.logAuditEvent(succeeded, ugi.toString(), addr, cmd, src, 
dst,
+        hdfsLogger.logAuditEvent(succeeded, ugiStr, addr, cmd, src, dst,
             status, CallerContext.getCurrent(), ugi, dtSecretManager);
       } else {
-        logger.logAuditEvent(succeeded, ugi.toString(), addr,
-            cmd, src, dst, status);
+        logger.logAuditEvent(succeeded, ugiStr, addr, cmd, src, dst, status);
       }
     }
   }
@@ -622,15 +614,13 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
       // sharedEditsDirs.
       if (!editsDirs.contains(u) &&
           !sharedEditsDirs.contains(u)) {
-        throw new IllegalArgumentException(
-            "Required edits directory " + u.toString() + " not present in " +
-            DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY + ". " +
-            DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY + "=" +
-            editsDirs.toString() + "; " +
-            DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_REQUIRED_KEY + "=" +
-            requiredEditsDirs.toString() + ". " +
-            DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY + "=" +
-            sharedEditsDirs.toString() + ".");
+        throw new IllegalArgumentException("Required edits directory " + u
+            + " not found: "
+            + DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY + "=" + editsDirs + "; "
+            + DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_REQUIRED_KEY
+            + "=" + requiredEditsDirs + "; "
+            + DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY
+            + "=" + sharedEditsDirs);
       }
     }
 
@@ -704,11 +694,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
   FSNamesystem(Configuration conf, FSImage fsImage, boolean ignoreRetryCache)
       throws IOException {
     provider = DFSUtil.createKeyProviderCryptoExtension(conf);
-    if (provider == null) {
-      LOG.info("No KeyProvider found.");
-    } else {
-      LOG.info("Found KeyProvider: " + provider.toString());
-    }
+    LOG.info("KeyProvider: " + provider);
     if (conf.getBoolean(DFS_NAMENODE_AUDIT_LOG_ASYNC_KEY,
                         DFS_NAMENODE_AUDIT_LOG_ASYNC_DEFAULT)) {
       LOG.info("Enabling async auditlog");
@@ -2101,11 +2087,10 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
           .append(", clientMachine=").append(clientMachine)
           .append(", createParent=").append(createParent)
           .append(", replication=").append(replication)
-          .append(", createFlag=").append(flag.toString())
+          .append(", createFlag=").append(flag)
           .append(", blockSize=").append(blockSize)
           .append(", supportedVersions=")
-          .append(supportedVersions == null ? null : Arrays.toString
-              (supportedVersions));
+          .append(Arrays.toString(supportedVersions));
       NameNode.stateChangeLog.debug(builder.toString());
     }
     if (!DFSUtil.isValidName(src)) {
@@ -2532,42 +2517,40 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
     getEditLog().logSync();
   }
 
-  INodeFile checkLease(
-      String src, String holder, INode inode, long fileId) throws 
LeaseExpiredException, FileNotFoundException {
+  private String leaseExceptionString(String src, long fileId, String holder) {
+    final Lease lease = leaseManager.getLease(holder);
+    return src + " (inode " + fileId + ") " + (lease != null? lease.toString()
+        : "Holder " + holder + " does not have any open files.");
+  }
+
+  INodeFile checkLease(String src, String holder, INode inode, long fileId)
+      throws LeaseExpiredException, FileNotFoundException {
     assert hasReadLock();
-    final String ident = src + " (inode " + fileId + ")";
     if (inode == null) {
-      Lease lease = leaseManager.getLease(holder);
-      throw new FileNotFoundException(
-          "No lease on " + ident + ": File does not exist. "
-          + (lease != null ? lease.toString()
-              : "Holder " + holder + " does not have any open files."));
+      throw new FileNotFoundException("File does not exist: "
+          + leaseExceptionString(src, fileId, holder));
     }
     if (!inode.isFile()) {
-      Lease lease = leaseManager.getLease(holder);
-      throw new LeaseExpiredException(
-          "No lease on " + ident + ": INode is not a regular file. "
-              + (lease != null ? lease.toString()
-              : "Holder " + holder + " does not have any open files."));
+      throw new LeaseExpiredException("INode is not a regular file: "
+          + leaseExceptionString(src, fileId, holder));
     }
     final INodeFile file = inode.asFile();
     if (!file.isUnderConstruction()) {
-      Lease lease = leaseManager.getLease(holder);
-      throw new LeaseExpiredException(
-          "No lease on " + ident + ": File is not open for writing. "
-          + (lease != null ? lease.toString()
-              : "Holder " + holder + " does not have any open files."));
+      throw new LeaseExpiredException("File is not open for writing: "
+          + leaseExceptionString(src, fileId, holder));
     }
     // No further modification is allowed on a deleted file.
     // A file is considered deleted, if it is not in the inodeMap or is marked
     // as deleted in the snapshot feature.
     if (isFileDeleted(file)) {
-      throw new FileNotFoundException(src);
+      throw new FileNotFoundException("File is deleted: "
+          + leaseExceptionString(src, fileId, holder));
     }
-    String clientName = file.getFileUnderConstructionFeature().getClientName();
-    if (holder != null && !clientName.equals(holder)) {
-      throw new LeaseExpiredException("Lease mismatch on " + ident +
-          " owned by " + clientName + " but is accessed by " + holder);
+    final String owner = 
file.getFileUnderConstructionFeature().getClientName();
+    if (holder != null && !owner.equals(holder)) {
+      throw new LeaseExpiredException("Client (=" + holder
+          + ") is not the lease owner (=" + owner + ": "
+          + leaseExceptionString(src, fileId, holder));
     }
     return file;
   }
@@ -6866,7 +6849,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
       if (success) {
         getEditLog().logSync();
       }
-      String idStr = "{id: " + directive.getId().toString() + "}";
+      final String idStr = "{id: " + directive.getId() + "}";
       logAuditEvent(success, "modifyCacheDirective", idStr,
           directive.toString(), null);
     }
@@ -7279,6 +7262,14 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
    */
   @VisibleForTesting
   static class DefaultAuditLogger extends HdfsAuditLogger {
+    private static final ThreadLocal<StringBuilder> STRING_BUILDER =
+        new ThreadLocal<StringBuilder>() {
+          @Override
+          protected StringBuilder initialValue() {
+            return new StringBuilder();
+          }
+        };
+
     private boolean isCallerContextEnabled;
     private int callerContextMaxLen;
     private int callerSignatureMaxLen;
@@ -7313,7 +7304,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
 
       if (auditLog.isDebugEnabled() ||
           (auditLog.isInfoEnabled() && !debugCmdSet.contains(cmd))) {
-        final StringBuilder sb = auditBuffer.get();
+        final StringBuilder sb = STRING_BUILDER.get();
         sb.setLength(0);
         sb.append("allowed=").append(succeeded).append("\t");
         sb.append("ugi=").append(userName).append("\t");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4c0b9c90/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index be4d1c5..5f9e16d 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
@@ -312,7 +312,7 @@ public class NamenodeFsck implements 
DataEncryptionKeyFactory {
         sb.append("FSCK started by " +
             UserGroupInformation.getCurrentUser() + " from " +
             remoteAddress + " at " + new Date());
-        out.println(sb.toString());
+        out.println(sb);
         sb.append(" for blockIds: \n");
         for (String blk: blocks) {
           if(blk == null || !blk.contains(Block.BLOCK_FILE_PREFIX)) {
@@ -323,7 +323,7 @@ public class NamenodeFsck implements 
DataEncryptionKeyFactory {
           blockIdCK(blk);
           sb.append(blk + "\n");
         }
-        LOG.info(sb.toString());
+        LOG.info(sb);
         namenode.getNamesystem().logFsckEvent("/", remoteAddress);
         out.flush();
         return;
@@ -367,7 +367,7 @@ public class NamenodeFsck implements 
DataEncryptionKeyFactory {
         out.println(" Number of racks:\t\t" + networktopology.getNumOfRacks());
 
         if (this.showStoragePolcies) {
-          out.print(storageTypeSummary.toString());
+          out.print(storageTypeSummary);
         }
 
         out.println("FSCK ended at " + new Date() + " in "
@@ -633,7 +633,7 @@ public class NamenodeFsck implements 
DataEncryptionKeyFactory {
       report.append(blockNumber + ". " + blkName + " len=" + 
block.getNumBytes());
       if (totalReplicasPerBlock == 0) {
         report.append(" MISSING!");
-        res.addMissing(block.toString(), block.getNumBytes());
+        res.addMissing(blkName, block.getNumBytes());
         missing++;
         missize += block.getNumBytes();
       } else {
@@ -706,7 +706,7 @@ public class NamenodeFsck implements 
DataEncryptionKeyFactory {
         out.print(" OK\n");
       }
       if (showBlocks) {
-        out.print(report.toString() + "\n");
+        out.print(report + "\n");
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4c0b9c90/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java
index 1acab73..62f6cd2 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
+import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.log4j.Level;
@@ -198,14 +199,12 @@ public class TestFileAppend4 {
       t.join();
       LOG.info("Close finished.");
  
-      // We expect that close will get a "File is not open"
-      // error.
+      // We expect that close will get a "File is not open" error.
       Throwable thrownByClose = err.get();
       assertNotNull(thrownByClose);
-      assertTrue(thrownByClose instanceof IOException);
-      if (!thrownByClose.getMessage().contains(
-            "No lease on /testRecoverFinalized"))
-        throw thrownByClose;
+      assertTrue(thrownByClose instanceof LeaseExpiredException);
+      GenericTestUtils.assertExceptionContains("File is not open for writing",
+          thrownByClose);
     } finally {
       cluster.shutdown();
     }
@@ -281,10 +280,9 @@ public class TestFileAppend4 {
       // error.
       Throwable thrownByClose = err.get();
       assertNotNull(thrownByClose);
-      assertTrue(thrownByClose instanceof IOException);
-      if (!thrownByClose.getMessage().contains(
-            "Lease mismatch"))
-        throw thrownByClose;
+      assertTrue(thrownByClose instanceof LeaseExpiredException);
+      GenericTestUtils.assertExceptionContains("not the lease owner",
+          thrownByClose);
       
       // The appender should be able to close properly
       appenderStream.close();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4c0b9c90/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
index 09df4bd..05c98ac 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
@@ -165,7 +165,7 @@ public class TestFileCreation {
     cluster.waitActive();
     FileSystem fs = cluster.getFileSystem();
     try {
-      FsServerDefaults serverDefaults = fs.getServerDefaults();
+      FsServerDefaults serverDefaults = fs.getServerDefaults(new Path("/"));
       assertEquals(DFS_BLOCK_SIZE_DEFAULT, serverDefaults.getBlockSize());
       assertEquals(DFS_BYTES_PER_CHECKSUM_DEFAULT, 
serverDefaults.getBytesPerChecksum());
       assertEquals(DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT, 
serverDefaults.getWritePacketSize());
@@ -413,8 +413,7 @@ public class TestFileCreation {
         stm1.close();
         fail("Should have exception closing stm1 since it was deleted");
       } catch (IOException ioe) {
-        GenericTestUtils.assertExceptionContains("No lease on /testfile", ioe);
-        GenericTestUtils.assertExceptionContains("File does not exist.", ioe);
+        GenericTestUtils.assertExceptionContains("File does not exist", ioe);
       }
       
     } finally {
@@ -805,7 +804,6 @@ public class TestFileCreation {
   public static void testFileCreationNonRecursive(FileSystem fs) throws 
IOException {
     final Path path = new Path("/" + Time.now()
         + "-testFileCreationNonRecursive");
-    FSDataOutputStream out = null;
     IOException expectedException = null;
     final String nonExistDir = "/non-exist-" + Time.now();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4c0b9c90/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
index 148135b..9f80376 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
@@ -304,6 +304,6 @@ public class BlockManagerTestUtil {
    */
   public static void recheckDecommissionState(DatanodeManager dm)
       throws ExecutionException, InterruptedException {
-    dm.getDecomManager().runMonitor();
+    dm.getDecomManager().runMonitorForTest();
   }
 }

Reply via email to