瀏覽代碼

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 11 年之前
父節點
當前提交
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
     HDFS-5519. COMMIT handler should update the commit status after sync
     (brandonli)
     (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
 Release 2.2.0 - 2013-10-13
 
 
   INCOMPATIBLE CHANGES
   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 boolean isFileUnderConstruction, final long offset,
       final long length, final boolean needBlockToken, final boolean inSnapshot)
       final long length, final boolean needBlockToken, final boolean inSnapshot)
       throws IOException {
       throws IOException {
-    assert namesystem.hasReadOrWriteLock();
+    assert namesystem.hasReadLock();
     if (blocks == null) {
     if (blocks == null) {
       return null;
       return null;
     } else if (blocks.length == 0) {
     } 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() {
   public TreeMap<Long, PathBasedCacheEntry> getEntriesById() {
-    assert namesystem.hasReadOrWriteLock();
+    assert namesystem.hasReadLock();
     return entriesById;
     return entriesById;
   }
   }
   
   
   @VisibleForTesting
   @VisibleForTesting
   public GSet<CachedBlock, CachedBlock> getCachedBlocks() {
   public GSet<CachedBlock, CachedBlock> getCachedBlocks() {
-    assert namesystem.hasReadOrWriteLock();
+    assert namesystem.hasReadLock();
     return cachedBlocks;
     return cachedBlocks;
   }
   }
 
 
@@ -450,7 +450,7 @@ public final class CacheManager {
         listPathBasedCacheDirectives(long prevId,
         listPathBasedCacheDirectives(long prevId,
             PathBasedCacheDirective filter,
             PathBasedCacheDirective filter,
             FSPermissionChecker pc) throws IOException {
             FSPermissionChecker pc) throws IOException {
-    assert namesystem.hasReadOrWriteLock();
+    assert namesystem.hasReadLock();
     final int NUM_PRE_ALLOCATED_ENTRIES = 16;
     final int NUM_PRE_ALLOCATED_ENTRIES = 16;
     String filterPath = null;
     String filterPath = null;
     if (filter.getId() != null) {
     if (filter.getId() != null) {
@@ -607,7 +607,7 @@ public final class CacheManager {
 
 
   public BatchedListEntries<CachePoolInfo>
   public BatchedListEntries<CachePoolInfo>
       listCachePools(FSPermissionChecker pc, String prevKey) {
       listCachePools(FSPermissionChecker pc, String prevKey) {
-    assert namesystem.hasReadOrWriteLock();
+    assert namesystem.hasReadLock();
     final int NUM_PRE_ALLOCATED_ENTRIES = 16;
     final int NUM_PRE_ALLOCATED_ENTRIES = 16;
     ArrayList<CachePoolInfo> results = 
     ArrayList<CachePoolInfo> results = 
         new ArrayList<CachePoolInfo>(NUM_PRE_ALLOCATED_ENTRIES);
         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
   @Override
   public boolean hasReadLock() {
   public boolean hasReadLock() {
-    return this.fsLock.getReadHoldCount() > 0;
-  }
-  @Override
-  public boolean hasReadOrWriteLock() {
-    return hasReadLock() || hasWriteLock();
+    return this.fsLock.getReadHoldCount() > 0 || hasWriteLock();
   }
   }
 
 
   public int getReadHoldCount() {
   public int getReadHoldCount() {
@@ -2043,7 +2039,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    */
    */
   private void verifyParentDir(String src) throws FileNotFoundException,
   private void verifyParentDir(String src) throws FileNotFoundException,
       ParentNotDirectoryException, UnresolvedLinkException {
       ParentNotDirectoryException, UnresolvedLinkException {
-    assert hasReadOrWriteLock();
+    assert hasReadLock();
     Path parent = new Path(src).getParent();
     Path parent = new Path(src).getParent();
     if (parent != null) {
     if (parent != null) {
       final INode parentNode = dir.getINode(parent.toString());
       final INode parentNode = dir.getINode(parent.toString());
@@ -2651,7 +2647,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
                                 ExtendedBlock previous,
                                 ExtendedBlock previous,
                                 LocatedBlock[] onRetryBlock)
                                 LocatedBlock[] onRetryBlock)
           throws IOException  {
           throws IOException  {
-    assert hasReadOrWriteLock();
+    assert hasReadLock();
 
 
     checkBlock(previous);
     checkBlock(previous);
     onRetryBlock[0] = null;
     onRetryBlock[0] = null;
@@ -2848,7 +2844,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   private INodeFileUnderConstruction checkLease(String src, long fileId,
   private INodeFileUnderConstruction checkLease(String src, long fileId,
       String holder, INode inode) throws LeaseExpiredException,
       String holder, INode inode) throws LeaseExpiredException,
       FileNotFoundException {
       FileNotFoundException {
-    assert hasReadOrWriteLock();
+    assert hasReadLock();
     if (inode == null || !inode.isFile()) {
     if (inode == null || !inode.isFile()) {
       Lease lease = leaseManager.getLease(holder);
       Lease lease = leaseManager.getLease(holder);
       throw new LeaseExpiredException(
       throw new LeaseExpiredException(
@@ -3807,7 +3803,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   
   
   @Override
   @Override
   public boolean isInSnapshot(BlockInfoUnderConstruction blockUC) {
   public boolean isInSnapshot(BlockInfoUnderConstruction blockUC) {
-    assert hasReadOrWriteLock();
+    assert hasReadLock();
     final BlockCollection bc = blockUC.getBlockCollection();
     final BlockCollection bc = blockUC.getBlockCollection();
     if (bc == null || !(bc instanceof INodeFileUnderConstruction)) {
     if (bc == null || !(bc instanceof INodeFileUnderConstruction)) {
       return false;
       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. */
   /** Check if the current thread holds write lock. */
   public boolean hasWriteLock();
   public boolean hasWriteLock();
-
-  /** Check if the current thread holds read or write lock. */
-  public boolean hasReadOrWriteLock();
 }
 }