Преглед изворни кода

HADOOP-4204. Fix findbugs warnings related to unused variables, naive
Number subclass instantiation, Map iteration, and badly scoped inner
classes. Contributed by Suresh Srinivas.


git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@700589 13f79535-47bb-0310-9956-ffa450edef68

Christopher Douglas пре 16 година
родитељ
комит
c228be0194

+ 4 - 0
CHANGES.txt

@@ -37,6 +37,10 @@ Trunk (unreleased changes)
 
   BUG FIXES
 
+    HADOOP-4204. Fix findbugs warnings related to unused variables, naive
+    Number subclass instantiation, Map iteration, and badly scoped inner
+    classes. (Suresh Srinivas via cdouglas)
+
 Release 0.19.0 - Unreleased
 
   INCOMPATIBLE CHANGES

+ 4 - 6
src/contrib/streaming/src/java/org/apache/hadoop/streaming/PathFinder.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.streaming;
 
 import java.io.*;
 import java.util.*;
+import java.util.Map.Entry;
 
 /**
  * Maps a relative pathname to an absolute pathname using the
@@ -108,12 +109,9 @@ public class PathFinder
   private static void printEnvVariables() {
     System.out.println("Environment Variables: ");
     Map<String,String> map = System.getenv();
-    Set<String> keys = map.keySet();
-    Iterator iter = keys.iterator();
-    while(iter.hasNext()) {
-      String thiskey = (String)(iter.next()); 
-      String value = map.get(thiskey);
-      System.out.println(thiskey + " = " + value);
+    Set<Entry<String, String>> entrySet = map.entrySet();
+    for(Entry<String, String> entry : entrySet) {
+      System.out.println(entry.getKey() + " = " + entry.getValue());
     }
   }
 

+ 2 - 2
src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java

@@ -953,7 +953,7 @@ public class StreamJob implements Tool {
     return 0;
   }
   /** Support -jobconf x=y x1=y1 type options **/
