You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by su...@apache.org on 2012/11/09 19:15:16 UTC

svn commit: r1407571 - in /hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs: ./ src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ src/main/java/org/apache/hadoop/hdfs/server/namenode/ src/test/java/org/apache/hadoop/hdfs/

Author: suresh
Date: Fri Nov  9 18:15:14 2012
New Revision: 1407571

URL: http://svn.apache.org/viewvc?rev=1407571&view=rev
Log:
HDFS-4080. Merging change r1407566 from trunk

Modified:
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CorruptReplicasMap.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeDeath.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend2.java

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt?rev=1407571&r1=1407570&r2=1407571&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt Fri Nov  9 18:15:14 2012
@@ -1677,6 +1677,8 @@ Release 0.23.5 - UNRELEASED
   NEW FEATURES
 
   IMPROVEMENTS
+    HDFS-4080. Add a separate logger for block state change logs to enable turning
+    off those logs. (Kihwal Lee via suresh)
 
   OPTIMIZATIONS
 

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java?rev=1407571&r1=1407570&r2=1407571&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java Fri Nov  9 18:15:14 2012
@@ -233,7 +233,7 @@ public class BlockInfoUnderConstruction 
     setBlockUCState(BlockUCState.UNDER_RECOVERY);
     blockRecoveryId = recoveryId;
     if (replicas.size() == 0) {
-      NameNode.stateChangeLog.warn("BLOCK*"
+      NameNode.blockStateChangeLog.warn("BLOCK*"
         + " BlockInfoUnderConstruction.initLeaseRecovery:"
         + " No blocks found, lease removed.");
     }
@@ -245,7 +245,7 @@ public class BlockInfoUnderConstruction 
         primaryNodeIndex = j;
         DatanodeDescriptor primary = replicas.get(j).getExpectedLocation(); 
         primary.addBlockToBeRecovered(this);
-        NameNode.stateChangeLog.info("BLOCK* " + this
+        NameNode.blockStateChangeLog.info("BLOCK* " + this
           + " recovery started, primary=" + primary);
         return;
       }

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java?rev=1407571&r1=1407570&r2=1407571&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java Fri Nov  9 18:15:14 2012
@@ -84,6 +84,7 @@ import com.google.common.collect.Sets;
 public class BlockManager {
 
   static final Log LOG = LogFactory.getLog(BlockManager.class);
+  static final Log blockLog = NameNode.blockStateChangeLog;
 
   /** Default load factor of map */
   public static final float DEFAULT_MAP_LOAD_FACTOR = 0.75f;
@@ -873,7 +874,7 @@ public class BlockManager {
       final long size) throws UnregisteredNodeException {
     final DatanodeDescriptor node = getDatanodeManager().getDatanode(datanode);
     if (node == null) {
-      NameNode.stateChangeLog.warn("BLOCK* getBlocks: "
+      blockLog.warn("BLOCK* getBlocks: "
           + "Asking for blocks from an unrecorded node " + datanode);
       throw new HadoopIllegalArgumentException(
           "Datanode " + datanode + " not found.");
@@ -951,7 +952,7 @@ public class BlockManager {
       datanodes.append(node).append(" ");
     }
     if (datanodes.length() != 0) {
-      NameNode.stateChangeLog.info("BLOCK* addToInvalidates: " + b + " "
+      blockLog.info("BLOCK* addToInvalidates: " + b + " "
           + datanodes);
     }
   }
@@ -972,7 +973,7 @@ public class BlockManager {
       // ignore the request for now. This could happen when BlockScanner
       // thread of Datanode reports bad block before Block reports are sent
       // by the Datanode on startup
-      NameNode.stateChangeLog.info("BLOCK* findAndMarkBlockAsCorrupt: "
+      blockLog.info("BLOCK* findAndMarkBlockAsCorrupt: "
           + blk + " not found");
       return;
     }
@@ -989,7 +990,7 @@ public class BlockManager {
 
     BlockCollection bc = b.corrupted.getBlockCollection();
     if (bc == null) {
-      NameNode.stateChangeLog.info("BLOCK markBlockAsCorrupt: " + b
+      blockLog.info("BLOCK markBlockAsCorrupt: " + b
           + " cannot be marked as corrupt as it does not belong to any file");
       addToInvalidates(b.corrupted, node);
       return;
@@ -1014,7 +1015,7 @@ public class BlockManager {
    */
   private void invalidateBlock(BlockToMarkCorrupt b, DatanodeInfo dn
       ) throws IOException {
-    NameNode.stateChangeLog.info("BLOCK* invalidateBlock: " + b + " on " + dn);
+    blockLog.info("BLOCK* invalidateBlock: " + b + " on " + dn);
     DatanodeDescriptor node = getDatanodeManager().getDatanode(dn);
     if (node == null) {
       throw new IOException("Cannot invalidate " + b
@@ -1024,7 +1025,7 @@ public class BlockManager {
     // Check how many copies we have of the block
     NumberReplicas nr = countNodes(b.stored);
     if (nr.replicasOnStaleNodes() > 0) {
-      NameNode.stateChangeLog.info("BLOCK* invalidateBlocks: postponing " +
+      blockLog.info("BLOCK* invalidateBlocks: postponing " +
           "invalidation of " + b + " on " + dn + " because " +
           nr.replicasOnStaleNodes() + " replica(s) are located on nodes " +
           "with potentially out-of-date block reports");
@@ -1034,12 +1035,12 @@ public class BlockManager {
       // If we have at least one copy on a live node, then we can delete it.
       addToInvalidates(b.corrupted, dn);
       removeStoredBlock(b.stored, node);
-      if(NameNode.stateChangeLog.isDebugEnabled()) {
-        NameNode.stateChangeLog.debug("BLOCK* invalidateBlocks: "
+      if(blockLog.isDebugEnabled()) {
+        blockLog.debug("BLOCK* invalidateBlocks: "
             + b + " on " + dn + " listed for deletion.");
       }
     } else {
-      NameNode.stateChangeLog.info("BLOCK* invalidateBlocks: " + b
+      blockLog.info("BLOCK* invalidateBlocks: " + b
           + " on " + dn + " is the only copy and was not deleted");
     }
   }
@@ -1161,7 +1162,7 @@ public class BlockManager {
                    (blockHasEnoughRacks(block)) ) {
                 neededReplications.remove(block, priority); // remove from neededReplications
                 neededReplications.decrementReplicationIndex(priority);
-                NameNode.stateChangeLog.info("BLOCK* Removing " + block
+                blockLog.info("BLOCK* Removing " + block
                     + " from neededReplications as it has enough replicas");
                 continue;
               }
@@ -1236,7 +1237,7 @@ public class BlockManager {
               neededReplications.remove(block, priority); // remove from neededReplications
               neededReplications.decrementReplicationIndex(priority);
               rw.targets = null;
-              NameNode.stateChangeLog.info("BLOCK* Removing " + block
+              blockLog.info("BLOCK* Removing " + block
                   + " from neededReplications as it has enough replicas");
               continue;
             }
@@ -1262,8 +1263,8 @@ public class BlockManager {
           // The reason we use 'pending' is so we can retry
           // replications that fail after an appropriate amount of time.
           pendingReplications.increment(block, targets.length);
-          if(NameNode.stateChangeLog.isDebugEnabled()) {
-            NameNode.stateChangeLog.debug(
+          if(blockLog.isDebugEnabled()) {
+            blockLog.debug(
                 "BLOCK* block " + block
                 + " is moved from neededReplications to pendingReplications");
           }
@@ -1279,7 +1280,7 @@ public class BlockManager {
       namesystem.writeUnlock();
     }
 
-    if (NameNode.stateChangeLog.isInfoEnabled()) {
+    if (blockLog.isInfoEnabled()) {
       // log which blocks have been scheduled for replication
       for(ReplicationWork rw : work){
         DatanodeDescriptor[] targets = rw.targets;
@@ -1289,13 +1290,13 @@ public class BlockManager {
             targetList.append(' ');
             targetList.append(targets[k]);
           }
-          NameNode.stateChangeLog.info("BLOCK* ask " + rw.srcNode
+          blockLog.info("BLOCK* ask " + rw.srcNode
               + " to replicate " + rw.block + " to " + targetList);
         }
       }
     }
-    if(NameNode.stateChangeLog.isDebugEnabled()) {
-        NameNode.stateChangeLog.debug(
+    if(blockLog.isDebugEnabled()) {
+        blockLog.debug(
           "BLOCK* neededReplications = " + neededReplications.size()
           + " pendingReplications = " + pendingReplications.size());
     }
@@ -1505,7 +1506,7 @@ public class BlockManager {
       // To minimize startup time, we discard any second (or later) block reports
       // that we receive while still in startup phase.
       if (namesystem.isInStartupSafeMode() && !node.isFirstBlockReport()) {
-        NameNode.stateChangeLog.info("BLOCK* processReport: "
+        blockLog.info("BLOCK* processReport: "
             + "discarded non-initial block report from " + nodeID
             + " because namenode still in startup phase");
         return;
@@ -1537,7 +1538,7 @@ public class BlockManager {
 
     // Log the block report processing stats from Namenode perspective
     NameNode.getNameNodeMetrics().addBlockReport((int) (endTime - startTime));
-    NameNode.stateChangeLog.info("BLOCK* processReport: from "
+    blockLog.info("BLOCK* processReport: from "
         + nodeID + ", blocks: " + newReport.getNumberOfBlocks()
         + ", processing time: " + (endTime - startTime) + " msecs");
   }
@@ -1597,7 +1598,7 @@ public class BlockManager {
       addStoredBlock(b, node, null, true);
     }
     for (Block b : toInvalidate) {
-      NameNode.stateChangeLog.info("BLOCK* processReport: "
+      blockLog.info("BLOCK* processReport: "
           + b + " on " + node + " size " + b.getNumBytes()
           + " does not belong to any file");
       addToInvalidates(b, node);
@@ -2035,7 +2036,7 @@ assert storedBlock.findDatanode(dn) < 0 
     }
     if (storedBlock == null || storedBlock.getBlockCollection() == null) {
       // If this block does not belong to anyfile, then we are done.
-      NameNode.stateChangeLog.info("BLOCK* addStoredBlock: " + block + " on "
+      blockLog.info("BLOCK* addStoredBlock: " + block + " on "
           + node + " size " + block.getNumBytes()
           + " but it does not belong to any file");
       // we could add this block to invalidate set of this datanode.
@@ -2057,7 +2058,7 @@ assert storedBlock.findDatanode(dn) < 0 
       }
     } else {
       curReplicaDelta = 0;
-      NameNode.stateChangeLog.warn("BLOCK* addStoredBlock: "
+      blockLog.warn("BLOCK* addStoredBlock: "
           + "Redundant addStoredBlock request received for " + storedBlock
           + " on " + node + " size " + storedBlock.getNumBytes());
     }
@@ -2116,7 +2117,7 @@ assert storedBlock.findDatanode(dn) < 0 
   }
 
   private void logAddStoredBlock(BlockInfo storedBlock, DatanodeDescriptor node) {
-    if (!NameNode.stateChangeLog.isInfoEnabled()) {
+    if (!blockLog.isInfoEnabled()) {
       return;
     }
     
@@ -2127,7 +2128,7 @@ assert storedBlock.findDatanode(dn) < 0 
     storedBlock.appendStringTo(sb);
     sb.append(" size " )
       .append(storedBlock.getNumBytes());
-    NameNode.stateChangeLog.info(sb);
+    blockLog.info(sb);
   }
   /**
    * Invalidate corrupt replicas.
@@ -2154,7 +2155,7 @@ assert storedBlock.findDatanode(dn) < 0 
       try {
         invalidateBlock(new BlockToMarkCorrupt(blk, null), node);
       } catch (IOException e) {
-        NameNode.stateChangeLog.info("invalidateCorruptReplicas "
+        blockLog.info("invalidateCorruptReplicas "
             + "error in deleting bad block " + blk + " on " + node, e);
         gotException = true;
       }
@@ -2420,7 +2421,7 @@ assert storedBlock.findDatanode(dn) < 0 
       // upon giving instructions to the namenode.
       //
       addToInvalidates(b, cur);
-      NameNode.stateChangeLog.info("BLOCK* chooseExcessReplicates: "
+      blockLog.info("BLOCK* chooseExcessReplicates: "
                 +"("+cur+", "+b+") is added to invalidated blocks set");
     }
   }
@@ -2434,8 +2435,8 @@ assert storedBlock.findDatanode(dn) < 0 
     }
     if (excessBlocks.add(block)) {
       excessBlocksCount++;
-      if(NameNode.stateChangeLog.isDebugEnabled()) {
-        NameNode.stateChangeLog.debug("BLOCK* addToExcessReplicate:"
+      if(blockLog.isDebugEnabled()) {
+        blockLog.debug("BLOCK* addToExcessReplicate:"
             + " (" + dn + ", " + block
             + ") is added to excessReplicateMap");
       }
@@ -2447,15 +2448,15 @@ assert storedBlock.findDatanode(dn) < 0 
    * removed block is still valid.
    */
   public void removeStoredBlock(Block block, DatanodeDescriptor node) {
-    if(NameNode.stateChangeLog.isDebugEnabled()) {
-      NameNode.stateChangeLog.debug("BLOCK* removeStoredBlock: "
+    if(blockLog.isDebugEnabled()) {
+      blockLog.debug("BLOCK* removeStoredBlock: "
           + block + " from " + node);
     }
     assert (namesystem.hasWriteLock());
     {
       if (!blocksMap.removeNode(block, node)) {
-        if(NameNode.stateChangeLog.isDebugEnabled()) {
-          NameNode.stateChangeLog.debug("BLOCK* removeStoredBlock: "
+        if(blockLog.isDebugEnabled()) {
+          blockLog.debug("BLOCK* removeStoredBlock: "
               + block + " has already been removed from node " + node);
         }
         return;
@@ -2482,8 +2483,8 @@ assert storedBlock.findDatanode(dn) < 0 
       if (excessBlocks != null) {
         if (excessBlocks.remove(block)) {
           excessBlocksCount--;
-          if(NameNode.stateChangeLog.isDebugEnabled()) {
-            NameNode.stateChangeLog.debug("BLOCK* removeStoredBlock: "
+          if(blockLog.isDebugEnabled()) {
+            blockLog.debug("BLOCK* removeStoredBlock: "
                 + block + " is removed from excessBlocks");
           }
           if (excessBlocks.size() == 0) {
@@ -2526,7 +2527,7 @@ assert storedBlock.findDatanode(dn) < 0 
     if (delHint != null && delHint.length() != 0) {
       delHintNode = datanodeManager.getDatanode(delHint);
       if (delHintNode == null) {
-        NameNode.stateChangeLog.warn("BLOCK* blockReceived: " + block
+        blockLog.warn("BLOCK* blockReceived: " + block
             + " is expected to be removed from an unrecorded node " + delHint);
       }
     }
@@ -2561,7 +2562,7 @@ assert storedBlock.findDatanode(dn) < 0 
       addStoredBlock(b, node, delHintNode, true);
     }
     for (Block b : toInvalidate) {
-      NameNode.stateChangeLog.info("BLOCK* addBlock: block "
+      blockLog.info("BLOCK* addBlock: block "
           + b + " on " + node + " size " + b.getNumBytes()
           + " does not belong to any file");
       addToInvalidates(b, node);
@@ -2587,7 +2588,7 @@ assert storedBlock.findDatanode(dn) < 0 
     try {
       final DatanodeDescriptor node = datanodeManager.getDatanode(nodeID);
       if (node == null || !node.isAlive) {
-        NameNode.stateChangeLog
+        blockLog
             .warn("BLOCK* processIncrementalBlockReport"
                 + " is received from dead or unregistered node "
                 + nodeID);
@@ -2614,19 +2615,19 @@ assert storedBlock.findDatanode(dn) < 0 
           String msg = 
             "Unknown block status code reported by " + nodeID +
             ": " + rdbi;
-          NameNode.stateChangeLog.warn(msg);
+          blockLog.warn(msg);
           assert false : msg; // if assertions are enabled, throw.
           break;
         }
-        if (NameNode.stateChangeLog.isDebugEnabled()) {
-          NameNode.stateChangeLog.debug("BLOCK* block "
+        if (blockLog.isDebugEnabled()) {
+          blockLog.debug("BLOCK* block "
               + (rdbi.getStatus()) + ": " + rdbi.getBlock()
               + " is received from " + nodeID);
         }
       }
     } finally {
       namesystem.writeUnlock();
-      NameNode.stateChangeLog
+      blockLog
           .debug("*BLOCK* NameNode.processIncrementalBlockReport: " + "from "
               + nodeID
               +  " receiving: " + receiving + ", "

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CorruptReplicasMap.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CorruptReplicasMap.java?rev=1407571&r1=1407570&r2=1407571&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CorruptReplicasMap.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CorruptReplicasMap.java Fri Nov  9 18:15:14 2012
@@ -63,13 +63,13 @@ public class CorruptReplicasMap{
     
     if (!nodes.contains(dn)) {
       nodes.add(dn);
-      NameNode.stateChangeLog.info("BLOCK NameSystem.addToCorruptReplicasMap: "+
+      NameNode.blockStateChangeLog.info("BLOCK NameSystem.addToCorruptReplicasMap: "+
                                    blk.getBlockName() +
                                    " added as corrupt on " + dn +
                                    " by " + Server.getRemoteIp() +
                                    reasonText);
     } else {
-      NameNode.stateChangeLog.info("BLOCK NameSystem.addToCorruptReplicasMap: "+
+      NameNode.blockStateChangeLog.info("BLOCK NameSystem.addToCorruptReplicasMap: "+
                                    "duplicate requested for " + 
                                    blk.getBlockName() + " to add as corrupt " +
                                    "on " + dn +

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java?rev=1407571&r1=1407570&r2=1407571&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/InvalidateBlocks.java Fri Nov  9 18:15:14 2012
@@ -88,7 +88,7 @@ class InvalidateBlocks {
     if (set.add(block)) {
       numBlocks++;
       if (log) {
-        NameNode.stateChangeLog.info("BLOCK* " + getClass().getSimpleName()
+        NameNode.blockStateChangeLog.info("BLOCK* " + getClass().getSimpleName()
             + ": add " + block + " to " + datanode);
       }
     }

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java?rev=1407571&r1=1407570&r2=1407571&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java Fri Nov  9 18:15:14 2012
@@ -191,8 +191,8 @@ class UnderReplicatedBlocks implements I
     int priLevel = getPriority(block, curReplicas, decomissionedReplicas,
                                expectedReplicas);
     if(priLevel != LEVEL && priorityQueues.get(priLevel).add(block)) {
-      if(NameNode.stateChangeLog.isDebugEnabled()) {
-        NameNode.stateChangeLog.debug(
+      if(NameNode.blockStateChangeLog.isDebugEnabled()) {
+        NameNode.blockStateChangeLog.debug(
           "BLOCK* NameSystem.UnderReplicationBlock.add:"
           + block
           + " has only " + curReplicas
@@ -234,8 +234,8 @@ class UnderReplicatedBlocks implements I
   boolean remove(Block block, int priLevel) {
     if(priLevel >= 0 && priLevel < LEVEL 
         && priorityQueues.get(priLevel).remove(block)) {
-      if(NameNode.stateChangeLog.isDebugEnabled()) {
-        NameNode.stateChangeLog.debug(
+      if(NameNode.blockStateChangeLog.isDebugEnabled()) {
+        NameNode.blockStateChangeLog.debug(
           "BLOCK* NameSystem.UnderReplicationBlock.remove: "
           + "Removing block " + block
           + " from priority queue "+ priLevel);
@@ -246,8 +246,8 @@ class UnderReplicatedBlocks implements I
       // not found in the queue for the given priority level.
       for (int i = 0; i < LEVEL; i++) {
         if (priorityQueues.get(i).remove(block)) {
-          if(NameNode.stateChangeLog.isDebugEnabled()) {
-            NameNode.stateChangeLog.debug(
+          if(NameNode.blockStateChangeLog.isDebugEnabled()) {
+            NameNode.blockStateChangeLog.debug(
               "BLOCK* NameSystem.UnderReplicationBlock.remove: "
               + "Removing block " + block
               + " from priority queue "+ i);
@@ -297,8 +297,8 @@ class UnderReplicatedBlocks implements I
       remove(block, oldPri);
     }
     if(curPri != LEVEL && priorityQueues.get(curPri).add(block)) {
-      if(NameNode.stateChangeLog.isDebugEnabled()) {
-        NameNode.stateChangeLog.debug(
+      if(NameNode.blockStateChangeLog.isDebugEnabled()) {
+        NameNode.blockStateChangeLog.debug(
           "BLOCK* NameSystem.UnderReplicationBlock.update:"
           + block
           + " has only "+ curReplicas

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java?rev=1407571&r1=1407570&r2=1407571&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java Fri Nov  9 18:15:14 2012
@@ -229,6 +229,7 @@ public class NameNode {
   public static final int DEFAULT_PORT = 8020;
   public static final Log LOG = LogFactory.getLog(NameNode.class.getName());
   public static final Log stateChangeLog = LogFactory.getLog("org.apache.hadoop.hdfs.StateChange");
+  public static final Log blockStateChangeLog = LogFactory.getLog("BlockStateChange");
   public static final HAState ACTIVE_STATE = new ActiveState();
   public static final HAState STANDBY_STATE = new StandbyState();
   

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java?rev=1407571&r1=1407570&r2=1407571&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java Fri Nov  9 18:15:14 2012
@@ -133,6 +133,7 @@ class NameNodeRpcServer implements Namen
   
   private static final Log LOG = NameNode.LOG;
   private static final Log stateChangeLog = NameNode.stateChangeLog;
+  private static final Log blockStateChangeLog = NameNode.blockStateChangeLog;
   
   // Dependencies from other parts of NN.
   protected final FSNamesystem namesystem;
@@ -877,8 +878,8 @@ class NameNodeRpcServer implements Namen
       String poolId, StorageBlockReport[] reports) throws IOException {
     verifyRequest(nodeReg);
     BlockListAsLongs blist = new BlockListAsLongs(reports[0].getBlocks());
-    if(stateChangeLog.isDebugEnabled()) {
-      stateChangeLog.debug("*BLOCK* NameNode.blockReport: "
+    if(blockStateChangeLog.isDebugEnabled()) {
+      blockStateChangeLog.debug("*BLOCK* NameNode.blockReport: "
            + "from " + nodeReg + " " + blist.getNumberOfBlocks()
            + " blocks");
     }
@@ -893,8 +894,8 @@ class NameNodeRpcServer implements Namen
   public void blockReceivedAndDeleted(DatanodeRegistration nodeReg, String poolId,
       StorageReceivedDeletedBlocks[] receivedAndDeletedBlocks) throws IOException {
     verifyRequest(nodeReg);
-    if(stateChangeLog.isDebugEnabled()) {
-      stateChangeLog.debug("*BLOCK* NameNode.blockReceivedAndDeleted: "
+    if(blockStateChangeLog.isDebugEnabled()) {
+      blockStateChangeLog.debug("*BLOCK* NameNode.blockReceivedAndDeleted: "
           +"from "+nodeReg+" "+receivedAndDeletedBlocks.length
           +" blocks.");
     }

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeDeath.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeDeath.java?rev=1407571&r1=1407570&r2=1407571&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeDeath.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeDeath.java Fri Nov  9 18:15:14 2012
@@ -47,6 +47,7 @@ import org.junit.Test;
 public class TestDatanodeDeath {
   {
     ((Log4JLogger)NameNode.stateChangeLog).getLogger().setLevel(Level.ALL);
+    ((Log4JLogger)NameNode.blockStateChangeLog).getLogger().setLevel(Level.ALL);
     ((Log4JLogger)LeaseManager.LOG).getLogger().setLevel(Level.ALL);
     ((Log4JLogger)LogFactory.getLog(FSNamesystem.class)).getLogger().setLevel(Level.ALL);
     ((Log4JLogger)DataNode.LOG).getLogger().setLevel(Level.ALL);

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend2.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend2.java?rev=1407571&r1=1407570&r2=1407571&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend2.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend2.java Fri Nov  9 18:15:14 2012
@@ -51,6 +51,7 @@ public class TestFileAppend2 {
 
   {
     ((Log4JLogger)NameNode.stateChangeLog).getLogger().setLevel(Level.ALL);
+    ((Log4JLogger)NameNode.blockStateChangeLog).getLogger().setLevel(Level.ALL);
     ((Log4JLogger)LeaseManager.LOG).getLogger().setLevel(Level.ALL);
     ((Log4JLogger)LogFactory.getLog(FSNamesystem.class)).getLogger().setLevel(Level.ALL);
     ((Log4JLogger)DataNode.LOG).getLogger().setLevel(Level.ALL);