소스 검색

HDFS-4887. TestNNThroughputBenchmark exits abruptly. Contributed by Kihwal Lee.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1501841 13f79535-47bb-0310-9956-ffa450edef68
Kihwal Lee 12 년 전
부모
커밋
3a3109f8bf

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -652,6 +652,8 @@ Release 2.1.0-beta - 2013-07-02
     HDFS-4943. WebHdfsFileSystem does not work when original file path has
     encoded chars.  (Jerry He via szetszwo)
 
+    HDFS-4887. TestNNThroughputBenchmark exits abruptly. (kihwal)
+
   BREAKDOWN OF HDFS-347 SUBTASKS AND RELATED JIRAS
 
     HDFS-4353. Encapsulate connections to peers in Peer and PeerServer classes.

+ 12 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -227,6 +227,9 @@ public class BlockManager {
 
   /** for block replicas placement */
   private BlockPlacementPolicy blockplacement;
+
+  /** Check whether name system is running before terminating */
+  private boolean checkNSRunning = true;
   
   public BlockManager(final Namesystem namesystem, final FSClusterStats stats,
       final Configuration conf) throws IOException {
@@ -356,6 +359,12 @@ public class BlockManager {
     return blockTokenSecretManager;
   }
 
+  /** Allow silent termination of replication monitor for testing */
+  @VisibleForTesting
+  void enableRMTerminationForTesting() {
+    checkNSRunning = false;
+  }
+
   private boolean isBlockTokenEnabled() {
     return blockTokenSecretManager != null;
   }
@@ -3129,6 +3138,9 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
                   + " while shutting down.", t);
             }
             break;
+          } else if (!checkNSRunning && t instanceof InterruptedException) {
+            LOG.info("Stopping ReplicationMonitor for testing.");
+            break;
           }
           LOG.fatal("ReplicationMonitor thread received Runtime exception. ", t);
           terminate(1, t);

+ 16 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java

@@ -107,6 +107,22 @@ public class BlockManagerTestUtil {
     return blockManager.replicationThread;
   }
   
+  /**
+   * Stop the replication monitor thread
+   * @param blockManager
+   */
+  public static void stopReplicationThread(final BlockManager blockManager) 
+      throws IOException {
+    blockManager.enableRMTerminationForTesting();
+    blockManager.replicationThread.interrupt();
+    try {
+      blockManager.replicationThread.join();
+    } catch(InterruptedException ie) {
+      throw new IOException(
+          "Interrupted while trying to stop ReplicationMonitor");
+    }
+  }
+
   /**
    * @param blockManager
    * @return corruptReplicas from block manager

+ 2 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java

@@ -1194,14 +1194,8 @@ public class NNThroughputBenchmark {
       // start data-nodes; create a bunch of files; generate block reports.
       blockReportObject.generateInputs(ignore);
       // stop replication monitor
-      BlockManagerTestUtil.getReplicationThread(namesystem.getBlockManager())
-          .interrupt();
-      try {
-        BlockManagerTestUtil.getReplicationThread(namesystem.getBlockManager())
-            .join();
-      } catch(InterruptedException ei) {
-        return;
-      }
+      BlockManagerTestUtil.stopReplicationThread(namesystem.getBlockManager());
+
       // report blocks once
       int nrDatanodes = blockReportObject.getNumDatanodes();
       for(int idx=0; idx < nrDatanodes; idx++) {