浏览代码

HADOOP-3965. Convert DataBlockScanner into a package private class. Contributed by Konstantin Shvachko.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@689230 13f79535-47bb-0310-9956-ffa450edef68
Konstantin Shvachko 17 年之前
父节点
当前提交
2b6db41dbc

+ 2 - 0
CHANGES.txt

@@ -224,6 +224,8 @@ Trunk (unreleased changes)
     HADOOP-2330. Preallocate HDFS transaction log to improve performance.
     (dhruba and hairong)
 
+    HADOOP-3965. Convert DataBlockScanner into a package private class. (shv)
+
   OPTIMIZATIONS
 
     HADOOP-3556. Removed lock contention in MD5Hash by changing the 

+ 3 - 3
src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataBlockScanner.java

@@ -57,7 +57,7 @@ import org.apache.hadoop.util.StringUtils;
  * Currently it does not modify the metadata for block.
  */
 
-public class DataBlockScanner implements Runnable {
+class DataBlockScanner implements Runnable {
   
   public static final Log LOG = LogFactory.getLog(DataBlockScanner.class);
   
@@ -122,7 +122,7 @@ public class DataBlockScanner implements Runnable {
              compareTo((BlockScanInfo)other) == 0;
     }
     
-    public long getLastScanTime() {
+    long getLastScanTime() {
       return ( lastScanType == ScanType.NONE) ? 0 : lastScanTime;
     }
     
@@ -283,7 +283,7 @@ public class DataBlockScanner implements Runnable {
   }
 
   /** @return the last scan time */
-  public synchronized long getLastScanTime(Block block) {
+  synchronized long getLastScanTime(Block block) {
     if (!isInitiliazed()) {
       return 0;
     }

+ 1 - 1
src/test/org/apache/hadoop/hdfs/MiniDFSCluster.java

@@ -514,7 +514,7 @@ public class MiniDFSCluster {
   }
   
   /** @return the datanode having the ipc server listen port */
-  DataNode getDataNode(int ipcPort) {
+  public DataNode getDataNode(int ipcPort) {
     for(DataNode dn : getDataNodes()) {
       if (dn.ipcServer.getListenerAddress().getPort() == ipcPort) {
         return dn;

+ 1 - 0
src/test/org/apache/hadoop/hdfs/TestLeaseRecovery.java

@@ -26,6 +26,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.TestInterDatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.BlockMetaDataInfo;
 import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
 

+ 2 - 1
src/test/org/apache/hadoop/hdfs/TestLeaseRecovery2.java

@@ -23,6 +23,7 @@ import java.util.*;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.datanode.TestInterDatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
 
 public class TestLeaseRecovery2 extends junit.framework.TestCase {
@@ -90,7 +91,7 @@ public class TestLeaseRecovery2 extends junit.framework.TestCase {
       dfs.dfs.clientName += "_1";
       while (true) {
         try {
-          FSDataOutputStream newstm = dfs.create(filepath, false,
+          dfs.create(filepath, false,
             dfs.getConf().getInt("io.file.buffer.size", 4096),
             (short)repl, (long)BLOCK_SIZE);
           assertTrue("Creation of an existing file should never succeed.", false);

+ 10 - 6
src/test/org/apache/hadoop/hdfs/TestInterDatanodeProtocol.java → src/test/org/apache/hadoop/hdfs/server/datanode/TestInterDatanodeProtocol.java

@@ -15,13 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs;
+package org.apache.hadoop.hdfs.server.datanode;
 
 import java.io.IOException;
 import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.*;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -36,7 +39,7 @@ import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
  * This tests InterDataNodeProtocol for block handling. 
  */
 public class TestInterDatanodeProtocol extends junit.framework.TestCase {
-  static void checkMetaInfo(Block b, InterDatanodeProtocol idp,
+  public static void checkMetaInfo(Block b, InterDatanodeProtocol idp,
       DataBlockScanner scanner) throws IOException {
     BlockMetaDataInfo metainfo = idp.getBlockMetaDataInfo(b);
     assertEquals(b.getBlockId(), metainfo.getBlockId());
@@ -47,8 +50,9 @@ public class TestInterDatanodeProtocol extends junit.framework.TestCase {
     }
   }
 
-  static LocatedBlock getLastLocatedBlock(ClientProtocol namenode, String src
-      ) throws IOException {
+  public static LocatedBlock getLastLocatedBlock(
+      ClientProtocol namenode, String src
+  ) throws IOException {
     //get block info for the last block
     LocatedBlocks locations = namenode.getBlockLocations(src, 0, Long.MAX_VALUE);
     List<LocatedBlock> blocks = locations.getLocatedBlocks();
@@ -76,10 +80,10 @@ public class TestInterDatanodeProtocol extends junit.framework.TestCase {
       String filestr = "/foo";
       Path filepath = new Path(filestr);
       DFSTestUtil.createFile(dfs, filepath, 1024L, (short)3, 0L);
-      assertTrue(dfs.dfs.exists(filestr));
+      assertTrue(dfs.getClient().exists(filestr));
 
       //get block info
-      LocatedBlock locatedblock = getLastLocatedBlock(dfs.dfs.namenode, filestr);
+      LocatedBlock locatedblock = getLastLocatedBlock(dfs.getClient().namenode, filestr);
       DatanodeInfo[] datanodeinfo = locatedblock.getLocations();
       assertTrue(datanodeinfo.length > 0);