Browse Source

HDFS-4143. Change blocks to private in INodeFile and renames isLink() to isSymlink() in INode.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1405237 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 12 years ago
parent
commit
d174f574ba

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

@@ -445,6 +445,9 @@ Release 2.0.3-alpha - Unreleased
 
     HDFS-3916. libwebhdfs testing code cleanup. (Jing Zhao via suresh)
 
+    HDFS-4143. Change blocks to private in INodeFile and renames isLink() to
+    isSymlink() in INode. (szetszwo)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 13 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -313,7 +313,7 @@ public class FSDirectory implements Closeable {
       }
       if(newParent == null)
         return null;
-      if(!newNode.isDirectory() && !newNode.isLink()) {
+      if(!newNode.isDirectory() && !newNode.isSymlink()) {
         // Add file->block mapping
         INodeFile newF = (INodeFile)newNode;
         BlockInfo[] blocks = newF.getBlocks();
@@ -533,7 +533,7 @@ public class FSDirectory implements Closeable {
     if (dst.equals(src)) {
       return true;
     }
-    if (srcInode.isLink() && 
+    if (srcInode.isSymlink() && 
         dst.equals(((INodeSymlink)srcInode).getLinkValue())) {
       throw new FileAlreadyExistsException(
           "Cannot rename symlink "+src+" to its target "+dst);
@@ -655,7 +655,7 @@ public class FSDirectory implements Closeable {
       throw new FileAlreadyExistsException(
           "The source "+src+" and destination "+dst+" are the same");
     }
-    if (srcInode.isLink() && 
+    if (srcInode.isSymlink() && 
         dst.equals(((INodeSymlink)srcInode).getLinkValue())) {
       throw new FileAlreadyExistsException(
           "Cannot rename symlink "+src+" to its target "+dst);
@@ -819,7 +819,7 @@ public class FSDirectory implements Closeable {
     if (inode == null) {
       return null;
     }
-    assert !inode.isLink();
+    assert !inode.isSymlink();
     if (inode.isDirectory()) {
       return null;
     }
@@ -851,7 +851,7 @@ public class FSDirectory implements Closeable {
       if (inode == null) {
         throw new FileNotFoundException("File does not exist: " + filename);
       }
-      if (inode.isDirectory() || inode.isLink()) {
+      if (inode.isDirectory() || inode.isSymlink()) {
         throw new IOException("Getting block size of non-file: "+ filename); 
       }
       return ((INodeFile)inode).getPreferredBlockSize();
@@ -868,7 +868,7 @@ public class FSDirectory implements Closeable {
       if (inode == null) {
          return false;
       }
-      return inode.isDirectory() || inode.isLink() 
+      return inode.isDirectory() || inode.isSymlink() 
         ? true 
         : ((INodeFile)inode).getBlocks() != null;
     } finally {
@@ -968,7 +968,7 @@ public class FSDirectory implements Closeable {
     for(String src : srcs) {
       INodeFile srcInode = (INodeFile)getINode(src);
       allSrcInodes[i++] = srcInode;
-      totalBlocks += srcInode.blocks.length;  
+      totalBlocks += srcInode.numBlocks();  
     }
     trgInode.appendBlocks(allSrcInodes, totalBlocks); // copy the blocks
     
@@ -977,7 +977,7 @@ public class FSDirectory implements Closeable {
     for(INodeFile nodeToRemove: allSrcInodes) {
       if(nodeToRemove == null) continue;
       
-      nodeToRemove.blocks = null;
+      nodeToRemove.setBlocks(null);
       trgParent.removeChild(nodeToRemove);
       count++;
     }
@@ -1208,7 +1208,7 @@ public class FSDirectory implements Closeable {
         return null;
       if (targetNode.isDirectory())
         return null;
-      if (targetNode.isLink()) 
+      if (targetNode.isSymlink()) 
         return null;
       return ((INodeFile)targetNode).getBlocks();
     } finally {
@@ -1822,7 +1822,7 @@ public class FSDirectory implements Closeable {
       if (child.isDirectory()) {
         updateCountForINodeWithQuota((INodeDirectory)child, 
                                      counts, nodesInPath);
-      } else if (child.isLink()) {
+      } else if (child.isSymlink()) {
         counts.nsCount += 1;
       } else { // reduce recursive calls
         counts.nsCount += 1;
@@ -2051,7 +2051,7 @@ public class FSDirectory implements Closeable {
         node.getFsPermission(),
         node.getUserName(),
         node.getGroupName(),
-        node.isLink() ? ((INodeSymlink)node).getSymlink() : null,
+        node.isSymlink() ? ((INodeSymlink)node).getSymlink() : null,
         path);
   }
 
@@ -2087,7 +2087,7 @@ public class FSDirectory implements Closeable {
           node.getFsPermission(),
           node.getUserName(),
           node.getGroupName(),
-          node.isLink() ? ((INodeSymlink)node).getSymlink() : null,
+          node.isSymlink() ? ((INodeSymlink)node).getSymlink() : null,
           path,
           loc);
       }
@@ -2158,7 +2158,7 @@ public class FSDirectory implements Closeable {
    */
   void cacheName(INode inode) {
     // Name is cached only for files
-    if (inode.isDirectory() || inode.isLink()) {
+    if (inode.isDirectory() || inode.isSymlink()) {
       return;
     }
     ByteArray name = new ByteArray(inode.getLocalNameBytes());

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java

@@ -162,7 +162,7 @@ public class FSImageSerialization {
       PermissionStatus.write(out, node.getUserName(),
                              node.getGroupName(),
                              filePerm);
-    } else if (node.isLink()) {
+    } else if (node.isSymlink()) {
       out.writeShort(0);  // replication
       out.writeLong(0);   // modification time
       out.writeLong(0);   // access time

+ 11 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -1390,7 +1390,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
           + target + " is under construction");
     }
     // per design target shouldn't be empty and all the blocks same size
-    if(trgInode.blocks.length == 0) {
+    if(trgInode.numBlocks() == 0) {
       throw new HadoopIllegalArgumentException("concat: target file "
           + target + " is empty");
     }
@@ -1398,10 +1398,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     long blockSize = trgInode.getPreferredBlockSize();
 
     // check the end block to be full
-    if(blockSize != trgInode.blocks[trgInode.blocks.length-1].getNumBytes()) {
+    final BlockInfo last = trgInode.getLastBlock();
+    if(blockSize != last.getNumBytes()) {
       throw new HadoopIllegalArgumentException("The last block in " + target
-          + " is not full; last block size = "
-          + trgInode.blocks[trgInode.blocks.length-1].getNumBytes()
+          + " is not full; last block size = " + last.getNumBytes()
           + " but file block size = " + blockSize);
     }
 
@@ -1418,7 +1418,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       final INodeFile srcInode = INodeFile.valueOf(dir.getINode(src), src);
       if(src.isEmpty() 
           || srcInode.isUnderConstruction()
-          || srcInode.blocks.length == 0) {
+          || srcInode.numBlocks() == 0) {
         throw new HadoopIllegalArgumentException("concat: source file " + src
             + " is invalid or empty or underConstruction");
       }
@@ -1435,15 +1435,16 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       //boolean endBlock=false;
       // verify that all the blocks are of the same length as target
       // should be enough to check the end blocks
-      int idx = srcInode.blocks.length-1;
+      final BlockInfo[] srcBlocks = srcInode.getBlocks();
+      int idx = srcBlocks.length-1;
       if(endSrc)
-        idx = srcInode.blocks.length-2; // end block of endSrc is OK not to be full
-      if(idx >= 0 && srcInode.blocks[idx].getNumBytes() != blockSize) {
+        idx = srcBlocks.length-2; // end block of endSrc is OK not to be full
+      if(idx >= 0 && srcBlocks[idx].getNumBytes() != blockSize) {
         throw new HadoopIllegalArgumentException("concat: the soruce file "
             + src + " and the target file " + target
             + " should have the same blocks sizes: target block size is "
             + blockSize + " but the size of source block " + idx + " is "
-            + srcInode.blocks[idx].getNumBytes());
+            + srcBlocks[idx].getNumBytes());
       }
 
       si.add(srcInode);
@@ -1678,7 +1679,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       if (parentNode == null) {
         throw new FileNotFoundException("Parent directory doesn't exist: "
             + parent.toString());
-      } else if (!parentNode.isDirectory() && !parentNode.isLink()) {
+      } else if (!parentNode.isDirectory() && !parentNode.isSymlink()) {
         throw new ParentNotDirectoryException("Parent path is not a directory: "
             + parent.toString());
       }

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

@@ -183,7 +183,9 @@ abstract class INode implements Comparable<byte[]> {
   /**
    * Check whether it's a directory
    */
-  abstract boolean isDirectory();
+  public boolean isDirectory() {
+    return false;
+  }
 
   /**
    * Collect all the blocks in all children of this INode.
@@ -332,7 +334,7 @@ abstract class INode implements Comparable<byte[]> {
   /**
    * Check whether it's a symlink
    */
-  public boolean isLink() {
+  public boolean isSymlink() {
     return false;
   }
 

+ 3 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java

@@ -78,11 +78,9 @@ class INodeDirectory extends INode {
     this.children = other.getChildren();
   }
   
-  /**
-   * Check whether it's a directory
-   */
+  /** @return true unconditionally. */
   @Override
-  public boolean isDirectory() {
+  public final boolean isDirectory() {
     return true;
   }
 
@@ -207,7 +205,7 @@ class INodeDirectory extends INode {
       if (index >= 0) {
         existing.inodes[index] = curNode;
       }
-      if (curNode.isLink() && (!lastComp || (lastComp && resolveLink))) {
+      if (curNode.isSymlink() && (!lastComp || (lastComp && resolveLink))) {
         final String path = constructPath(components, 0, components.length);
         final String preceding = constructPath(components, 0, count);
         final String remainder =

+ 11 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -55,7 +55,7 @@ public class INodeFile extends INode implements BlockCollection {
 
   private long header;
 
-  BlockInfo blocks[] = null;
+  private BlockInfo[] blocks;
 
   INodeFile(PermissionStatus permissions, BlockInfo[] blklist,
                       short replication, long modificationTime,
@@ -63,7 +63,7 @@ public class INodeFile extends INode implements BlockCollection {
     super(permissions, modificationTime, atime);
     this.setReplication(replication);
     this.setPreferredBlockSize(preferredBlockSize);
-    blocks = blklist;
+    this.blocks = blklist;
   }
 
   /**
@@ -76,11 +76,6 @@ public class INodeFile extends INode implements BlockCollection {
     super.setPermission(permission.applyUMask(UMASK));
   }
 
-  @Override
-  boolean isDirectory() {
-    return false;
-  }
-
   /** @return the replication factor of the file. */
   @Override
   public short getBlockReplication() {
@@ -128,7 +123,7 @@ public class INodeFile extends INode implements BlockCollection {
     for(BlockInfo bi: newlist) {
       bi.setBlockCollection(this);
     }
-    this.blocks = newlist;
+    setBlocks(newlist);
   }
   
   /**
@@ -136,14 +131,13 @@ public class INodeFile extends INode implements BlockCollection {
    */
   void addBlock(BlockInfo newblock) {
     if (this.blocks == null) {
-      this.blocks = new BlockInfo[1];
-      this.blocks[0] = newblock;
+      this.setBlocks(new BlockInfo[]{newblock});
     } else {
       int size = this.blocks.length;
       BlockInfo[] newlist = new BlockInfo[size + 1];
       System.arraycopy(this.blocks, 0, newlist, 0, size);
       newlist[size] = newblock;
-      this.blocks = newlist;
+      this.setBlocks(newlist);
     }
   }
 
@@ -152,6 +146,11 @@ public class INodeFile extends INode implements BlockCollection {
     this.blocks[idx] = blk;
   }
 
+  /** Set the blocks. */
+  public void setBlocks(BlockInfo[] blocks) {
+    this.blocks = blocks;
+  }
+
   @Override
   int collectSubtreeBlocksAndClear(List<Block> v) {
     parent = null;
@@ -161,7 +160,7 @@ public class INodeFile extends INode implements BlockCollection {
         blk.setBlockCollection(null);
       }
     }
-    blocks = null;
+    setBlocks(null);
     return 1;
   }
   

+ 10 - 16
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.IOException;
+import java.util.Arrays;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.permission.PermissionStatus;
@@ -28,8 +29,6 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.MutableBlockCollection;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 
-import com.google.common.base.Joiner;
-
 /**
  * I-node for file being written.
  */
@@ -109,9 +108,9 @@ class INodeFileUnderConstruction extends INodeFile implements MutableBlockCollec
   // use the modification time as the access time
   //
   INodeFile convertToInodeFile() {
-    assert allBlocksComplete() :
-      "Can't finalize inode " + this + " since it contains " +
-      "non-complete blocks! Blocks are: " + blocksAsString();
+    assert allBlocksComplete() : "Can't finalize inode " + this
+      + " since it contains non-complete blocks! Blocks are "
+      + Arrays.asList(getBlocks());
     INodeFile obj = new INodeFile(getPermissionStatus(),
                                   getBlocks(),
                                   getBlockReplication(),
@@ -126,7 +125,7 @@ class INodeFileUnderConstruction extends INodeFile implements MutableBlockCollec
    * @return true if all of the blocks in this file are marked as completed.
    */
   private boolean allBlocksComplete() {
-    for (BlockInfo b : blocks) {
+    for (BlockInfo b : getBlocks()) {
       if (!b.isComplete()) {
         return false;
       }
@@ -139,6 +138,7 @@ class INodeFileUnderConstruction extends INodeFile implements MutableBlockCollec
    * the last one on the list.
    */
   void removeLastBlock(Block oldblock) throws IOException {
+    final BlockInfo[] blocks = getBlocks();
     if (blocks == null) {
       throw new IOException("Trying to delete non-existant block " + oldblock);
     }
@@ -150,7 +150,7 @@ class INodeFileUnderConstruction extends INodeFile implements MutableBlockCollec
     //copy to a new list
     BlockInfo[] newlist = new BlockInfo[size_1];
     System.arraycopy(blocks, 0, newlist, 0, size_1);
-    blocks = newlist;
+    setBlocks(newlist);
   }
 
   /**
@@ -159,11 +159,9 @@ class INodeFileUnderConstruction extends INodeFile implements MutableBlockCollec
    */
   @Override
   public BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock,
-                                          DatanodeDescriptor[] targets)
-  throws IOException {
-    if (blocks == null || blocks.length == 0) {
-      throw new IOException("Trying to update non-existant block. " +
-          "File is empty.");
+      DatanodeDescriptor[] targets) throws IOException {
+    if (numBlocks() == 0) {
+      throw new IOException("Failed to set last block: File is empty.");
     }
     BlockInfoUnderConstruction ucBlock =
       lastBlock.convertToBlockUnderConstruction(
@@ -172,8 +170,4 @@ class INodeFileUnderConstruction extends INodeFile implements MutableBlockCollec
     setBlock(numBlocks()-1, ucBlock);
     return ucBlock;
   }
-  
-  private String blocksAsString() {
-    return Joiner.on(",").join(this.blocks);
-  }
 }

+ 1 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeSymlink.java

@@ -41,7 +41,7 @@ public class INodeSymlink extends INode {
   }
 
   @Override
-  public boolean isLink() {
+  public boolean isSymlink() {
     return true;
   }
   
@@ -73,9 +73,4 @@ public class INodeSymlink extends INode {
     summary[1]++; // Increment the file count
     return summary;
   }
-
-  @Override
-  public boolean isDirectory() {
-    return false;
-  }
 }

+ 6 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java

@@ -64,6 +64,7 @@ import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck.Result;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.tools.DFSck;
@@ -678,11 +679,11 @@ public class TestFsck {
       DFSTestUtil.waitReplication(fs, filePath, (short)1);
       
       // intentionally corrupt NN data structure
-      INodeFile node = 
-        (INodeFile)cluster.getNamesystem().dir.rootDir.getNode(fileName,
-                                                               true);
-      assertEquals(node.blocks.length, 1);
-      node.blocks[0].setNumBytes(-1L);  // set the block length to be negative
+      INodeFile node = (INodeFile)cluster.getNamesystem().dir.rootDir.getNode(
+          fileName, true);
+      final BlockInfo[] blocks = node.getBlocks(); 
+      assertEquals(blocks.length, 1);
+      blocks[0].setNumBytes(-1L);  // set the block length to be negative
       
       // run fsck and expect a failure with -1 as the error code
       String outStr = runFsck(conf, -1, true, fileName);