Browse Source

HADOOP-1211. Remove deprecated constructor and unused static members in DataNode class. Contributed by Konstantin Shvachko.

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk@527401 13f79535-47bb-0310-9956-ffa450edef68
Thomas White 18 years ago
parent
commit
9f48a9721d

+ 3 - 0
CHANGES.txt

@@ -144,6 +144,9 @@ Trunk (unreleased changes)
     unit test configuration.  Using the default is more robust and
     has almost the same run time.  (Arun C Murthy via tomwhite)
 
+45. HADOOP-1211.  Remove deprecated constructor and unused static 
+    members in DataNode class.  (Konstantin Shvachko via tomwhite)
+
 
 Release 0.12.3 - 2007-04-06
 

+ 2 - 2
src/java/org/apache/hadoop/dfs/DataNode.java

@@ -277,8 +277,8 @@ public class DataNode implements FSConstants, Runnable {
     }
 
     private NamespaceInfo handshake() throws IOException {
-      NamespaceInfo nsInfo;
-      while( true ) {
+      NamespaceInfo nsInfo = new NamespaceInfo();
+      while (shouldRun) {
         try {
           nsInfo = namenode.versionRequest();
           break;

+ 0 - 14
src/java/org/apache/hadoop/dfs/DatanodeRegistration.java

@@ -45,20 +45,6 @@ class DatanodeRegistration extends DatanodeID implements Writable {
     this.storageInfo = new StorageInfo( storage );
   }
 
-  /**
-   * Create DatanodeRegistration
-   * @deprecated 
-   * use {@link #DatanodeRegistration(String, int, DataStorage)} instead
-   */
-  public DatanodeRegistration(int version, 
-                              String nodeName, 
-                              String storageID,
-                              int infoPort,
-                              String registrationID ) {
-    super( nodeName, storageID, infoPort );
-    this.storageInfo = new StorageInfo();
-  }
-
   /**
    */
   public int getVersion() {

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

@@ -256,7 +256,6 @@ class FSDataset implements FSConstants {
     class FSVolume {
       static final double USABLE_DISK_PCT_DEFAULT = 0.98f; 
 
-      private File dir; // TODO this field is redundant equals this.dataDir.dir.getParent()
       private FSDir dataDir;
       private File tmpDir;
       private DF usage;
@@ -267,9 +266,9 @@ class FSDataset implements FSConstants {
         this.reserved = conf.getLong("dfs.datanode.du.reserved", 0);
         this.usableDiskPct = conf.getFloat("dfs.datanode.du.pct",
             (float) USABLE_DISK_PCT_DEFAULT);
-        this.dir = currentDir.getParentFile();
+        File parent = currentDir.getParentFile();
         this.dataDir = new FSDir( currentDir );
-        this.tmpDir = new File(dir, "tmp");
+        this.tmpDir = new File(parent, "tmp");
         if (tmpDir.exists()) {
           FileUtil.fullyDelete(tmpDir);
         }
@@ -278,7 +277,7 @@ class FSDataset implements FSConstants {
             throw new IOException("Mkdirs failed to create " + tmpDir.toString());
           }
         }
-        this.usage = new DF(dir, conf);
+        this.usage = new DF(parent, conf);
       }
       
       long getCapacity() throws IOException {
@@ -342,7 +341,7 @@ class FSDataset implements FSConstants {
       }
       
       public String toString() {
-        return dir.getAbsolutePath();
+        return dataDir.dir.getAbsolutePath();
       }
     }
     

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

@@ -1102,7 +1102,8 @@ class FSNamesystem implements FSConstants {
      */
     private synchronized void dumpRecentInvalidateSets(PrintWriter out) {
       Collection<Collection<Block>> values = recentInvalidateSets.values();
-      Iterator it = recentInvalidateSets.entrySet().iterator();
+      Iterator<Map.Entry<String,Collection<Block>>> it = 
+                                recentInvalidateSets.entrySet().iterator();
       if (values.size() == 0) {
         out.println("Metasave: Blocks waiting deletion: 0");
         return;
@@ -1110,10 +1111,10 @@ class FSNamesystem implements FSConstants {
       out.println("Metasave: Blocks waiting deletion from " +
                    values.size() + " datanodes.");
       while (it.hasNext()) {
-        Map.Entry entry = (Map.Entry) it.next();
+        Map.Entry<String,Collection<Block>> entry = it.next();
         String storageId = (String) entry.getKey();
         DatanodeDescriptor node = datanodeMap.get(storageId);
-        Collection<Block> blklist = (Collection<Block>) entry.getValue();
+        Collection<Block> blklist = entry.getValue();
         if (blklist.size() > 0) {
           out.print(node.getName());
           for (Iterator jt = blklist.iterator(); jt.hasNext();) {

+ 0 - 13
src/java/org/apache/hadoop/dfs/NameNode.java

@@ -224,19 +224,6 @@ public class NameNode implements ClientProtocol, DatanodeProtocol, FSConstants {
       init( bindAddress, port, conf );
     }
 
-    /** Return the configured directories where name data is stored. 
-     * @deprecated
-     */
-    static File[] getDirs(Configuration conf) {
-      String[] dirNames = conf.getStrings("dfs.name.dir");
-      if (dirNames == null) { dirNames = new String[] {"/tmp/hadoop/dfs/name"}; }
-      File[] dirs = new File[dirNames.length];
-      for (int idx = 0; idx < dirs.length; idx++) {
-        dirs[idx] = new File(dirNames[idx]);
-      }
-      return dirs;
-    }
-
     /**
      * Wait for service to finish.
      * (Normally, it runs forever.)

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

@@ -23,8 +23,6 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.RandomAccessFile;
 import java.nio.channels.FileLock;
-import java.util.AbstractList;
-import java.util.ArrayList;
 import java.util.List;
 import java.util.Iterator;
 import java.util.Properties;

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

@@ -20,6 +20,8 @@ package org.apache.hadoop.dfs;
 import java.io.*;
 import java.net.*;
 import java.util.ArrayList;
+import java.util.Collection;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.dfs.FSConstants.StartupOption;
 import org.apache.hadoop.fs.*;
@@ -300,11 +302,11 @@ public class MiniDFSCluster {
   /**
    * Get the directories where the namenode stores its state.
    */
-  public File[] getNameDirs() {
-    return NameNode.getDirs(conf);
+  public Collection<File> getNameDirs() {
+    return FSNamesystem.getNamespaceDirs(conf);
   }
 
-   /**
+  /**
    * Wait until the cluster is active and running.
    */
   public void waitActive() throws IOException {

+ 16 - 14
src/test/org/apache/hadoop/dfs/TestCheckpoint.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.dfs;
 
 import junit.framework.TestCase;
 import java.io.*;
+import java.util.Collection;
 import java.util.Random;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -68,12 +69,12 @@ public class TestCheckpoint extends TestCase {
   /**
    * put back the old namedir
    */
-  private void resurrectNameDir(File[] namedirs) 
+  private void resurrectNameDir(File namedir) 
   throws IOException {
-    String parentdir = namedirs[0].getParent();
-    String name = namedirs[0].getName();
+    String parentdir = namedir.getParent();
+    String name = namedir.getName();
     File oldname =  new File(parentdir, name + ".old");
-    if (!oldname.renameTo(namedirs[0])) {
+    if (!oldname.renameTo(namedir)) {
       assertTrue(false);
     }
   }
@@ -81,12 +82,12 @@ public class TestCheckpoint extends TestCase {
   /**
    * remove one namedir
    */
-  private void removeOneNameDir(File[] namedirs) 
+  private void removeOneNameDir(File namedir) 
   throws IOException {
-    String parentdir = namedirs[0].getParent();
-    String name = namedirs[0].getName();
+    String parentdir = namedir.getParent();
+    String name = namedir.getName();
     File newname =  new File(parentdir, name + ".old");
-    if (!namedirs[0].renameTo(newname)) {
+    if (!namedir.renameTo(newname)) {
       assertTrue(false);
     }
   }
@@ -94,27 +95,28 @@ public class TestCheckpoint extends TestCase {
   /*
    * Verify that namenode does not startup if one namedir is bad.
    */
-  private void testNamedirError(Configuration conf, File[] namedirs) 
+  private void testNamedirError(Configuration conf, Collection<File> namedirs) 
   throws IOException {
     System.out.println("Starting testNamedirError");
-    Path file1 = new Path("checkpoint.dat");
     MiniDFSCluster cluster = null;
 
-    if (namedirs.length <= 1) {
+    if (namedirs.size() <= 1) {
       return;
     }
     
     //
     // Remove one namedir & Restart cluster. This should fail.
     //
-    removeOneNameDir(namedirs);
+    File first = namedirs.iterator().next();
+    removeOneNameDir(first);
     try {
       cluster = new MiniDFSCluster(conf, 0, false, null);
+      cluster.shutdown();
       assertTrue(false);
     } catch (Throwable t) {
       // no nothing
     }
-    resurrectNameDir(namedirs); // put back namedir
+    resurrectNameDir(first); // put back namedir
   }
 
   /*
@@ -241,7 +243,7 @@ public class TestCheckpoint extends TestCase {
   public void testCheckpoint() throws IOException {
     Path file1 = new Path("checkpoint.dat");
     Path file2 = new Path("checkpoint2.dat");
-    File[] namedirs = null;
+    Collection<File> namedirs = null;
 
     Configuration conf = new Configuration();
     MiniDFSCluster cluster = new MiniDFSCluster(conf, numDatanodes, true, null);