Bläddra i källkod

HADOOP-932. File locking interfaces and implementations (that were
earlier deprecated) are removed. Client Protocol version changed
from 15 to 16. (Raghu Angadi via dhruba)



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

Dhruba Borthakur 18 år sedan
förälder
incheckning
0052e04b2e

+ 4 - 0
CHANGES.txt

@@ -9,6 +9,10 @@ Trunk (unreleased changes)
     file creation.  ClientProtocol version changed from 14 to 15.
     (Tsz Wo (Nicholas), SZE via dhruba)
 
+    HADOOP-932.  File locking interfaces and implementations (that were
+    earlier deprecated) are removed.  Client Protocol version changed 
+    from 15 to 16.  (Raghu Angadi via dhruba)
+
   NEW FEATURES
 
     HADOOP-1636.  Allow configuration of the number of jobs kept in

+ 2 - 20
src/java/org/apache/hadoop/dfs/ClientProtocol.java

@@ -31,9 +31,9 @@ interface ClientProtocol extends VersionedProtocol {
 
   /**
    * Compared to the previous version the following changes have been introduced:
-   * 15: create(...) should only create a file but not return block.
+   * 16 : removed deprecated obtainLock() and releaseLock(). 
    */
-  public static final long versionID = 15L;
+  public static final long versionID = 16L;
   
   ///////////////////////////////////////
   // File contents
@@ -208,24 +208,6 @@ interface ClientProtocol extends VersionedProtocol {
   ///////////////////////////////////////
   // System issues and management
   ///////////////////////////////////////
-  /**
-   * obtainLock() is used for lock managemnet.  It returns true if
-   * the lock has been seized correctly.  It returns false if the
-   * lock could not be obtained, and the client should try again.
-   *
-   * Locking is a part of most filesystems and is useful for a
-   * number of inter-process synchronization tasks.
-   */
-  /** @deprecated */ @Deprecated
-    public boolean obtainLock(String src, String clientName, boolean exclusive) throws IOException;
-
-  /**
-   * releaseLock() is called if the client would like to release
-   * a held lock.  It returns true if the lock is correctly released.
-   * It returns false if the client should wait and try again.
-   */
-  /** @deprecated */ @Deprecated
-    public boolean releaseLock(String src, String clientName) throws IOException;
 
   /**
    * Client programs can cause stateful changes in the NameNode

+ 0 - 37
src/java/org/apache/hadoop/dfs/DFSClient.java

@@ -508,43 +508,6 @@ class DFSClient implements FSConstants {
     return namenode.mkdirs(src.toString());
   }
 
-  /**
-   */
-  public void lock(UTF8 src, boolean exclusive) throws IOException {
-    long start = System.currentTimeMillis();
-    boolean hasLock = false;
-    while (!hasLock) {
-      hasLock = namenode.obtainLock(src.toString(), clientName, exclusive);
-      if (!hasLock) {
-        try {
-          Thread.sleep(400);
-          if (System.currentTimeMillis() - start > 5000) {
-            LOG.info("Waiting to retry lock for " + (System.currentTimeMillis() - start) + " ms.");
-            Thread.sleep(2000);
-          }
-        } catch (InterruptedException ie) {
-        }
-      }
-    }
-  }
-
-  /**
-   *
-   */
-  public void release(UTF8 src) throws IOException {
-    boolean hasReleased = false;
-    while (!hasReleased) {
-      hasReleased = namenode.releaseLock(src.toString(), clientName);
-      if (!hasReleased) {
-        LOG.info("Could not release.  Retrying...");
-        try {
-          Thread.sleep(2000);
-        } catch (InterruptedException ie) {
-        }
-      }
-    }
-  }
-
   /**
    * Pick the best node from which to stream the data.
    * Entries in <i>nodes</i> are already in the priority order

+ 0 - 10
src/java/org/apache/hadoop/dfs/DistributedFileSystem.java

@@ -188,16 +188,6 @@ public class DistributedFileSystem extends FileSystem {
     return dfs.mkdirs(getPath(f));
   }
 
-  /** @deprecated */ @Deprecated
-  public void lock(Path f, boolean shared) throws IOException {
-    dfs.lock(getPath(f), !shared);
-  }
-
-  /** @deprecated */ @Deprecated
-  public void release(Path f) throws IOException {
-    dfs.release(getPath(f));
-  }
-
   public void close() throws IOException {
     super.close();
     dfs.close();

+ 0 - 36
src/java/org/apache/hadoop/dfs/FSDirectory.java

@@ -412,8 +412,6 @@ class FSDirectory implements FSConstants {
 
   FSNamesystem namesystem = null;
   INode rootDir = new INode("");
-  TreeMap<StringBytesWritable, TreeSet<StringBytesWritable>> activeLocks =
-    new TreeMap<StringBytesWritable, TreeSet<StringBytesWritable>>();
   FSImage fsImage;  
   boolean ready = false;
   // Metrics record
@@ -709,40 +707,6 @@ class FSDirectory implements FSConstants {
     }
   }
 
-  /**
-   */
-  public int obtainLock(String src, String holder, boolean exclusive) throws IOException {
-    StringBytesWritable srcSBW = new StringBytesWritable(src);
-    TreeSet<StringBytesWritable> holders = activeLocks.get(srcSBW);
-    if (holders == null) {
-      holders = new TreeSet<StringBytesWritable>();
-      activeLocks.put(srcSBW, holders);
-    }
-    if (exclusive && holders.size() > 0) {
-      return STILL_WAITING;
-    } else {
-      holders.add(new StringBytesWritable(holder));
-      return COMPLETE_SUCCESS;
-    }
-  }
-
-  /**
-   */
-  public int releaseLock(String src, String holder) throws IOException {
-    StringBytesWritable srcSBW = new StringBytesWritable(src);
-    StringBytesWritable holderSBW = new StringBytesWritable(holder);
-    TreeSet<StringBytesWritable> holders = activeLocks.get(srcSBW);
-    if (holders != null && holders.contains(holderSBW)) {
-      holders.remove(holderSBW);
-      if (holders.size() == 0) {
-        activeLocks.remove(srcSBW);
-      }
-      return COMPLETE_SUCCESS;
-    } else {
-      return OPERATION_FAILED;
-    }
-  }
-
   /**
    * Get a listing of files given path 'src'
    *

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

@@ -1357,8 +1357,6 @@ class FSNamesystem implements FSConstants {
     }
     public void releaseLocks() throws IOException {
       String holderStr = holder.getString();
-      for (Iterator<StringBytesWritable> it = locks.iterator(); it.hasNext();)
-        internalReleaseLock(it.next().getString(), holderStr);
       locks.clear();
       for (Iterator<StringBytesWritable> it = creates.iterator(); it.hasNext();)
         internalReleaseCreate(it.next().getString(), holderStr);
@@ -1458,31 +1456,6 @@ class FSNamesystem implements FSConstants {
     leases.remove(new StringBytesWritable(holder));
   }
 
-  /**
-   * Get a lock (perhaps exclusive) on the given file
-   */
-  /** @deprecated */
-  @Deprecated
-  public synchronized int obtainLock(UTF8 src, 
-                                     UTF8 holder, 
-                                     boolean exclusive) throws IOException {
-    if (isInSafeMode())
-      throw new SafeModeException("Cannot lock file " + src, safeMode);
-    return OPERATION_FAILED;
-  }
-
-  /**
-   * Release the lock on the given file
-   */
-  /** @deprecated */
-  @Deprecated
-  public synchronized int releaseLock(UTF8 src, UTF8 holder) {
-    return OPERATION_FAILED;
-  }
-  private int internalReleaseLock(String src, String holder) throws IOException {
-    return dir.releaseLock(src, holder);
-  }
-
   /**
    * Release a pending file creation lock.
    * @param src The filename

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

@@ -434,30 +434,6 @@ public class NameNode implements ClientProtocol, DatanodeProtocol, FSConstants {
     return namesystem.mkdirs(src);
   }
 
-  /** @deprecated */ @Deprecated
-    public boolean obtainLock(String src, String clientName, boolean exclusive) throws IOException {
-    int returnCode = namesystem.obtainLock(new UTF8(src), new UTF8(clientName), exclusive);
-    if (returnCode == COMPLETE_SUCCESS) {
-      return true;
-    } else if (returnCode == STILL_WAITING) {
-      return false;
-    } else {
-      throw new IOException("Failure when trying to obtain lock on " + src);
-    }
-  }
-
-  /** @deprecated */ @Deprecated
-    public boolean releaseLock(String src, String clientName) throws IOException {
-    int returnCode = namesystem.releaseLock(new UTF8(src), new UTF8(clientName));
-    if (returnCode == COMPLETE_SUCCESS) {
-      return true;
-    } else if (returnCode == STILL_WAITING) {
-      return false;
-    } else {
-      throw new IOException("Failure when trying to release lock on " + src);
-    }
-  }
-
   /**
    */
   public void renewLease(String clientName) throws IOException {

+ 0 - 26
src/java/org/apache/hadoop/fs/ChecksumFileSystem.java

@@ -442,32 +442,6 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
     return fs.mkdirs(f);
   }
 
-  @Override
-  public void lock(Path f, boolean shared) throws IOException {
-    if (fs.isDirectory(f)) {
-      fs.lock(f, shared);
-    } else {
-      Path checkFile = getChecksumFile(f);
-      if (fs.exists(checkFile)) {
-        fs.lock(checkFile, shared);
-      }
-      fs.lock(f, shared);
-    }
-  }
-
-  @Override
-  public void release(Path f) throws IOException {
-    if (fs.isDirectory(f)) {
-      fs.release(f);
-    } else {
-      Path checkFile = getChecksumFile(f);
-      if (fs.exists(checkFile)) {
-        fs.release(getChecksumFile(f));
-      }
-      fs.release(f);
-    }
-  }
-
   @Override
   public void copyFromLocalFile(boolean delSrc, Path src, Path dst)
     throws IOException {

+ 0 - 16
src/java/org/apache/hadoop/fs/FileSystem.java

@@ -740,22 +740,6 @@ public abstract class FileSystem extends Configured {
    */
   public abstract boolean mkdirs(Path f) throws IOException;
 
-  /**
-   * Obtain a lock on the given Path
-   * 
-   * @deprecated FS does not support file locks anymore.
-   */
-  @Deprecated
-  public void lock(Path f, boolean shared) throws IOException {}
-
-  /**
-   * Release the lock
-   * 
-   * @deprecated FS does not support file locks anymore.     
-   */
-  @Deprecated
-  public void release(Path f) throws IOException {}
-
   /**
    * The src file is on the local disk.  Add it to FS at
    * the given dst name and the source is kept intact afterwards

+ 0 - 20
src/java/org/apache/hadoop/fs/FilterFileSystem.java

@@ -189,26 +189,6 @@ public class FilterFileSystem extends FileSystem {
     return fs.mkdirs(f);
   }
 
-  /**
-   * Obtain a lock on the given Path
-   * 
-   * @deprecated FS does not support file locks anymore.
-   */
-  @Deprecated
-  public void lock(Path f, boolean shared) throws IOException {
-    fs.lock(f, shared);
-  }
-
-  /**
-   * Release the lock
-   * 
-   * @deprecated FS does not support file locks anymore.     
-   */
-  @Deprecated
-  public void release(Path f) throws IOException {
-    fs.release(f);
-  }
-
   /**
    * The src file is on the local disk.  Add it to FS at
    * the given dst name.

+ 0 - 16
src/java/org/apache/hadoop/mapred/PhasedFileSystem.java

@@ -268,22 +268,6 @@ public class PhasedFileSystem extends FilterFileSystem {
     throw new UnsupportedOperationException("Operation not supported");  
   }
 
-  /** @deprecated */ @Deprecated
-    @Override
-    public void lock(
-                     Path f, boolean shared)
-    throws IOException {
-    throw new UnsupportedOperationException("Operation not supported");  
-  }
-
-  /** @deprecated */ @Deprecated
-    @Override
-    public void release(
-                        Path f)
-    throws IOException {
-    throw new UnsupportedOperationException("Operation not supported");  
-  }
-
   @Override
   public void copyFromLocalFile(
                                 boolean delSrc, Path src, Path dst)