You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ae...@apache.org on 2017/09/07 21:36:42 UTC

[29/37] hadoop git commit: HDFS-12402. Refactor ErasureCodingPolicyManager and related codes. Contributed by Sammi Chen

HDFS-12402. Refactor ErasureCodingPolicyManager and related codes. Contributed by Sammi Chen


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/2adf8bed
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2adf8bed
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2adf8bed

Branch: refs/heads/HDFS-7240
Commit: 2adf8bed712e6d770a0d53eea198d8911ae1a258
Parents: 6f101e7
Author: Kai Zheng <ka...@intel.com>
Authored: Thu Sep 7 20:38:23 2017 +0800
Committer: Kai Zheng <ka...@intel.com>
Committed: Thu Sep 7 20:38:23 2017 +0800

----------------------------------------------------------------------
 .../hdfs/protocol/AddECPolicyResponse.java      |  4 +-
 .../hdfs/protocol/IllegalECPolicyException.java | 34 -----------------
 .../namenode/ErasureCodingPolicyManager.java    | 39 ++++++++++----------
 .../server/namenode/FSDirErasureCodingOp.java   |  3 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  3 +-
 .../hadoop/hdfs/TestDistributedFileSystem.java  |  4 +-
 6 files changed, 27 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2adf8bed/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddECPolicyResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddECPolicyResponse.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddECPolicyResponse.java
index e7a8435..4e55680 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddECPolicyResponse.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddECPolicyResponse.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs.protocol;
 
+import org.apache.hadoop.HadoopIllegalArgumentException;
+
 /**
  * A response of add an ErasureCoding policy.
  */
