소스 검색

HADOOP-11818 Minor improvements for erasurecode classes. Contributed by Rakesh R

Kai Zheng 10 년 전
부모
커밋
971bd5e885

+ 2 - 0
hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt

@@ -40,3 +40,5 @@
 
     HADOOP-11645. Erasure Codec API covering the essential aspects for an erasure code
     ( Kai Zheng via vinayakumarb )
+  
+    HADOOP-11818. Minor improvements for erasurecode classes. (Rakesh R via Kai Zheng)

+ 6 - 6
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java

@@ -17,8 +17,8 @@
  */
 package org.apache.hadoop.io.erasurecode;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.w3c.dom.*;
@@ -36,7 +36,7 @@ import java.util.*;
  * A EC schema loading utility that loads predefined EC schemas from XML file
  */
 public class SchemaLoader {
-  private static final Log LOG = LogFactory.getLog(SchemaLoader.class.getName());
+  private static final Logger LOG = LoggerFactory.getLogger(SchemaLoader.class.getName());
 
   /**
    * Load predefined ec schemas from configuration file. This file is
@@ -63,7 +63,7 @@ public class SchemaLoader {
   private List<ECSchema> loadSchema(File schemaFile)
       throws ParserConfigurationException, IOException, SAXException {
 
-    LOG.info("Loading predefined EC schema file " + schemaFile);
+    LOG.info("Loading predefined EC schema file {}", schemaFile);
 
     // Read and parse the schema file.
     DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
@@ -87,7 +87,7 @@ public class SchemaLoader {
           ECSchema schema = loadSchema(element);
             schemas.add(schema);
         } else {
-          LOG.warn("Bad element in EC schema configuration file: " +
+          LOG.warn("Bad element in EC schema configuration file: {}",
               element.getTagName());
         }
       }
@@ -109,7 +109,7 @@ public class SchemaLoader {
       URL url = Thread.currentThread().getContextClassLoader()
           .getResource(schemaFilePath);
       if (url == null) {
-        LOG.warn(schemaFilePath + " not found on the classpath.");
+        LOG.warn("{} not found on the classpath.", schemaFilePath);
         schemaFile = null;
       } else if (! url.getProtocol().equalsIgnoreCase("file")) {
         throw new RuntimeException(

+ 18 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java

@@ -1,3 +1,20 @@
+/**
+ * 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.io.erasurecode.coder;
 
 import org.apache.hadoop.conf.Configuration;
@@ -11,7 +28,7 @@ import org.apache.hadoop.io.erasurecode.rawcoder.XORRawDecoder;
 /**
  * Reed-Solomon erasure decoder that decodes a block group.
  *
- * It implements {@link ErasureDecoder}.
+ * It implements {@link ErasureCoder}.
  */
 public class RSErasureDecoder extends AbstractErasureDecoder {
   private RawErasureDecoder rsRawDecoder;

+ 18 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java

@@ -1,3 +1,20 @@
+/**
+ * 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.io.erasurecode.coder;
 
 import org.apache.hadoop.fs.CommonConfigurationKeys;
@@ -9,7 +26,7 @@ import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
 /**
  * Reed-Solomon erasure encoder that encodes a block group.
  *
- * It implements {@link ErasureEncoder}.
+ * It implements {@link ErasureCoder}.
  */
 public class RSErasureEncoder extends AbstractErasureEncoder {
   private RawErasureEncoder rawEncoder;

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java

@@ -25,7 +25,7 @@ import org.apache.hadoop.io.erasurecode.rawcoder.XORRawDecoder;
 /**
  * Xor erasure decoder that decodes a block group.
  *
- * It implements {@link ErasureDecoder}.
+ * It implements {@link ErasureCoder}.
  */
 public class XORErasureDecoder extends AbstractErasureDecoder {
 

+ 17 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java

@@ -1,3 +1,20 @@
+/**
+ * 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.io.erasurecode.rawcoder.util;
 
 /**