소스 검색

svn merge -c 1505362 Merging from trunk to branch-2.1-beta to fix HDFS-4602.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2.1-beta@1506567 13f79535-47bb-0310-9956-ffa450edef68
Kihwal Lee 12 년 전
부모
커밋
b63d675957

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

@@ -472,6 +472,8 @@ Release 2.1.0-beta - 2013-07-02
     HDFS-5018. Misspelled DFSConfigKeys#DFS_NAMENODE_STALE_DATANODE_INTERVAL_DEFAULT
     in javadoc of DatanodeInfo#isStale(). (Ted Yu via jing9)
 
+    HDFS-4602. TestBookKeeperHACheckpoints fails. (umamahesh)
+
   BREAKDOWN OF HDFS-347 SUBTASKS AND RELATED JIRAS
 
     HDFS-4353. Encapsulate connections to peers in Peer and PeerServer classes.

+ 13 - 9
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperHACheckpoints.java

@@ -17,19 +17,18 @@
  */
 package org.apache.hadoop.contrib.bkjournal;
 
-import org.apache.hadoop.hdfs.server.namenode.ha.TestStandbyCheckpoints;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
-
+import org.apache.hadoop.hdfs.server.namenode.ha.TestStandbyCheckpoints;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.junit.AfterClass;
 import org.junit.Before;
-import org.junit.After;
-
 import org.junit.BeforeClass;
-import org.junit.AfterClass;
+
+import com.google.common.collect.ImmutableList;
 
 /**
  * Runs the same tests as TestStandbyCheckpoints, but
@@ -40,6 +39,7 @@ public class TestBookKeeperHACheckpoints extends TestStandbyCheckpoints {
   static int numBookies = 3;
   static int journalCount = 0;
 
+  @SuppressWarnings("rawtypes")
   @Override
   @Before
   public void setupCluster() throws Exception {
@@ -51,7 +51,11 @@ public class TestBookKeeperHACheckpoints extends TestStandbyCheckpoints {
              BKJMUtil.createJournalURI("/checkpointing" + journalCount++)
              .toString());
     BKJMUtil.addJournalManagerDefinition(conf);
-
+    conf.setBoolean(DFSConfigKeys.DFS_IMAGE_COMPRESS_KEY, true);
+    conf.set(DFSConfigKeys.DFS_IMAGE_COMPRESSION_CODEC_KEY, SlowCodec.class
+        .getCanonicalName());
+    CompressionCodecFactory.setCodecClasses(conf, ImmutableList
+        .<Class> of(SlowCodec.class));
     MiniDFSNNTopology topology = new MiniDFSNNTopology()
       .addNameservice(new MiniDFSNNTopology.NSConf("ns1")
         .addNN(new MiniDFSNNTopology.NNConf("nn1").setHttpPort(10001))