You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by sz...@apache.org on 2008/11/01 01:00:18 UTC

svn commit: r709609 - in /hadoop/core/trunk: ./ src/hdfs/org/apache/hadoop/hdfs/server/namenode/

Author: szetszwo
Date: Fri Oct 31 17:00:17 2008
New Revision: 709609

URL: http://svn.apache.org/viewvc?rev=709609&view=rev
Log:
HADOOP-3461. Remove hdfs.StringBytesWritable. (szetszwo)

Removed:
    hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/StringBytesWritable.java
Modified:
    hadoop/core/trunk/CHANGES.txt
    hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
    hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java
    hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java

Modified: hadoop/core/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/CHANGES.txt?rev=709609&r1=709608&r2=709609&view=diff
==============================================================================
--- hadoop/core/trunk/CHANGES.txt (original)
+++ hadoop/core/trunk/CHANGES.txt Fri Oct 31 17:00:17 2008
@@ -53,6 +53,8 @@
 
     HADOOP-4454. Allow # comments in slaves file. (Rama Ramasamy via omalley)
 
+    HADOOP-3461. Remove hdfs.StringBytesWritable. (szetszwo)
+
   OPTIMIZATIONS
 
   BUG FIXES

Modified: hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java?rev=709609&r1=709608&r2=709609&view=diff
==============================================================================
--- hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java (original)
+++ hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java Fri Oct 31 17:00:17 2008
@@ -1043,7 +1043,7 @@
         // If the file is under construction , then it must be in our
         // leases. Find the appropriate lease record.
         //
-        Lease lease = leaseManager.getLease(new StringBytesWritable(holder));
+        Lease lease = leaseManager.getLease(holder);
         //
         // We found the lease for this file. And surprisingly the original
         // holder is trying to recreate this file. This should never occur.
