Selaa lähdekoodia

HDFS-12525. Ozone: OzoneClient: Verify bucket/volume name in create calls. Contributed by Nandakumar.

Yiqun Lin 7 vuotta sitten
vanhempi
commit
6b10723ec3

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java

@@ -50,6 +50,7 @@ public class ObjectStore {
    */
   public void createVolume(String volumeName) throws IOException {
     Preconditions.checkNotNull(volumeName);
+    OzoneClientUtils.verifyResourceName(volumeName);
     proxy.createVolume(volumeName);
   }
 
@@ -63,6 +64,7 @@ public class ObjectStore {
       throws IOException {
     Preconditions.checkNotNull(volumeName);
     Preconditions.checkNotNull(volumeArgs);
+    OzoneClientUtils.verifyResourceName(volumeName);
     proxy.createVolume(volumeName, volumeArgs);
   }
 
@@ -74,6 +76,7 @@ public class ObjectStore {
    */
   public OzoneVolume getVolume(String volumeName) throws IOException {
     Preconditions.checkNotNull(volumeName);
+    OzoneClientUtils.verifyResourceName(volumeName);
     OzoneVolume volume = proxy.getVolumeDetails(volumeName);
     volume.setClientProxy(proxy);
     return volume;
@@ -86,6 +89,7 @@ public class ObjectStore {
    */
   public void deleteVolume(String volumeName) throws IOException {
     Preconditions.checkNotNull(volumeName);
+    OzoneClientUtils.verifyResourceName(volumeName);
     proxy.deleteVolume(volumeName);
   }
 }

+ 80 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientUtils.java

@@ -28,6 +28,7 @@ import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConfiguration;
+import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.scm.ScmConfigKeys;
 import org.apache.http.client.config.RequestConfig;
 import org.apache.http.client.methods.HttpRequestBase;
@@ -731,4 +732,83 @@ public final class OzoneClientUtils {
         .build();
     return client;
   }
+
+  /**
+   * verifies that bucket name / volume name is a valid DNS name.
+   *
+   * @param resName Bucket or volume Name to be validated
+   *
+   * @throws IllegalArgumentException
+   */
+  public static void verifyResourceName(String resName)
+      throws IllegalArgumentException {
+
+    if (resName == null) {
+      throw new IllegalArgumentException("Bucket or Volume name is null");
+    }
+
+    if ((resName.length() < OzoneConsts.OZONE_MIN_BUCKET_NAME_LENGTH) ||
+        (resName.length() > OzoneConsts.OZONE_MAX_BUCKET_NAME_LENGTH)) {
+      throw new IllegalArgumentException(
+          "Bucket or Volume length is illegal, " +
+              "valid length is 3-63 characters");
+    }
+
+    if ((resName.charAt(0) == '.') || (resName.charAt(0) == '-')) {
+      throw new IllegalArgumentException(
+          "Bucket or Volume name cannot start with a period or dash");
+    }
+
+    if ((resName.charAt(resName.length() - 1) == '.') ||
+        (resName.charAt(resName.length() - 1) == '-')) {
+      throw new IllegalArgumentException(
+          "Bucket or Volume name cannot end with a period or dash");
+    }
+
+    boolean isIPv4 = true;
+    char prev = (char) 0;
+
+    for (int index = 0; index < resName.length(); index++) {
+      char currChar = resName.charAt(index);
+
+      if (currChar != '.') {
+        isIPv4 = ((currChar >= '0') && (currChar <= '9')) && isIPv4;
+      }
+
+      if (currChar > 'A' && currChar < 'Z') {
+        throw new IllegalArgumentException(
+            "Bucket or Volume name does not support uppercase characters");
+      }
+
+      if ((currChar != '.') && (currChar != '-')) {
+        if ((currChar < '0') || (currChar > '9' && currChar < 'a') ||
+            (currChar > 'z')) {
+          throw new IllegalArgumentException("Bucket or Volume name has an " +
+              "unsupported character : " +
+              currChar);
+        }
+      }
+
+      if ((prev == '.') && (currChar == '.')) {
+        throw new IllegalArgumentException("Bucket or Volume name should not " +
+            "have two contiguous periods");
+      }
+
+      if ((prev == '-') && (currChar == '.')) {
+        throw new IllegalArgumentException(
+            "Bucket or Volume name should not have period after dash");
+      }
+
+      if ((prev == '.') && (currChar == '-')) {
+        throw new IllegalArgumentException(
+            "Bucket or Volume name should not have dash after period");
+      }
+      prev = currChar;
+    }
+
+    if (isIPv4) {
+      throw new IllegalArgumentException(
+          "Bucket or Volume name cannot be an IPv4 address or all numeric");
+    }
+  }
 }

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/ozone/client/OzoneVolume.java

@@ -148,6 +148,7 @@ public class OzoneVolume {
       throws IOException {
     Preconditions.checkNotNull(proxy, "Client proxy is not set.");
     Preconditions.checkNotNull(bucketName);
+    OzoneClientUtils.verifyResourceName(bucketName);
     proxy.createBucket(name, bucketName);
   }
 
@@ -162,6 +163,7 @@ public class OzoneVolume {
     Preconditions.checkNotNull(proxy, "Client proxy is not set.");
     Preconditions.checkNotNull(bucketName);
     Preconditions.checkNotNull(bucketArgs);
+    OzoneClientUtils.verifyResourceName(bucketName);
     proxy.createBucket(name, bucketName, bucketArgs);
   }
 
@@ -174,6 +176,7 @@ public class OzoneVolume {
   public OzoneBucket getBucket(String bucketName) throws IOException {
     Preconditions.checkNotNull(proxy, "Client proxy is not set.");
     Preconditions.checkNotNull(bucketName);
+    OzoneClientUtils.verifyResourceName(bucketName);
     OzoneBucket bucket = proxy.getBucketDetails(name, bucketName);
     bucket.setClientProxy(proxy);
     return bucket;
@@ -187,6 +190,7 @@ public class OzoneVolume {
   public void deleteBucket(String bucketName) throws IOException {
     Preconditions.checkNotNull(proxy, "Client proxy is not set.");
     Preconditions.checkNotNull(bucketName);
+    OzoneClientUtils.verifyResourceName(bucketName);
     proxy.deleteBucket(name, bucketName);
   }
 }

+ 2 - 69
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/utils/OzoneUtils.java

@@ -24,6 +24,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.client.OzoneClientUtils;
 import org.apache.hadoop.ozone.client.io.LengthInputStream;
 import org.apache.hadoop.ozone.web.exceptions.ErrorTable;
 import org.apache.hadoop.ozone.web.exceptions.OzoneException;
@@ -86,75 +87,7 @@ public final class OzoneUtils {
    */
   public static void verifyResourceName(String resName)
       throws IllegalArgumentException {
-
-    if (resName == null) {
-      throw new IllegalArgumentException("Bucket or Volume name is null");
-    }
-
-    if ((resName.length() < OzoneConsts.OZONE_MIN_BUCKET_NAME_LENGTH) ||
-        (resName.length() > OzoneConsts.OZONE_MAX_BUCKET_NAME_LENGTH)) {
-      throw new IllegalArgumentException(
-          "Bucket or Volume length is illegal, " +
-              "valid length is 3-63 characters");
-    }
-
-    if ((resName.charAt(0) == '.') || (resName.charAt(0) == '-')) {
-      throw new IllegalArgumentException(
-          "Bucket or Volume name cannot start with a period or dash");
-    }
-
-    if ((resName.charAt(resName.length() - 1) == '.') ||
-        (resName.charAt(resName.length() - 1) == '-')) {
-      throw new IllegalArgumentException(
-          "Bucket or Volume name cannot end with a period or dash");
-    }
-
-    boolean isIPv4 = true;
-    char prev = (char) 0;
-
-    for (int index = 0; index < resName.length(); index++) {
-      char currChar = resName.charAt(index);
-
-      if (currChar != '.') {
-        isIPv4 = ((currChar >= '0') && (currChar <= '9')) && isIPv4;
-      }
-
-      if (currChar > 'A' && currChar < 'Z') {
-        throw new IllegalArgumentException(
-            "Bucket or Volume name does not support uppercase characters");
-      }
-
-
-      if ((currChar != '.') && (currChar != '-')) {
-        if ((currChar < '0') || (currChar > '9' && currChar < 'a') ||
-            (currChar > 'z')) {
-          throw new IllegalArgumentException("Bucket or Volume name has an " +
-                                                 "unsupported character : " +
-                                                 currChar);
-        }
-      }
-
-      if ((prev == '.') && (currChar == '.')) {
-        throw new IllegalArgumentException("Bucket or Volume name should not " +
-                                               "have two contiguous periods");
-      }
-
-      if ((prev == '-') && (currChar == '.')) {
-        throw new IllegalArgumentException(
-            "Bucket or Volume name should not have period after dash");
-      }
-
-      if ((prev == '.') && (currChar == '-')) {
-        throw new IllegalArgumentException(
-            "Bucket or Volume name should not have dash after period");
-      }
-      prev = currChar;
-    }
-
-    if (isIPv4) {
-      throw new IllegalArgumentException(
-          "Bucket or Volume name cannot be an IPv4 address or all numeric");
-    }
+    OzoneClientUtils.verifyResourceName(resName);
   }
 
   /**

+ 19 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClient.java

@@ -116,6 +116,14 @@ public class TestOzoneRpcClient {
     Assert.assertEquals(1000000000L, volume.getQuota());
   }
 
+  @Test
+  public void testInvalidVolumeCreation() throws IOException {
+    thrown.expectMessage("Bucket or Volume name has an unsupported" +
+        " character : #");
+    String volumeName = "invalid#name";
+    store.createVolume(volumeName);
+  }
+
   @Test
   public void testVolumeAlreadyExist()
       throws IOException, OzoneException {
@@ -247,6 +255,17 @@ public class TestOzoneRpcClient {
     Assert.assertTrue(bucket.getAcls().contains(userAcl));
   }
 
+  @Test
+  public void testInvalidBucketCreation() throws IOException {
+    thrown.expectMessage("Bucket or Volume name has an unsupported" +
+        " character : #");
+    String volumeName = UUID.randomUUID().toString();
+    String bucketName = "invalid#bucket";
+    store.createVolume(volumeName);
+    OzoneVolume volume = store.getVolume(volumeName);
+    volume.createBucket(bucketName);
+  }
+
   @Test
   public void testAddBucketAcl()
       throws IOException, OzoneException {