-  class MultiPropertyOption extends PropertyOption{
+  static class MultiPropertyOption extends PropertyOption{
     private String optionString; 
     MultiPropertyOption(){
       super(); 
@@ -1051,7 +1051,7 @@ public class StreamJob implements Tool {
 
   protected RunningJob running_;
   protected JobID jobId_;
-  protected static String LINK_URI = "You need to specify the uris as hdfs://host:port/#linkname," +
+  protected static final String LINK_URI = "You need to specify the uris as hdfs://host:port/#linkname," +
     "Please specify a different link name for all of your caching URIs";
 
 }

+ 5 - 8
src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamXmlRecordReader.java

@@ -128,8 +128,6 @@ public class StreamXmlRecordReader extends StreamBaseRecordReader {
                                      DataOutputBuffer outBufOrNull) throws IOException {
     byte[] buf = new byte[Math.max(lookAhead_, maxRecSize_)];
     int read = 0;
-    boolean success = true;
-    long skippedBytes = 0;
     bin_.mark(Math.max(lookAhead_, maxRecSize_) + 2); //mark to invalidate if we read more
     read = bin_.read(buf);
     if (read == -1) return false;
@@ -142,12 +140,9 @@ public class StreamXmlRecordReader extends StreamBaseRecordReader {
     int bufPos = 0;
     int state = synched_ ? CDATA_OUT : CDATA_UNK;
     int s = 0;
-    int matchLen = 0;
-    int LL = 120000 * 10;
 
     while (match.find(bufPos)) {
       int input;
-      matchLen = match.group(0).length();
       if (match.group(1) != null) {
         input = CDATA_BEGIN;
       } else if (match.group(2) != null) {
@@ -164,7 +159,6 @@ public class StreamXmlRecordReader extends StreamBaseRecordReader {
       }
       state = nextState(state, input, match.start());
       if (state == RECORD_ACCEPT) {
-        bufPos = match.end();
         break;
       }
       bufPos = match.end();
@@ -177,7 +171,11 @@ public class StreamXmlRecordReader extends StreamBaseRecordReader {
     if (matched) {
       int endPos = includePat ? firstMatchEnd_ : firstMatchStart_;
       bin_.reset();
-      skippedBytes = bin_.skip(endPos); //Skip succeeds as we have already read this is buffer
+
+      for (long skiplen = endPos; skiplen > 0; ) {
+        skiplen -= bin_.skip(skiplen); // Skip succeeds as we have read this buffer
+      }
+
       pos_ += endPos;
       if (outBufOrNull != null) {
         outBufOrNull.writeBytes(sbuf.substring(0,endPos));
@@ -299,6 +297,5 @@ public class StreamXmlRecordReader extends StreamBaseRecordReader {
   int firstMatchStart_ = 0; // candidate record boundary. Might just be CDATA.
   int firstMatchEnd_ = 0;
 
-  boolean isRecordMatch_;
   boolean synched_;
 }

+ 13 - 8
src/core/org/apache/hadoop/metrics/spi/AbstractMetricsContext.java

@@ -31,6 +31,8 @@ import java.util.Set;
 import java.util.Timer;
 import java.util.TimerTask;
 import java.util.TreeMap;
+import java.util.Map.Entry;
+
 import org.apache.hadoop.metrics.ContextFactory;
 import org.apache.hadoop.metrics.MetricsContext;
 import org.apache.hadoop.metrics.MetricsException;
@@ -298,9 +300,9 @@ public abstract class AbstractMetricsContext implements MetricsContext {
     for (String recordName : bufferedData.keySet()) {
       RecordMap recordMap = bufferedData.get(recordName);
       synchronized (recordMap) {
-        for (TagMap tagMap : recordMap.keySet()) {
-          MetricMap metricMap = recordMap.get(tagMap);
-          OutputRecord outRec = new OutputRecord(tagMap, metricMap);
+        Set<Entry<TagMap, MetricMap>> entrySet = recordMap.entrySet ();
+        for (Entry<TagMap, MetricMap> entry : entrySet) {
+          OutputRecord outRec = new OutputRecord(entry.getKey(), entry.getValue());
           emitRecord(contextName, recordName, outRec);
         }
       }
@@ -338,8 +340,11 @@ public abstract class AbstractMetricsContext implements MetricsContext {
         TagMap tagMap = new TagMap(tagTable); // clone tags
         recordMap.put(tagMap, metricMap);
       }
-      for (String metricName : metricUpdates.keySet()) {
-        MetricValue updateValue = metricUpdates.get(metricName);
+
+      Set<Entry<String, MetricValue>> entrySet = metricUpdates.entrySet();
+      for (Entry<String, MetricValue> entry : entrySet) {
+        String metricName = entry.getKey ();
+        MetricValue updateValue = entry.getValue ();
         Number updateNumber = updateValue.getNumber();
         Number currentNumber = metricMap.get(metricName);
         if (currentNumber == null || updateValue.isAbsolute()) {
@@ -363,16 +368,16 @@ public abstract class AbstractMetricsContext implements MetricsContext {
    */
   private Number sum(Number a, Number b) {
     if (a instanceof Integer) {
-      return new Integer(a.intValue() + b.intValue());
+      return Integer.valueOf(a.intValue() + b.intValue());
     }
     else if (a instanceof Float) {
       return new Float(a.floatValue() + b.floatValue());
     }
     else if (a instanceof Short) {
-      return new Short((short)(a.shortValue() + b.shortValue()));
+      return Short.valueOf((short)(a.shortValue() + b.shortValue()));
     }
     else if (a instanceof Byte) {
-      return new Byte((byte)(a.byteValue() + b.byteValue()));
+      return Byte.valueOf((byte)(a.byteValue() + b.byteValue()));
     }
     else if (a instanceof Long) {
       return Long.valueOf((a.longValue() + b.longValue()));

+ 9 - 9
src/core/org/apache/hadoop/metrics/spi/MetricsRecordImpl.java

@@ -77,7 +77,7 @@ public class MetricsRecordImpl implements MetricsRecord {
    * @throws MetricsException if the tagName conflicts with the configuration
    */
   public void setTag(String tagName, int tagValue) {
-    tagTable.put(tagName, new Integer(tagValue));
+    tagTable.put(tagName, Integer.valueOf(tagValue));
   }
     
   /**
@@ -99,7 +99,7 @@ public class MetricsRecordImpl implements MetricsRecord {
    * @throws MetricsException if the tagName conflicts with the configuration
    */
   public void setTag(String tagName, short tagValue) {
-    tagTable.put(tagName, new Short(tagValue));
+    tagTable.put(tagName, Short.valueOf(tagValue));
   }
     
   /**
@@ -110,7 +110,7 @@ public class MetricsRecordImpl implements MetricsRecord {
    * @throws MetricsException if the tagName conflicts with the configuration
    */
   public void setTag(String tagName, byte tagValue) {
-    tagTable.put(tagName, new Byte(tagValue));
+    tagTable.put(tagName, Byte.valueOf(tagValue));
   }
     
   /**
@@ -129,7 +129,7 @@ public class MetricsRecordImpl implements MetricsRecord {
    * conflicts with the configuration
    */
   public void setMetric(String metricName, int metricValue) {
-    setAbsolute(metricName, new Integer(metricValue));
+    setAbsolute(metricName, Integer.valueOf(metricValue));
   }
     
   /**
@@ -153,7 +153,7 @@ public class MetricsRecordImpl implements MetricsRecord {
    * conflicts with the configuration
    */
   public void setMetric(String metricName, short metricValue) {
-    setAbsolute(metricName, new Short(metricValue));
+    setAbsolute(metricName, Short.valueOf(metricValue));
   }
     
   /**
@@ -165,7 +165,7 @@ public class MetricsRecordImpl implements MetricsRecord {
    * conflicts with the configuration
    */
   public void setMetric(String metricName, byte metricValue) {
-    setAbsolute(metricName, new Byte(metricValue));
+    setAbsolute(metricName, Byte.valueOf(metricValue));
   }
     
   /**
@@ -189,7 +189,7 @@ public class MetricsRecordImpl implements MetricsRecord {
    * conflicts with the configuration
    */
   public void incrMetric(String metricName, int metricValue) {
-    setIncrement(metricName, new Integer(metricValue));
+    setIncrement(metricName, Integer.valueOf(metricValue));
   }
     
   /**
@@ -213,7 +213,7 @@ public class MetricsRecordImpl implements MetricsRecord {
    * conflicts with the configuration
    */
   public void incrMetric(String metricName, short metricValue) {
-    setIncrement(metricName, new Short(metricValue));
+    setIncrement(metricName, Short.valueOf(metricValue));
   }
     
   /**
@@ -225,7 +225,7 @@ public class MetricsRecordImpl implements MetricsRecord {
    * conflicts with the configuration
    */
   public void incrMetric(String metricName, byte metricValue) {
-    setIncrement(metricName, new Byte(metricValue));
+    setIncrement(metricName, Byte.valueOf(metricValue));
   }
     
   /**

+ 1 - 1
src/core/org/apache/hadoop/net/NodeBase.java

@@ -23,7 +23,7 @@ package org.apache.hadoop.net;
 
 public class NodeBase implements Node {
   public final static char PATH_SEPARATOR = '/';
-  public static String PATH_SEPARATOR_STR = "/";
+  public final static String PATH_SEPARATOR_STR = "/";
   public final static String ROOT = ""; // string representation of root
   
   protected String name; //host:port#

+ 1 - 1
src/core/org/apache/hadoop/record/XmlRecordInput.java

@@ -47,7 +47,7 @@ public class XmlRecordInput implements RecordInput {
     public String getType() { return type; }
   }
     
-  private class XMLParser extends DefaultHandler {
+  private static class XMLParser extends DefaultHandler {
     private boolean charsValid = false;
         
     private ArrayList<Value> valList;

+ 2 - 2
src/hdfs/org/apache/hadoop/hdfs/server/namenode/JspHelper.java

@@ -50,12 +50,12 @@ public class JspHelper {
 
   static FSNamesystem fsn = null;
   public static InetSocketAddress nameNodeAddr;
-  public static Configuration conf = new Configuration();
+  public static final Configuration conf = new Configuration();
   public static final UnixUserGroupInformation webUGI
   = UnixUserGroupInformation.createImmutable(
       conf.getStrings(WEB_UGI_PROPERTY_NAME));
 
-  public static int defaultChunkSizeToView = 
+  public static final int defaultChunkSizeToView = 
     conf.getInt("dfs.default.chunk.view.size", 32 * 1024);
   static Random rand = new Random();
 

+ 1 - 1
src/mapred/org/apache/hadoop/mapred/pipes/PipesNonJavaInputFormat.java

@@ -66,7 +66,7 @@ implements InputFormat<FloatWritable, NullWritable> {
    * {@link #next(FloatWritable, NullWritable)} with the progress as the
    * <code>key</code>.
    */
-  class PipesDummyRecordReader implements RecordReader<FloatWritable, NullWritable> {
+  static class PipesDummyRecordReader implements RecordReader<FloatWritable, NullWritable> {
     float progress = 0.0f;
     
     public PipesDummyRecordReader(Configuration job, InputSplit split)