Преглед на файлове

HADOOP-1414. Fix a number of issues identified by FindBugs as 'Bad Practice.' Contributed by Dhruba.

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk@546240 13f79535-47bb-0310-9956-ffa450edef68
Doug Cutting преди 18 години
родител
ревизия
c4b23d4a49

+ 4 - 0
CHANGES.txt

@@ -81,6 +81,10 @@ Trunk (unreleased changes)
  26. HADOOP-1446.  Update the TaskTracker metrics while the task is
      running. (Devaraj via omalley)
 
+ 27. HADOOP-1414.  Fix a number of issues identified by FindBugs as
+     "Bad Practice".  (Dhruba Borthakur via cutting)
+
+
 Release 0.13.0 - 2007-06-08
 
  1. HADOOP-1047.  Fix TestReplication to succeed more reliably.

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

@@ -526,7 +526,7 @@ public class StreamJob {
   public void fail(String message) {
     if (mayExit_) {
       System.err.println(message);
-      System.exit(1);
+      throw new RuntimeException(message);
     } else {
       throw new IllegalArgumentException(message);
     }

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

@@ -30,6 +30,7 @@ import java.util.jar.*;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.mapred.FileSplit;
 import org.apache.hadoop.mapred.JobConf;
 
@@ -320,10 +321,10 @@ public class StreamUtil {
   static String slurpHadoop(Path p, FileSystem fs) throws IOException {
     int len = (int) fs.getLength(p);
     byte[] buf = new byte[len];
-    InputStream in = fs.open(p);
+    FSDataInputStream in = fs.open(p);
     String contents = null;
     try {
-      in.read(buf, 0, len);
+      in.readFully(in.getPos(), buf);
       contents = new String(buf, "UTF-8");
     } finally {
       in.close();

+ 3 - 0
src/java/org/apache/hadoop/dfs/Block.java

@@ -131,6 +131,9 @@ class Block implements Writable, Comparable {
     }
   }
   public boolean equals(Object o) {
+    if (!(o instanceof Block)) {
+      return false;
+    }
     return blkid == ((Block)o).blkid;
   }
     

+ 6 - 0
src/java/org/apache/hadoop/dfs/DatanodeID.java

@@ -97,6 +97,12 @@ public class DatanodeID implements WritableComparable {
   }
 
   public boolean equals(Object to) {
+    if (this == to) {
+      return true;
+    }
+    if (!(to instanceof DatanodeID)) {
+      return false;
+    }
     return (name.equals(((DatanodeID)to).getName()) &&
             storageID.equals(((DatanodeID)to).getStorageID()));
   }

+ 1 - 1
src/java/org/apache/hadoop/dfs/FSEditLog.java

@@ -367,7 +367,7 @@ class FSEditLog {
           } catch (IOException e) {
             FSNamesystem.LOG.error("Unable to append to edit log. " +
                                    "Fatal Error.");
-            System.exit(-1);
+            throw new RuntimeException("Unable to append to edit log. ");
           }
         }
       }

+ 8 - 0
src/java/org/apache/hadoop/dfs/FSImage.java

@@ -937,6 +937,14 @@ class FSImage extends Storage {
       return node.compareTo(o);
     }
 
+    public boolean equals(Object o) {
+      return node.equals(o);
+    }
+
+    public int hashCode() {
+      return node.hashCode();
+    }
+
     /////////////////////////////////////////////////
     // Writable
     /////////////////////////////////////////////////

+ 16 - 0
src/java/org/apache/hadoop/dfs/FSNamesystem.java

@@ -1234,6 +1234,22 @@ class FSNamesystem implements FSConstants {
         return l1.holder.compareTo(l2.holder);
       }
     }
+
+    public boolean equals(Object o) {
+      if (!(o instanceof Lease)) {
+        return false;
+      }
+      Lease obj = (Lease) o;
+      if (lastUpdate == obj.lastUpdate &&
+          holder.equals(obj.holder)) {
+        return true;
+      }
+      return false;
+    }
+
+    public int hashCode() {
+      return holder.hashCode();
+    }
   }
   /******************************************************
    * LeaseMonitor checks for leases that have expired,

+ 10 - 2
src/java/org/apache/hadoop/dfs/Storage.java

@@ -140,13 +140,17 @@ abstract class Storage extends StorageInfo {
     
     void read(File from) throws IOException {
       RandomAccessFile file = new RandomAccessFile(from, "rws");
+      FileInputStream in = null;
       try {
-        FileInputStream in = new FileInputStream(file.getFD());
+        in = new FileInputStream(file.getFD());
         file.seek(0);
         Properties props = new Properties();
         props.load(in);
         getFields(props, this);
       } finally {
+        if (in != null) {
+          in.close();
+        }
         file.close();
       }
     }
@@ -165,11 +169,15 @@ abstract class Storage extends StorageInfo {
       Properties props = new Properties();
       setFields(props, this);
       RandomAccessFile file = new RandomAccessFile(to, "rws");
+      FileOutputStream out = null;
       try {
         file.seek(0);
-        FileOutputStream out = new FileOutputStream(file.getFD());
+        out = new FileOutputStream(file.getFD());
         props.store(out, null);
       } finally {
+        if (out != null) {
+          out.close();
+        }
         file.close();
       }
     }

+ 9 - 7
src/java/org/apache/hadoop/dfs/TransferFsImage.java

@@ -134,14 +134,14 @@ class TransferFsImage implements FSConstants {
     URLConnection connection = url.openConnection();
     InputStream stream = connection.getInputStream();
     FileOutputStream[] output = null;
-    if (localPath != null) {
-      output = new FileOutputStream[localPath.length];
-      for (int i = 0; i < output.length; i++) {
-        output[i] = new FileOutputStream(localPath[i]);
-      }
-    }
 
     try {
+      if (localPath != null) {
+        output = new FileOutputStream[localPath.length];
+        for (int i = 0; i < output.length; i++) {
+          output[i] = new FileOutputStream(localPath[i]);
+        }
+      }
       int num = 1;
       while (num > 0) {
         num = stream.read(buf);
@@ -155,7 +155,9 @@ class TransferFsImage implements FSConstants {
       stream.close();
       if (localPath != null) {
         for (int i = 0; i < output.length; i++) {
-          output[i].close();
+          if (output[i] != null) {
+            output[i].close();
+          }
         }
       }
     }

+ 3 - 3
src/java/org/apache/hadoop/fs/FsShell.java

@@ -123,7 +123,7 @@ public class FsShell extends ToolBase {
   void copyToLocal(String[]argv, int pos) throws IOException {
     if (argv.length-pos<2 || (argv.length-pos==2 && argv[pos].equalsIgnoreCase("-crc"))) {
       System.err.println("Usage: -get [-crc] <src> <dst>");
-      System.exit(-1);
+      throw new RuntimeException("Usage: -get [-crc] <src> <dst>");
     }
     boolean copyCrc = false;
     if ("-crc".equalsIgnoreCase(argv[pos])) {
@@ -221,7 +221,7 @@ public class FsShell extends ToolBase {
   private void setReplication(String[] cmd, int pos) throws IOException {
     if (cmd.length-pos<2 || (cmd.length-pos==2 && cmd[pos].equalsIgnoreCase("-R"))) {
       System.err.println("Usage: [-R] <repvalue> <path>");
-      System.exit(-1);
+      throw new RuntimeException("Usage: [-R] <repvalue> <path>");
     }
       
     boolean recursive = false;
@@ -238,7 +238,7 @@ public class FsShell extends ToolBase {
       pos++;
     } catch (NumberFormatException e) {
       System.err.println("Cannot set replication to: " + cmd[pos]);
-      System.exit(-1);
+      throw new RuntimeException("Cannot set replication to: " + cmd[pos]);
     }
       
     setReplication(rep, cmd[pos], recursive);

+ 2 - 1
src/java/org/apache/hadoop/fs/LocalFileSystem.java

@@ -29,6 +29,7 @@ import java.util.*;
  *****************************************************************/
 public class LocalFileSystem extends ChecksumFileSystem {
   static final URI NAME = URI.create("file:///");
+  static private Random rand = new Random();
 
   public LocalFileSystem() {
     super(new RawLocalFileSystem());
@@ -87,7 +88,7 @@ public class LocalFileSystem extends ChecksumFileSystem {
           throw new IOException("Mkdirs failed to create " + badDir.toString());
         }
       }
-      String suffix = "." + new Random().nextInt();
+      String suffix = "." + rand.nextInt();
       File badFile = new File(badDir, f.getName()+suffix);
       LOG.warn("Moving bad file " + f + " to " + badFile);
       in.close();                               // close it first

+ 32 - 4
src/java/org/apache/hadoop/io/SequenceFile.java

@@ -576,6 +576,11 @@ public class SequenceFile {
       }
       return true;
     }
