Explorar el Código

HDFS-5372. In FSNamesystem, hasReadLock() returns false if the current thread holds the write lock (Contributed by Vinay)


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1542887 13f79535-47bb-0310-9956-ffa450edef68
Uma Maheswara Rao G hace 11 años
padre
commit
e3d7ef36ef

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -666,6 +666,9 @@ Release 2.2.1 - UNRELEASED
     HDFS-5519. COMMIT handler should update the commit status after sync
     (brandonli)
 
+    HDFS-5372. In FSNamesystem, hasReadLock() returns false if the current thread 
+    holds the write lock (VinayaKumar B via umamahesh)
+
 Release 2.2.0 - 2013-10-13
 
   INCOMPATIBLE CHANGES

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -817,7 +817,7 @@ public class BlockManager {
       final boolean isFileUnderConstruction, final long offset,
       final long length, final boolean needBlockToken, final boolean inSnapshot)
       throws IOException {
-    assert namesystem.hasReadOrWriteLock();
+    assert namesystem.hasReadLock();
     if (blocks == null) {
       return null;
     } else if (blocks.length == 0) {

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java

@@ -237,13 +237,13 @@ public final class CacheManager {
   }
 
   public TreeMap<Long, PathBasedCacheEntry> getEntriesById() {
-    assert namesystem.hasReadOrWriteLock();
+    assert namesystem.hasReadLock();
     return entriesById;
   }
   
   @VisibleForTesting
   public GSet<CachedBlock, CachedBlock> getCachedBlocks() {
-    assert namesystem.hasReadOrWriteLock();
+    assert namesystem.hasReadLock();
     return cachedBlocks;
   }
 
@@ -450,7 +450,7 @@ public final class CacheManager {
         listPathBasedCacheDirectives(long prevId,
             PathBasedCacheDirective filter,
             FSPermissionChecker pc) throws IOException {
-    assert namesystem.hasReadOrWriteLock();
+    assert namesystem.hasReadLock();
     final int NUM_PRE_ALLOCATED_ENTRIES = 16;
     String filterPath = null;
     if (filter.getId() != null) {
@@ -607,7 +607,7 @@ public final class CacheManager {
 
   public BatchedListEntries<CachePoolInfo>
       listCachePools(FSPermissionChecker pc, String prevKey) {
-    assert namesystem.hasReadOrWriteLock();
+    assert namesystem.hasReadLock();
     final int NUM_PRE_ALLOCATED_ENTRIES = 16;
     ArrayList<CachePoolInfo> results = 
         new ArrayList<CachePoolInfo>(NUM_PRE_ALLOCATED_ENTRIES);

+ 5 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -1295,11 +1295,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   }
   @Override
   public boolean hasReadLock() {
-    return this.fsLock.getReadHoldCount() > 0;
-  }
-  @Override
-  public boolean hasReadOrWriteLock() {
-    return hasReadLock() || hasWriteLock();
+    return this.fsLock.getReadHoldCount() > 0 || hasWriteLock();
   }
 
   public int getReadHoldCount() {
@@ -2043,7 +2039,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    */
   private void verifyParentDir(String src) throws FileNotFoundException,
       ParentNotDirectoryException, UnresolvedLinkException {
-    assert hasReadOrWriteLock();
+    assert hasReadLock();
     Path parent = new Path(src).getParent();
     if (parent != null) {
       final INode parentNode = dir.getINode(parent.toString());
@@ -2651,7 +2647,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
                                 ExtendedBlock previous,
                                 LocatedBlock[] onRetryBlock)
           throws IOException  {
-    assert hasReadOrWriteLock();
+    assert hasReadLock();
 
     checkBlock(previous);
     onRetryBlock[0] = null;
@@ -2848,7 +2844,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   private INodeFileUnderConstruction checkLease(String src, long fileId,
       String holder, INode inode) throws LeaseExpiredException,
       FileNotFoundException {
-    assert hasReadOrWriteLock();
+    assert hasReadLock();
     if (inode == null || !inode.isFile()) {
       Lease lease = leaseManager.getLease(holder);
       throw new LeaseExpiredException(
@@ -3807,7 +3803,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   
   @Override
   public boolean isInSnapshot(BlockInfoUnderConstruction blockUC) {
-    assert hasReadOrWriteLock();
+    assert hasReadLock();
     final BlockCollection bc = blockUC.getBlockCollection();
     if (bc == null || !(bc instanceof INodeFileUnderConstruction)) {
       return false;

+ 0 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/RwLock.java

@@ -39,7 +39,4 @@ public interface RwLock {
 
   /** Check if the current thread holds write lock. */
   public boolean hasWriteLock();
-
-  /** Check if the current thread holds read or write lock. */
-  public boolean hasReadOrWriteLock();
 }