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 we...@apache.org on 2019/10/04 00:23:00 UTC

[hadoop] branch branch-3.1 updated: HDFS-14113. EC : Add Configuration to restrict UserDefined Policies. Contributed by Ayush Saxena.

This is an automated email from the ASF dual-hosted git repository.

weichiu pushed a commit to branch branch-3.1
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/branch-3.1 by this push:
     new c417916  HDFS-14113. EC : Add Configuration to restrict UserDefined Policies. Contributed by Ayush Saxena.
c417916 is described below

commit c41791671c1dd31502c00967da958f2b1037e674
Author: Vinayakumar B <vi...@apache.org>
AuthorDate: Thu Dec 6 18:20:28 2018 +0530

    HDFS-14113. EC : Add Configuration to restrict UserDefined Policies. Contributed by Ayush Saxena.
    
    (cherry picked from commit c03024a5305bea1a40c87a4abc0793802bea5c75)
---
 .../java/org/apache/hadoop/hdfs/DFSConfigKeys.java |  4 ++
 .../namenode/ErasureCodingPolicyManager.java       | 13 ++++
 .../src/main/resources/hdfs-default.xml            |  8 +++
 .../hadoop/hdfs/TestErasureCodingAddConfig.java    | 79 ++++++++++++++++++++++
 4 files changed, 104 insertions(+)

diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index 2f10be5..d6ed729 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -649,6 +649,10 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
 
   public static final String  DFS_NAMENODE_EC_POLICIES_MAX_CELLSIZE_KEY = "dfs.namenode.ec.policies.max.cellsize";
   public static final int     DFS_NAMENODE_EC_POLICIES_MAX_CELLSIZE_DEFAULT = 4 * 1024 * 1024;
+  public static final String DFS_NAMENODE_EC_POLICIES_USERPOLICIES_ALLOWED_KEY =
+      "dfs.namenode.ec.userdefined.policy.allowed";
+  public static final boolean
+      DFS_NAMENODE_EC_POLICIES_USERPOLICIES_ALLOWED_KEY_DEFAULT = true;
   public static final String  DFS_NAMENODE_EC_SYSTEM_DEFAULT_POLICY =
       "dfs.namenode.ec.system.default.policy";
   public static final String  DFS_NAMENODE_EC_SYSTEM_DEFAULT_POLICY_DEFAULT =
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 53732b0..a484337 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
@@ -56,6 +56,9 @@ public final class ErasureCodingPolicyManager {
   private int maxCellSize =
       DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_MAX_CELLSIZE_DEFAULT;
 
+  private boolean userDefinedAllowed =
+      DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_USERPOLICIES_ALLOWED_KEY_DEFAULT;
+
   // Supported storage policies for striped EC files
   private static final byte[] SUITABLE_STORAGE_POLICIES_FOR_EC_STRIPED_MODE =
       new byte[]{
@@ -141,6 +144,11 @@ public final class ErasureCodingPolicyManager {
     maxCellSize = conf.getInt(
         DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_MAX_CELLSIZE_KEY,
         DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_MAX_CELLSIZE_DEFAULT);
+
+    userDefinedAllowed = conf.getBoolean(
+        DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_USERPOLICIES_ALLOWED_KEY,
+        DFSConfigKeys.
+            DFS_NAMENODE_EC_POLICIES_USERPOLICIES_ALLOWED_KEY_DEFAULT);
   }
 
   /**
@@ -276,6 +284,11 @@ public final class ErasureCodingPolicyManager {
    */
   public synchronized ErasureCodingPolicy addPolicy(
       ErasureCodingPolicy policy) {
+    if (!userDefinedAllowed) {
+      throw new HadoopIllegalArgumentException(
+          "Addition of user defined erasure coding policy is disabled.");
+    }
+
     if (!CodecUtil.hasCodec(policy.getCodecName())) {
       throw new HadoopIllegalArgumentException("Codec name "
           + policy.getCodecName() + " is not supported");
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index e6415cf..97aaec2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -3184,6 +3184,14 @@
 </property>
 
 <property>
+  <name>dfs.namenode.ec.userdefined.policy.allowed</name>
+  <value>true</value>
+  <description>If set to false, doesn't allow addition of user defined
+    erasure coding policies.
+  </description>
+</property>
+
+<property>
   <name>dfs.datanode.ec.reconstruction.stripedread.timeout.millis</name>
   <value>5000</value>
   <description>Datanode striped read timeout in milliseconds.
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingAddConfig.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingAddConfig.java
new file mode 100644
index 0000000..24c88bd
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingAddConfig.java
@@ -0,0 +1,79 @@
+/**
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.junit.Test;
+
+/**
+ * Test that ensures addition of user defined EC policies is allowed only when
+ * dfs.namenode.ec.userdefined.policy.allowed is set to true.
+ */
+public class TestErasureCodingAddConfig {
+
+  @Test
+  public void testECAddPolicyConfigDisable() throws IOException {
+    Configuration conf = new HdfsConfiguration();
+    conf.setBoolean(
+        DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_USERPOLICIES_ALLOWED_KEY,
+        false);
+    try (MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(0).build()) {
+      cluster.waitActive();
+      DistributedFileSystem fs = cluster.getFileSystem();
+
+      ErasureCodingPolicy newPolicy1 =
+          new ErasureCodingPolicy(new ECSchema("rs", 5, 3), 1024 * 1024);
+
+      AddErasureCodingPolicyResponse[] response =
+          fs.addErasureCodingPolicies(new ErasureCodingPolicy[] {newPolicy1});
+
+      assertFalse(response[0].isSucceed());
+      assertEquals(
+          "Addition of user defined erasure coding policy is disabled.",
+          response[0].getErrorMsg());
+    }
+  }
+
+  @Test
+  public void testECAddPolicyConfigEnable() throws IOException {
+    Configuration conf = new HdfsConfiguration();
+    conf.setBoolean(
+        DFSConfigKeys.DFS_NAMENODE_EC_POLICIES_USERPOLICIES_ALLOWED_KEY, true);
+    try (MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(0).build()) {
+      DistributedFileSystem fs = cluster.getFileSystem();
+      ErasureCodingPolicy newPolicy1 =
+          new ErasureCodingPolicy(new ECSchema("rs", 5, 3), 1024 * 1024);
+      AddErasureCodingPolicyResponse[] response =
+          fs.addErasureCodingPolicies(new ErasureCodingPolicy[] {newPolicy1});
+      assertTrue(response[0].isSucceed());
+      assertNull(response[0].getErrorMsg());
+    }
+  }
+}
\ No newline at end of file


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