+
+    public int hashCode() {
+      assert false : "hashCode not designed";
+      return 42; // any arbitrary constant will do 
+    }
     
     public String toString() {
       StringBuffer sb = new StringBuffer();
@@ -1298,11 +1303,16 @@ public class SequenceFile {
                                          CompressionInputStream filter) throws IOException {
       // Read data into a temporary buffer
       DataOutputBuffer dataBuffer = new DataOutputBuffer();
-      int dataBufferLength = WritableUtils.readVInt(in);
-      dataBuffer.write(in, dataBufferLength);
+
+      try {
+        int dataBufferLength = WritableUtils.readVInt(in);
+        dataBuffer.write(in, dataBufferLength);
       
-      // Set up 'buffer' connected to the input-stream
-      buffer.reset(dataBuffer.getData(), 0, dataBuffer.getLength());
+        // Set up 'buffer' connected to the input-stream
+        buffer.reset(dataBuffer.getData(), 0, dataBuffer.getLength());
+      } finally {
+        dataBuffer.close();
+      }
 
       // Reset the codec
       filter.resetState();
@@ -2550,6 +2560,24 @@ public class SequenceFile {
           compareTo(that.segmentPathName.toString());
       }
 
+      public boolean equals(Object o) {
+        if (!(o instanceof SegmentDescriptor)) {
+          return false;
+        }
+        SegmentDescriptor that = (SegmentDescriptor)o;
+        if (this.segmentLength == that.segmentLength &&
+            this.segmentOffset == that.segmentOffset &&
+            this.segmentPathName.toString().equals(
+              that.segmentPathName.toString())) {
+          return true;
+        }
+        return false;
+      }
+
+      public int hashCode() {
+        return 37 * 17 + (int) (segmentOffset^(segmentOffset>>>32));
+      }
+
       /** Fills up the rawKey object with the key returned by the Reader
        * @return true if there is a key returned; false, otherwise
        * @throws IOException

+ 1 - 1
src/java/org/apache/hadoop/io/Text.java

@@ -215,7 +215,7 @@ public class Text implements WritableComparable {
     try {
       return decode(bytes, 0, length);
     } catch (CharacterCodingException e) { 
-      return null;
+      throw new RuntimeException("Should not have happened " + e.toString()); 
     }
   }
   

+ 5 - 4
src/java/org/apache/hadoop/mapred/JobClient.java

@@ -717,10 +717,11 @@ public class JobClient extends ToolBase implements MRConstants  {
     
   public int run(String[] argv) throws Exception {
     if (argv.length < 2) {
-      System.out.println("JobClient -submit <job> | -status <id> |" + 
-                         " -events <id> |" +
-                         " -kill <id> [-jt <jobtracker:port>|<config>]");
-      System.exit(-1);
+      String cmd = "JobClient -submit <job> | -status <id> |" + 
+                   " -events <id> |" +
+                   " -kill <id> [-jt <jobtracker:port>|<config>]";
+      System.out.println(cmd);
+      throw new RuntimeException("JobClient:" + cmd);
     }
 
     // initialize JobClient

+ 13 - 0
src/java/org/apache/hadoop/mapred/JobEndNotifier.java

@@ -196,7 +196,20 @@ public class JobEndNotifier {
       return (int)(delayTime - ((JobEndStatusInfo)d).delayTime);
     }
 
+    public boolean equals(Object o) {
+      if (!(o instanceof JobEndStatusInfo)) {
+        return false;
+      }
+      if (delayTime == ((JobEndStatusInfo)o).delayTime) {
+        return true;
+      }
+      return false;
+    }
 
+    public int hashCode() {
+      return 37 * 17 + (int) (delayTime^(delayTime>>>32));
+    }
+      
     public String toString() {
       return "URL: " + uri + " remaining retries: " + retryAttempts +
         " interval: " + retryInterval;

+ 12 - 8
src/java/org/apache/hadoop/mapred/JobHistory.java

@@ -115,15 +115,19 @@ public class JobHistory {
    */
   public static void parseHistory(File path, Listener l) throws IOException{
     BufferedReader reader = new BufferedReader(new FileReader(path));
-    String line = null; 
-    StringBuffer buf = new StringBuffer(); 
-    while ((line = reader.readLine())!= null){
-      buf.append(line); 
-      if (!line.trim().endsWith("\"")){
-        continue; 
+    try {
+      String line = null; 
+      StringBuffer buf = new StringBuffer(); 
+      while ((line = reader.readLine())!= null){
+        buf.append(line); 
+        if (!line.trim().endsWith("\"")){
+          continue; 
+        }
+        parseLine(buf.toString(), l);
+        buf = new StringBuffer(); 
       }
-      parseLine(buf.toString(), l);
-      buf = new StringBuffer(); 
+    } finally {
+      try { reader.close(); } catch (IOException ex) {}
     }
   }
   /**

+ 35 - 27
src/java/org/apache/hadoop/mapred/TaskLog.java

@@ -402,21 +402,24 @@ class TaskLog {
       // Copy log data into buffer
       byte[] b = new byte[totalLogSize];
       SequenceInputStream in = new SequenceInputStream(streams.elements());
-      int bytesRead = 0, totalBytesRead = 0;
-      int off = 0, len = totalLogSize;
-      LOG.debug("Attempting to read " + len + " bytes from logs");
-      while ((bytesRead = in.read(b, off, len)) > 0) {
-        LOG.debug("Got " + bytesRead + " bytes");
-        off += bytesRead;
-        len -= bytesRead;
+      try {
+        int bytesRead = 0, totalBytesRead = 0;
+        int off = 0, len = totalLogSize;
+        LOG.debug("Attempting to read " + len + " bytes from logs");
+        while ((bytesRead = in.read(b, off, len)) > 0) {
+          LOG.debug("Got " + bytesRead + " bytes");
+          off += bytesRead;
+          len -= bytesRead;
         
-        totalBytesRead += bytesRead;
-      }
+          totalBytesRead += bytesRead;
+        }
 
-      if (totalBytesRead != totalLogSize) {
-        LOG.debug("Didn't not read all requisite data in logs!");
+        if (totalBytesRead != totalLogSize) {
+          LOG.debug("Didn't not read all requisite data in logs!");
+        }
+      } finally {
+        try { in.close(); } catch (IOException ex) {}
       }
-      
       return b;
     }
     
@@ -527,23 +530,28 @@ class TaskLog {
       
       // Copy requisite data into user buffer
       SequenceInputStream in = new SequenceInputStream(streams.elements());
-      if (streams.size() == (stopIndex - startIndex +1)) {
-        // Skip to get to 'logOffset' if logs haven't been purged
-        long skipBytes = 
-          in.skip(logOffset - indexRecords[startIndex].splitOffset);
-        LOG.debug("Skipped " + skipBytes + " bytes from " + 
-                  startIndex + " stream");
-      }
-      int bytesRead = 0, totalBytesRead = 0;
-      len = Math.min((int)logLength, len);
-      LOG.debug("Attempting to read " + len + " bytes from logs");
-      while ((bytesRead = in.read(b, off, len)) > 0) {
-        off += bytesRead;
-        len -= bytesRead;
+      int totalBytesRead = 0;
+      try {
+        if (streams.size() == (stopIndex - startIndex +1)) {
+          // Skip to get to 'logOffset' if logs haven't been purged
+          long skipBytes = 
+            in.skip(logOffset - indexRecords[startIndex].splitOffset);
+          LOG.debug("Skipped " + skipBytes + " bytes from " + 
+                    startIndex + " stream");
+        }
+        int bytesRead = 0;
+        len = Math.min((int)logLength, len);
+        LOG.debug("Attempting to read " + len + " bytes from logs");
+        while ((bytesRead = in.read(b, off, len)) > 0) {
+          off += bytesRead;
+          len -= bytesRead;
         
-        totalBytesRead += bytesRead;
+          totalBytesRead += bytesRead;
+        }
+      } finally {
+        try { in.close(); } catch (IOException e) {}
       }
-      
+
       return totalBytesRead;
     }
 

+ 3 - 1
src/java/org/apache/hadoop/record/Buffer.java

@@ -240,6 +240,8 @@ public class Buffer implements Comparable, Cloneable {
   
   // inherit javadoc
   public Object clone() throws CloneNotSupportedException {
-    return new Buffer(this.get().clone(), 0, this.getCount());
+    Buffer result = (Buffer) super.clone();
+    result.copy(this.get(), 0, this.getCount());
+    return result;
   }
 }

+ 4 - 0
src/java/org/apache/hadoop/util/CopyFiles.java

@@ -652,6 +652,10 @@ public class CopyFiles extends ToolBase {
       if (fis != null) {
         fis.close();
       }
+    } finally {
+      if (fis != null) {
+        fis.close();
+      }
     }
 
     return !uris.isEmpty()? uris.toArray(new String[0]): null;

+ 5 - 1
src/java/org/apache/hadoop/util/HostsFileReader.java

@@ -23,8 +23,9 @@ public class HostsFileReader {
 
   private void readFileToSet(String filename, Set<String> set) throws IOException {
     FileInputStream fis = new FileInputStream(new File(filename));
+    BufferedReader reader = null;
     try {
-      BufferedReader reader = new BufferedReader(new InputStreamReader(fis));
+      reader = new BufferedReader(new InputStreamReader(fis));
       String line;
       while ((line = reader.readLine()) != null) {
         String[] nodes = line.split("[ \t\n\f\r]+");
@@ -35,6 +36,9 @@ public class HostsFileReader {
         }
       }   
     } finally {
+      if (reader != null) {
+        reader.close();
+      }
       fis.close();
     }  
   }