Переглянути джерело

HADOOP-12808. Rename the RS coder from HDFS-RAID as legacy. Contributed by Rui Li.

Change-Id: Icb64eb60833fe0139aadb6da9aa666f664defc0e
Zhe Zhang 9 роки тому
батько
коміт
efdc0070d8

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

@@ -653,6 +653,9 @@ Trunk (Unreleased)
       HADOOP-12041. Implement another Reed-Solomon coder in pure Java.
       (Kai Zheng via zhz)
 
+      HADOOP-12808. Rename the RS coder from HDFS-RAID as legacy.
+      (Rui Li via zhz)
+
 Release 2.9.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java

@@ -43,7 +43,7 @@ public final class CodecUtil {
         CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY,
         true, numDataUnits, numParityUnits);
     if (rawCoder == null) {
-      rawCoder = new RSRawEncoder(numDataUnits, numParityUnits);
+      rawCoder = new RSRawEncoderLegacy(numDataUnits, numParityUnits);
     }
 
     return (RawErasureEncoder) rawCoder;
@@ -62,7 +62,7 @@ public final class CodecUtil {
         CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY,
         false, numDataUnits, numParityUnits);
     if (rawCoder == null) {
-      rawCoder = new RSRawDecoder(numDataUnits, numParityUnits);
+      rawCoder = new RSRawDecoderLegacy(numDataUnits, numParityUnits);
     }
 
     return (RawErasureDecoder) rawCoder;

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java → hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoderLegacy.java

@@ -34,7 +34,7 @@ import java.nio.ByteBuffer;
  * addressed in HADOOP-11871.
  */
 @InterfaceAudience.Private
-public class RSRawDecoder extends AbstractRawErasureDecoder {
+public class RSRawDecoderLegacy extends AbstractRawErasureDecoder {
   // To describe and calculate the needed Vandermonde matrix
   private int[] errSignature;
   private int[] primitivePower;
@@ -61,7 +61,7 @@ public class RSRawDecoder extends AbstractRawErasureDecoder {
   private ByteBuffer[] adjustedDirectBufferOutputsParameter =
       new ByteBuffer[getNumParityUnits()];
 
-  public RSRawDecoder(int numDataUnits, int numParityUnits) {
+  public RSRawDecoderLegacy(int numDataUnits, int numParityUnits) {
     super(numDataUnits, numParityUnits);
     if (numDataUnits + numParityUnits >= RSUtil.GF.getFieldSize()) {
       throw new HadoopIllegalArgumentException(

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java → hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoderLegacy.java

@@ -29,10 +29,10 @@ import java.util.Arrays;
  * when possible.
  */
 @InterfaceAudience.Private
-public class RSRawEncoder extends AbstractRawErasureEncoder {
+public class RSRawEncoderLegacy extends AbstractRawErasureEncoder {
   private int[] generatingPolynomial;
 
-  public RSRawEncoder(int numDataUnits, int numParityUnits) {
+  public RSRawEncoderLegacy(int numDataUnits, int numParityUnits) {
     super(numDataUnits, numParityUnits);
 
     assert (getNumDataUnits() + getNumParityUnits() < RSUtil.GF.getFieldSize());

+ 4 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawErasureCoderFactory.java → hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawErasureCoderFactoryLegacy.java

@@ -20,18 +20,18 @@ package org.apache.hadoop.io.erasurecode.rawcoder;
 import org.apache.hadoop.classification.InterfaceAudience;
 
 /**
- * A raw coder factory for raw Reed-Solomon coder in Java.
+ * A raw coder factory for the legacy raw Reed-Solomon coder in Java.
  */
 @InterfaceAudience.Private
-public class RSRawErasureCoderFactory implements RawErasureCoderFactory {
+public class RSRawErasureCoderFactoryLegacy implements RawErasureCoderFactory {
 
   @Override
   public RawErasureEncoder createEncoder(int numDataUnits, int numParityUnits) {
-    return new RSRawEncoder(numDataUnits, numParityUnits);
+    return new RSRawEncoderLegacy(numDataUnits, numParityUnits);
   }
 
   @Override
   public RawErasureDecoder createDecoder(int numDataUnits, int numParityUnits) {
-    return new RSRawDecoder(numDataUnits, numParityUnits);
+    return new RSRawDecoderLegacy(numDataUnits, numParityUnits);
   }
 }

+ 27 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/package-info.java

@@ -0,0 +1,27 @@
+/**
+ * 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.
+ */
+
+/**
+ * General helpers for implementing raw erasure coders.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.io.erasurecode.rawcoder.util;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

+ 2 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestHHXORErasureCoder.java

@@ -19,7 +19,7 @@ package org.apache.hadoop.io.erasurecode.coder;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.io.erasurecode.rawcoder.RSRawErasureCoderFactory;
+import org.apache.hadoop.io.erasurecode.rawcoder.RSRawErasureCoderFactoryLegacy;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -51,7 +51,7 @@ public class TestHHXORErasureCoder extends TestHHErasureCoderBase {
      */
     Configuration conf = new Configuration();
     conf.set(CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY,
-        RSRawErasureCoderFactory.class.getCanonicalName());
+        RSRawErasureCoderFactoryLegacy.class.getCanonicalName());
     prepare(conf, 10, 4, new int[]{0}, new int[0]);
 
     testCoding(true);

+ 2 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java

@@ -19,7 +19,7 @@ package org.apache.hadoop.io.erasurecode.coder;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.io.erasurecode.rawcoder.RSRawErasureCoderFactory;
+import org.apache.hadoop.io.erasurecode.rawcoder.RSRawErasureCoderFactoryLegacy;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -58,7 +58,7 @@ public class TestRSErasureCoder extends TestErasureCoderBase {
      */
     Configuration conf = new Configuration();
     conf.set(CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY,
-        RSRawErasureCoderFactory.class.getCanonicalName());
+        RSRawErasureCoderFactoryLegacy.class.getCanonicalName());
     prepare(conf, 10, 4, new int[]{0}, new int[0]);
 
     testCoding(true);

+ 4 - 5
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java → hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoderLegacy.java

@@ -18,17 +18,16 @@
 package org.apache.hadoop.io.erasurecode.rawcoder;
 
 import org.junit.Before;
-import org.junit.Test;
 
 /**
- * Test raw Reed-solomon coder implemented in Java.
+ * Test the legacy raw Reed-solomon coder implemented in Java.
  */
-public class TestRSRawCoder extends TestRSRawCoderBase {
+public class TestRSRawCoderLegacy extends TestRSRawCoderBase {
 
   @Before
   public void setup() {
-    this.encoderClass = RSRawEncoder.class;
-    this.decoderClass = RSRawDecoder.class;
+    this.encoderClass = RSRawEncoderLegacy.class;
+    this.decoderClass = RSRawDecoderLegacy.class;
     setAllowDump(false); // Change to true to allow verbose dump for debugging
   }
 }