@@ -38,7 +40,7 @@ public class AddECPolicyResponse {
   }
 
   public AddECPolicyResponse(ErasureCodingPolicy policy,
-      IllegalECPolicyException e) {
+      HadoopIllegalArgumentException e) {
     this(policy, e.getMessage());
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2adf8bed/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/IllegalECPolicyException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/IllegalECPolicyException.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/IllegalECPolicyException.java
deleted file mode 100644
index 03ce2a5..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/IllegalECPolicyException.java
+++ /dev/null
@@ -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.protocol;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * An Exception indicates the error when adding an ErasureCoding policy.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class IllegalECPolicyException extends Exception {
-  static final long serialVersionUID = 1L;
-
-  public IllegalECPolicyException(String msg) {
-    super(msg);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2adf8bed/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
index 74b5ebf..4c75709 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java
@@ -19,11 +19,11 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.lang.ArrayUtils;
+import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyState;
-import org.apache.hadoop.hdfs.protocol.IllegalECPolicyException;
 import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -144,7 +144,7 @@ public final class ErasureCodingPolicyManager {
             policyName,
             DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_ENABLED_KEY,
             names);
-        throw new IllegalArgumentException(msg);
+        throw new HadoopIllegalArgumentException(msg);
       }
       enabledPoliciesByName.put(ecPolicy.getName(), ecPolicy);
     }
@@ -230,33 +230,34 @@ public final class ErasureCodingPolicyManager {
    * Add an erasure coding policy.
    * @return the added policy
    */
-  public synchronized ErasureCodingPolicy addPolicy(ErasureCodingPolicy policy)
-      throws IllegalECPolicyException {
+  public synchronized ErasureCodingPolicy addPolicy(
+      ErasureCodingPolicy policy) {
     // Set policy state into DISABLED when adding into Hadoop.
     policy.setState(ErasureCodingPolicyState.DISABLED);
 
     if (!CodecUtil.hasCodec(policy.getCodecName())) {
-      throw new IllegalECPolicyException("Codec name "
+      throw new HadoopIllegalArgumentException("Codec name "
           + policy.getCodecName() + " is not supported");
     }
 
     if (policy.getCellSize() > maxCellSize) {
-      throw new IllegalECPolicyException("Cell size " + policy.getCellSize()
-          + " should not exceed maximum " + maxCellSize + " byte");
+      throw new HadoopIllegalArgumentException("Cell size " +
+          policy.getCellSize() + " should not exceed maximum " +
+          maxCellSize + " bytes");
     }
 
     String assignedNewName = ErasureCodingPolicy.composePolicyName(
         policy.getSchema(), policy.getCellSize());
     for (ErasureCodingPolicy p : getPolicies()) {
       if (p.getName().equals(assignedNewName)) {
-        throw new IllegalECPolicyException("The policy name " + assignedNewName
-            + " already exists");
+        throw new HadoopIllegalArgumentException("The policy name " +
+            assignedNewName + " already exists");
       }
       if (p.getSchema().equals(policy.getSchema()) &&
           p.getCellSize() == policy.getCellSize()) {
-        throw new IllegalECPolicyException("A policy with same schema "
+        throw new HadoopIllegalArgumentException("A policy with same schema "
             + policy.getSchema().toString() + " and cell size "
-            + p.getCellSize() + " is already exists");
+            + p.getCellSize() + " already exists");
       }
     }
     policy.setName(assignedNewName);
@@ -281,12 +282,12 @@ public final class ErasureCodingPolicyManager {
   public synchronized void removePolicy(String name) {
     ErasureCodingPolicy ecPolicy = policiesByName.get(name);
     if (ecPolicy == null) {
-      throw new IllegalArgumentException("The policy name " +
-          name + " does not exists");
+      throw new HadoopIllegalArgumentException("The policy name " +
+          name + " does not exist");
     }
 
     if (ecPolicy.isSystemPolicy()) {
-      throw new IllegalArgumentException("System erasure coding policy " +
+      throw new HadoopIllegalArgumentException("System erasure coding policy " +
           name + " cannot be removed");
     }
 
@@ -317,8 +318,8 @@ public final class ErasureCodingPolicyManager {
   public synchronized void disablePolicy(String name) {
     ErasureCodingPolicy ecPolicy = policiesByName.get(name);
     if (ecPolicy == null) {
-      throw new IllegalArgumentException("The policy name " +
-          name + " does not exists");
+      throw new HadoopIllegalArgumentException("The policy name " +
+          name + " does not exist");
     }
 
     if (enabledPoliciesByName.containsKey(name)) {
@@ -336,8 +337,8 @@ public final class ErasureCodingPolicyManager {
   public synchronized void enablePolicy(String name) {
     ErasureCodingPolicy ecPolicy = policiesByName.get(name);
     if (ecPolicy == null) {
-      throw new IllegalArgumentException("The policy name " +
-          name + " does not exists");
+      throw new HadoopIllegalArgumentException("The policy name " +
+          name + " does not exist");
     }
 
     enabledPoliciesByName.put(name, ecPolicy);
@@ -346,4 +347,4 @@ public final class ErasureCodingPolicyManager {
         enabledPoliciesByName.values().toArray(new ErasureCodingPolicy[0]);
     LOG.info("Enable the erasure coding policy " + name);
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2adf8bed/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
index 4f4befe..deb03af 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java
@@ -27,7 +27,6 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
-import org.apache.hadoop.hdfs.protocol.IllegalECPolicyException;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.WritableUtils;
@@ -212,7 +211,7 @@ final class FSDirErasureCodingOp {
   }
 
   static ErasureCodingPolicy addErasureCodePolicy(final FSNamesystem fsn,
-      ErasureCodingPolicy policy) throws IllegalECPolicyException {
+      ErasureCodingPolicy policy) {
     Preconditions.checkNotNull(policy);
     return fsn.getErasureCodingPolicyManager().addPolicy(policy);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2adf8bed/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index e5604c4..c30999b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -201,7 +201,6 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.ReencryptAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.protocol.IllegalECPolicyException;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -7207,7 +7206,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
               FSDirErasureCodingOp.addErasureCodePolicy(this, policy);
           addECPolicyName = newPolicy.getName();
           responses.add(new AddECPolicyResponse(newPolicy));
-        } catch (IllegalECPolicyException e) {
+        } catch (HadoopIllegalArgumentException e) {
           responses.add(new AddECPolicyResponse(policy, e));
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2adf8bed/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
index bf2b002..8e54e5f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
@@ -1617,7 +1617,7 @@ public class TestDistributedFileSystem {
         fs.enableErasureCodingPolicy("notExistECName");
         Assert.fail("enable the policy that doesn't exist should fail");
       } catch (Exception e) {
-        GenericTestUtils.assertExceptionContains("does not exists", e);
+        GenericTestUtils.assertExceptionContains("does not exist", e);
         // pass
       }
 
@@ -1626,7 +1626,7 @@ public class TestDistributedFileSystem {
         fs.disableErasureCodingPolicy("notExistECName");
         Assert.fail("disable the policy that doesn't exist should fail");
       } catch (Exception e) {
-        GenericTestUtils.assertExceptionContains("does not exists", e);
+        GenericTestUtils.assertExceptionContains("does not exist", e);
         // pass
       }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org