@@ -1327,7 +1327,7 @@
                                                      throws IOException {
 
     if (file == null || file.isDirectory()) {
-      Lease lease = leaseManager.getLease(new StringBytesWritable(holder));
+      Lease lease = leaseManager.getLease(holder);
       throw new LeaseExpiredException("No lease on " + src +
                                       " File does not exist. " +
                                       (lease != null ? lease.toString() :
@@ -1335,7 +1335,7 @@
                                        " does not have any open files."));
     }
     if (!file.isUnderConstruction()) {
-      Lease lease = leaseManager.getLease(new StringBytesWritable(holder));
+      Lease lease = leaseManager.getLease(holder);
       throw new LeaseExpiredException("No lease on " + src + 
                                       " File is not open for writing. " +
                                       (lease != null ? lease.toString() :
@@ -4553,10 +4553,7 @@
       out.writeInt(leaseManager.countPath()); // write the size
 
       for (Lease lease : leaseManager.getSortedLeases()) {
-        Collection<StringBytesWritable> files = lease.getPaths();
-        for (Iterator<StringBytesWritable> i = files.iterator(); i.hasNext();){
-          String path = i.next().getString();
-
+        for(String path : lease.getPaths()) {
           // verify that path exists in namespace
           INode node = dir.getFileINode(path);
           if (node == null) {

Modified: hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java?rev=709609&r1=709608&r2=709609&view=diff
==============================================================================
--- hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java (original)
+++ hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java Fri Oct 31 17:00:17 2008
@@ -25,28 +25,25 @@
 
 
 public class INodeFileUnderConstruction extends INodeFile {
-  StringBytesWritable clientName = null;         // lease holder
-  StringBytesWritable clientMachine = null;
-  DatanodeDescriptor clientNode = null; // if client is a cluster node too.
+  final String clientName;         // lease holder
+  private final String clientMachine;
+  private final DatanodeDescriptor clientNode; // if client is a cluster node too.
 
   private int primaryNodeIndex = -1; //the node working on lease recovery
   private DatanodeDescriptor[] targets = null;   //locations for last block
   private long lastRecoveryTime = 0;
   
-  INodeFileUnderConstruction() {}
-
   INodeFileUnderConstruction(PermissionStatus permissions,
                              short replication,
                              long preferredBlockSize,
                              long modTime,
                              String clientName,
                              String clientMachine,
-                             DatanodeDescriptor clientNode) 
-                             throws IOException {
+                             DatanodeDescriptor clientNode) {
     super(permissions.applyUMask(UMASK), 0, replication, modTime, modTime,
         preferredBlockSize);
-    this.clientName = new StringBytesWritable(clientName);
-    this.clientMachine = new StringBytesWritable(clientMachine);
+    this.clientName = clientName;
+    this.clientMachine = clientMachine;
     this.clientNode = clientNode;
   }
 
@@ -58,22 +55,21 @@
                              PermissionStatus perm,
                              String clientName,
                              String clientMachine,
-                             DatanodeDescriptor clientNode)
-                             throws IOException {
+                             DatanodeDescriptor clientNode) {
     super(perm, blocks, blockReplication, modificationTime, modificationTime,
           preferredBlockSize);
     setLocalName(name);
-    this.clientName = new StringBytesWritable(clientName);
-    this.clientMachine = new StringBytesWritable(clientMachine);
+    this.clientName = clientName;
+    this.clientMachine = clientMachine;
     this.clientNode = clientNode;
   }
 
-  String getClientName() throws IOException {
-    return clientName.getString();
+  String getClientName() {
+    return clientName;
   }
 
-  String getClientMachine() throws IOException {
-    return clientMachine.getString();
+  String getClientMachine() {
+    return clientMachine;
   }
 
   DatanodeDescriptor getClientNode() {

Modified: hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java?rev=709609&r1=709608&r2=709609&view=diff
==============================================================================
--- hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java (original)
+++ hadoop/core/trunk/src/hdfs/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java Fri Oct 31 17:00:17 2008
@@ -20,7 +20,6 @@
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.SortedMap;
@@ -67,7 +66,7 @@
   // Used for handling lock-leases
   // Mapping: leaseHolder -> Lease
   //
-  private SortedMap<StringBytesWritable, Lease> leases = new TreeMap<StringBytesWritable, Lease>();
+  private SortedMap<String, Lease> leases = new TreeMap<String, Lease>();
   // Set of: Lease
   private SortedSet<Lease> sortedLeases = new TreeSet<Lease>();
 
@@ -79,7 +78,7 @@
 
   LeaseManager(FSNamesystem fsnamesystem) {this.fsnamesystem = fsnamesystem;}
 
-  Lease getLease(StringBytesWritable holder) throws IOException {
+  Lease getLease(String holder) {
     return leases.get(holder);
   }
   
@@ -103,8 +102,7 @@
   /**
    * Adds (or re-adds) the lease for the specified file.
    */
-  synchronized void addLease(StringBytesWritable holder, String src
-      ) throws IOException {
+  synchronized void addLease(String holder, String src) {
     Lease lease = getLease(holder);
     if (lease == null) {
       lease = new Lease(holder);
@@ -114,13 +112,13 @@
       renewLease(lease);
     }
     sortedLeasesByPath.put(src, lease);
-    lease.paths.add(new StringBytesWritable(src));
+    lease.paths.add(src);
   }
 
   /**
    * Remove the specified lease and src.
    */
-  synchronized void removeLease(Lease lease, String src) throws IOException {
+  synchronized void removeLease(Lease lease, String src) {
     sortedLeasesByPath.remove(src);
     if (!lease.removePath(src)) {
       LOG.error(src + " not found in lease.paths (=" + lease.paths + ")");
@@ -137,8 +135,7 @@
   /**
    * Remove the lease for the specified holder and src
    */
-  synchronized void removeLease(StringBytesWritable holder, String src
-      ) throws IOException {
+  synchronized void removeLease(String holder, String src) {
     Lease lease = getLease(holder);
     if (lease != null) {
       removeLease(lease, src);
@@ -164,8 +161,8 @@
   /**
    * Renew the lease(s) held by the given client
    */
-  synchronized void renewLease(String holder) throws IOException {
-    renewLease(getLease(new StringBytesWritable(holder)));
+  synchronized void renewLease(String holder) {
+    renewLease(getLease(holder));
   }
   synchronized void renewLease(Lease lease) {
     if (lease != null) {
@@ -183,12 +180,12 @@
    * expire, all the corresponding locks can be released.
    *************************************************************/
   class Lease implements Comparable<Lease> {
-    private StringBytesWritable holder;
+    private final String holder;
     private long lastUpdate;
-    private Collection<StringBytesWritable> paths = new TreeSet<StringBytesWritable>();
+    private final Collection<String> paths = new TreeSet<String>();
   
     /** Only LeaseManager object can create a lease */
-    private Lease(StringBytesWritable holder) throws IOException {
+    private Lease(String holder) {
       this.holder = holder;
       renew();
     }
@@ -211,8 +208,7 @@
      * @return the path associated with the pendingFile and null if not found.
      */
     private String findPath(INodeFileUnderConstruction pendingFile) {
-      for(Iterator<StringBytesWritable> i = paths.iterator(); i.hasNext(); ) {
-        String src = i.next().toString();
+      for(String src : paths) {
         if (fsnamesystem.dir.getFileINode(src) == pendingFile) {
           return src;
         }
@@ -223,8 +219,8 @@
     /** Does this lease contain any path? */
     boolean hasPath() {return !paths.isEmpty();}
 
-    boolean removePath(String src) throws IOException {
-      return paths.remove(new StringBytesWritable(src));
+    boolean removePath(String src) {
+      return paths.remove(src);
     }
 
     /** {@inheritDoc} */
@@ -266,18 +262,18 @@
       return holder.hashCode();
     }
     
-    Collection<StringBytesWritable> getPaths() {
+    Collection<String> getPaths() {
       return paths;
     }
     
-    void replacePath(String oldpath, String newpath) throws IOException {
-      paths.remove(new StringBytesWritable(oldpath));
-      paths.add(new StringBytesWritable(newpath));
+    void replacePath(String oldpath, String newpath) {
+      paths.remove(oldpath);
+      paths.add(newpath);
     }
   }
 
   synchronized void changeLease(String src, String dst,
-      String overwrite, String replaceBy) throws IOException {
+      String overwrite, String replaceBy) {
     if (LOG.isDebugEnabled()) {
       LOG.debug(getClass().getSimpleName() + ".changelease: " +
                " src=" + src + ", dest=" + dst + 
@@ -298,7 +294,7 @@
     }
   }
 
-  synchronized void removeLeaseWithPrefixPath(String prefix) throws IOException {
+  synchronized void removeLeaseWithPrefixPath(String prefix) {
     for(Map.Entry<String, Lease> entry : findLeaseWithPrefixPath(prefix, sortedLeasesByPath)) {
       if (LOG.isDebugEnabled()) {
         LOG.debug(LeaseManager.class.getSimpleName()
@@ -352,8 +348,8 @@
                 if (top.expiredHardLimit()) {
                   LOG.info("Lease Monitor: Removing lease " + top
                       + ", sortedLeases.size()=: " + sortedLeases.size());
-                  for(StringBytesWritable s : top.paths) {
-                    fsnamesystem.internalReleaseLease(top, s.getString());
+                  for(String s : top.paths) {
+                    fsnamesystem.internalReleaseLease(top, s);
                   }
                 } else {
                   break;