Browse Source

HADOOP-1693. Remove un-needed log fields. Contributed by Konstantin.

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk@566774 13f79535-47bb-0310-9956-ffa450edef68
Doug Cutting 18 years ago
parent
commit
52bdca20e6

+ 3 - 0
CHANGES.txt

@@ -47,6 +47,9 @@ Trunk (unreleased changes)
     HADOOP-1595.  dfsshell can wait for a file to achieve its intended
     replication target. (Tsz Wo (Nicholas), SZE via dhruba)
 
+    HADOOP-1693.  Remove un-needed log fields in DFS replication classes,
+    since the log may be accessed statically. (Konstantin Shvachko via cutting)
+
 
 Release 0.14.0 - 2007-08-17
 

+ 4 - 5
src/java/org/apache/hadoop/dfs/FSNamesystem.java

@@ -217,7 +217,7 @@ class FSNamesystem implements FSConstants {
     this.dir.loadFSImage(getNamespaceDirs(conf), startOpt);
     this.safeMode = new SafeModeInfo(conf);
     setBlockTotal();
-    pendingReplications = new PendingReplicationBlocks(LOG);
+    pendingReplications = new PendingReplicationBlocks();
     this.hbthread = new Daemon(new HeartbeatMonitor());
     this.lmthread = new Daemon(new LeaseMonitor());
     this.replthread = new Daemon(new ReplicationMonitor());
@@ -277,10 +277,9 @@ class FSNamesystem implements FSConstants {
   private void setConfigurationParameters(Configuration conf) 
                                           throws IOException {
     this.replicator = new ReplicationTargetChooser(
-                                                   conf.getBoolean("dfs.replication.considerLoad", true),
-                                                   this,
-                                                   clusterMap,
-                                                   LOG);
+                         conf.getBoolean("dfs.replication.considerLoad", true),
+                         this,
+                         clusterMap);
     this.defaultReplication = conf.getInt("dfs.replication", 3);
     this.maxReplication = conf.getInt("dfs.replication.max", 512);
     this.minReplication = conf.getInt("dfs.replication.min", 1);

+ 5 - 11
src/java/org/apache/hadoop/dfs/PendingReplicationBlocks.java

@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.dfs;
 
-import org.apache.commons.logging.*;
 import org.apache.hadoop.util.*;
 import java.io.*;
 import java.util.*;
@@ -35,7 +34,6 @@ import java.sql.Time;
  *
  ***************************************************/
 class PendingReplicationBlocks {
-  private Log LOG = null;
   private Map<Block, PendingBlockInfo> pendingReplications;
   private ArrayList<Block> timedOutItems;
   Daemon timerThread = null;
@@ -53,8 +51,7 @@ class PendingReplicationBlocks {
     init();
   }
 
-  PendingReplicationBlocks(Log log) {
-    this.LOG = log;
+  PendingReplicationBlocks() {
     init();
   }
 
@@ -184,10 +181,8 @@ class PendingReplicationBlocks {
           pendingReplicationCheck();
           Thread.sleep(period);
         } catch (InterruptedException ie) {
-          if (LOG != null) {
-            LOG.warn("PendingReplicationMonitor thread received exception. " 
-                     + ie);
-          }
+          FSNamesystem.LOG.warn(
+                "PendingReplicationMonitor thread received exception. " + ie);
         }
       }
     }
@@ -207,9 +202,8 @@ class PendingReplicationBlocks {
             synchronized (timedOutItems) {
               timedOutItems.add(block);
             }
-            if (LOG != null) {
-              LOG.warn("PendingReplicationMonitor timed out block " + block);
-            }
+            FSNamesystem.LOG.warn(
+                "PendingReplicationMonitor timed out block " + block);
             iter.remove();
           }
         }

+ 3 - 5
src/java/org/apache/hadoop/dfs/ReplicationTargetChooser.java

@@ -33,15 +33,13 @@ import java.util.*;
 class ReplicationTargetChooser {
   private final boolean considerLoad; 
   private NetworkTopology clusterMap;
-  private Log logr;
   private FSNamesystem fs;
     
   ReplicationTargetChooser(boolean considerLoad,  FSNamesystem fs,
-                           NetworkTopology clusterMap, Log logr) {
+                           NetworkTopology clusterMap) {
     this.considerLoad = considerLoad;
     this.fs = fs;
     this.clusterMap = clusterMap;
-    this.logr = logr;
   }
     
   private static class NotEnoughReplicasException extends Exception {
@@ -175,7 +173,7 @@ class ReplicationTargetChooser {
                      blocksize, maxNodesPerRack, results);
       }
     } catch (NotEnoughReplicasException e) {
-      logr.warn("Not able to place enough replicas, still in need of "
+      FSNamesystem.LOG.warn("Not able to place enough replicas, still in need of "
                + numOfReplicas);
     }
     return writer;
@@ -386,7 +384,7 @@ class ReplicationTargetChooser {
                                long blockSize, int maxTargetPerLoc,
                                boolean considerLoad,
                                List<DatanodeDescriptor> results) {
-      
+    Log logr = FSNamesystem.LOG;
     // check if the node is (being) decommissed
     if (node.isDecommissionInProgress() || node.isDecommissioned()) {
       logr.debug("Node "+node.getPath()+