Browse Source

HDFS-5383. fix broken caching unit tests (Andrew Wang)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-4949@1533253 13f79535-47bb-0310-9956-ffa450edef68
Andrew Wang 11 years ago
parent
commit
dcb0b85333

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-4949.txt

@@ -99,3 +99,5 @@ HDFS-4949 (Unreleased)
 
 
     HDFS-5373. hdfs cacheadmin -addDirective short usage does not mention
     HDFS-5373. hdfs cacheadmin -addDirective short usage does not mention
     -replication parameter. (cnauroth)
     -replication parameter. (cnauroth)
+
+    HDFS-5383. fix broken caching unit tests. (Andrew Wang)

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

@@ -662,7 +662,6 @@ public final class CacheManager {
    * @throws IOException
    * @throws IOException
    */
    */
   public void loadState(DataInput in) throws IOException {
   public void loadState(DataInput in) throws IOException {
-    assert namesystem.hasWriteLock();
     nextEntryId = in.readLong();
     nextEntryId = in.readLong();
     // pools need to be loaded first since entries point to their parent pool
     // pools need to be loaded first since entries point to their parent pool
     loadPools(in);
     loadPools(in);

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestClientNamenodeProtocolServerSideTranslatorPB.java

@@ -41,6 +41,7 @@ public class TestClientNamenodeProtocolServerSideTranslatorPB {
             setDirective(PathBasedCacheDirectiveProto.newBuilder().
             setDirective(PathBasedCacheDirectiveProto.newBuilder().
                 setPath("").
                 setPath("").
                 setPool("pool").
                 setPool("pool").
+                setReplication(1).
                 build()).
                 build()).
             build();
             build();
     ClientNamenodeProtocolServerSideTranslatorPB translator =
     ClientNamenodeProtocolServerSideTranslatorPB translator =

+ 6 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java

@@ -18,7 +18,8 @@
 package org.apache.hadoop.hdfs.server.datanode;
 package org.apache.hadoop.hdfs.server.datanode;
 
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
-import static org.mockito.Matchers.*;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyInt;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.doReturn;
 
 
 import java.io.FileInputStream;
 import java.io.FileInputStream;
@@ -41,7 +42,7 @@ import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.namenode.FSImage;
 import org.apache.hadoop.hdfs.server.namenode.FSImage;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
+import org.apache.hadoop.hdfs.server.protocol.BlockIdCommand;
 import org.apache.hadoop.hdfs.server.protocol.CacheReport;
 import org.apache.hadoop.hdfs.server.protocol.CacheReport;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
@@ -139,11 +140,11 @@ public class TestFsDatasetCache {
   private static DatanodeCommand getResponse(HdfsBlockLocation[] locs,
   private static DatanodeCommand getResponse(HdfsBlockLocation[] locs,
       int action) {
       int action) {
     String bpid = locs[0].getLocatedBlock().getBlock().getBlockPoolId();
     String bpid = locs[0].getLocatedBlock().getBlock().getBlockPoolId();
-    Block[] blocks = new Block[locs.length];
+    long[] blocks = new long[locs.length];
     for (int i=0; i<locs.length; i++) {
     for (int i=0; i<locs.length; i++) {
-      blocks[i] = locs[i].getLocatedBlock().getBlock().getLocalBlock();
+      blocks[i] = locs[i].getLocatedBlock().getBlock().getBlockId();
     }
     }
-    return new BlockCommand(action, bpid, blocks);
+    return new BlockIdCommand(action, bpid, blocks);
   }
   }
 
 
   private static long[] getBlockSizes(HdfsBlockLocation[] locs)
   private static long[] getBlockSizes(HdfsBlockLocation[] locs)