Ver Fonte

HDFS-3052. Change INodeFile and INodeFileUnderConstruction to package private.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1348998 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze há 13 anos atrás
pai
commit
ca39e780c4

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

@@ -214,6 +214,9 @@ Branch-2 ( Unreleased changes )
     HDFS-1013. Miscellaneous improvements to HTML markup for web UIs
     (Eugene Koontz via todd)
 
+    HDFS-3052. Change INodeFile and INodeFileUnderConstruction to package
+    private.  (szetszwo)
+
   OPTIMIZATIONS
 
     HDFS-2982. Startup performance suffers when there are many edit log

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java

@@ -30,8 +30,9 @@ import org.apache.hadoop.hdfs.util.LightWeightGSet;
  * the block are stored.
  */
 @InterfaceAudience.Private
-public class BlockInfo extends Block implements
-    LightWeightGSet.LinkedElement {
+public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
+  public static final BlockInfo[] EMPTY_ARRAY = {}; 
+
   private BlockCollection bc;
 
   /** For implementing {@link LightWeightGSet.LinkedElement} interface */

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

@@ -277,7 +277,7 @@ public class FSDirectory implements Closeable {
           preferredBlockSize, modificationTime, clientName, 
           clientMachine, null);
     } else {
-      newNode = new INodeFile(permissions, 0, replication,
+      newNode = new INodeFile(permissions, BlockInfo.EMPTY_ARRAY, replication,
                               modificationTime, atime, preferredBlockSize);
     }
 

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

@@ -1605,7 +1605,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
 
     try {
-      INode myFile = dir.getFileINode(src);
+      INodeFile myFile = dir.getFileINode(src);
       recoverLeaseInternal(myFile, src, holder, clientMachine, false);
 
       try {
@@ -1681,22 +1681,20 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * @throws UnresolvedLinkException
    * @throws IOException
    */
-  public LocatedBlock prepareFileForWrite(String src, INode file,
+  LocatedBlock prepareFileForWrite(String src, INodeFile file,
       String leaseHolder, String clientMachine, DatanodeDescriptor clientNode,
-      boolean writeToEditLog)
-      throws UnresolvedLinkException, IOException {
-    INodeFile node = (INodeFile) file;
+      boolean writeToEditLog) throws IOException {
     INodeFileUnderConstruction cons = new INodeFileUnderConstruction(
-                                    node.getLocalNameBytes(),
-                                    node.getReplication(),
-                                    node.getModificationTime(),
-                                    node.getPreferredBlockSize(),
-                                    node.getBlocks(),
-                                    node.getPermissionStatus(),
+                                    file.getLocalNameBytes(),
+                                    file.getReplication(),
+                                    file.getModificationTime(),
+                                    file.getPreferredBlockSize(),
+                                    file.getBlocks(),
+                                    file.getPermissionStatus(),
                                     leaseHolder,
                                     clientMachine,
                                     clientNode);
-    dir.replaceNode(src, node, cons);
+    dir.replaceNode(src, file, cons);
     leaseManager.addLease(cons.getClientName(), src);
     
     LocatedBlock ret = blockManager.convertLastBlockToUnderConstruction(cons);

+ 2 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -25,13 +25,13 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 
 /** I-node for closed file. */
 @InterfaceAudience.Private
-public class INodeFile extends INode implements BlockCollection {
+class INodeFile extends INode implements BlockCollection {
   static final FsPermission UMASK = FsPermission.createImmutable((short)0111);
 
   //Number of bits for Block size
@@ -45,13 +45,6 @@ public class INodeFile extends INode implements BlockCollection {
 
   BlockInfo blocks[] = null;
 
-  INodeFile(PermissionStatus permissions,
-            int nrBlocks, short replication, long modificationTime,
-            long atime, long preferredBlockSize) {
-    this(permissions, new BlockInfo[nrBlocks], replication,
-        modificationTime, atime, preferredBlockSize);
-  }
-
   INodeFile(PermissionStatus permissions, BlockInfo[] blklist,
                       short replication, long modificationTime,
                       long atime, long preferredBlockSize) {

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

@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.IOException;
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
@@ -32,8 +33,8 @@ import com.google.common.base.Joiner;
 /**
  * I-node for file being written.
  */
-public class INodeFileUnderConstruction extends INodeFile 
-                                        implements MutableBlockCollection {
+@InterfaceAudience.Private
+class INodeFileUnderConstruction extends INodeFile implements MutableBlockCollection {
   private  String clientName;         // lease holder
   private final String clientMachine;
   private final DatanodeDescriptor clientNode; // if client is a cluster node too.
@@ -45,7 +46,7 @@ public class INodeFileUnderConstruction extends INodeFile
                              String clientName,
                              String clientMachine,
                              DatanodeDescriptor clientNode) {
-    super(permissions.applyUMask(UMASK), 0, replication,
+    super(permissions.applyUMask(UMASK), BlockInfo.EMPTY_ARRAY, replication,
         modTime, modTime, preferredBlockSize);
     this.clientName = clientName;
     this.clientMachine = clientMachine;

+ 10 - 15
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPersistBlocks.java

@@ -18,39 +18,34 @@
 
 package org.apache.hadoop.hdfs;
 
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Random;
+
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
-import org.apache.hadoop.hdfs.server.namenode.FSEditLog;
 import org.apache.hadoop.hdfs.server.namenode.FSImage;
-import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import org.apache.hadoop.hdfs.server.namenode.INodeFileUnderConstruction;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.log4j.Level;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.URI;
-import java.util.Collection;
-import java.util.List;
-import java.util.Random;
-import static org.junit.Assert.*;
 import org.junit.Test;
 
-import com.google.common.collect.Lists;
-
 /**
  * A JUnit test for checking if restarting DFS preserves the
  * blocks that are part of an unclosed file.

+ 6 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java

@@ -17,7 +17,9 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -29,14 +31,9 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.net.NetworkTopology;
 import org.junit.Before;
 import org.junit.Test;
@@ -381,11 +378,11 @@ public class TestBlockManager {
   }
   
   private BlockInfo addBlockOnNodes(long blockId, List<DatanodeDescriptor> nodes) {
-    INodeFile iNode = Mockito.mock(INodeFile.class);
-    Mockito.doReturn((short)3).when(iNode).getReplication();
+    BlockCollection bc = Mockito.mock(BlockCollection.class);
+    Mockito.doReturn((short)3).when(bc).getReplication();
     BlockInfo blockInfo = blockOnNodes(blockId, nodes);
 
-    bm.blocksMap.addBlockCollection(blockInfo, iNode);
+    bm.blocksMap.addBlockCollection(blockInfo, bc);
     return blockInfo;
   }