浏览代码

HDFS-11823. Extend TestDFSStripedIutputStream/TestDFSStripedOutputStream with a random EC policy. Contributed by Takanobu Asanuma.

Jing Zhao 8 年之前
父节点
当前提交
a02b908e6a

+ 12 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java

@@ -49,6 +49,7 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Random;
 import java.util.Set;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -568,4 +569,15 @@ public class StripedFileTestUtil {
   public static ErasureCodingPolicy getDefaultECPolicy() {
     return SystemErasureCodingPolicies.getPolicies().get(0);
   }
+
+  /**
+   * Get non-default Erasure Coding Policy randomly.
+   * @return ErasureCodingPolicy
+   */
+  public static ErasureCodingPolicy getRandomNonDefaultECPolicy() {
+    Random rand = new Random();
+    List<ErasureCodingPolicy> policies = SystemErasureCodingPolicies
+            .getPolicies();
+    return policies.get(1 + rand.nextInt(policies.size() - 1));
+  }
 }

+ 17 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRSDefault10x4StripedInputStream.java → hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStreamWithRandomECPolicy.java

@@ -17,18 +17,29 @@
  */
 package org.apache.hadoop.hdfs;
 
-import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 
 /**
- * This tests read operation of DFS striped file with RS-10-4-64k
- *  erasure code policy.
+ * This tests read operation of DFS striped file with a random erasure code
+ * policy except for the default policy.
  */
-public class TestDFSRSDefault10x4StripedInputStream extends
+public class TestDFSStripedInputStreamWithRandomECPolicy extends
     TestDFSStripedInputStream {
 
+  private static final Log LOG = LogFactory.getLog(
+      TestDFSStripedInputStreamWithRandomECPolicy.class.getName());
+
+  private ErasureCodingPolicy ecPolicy;
+
+  public TestDFSStripedInputStreamWithRandomECPolicy() {
+    ecPolicy = StripedFileTestUtil.getRandomNonDefaultECPolicy();
+    LOG.info(ecPolicy);
+  }
+
+  @Override
   public ErasureCodingPolicy getEcPolicy() {
-    return SystemErasureCodingPolicies.getByID(
-        SystemErasureCodingPolicies.RS_10_4_POLICY_ID);
+    return ecPolicy;
   }
 }

+ 17 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRSDefault10x4StripedOutputStreamWithFailure.java → hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailureWithRandomECPolicy.java

@@ -17,19 +17,29 @@
  */
 package org.apache.hadoop.hdfs;
 
-import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 
 /**
- * This tests write operation of DFS striped file with RS-10-4-64k
- *  erasure code policy under Datanode failure conditions.
+ * This tests write operation of DFS striped file with a random erasure code
+ * policy except for the default policy under Datanode failure conditions.
  */
-public class TestDFSRSDefault10x4StripedOutputStreamWithFailure
-    extends TestDFSStripedOutputStreamWithFailure {
+public class TestDFSStripedOutputStreamWithFailureWithRandomECPolicy extends
+    TestDFSStripedOutputStreamWithFailure {
+
+  private static final Log LOG = LogFactory.getLog(
+      TestDFSStripedOutputStreamWithRandomECPolicy.class.getName());
+
+  private ErasureCodingPolicy ecPolicy;
+
+  public TestDFSStripedOutputStreamWithFailureWithRandomECPolicy() {
+    ecPolicy = StripedFileTestUtil.getRandomNonDefaultECPolicy();
+    LOG.info(ecPolicy);
+  }
 
   @Override
   public ErasureCodingPolicy getEcPolicy() {
-    return SystemErasureCodingPolicies.getByID(
-        SystemErasureCodingPolicies.RS_10_4_POLICY_ID);
+    return ecPolicy;
   }
 }

+ 17 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRSDefault10x4StripedOutputStream.java → hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithRandomECPolicy.java

@@ -17,19 +17,29 @@
  */
 package org.apache.hadoop.hdfs;
 
-import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 
 /**
- * This tests write operation of DFS striped file with RS-10-4-64k
- *  erasure code policy.
+ * This tests write operation of DFS striped file with a random erasure code
+ * policy except for the default policy.
  */
-public class TestDFSRSDefault10x4StripedOutputStream
-    extends TestDFSStripedOutputStream {
+public class TestDFSStripedOutputStreamWithRandomECPolicy extends
+    TestDFSStripedOutputStream {
+
+  private static final Log LOG = LogFactory.getLog(
+      TestDFSStripedOutputStreamWithRandomECPolicy.class.getName());
+
+  private ErasureCodingPolicy ecPolicy;
+
+  public TestDFSStripedOutputStreamWithRandomECPolicy() {
+    ecPolicy = StripedFileTestUtil.getRandomNonDefaultECPolicy();
+    LOG.info(ecPolicy);
+  }
 
   @Override
   public ErasureCodingPolicy getEcPolicy() {
-    return SystemErasureCodingPolicies.getByID(
-        SystemErasureCodingPolicies.RS_10_4_POLICY_ID);
+    return ecPolicy;
   }
 }

+ 0 - 32
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSXORStripedInputStream.java

@@ -1,32 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs;
-import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
-
-/**
- * This tests read operation of DFS striped file with XOR-2-1-64k erasure code
- * policy.
- */
-public class TestDFSXORStripedInputStream extends TestDFSStripedInputStream{
-
-  public ErasureCodingPolicy getEcPolicy() {
-    return SystemErasureCodingPolicies.getByID(
-        SystemErasureCodingPolicies.XOR_2_1_POLICY_ID);
-  }
-}

+ 0 - 34
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSXORStripedOutputStream.java

@@ -1,34 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs;
-
-import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
-
-/**
- * This tests write operation of DFS striped file with XOR-2-1-64k erasure code
- * policy.
- */
-public class TestDFSXORStripedOutputStream extends TestDFSStripedOutputStream{
-
-  @Override
-  public ErasureCodingPolicy getEcPolicy() {
-    return SystemErasureCodingPolicies.getByID(
-        SystemErasureCodingPolicies.XOR_2_1_POLICY_ID);
-  }
-}

+ 0 - 35
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSXORStripedOutputStreamWithFailure.java

@@ -1,35 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs;
-
-import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
-import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
-
-/**
- * This tests write operation of DFS striped file with XOR-2-1-64k erasure code
- * policy when there is data node failure.
- */
-public class TestDFSXORStripedOutputStreamWithFailure
-    extends TestDFSStripedOutputStreamWithFailure{
-
-  @Override
-  public ErasureCodingPolicy getEcPolicy() {
-    return SystemErasureCodingPolicies.getByID(
-        SystemErasureCodingPolicies.XOR_2_1_POLICY_ID);
-  }
-}