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 ar...@apache.org on 2016/03/11 23:22:46 UTC

[28/50] [abbrv] hadoop git commit: HDFS-8614. OzoneHandler : Add Quota Support. (Contributed by Anu Engineer)

HDFS-8614. OzoneHandler : Add Quota Support. (Contributed by Anu Engineer)


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

Branch: refs/heads/HDFS-7240
Commit: 15a3e758540892f3cc4d720eae36fd9524537381
Parents: e2f494c
Author: Arpit Agarwal <ar...@apache.org>
Authored: Wed Jun 17 08:50:19 2015 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Fri Mar 11 12:56:44 2016 -0800

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-7240.txt           |   3 +
 .../apache/hadoop/ozone/web/headers/Header.java |  33 ++++
 .../hadoop/ozone/web/request/OzoneQuota.java    | 180 +++++++++++++++++++
 .../org/apache/hadoop/ozone/web/TestQuota.java  | 115 ++++++++++++
 4 files changed, 331 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/15a3e758/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-7240.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-7240.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-7240.txt
index 71a1ec8..a259170 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-7240.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-7240.txt
@@ -7,3 +7,6 @@
     HDFS-8456. Ozone: Introduce STORAGE_CONTAINER_SERVICE as a new NodeType.
     (Arpit Agarwal)
 
+    HDFS-8641. OzoneHandler : Add Quota Support. (Anu Engineer via
+    Arpit Agarwal)
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/15a3e758/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/headers/Header.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/headers/Header.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/headers/Header.java
new file mode 100644
index 0000000..ac33208
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/headers/Header.java
@@ -0,0 +1,33 @@
+/*
+ * 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.ozone.web.headers;
+
+/**
+ * OZONE specific HTTP headers.
+ */
+public final class Header {
+  public static final String OZONE_QUOTA_BYTES = "BYTES";
+  public static final String OZONE_QUOTA_MB = "MB";
+  public static final String OZONE_QUOTA_GB = "GB";
+  public static final String OZONE_QUOTA_TB = "TB";
+  public static final String OZONE_QUOTA_REMOVE = "remove";
+  private Header() {
+    // Never constructed.
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/15a3e758/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/request/OzoneQuota.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/request/OzoneQuota.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/request/OzoneQuota.java
new file mode 100644
index 0000000..f0f6b98
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/request/OzoneQuota.java
@@ -0,0 +1,180 @@
+/*
+ * 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.ozone.web.request;
+
+
+import org.apache.hadoop.ozone.web.headers.Header;
+import org.codehaus.jackson.annotate.JsonIgnore;
+
+/**
+ * represents an OzoneQuota Object that can be applied to
+ * a storage volume.
+ */
+public class OzoneQuota {
+  private static final long MB_IN_BYTES = 1048576L;
+  private static final long GB_IN_BYTES = 1073741824L;
+  private static final long TB_IN_BYTES = 1099511627776L;
+
+
+  private Units unit;
+  private int size;
+
+  /** Quota Units.*/
+  public enum Units {UNDEFINED, BYTES, MB, GB, TB}
+
+  /**
+   * Returns size.
+   *
+   * @return int
+   */
+  public int getSize() {
+    return size;
+  }
+
+  /**
+   * Returns Units.
+   *
+   * @return Unit in MB, GB or TB
+   */
+  public Units getUnit() {
+    return unit;
+  }
+
+  /**
+   * Constructs a default Quota object.
+   */
+  public OzoneQuota() {
+    this.size = 0;
+    this.unit = Units.UNDEFINED;
+  }
+
+  /**
+   * Constructor for Ozone Quota.
+   *
+   * @param size - Integer Size
+   * @param unit MB, GB  or TB
+   */
+  public OzoneQuota(int size, Units unit) {
+    this.size = size;
+    this.unit = unit;
+  }
+
+  /**
+   * Parses a user provided string and returns the
+   * Quota Object.
+   *
+   * @param quotaString Quota String
+   *
+   * @return OzoneQuota object
+   *
+   * @throws IllegalArgumentException
+   */
+  public static OzoneQuota parseQuota(String quotaString)
+      throws IllegalArgumentException {
+
+    if ((quotaString == null) || (quotaString.isEmpty())) {
+      throw new IllegalArgumentException(
+          "Quota string cannot be null or empty.");
+    }
+
+    if (isRemove(quotaString)) {
+      throw new IllegalArgumentException("Remove is invalid in this context.");
+    }
+
+    String uppercase = quotaString.toUpperCase().replaceAll("\\s+", "");
+    String size = "";
+    int nSize;
+    Units currUnit = Units.MB;
+    Boolean found = false;
+    if (uppercase.endsWith(Header.OZONE_QUOTA_MB)) {
+      size = uppercase
+          .substring(0, uppercase.length() - Header.OZONE_QUOTA_MB.length());
+      currUnit = Units.MB;
+      found = true;
+    }
+
+    if (uppercase.endsWith(Header.OZONE_QUOTA_GB)) {
+      size = uppercase
+          .substring(0, uppercase.length() - Header.OZONE_QUOTA_GB.length());
+      currUnit = Units.GB;
+      found = true;
+    }
+
+    if (uppercase.endsWith(Header.OZONE_QUOTA_TB)) {
+      size = uppercase
+          .substring(0, uppercase.length() - Header.OZONE_QUOTA_TB.length());
+      currUnit = Units.TB;
+      found = true;
+    }
+
+    if (uppercase.endsWith(Header.OZONE_QUOTA_BYTES)) {
+      size = uppercase
+          .substring(0, uppercase.length() - Header.OZONE_QUOTA_BYTES.length());
+      currUnit = Units.BYTES;
+      found = true;
+    }
+
+    if (!found) {
+      throw new IllegalArgumentException(
+          "Quota unit not recognized. Supported values are BYTES, MB, GB and " +
+              "TB.");
+    }
+
+    nSize = Integer.parseInt(size);
+    if (nSize < 0) {
+      throw new IllegalArgumentException("Quota cannot be negative.");
+    }
+
+    return new OzoneQuota(nSize, currUnit);
+  }
+
+
+  /**
+   * Checks if Quota String is just as remove operation.
+   *
+   * @param quotaString User provided quota String
+   *
+   * @return True if it is Remove, false otherwise
+   */
+  public static boolean isRemove(String quotaString) {
+
+    return (quotaString != null) &&
+        (quotaString.compareToIgnoreCase(Header.OZONE_QUOTA_REMOVE) == 0);
+  }
+
+  /**
+   * Returns size in Bytes or -1 if there is no Quota.
+   */
+  @JsonIgnore
+  public long sizeInBytes() {
+    switch (this.unit) {
+    case BYTES:
+      return this.getSize();
+    case MB:
+      return this.getSize() * MB_IN_BYTES;
+    case GB:
+      return this.getSize() * GB_IN_BYTES;
+    case TB:
+      return this.getSize() * TB_IN_BYTES;
+    case UNDEFINED:
+    default:
+      return -1;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/15a3e758/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestQuota.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestQuota.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestQuota.java
new file mode 100644
index 0000000..c846147
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/web/TestQuota.java
@@ -0,0 +1,115 @@
+/**
+ * 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.ozone.web;
+
+import org.apache.hadoop.ozone.web.request.OzoneQuota;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+
+public class TestQuota {
+  @Test
+  public void TestParseQuota() {
+    HashMap<String, Boolean> testMatrix;
+    testMatrix = new HashMap<String, Boolean>();
+
+    testMatrix.put("10TB", Boolean.TRUE);
+    testMatrix.put("1 TB", Boolean.TRUE);
+    testMatrix.put("0MB", Boolean.TRUE);
+    testMatrix.put("0 TB", Boolean.TRUE);
+    testMatrix.put("    1000MB   ", Boolean.TRUE);
+
+    testMatrix.put("    1000MBMB   ", Boolean.FALSE);
+    testMatrix.put("    1000MB00   ", Boolean.FALSE);
+    testMatrix.put("1000ZMB", Boolean.FALSE);
+    testMatrix.put("MB1000", Boolean.FALSE);
+    testMatrix.put("9999", Boolean.FALSE);
+    testMatrix.put("1", Boolean.FALSE);
+    testMatrix.put("remove", Boolean.FALSE);
+    testMatrix.put("1UNDEFINED", Boolean.FALSE);
+    testMatrix.put(null, Boolean.FALSE);
+    testMatrix.put("", Boolean.FALSE);
+    testMatrix.put("-1000MB", Boolean.FALSE);
+    testMatrix.put("1024 bytes", Boolean.TRUE);
+    testMatrix.put("1bytes", Boolean.TRUE);
+    testMatrix.put("0bytes", Boolean.TRUE);
+    testMatrix.put("10000 BYTES",Boolean.TRUE );
+    testMatrix.put("BYTESbytes", Boolean.FALSE);
+    testMatrix.put("bytes", Boolean.FALSE);
+
+
+    Set<String> keys = testMatrix.keySet();
+    for (String key : keys) {
+      if (testMatrix.get(key)) {
+        OzoneQuota.parseQuota(key);
+      } else {
+        try {
+          OzoneQuota.parseQuota(key);
+          // should never get here since the isValid call will throw
+          fail(key);
+          fail("An exception was expected but did not happen.");
+        } catch (IllegalArgumentException e) {
+
+        }
+      }
+    }
+  }
+
+  @Test
+  public void TestVerifyQuota() {
+    OzoneQuota qt = OzoneQuota.parseQuota("10TB");
+    assertEquals(qt.getSize(), 10);
+    assertEquals(qt.getUnit(), OzoneQuota.Units.TB);
+    assertEquals(qt.sizeInBytes(), 10L * (1024L * 1024L * 1024L * 1024L));
+
+    qt = OzoneQuota.parseQuota("10MB");
+    assertEquals(qt.getSize(), 10);
+    assertEquals(qt.getUnit(), OzoneQuota.Units.MB);
+    assertEquals(qt.sizeInBytes(), 10L * (1024L * 1024L));
+
+    qt = OzoneQuota.parseQuota("10GB");
+    assertEquals(qt.getSize(), 10);
+    assertEquals(qt.getUnit(), OzoneQuota.Units.GB);
+    assertEquals(qt.sizeInBytes(), 10L * (1024L * 1024L * 1024L));
+
+    qt = OzoneQuota.parseQuota("10BYTES");
+    assertEquals(qt.getSize(), 10);
+    assertEquals(qt.getUnit(), OzoneQuota.Units.BYTES);
+    assertEquals(qt.sizeInBytes(), 10L);
+
+    OzoneQuota emptyQuota = new OzoneQuota();
+    assertEquals(emptyQuota.sizeInBytes(), -1L);
+    assertEquals(emptyQuota.getSize(), 0);
+    assertEquals(emptyQuota.getUnit(), OzoneQuota.Units.UNDEFINED);
+  }
+
+  @Test
+  public void TestVerifyRemove() {
+    assertTrue(OzoneQuota.isRemove("remove"));
+    assertFalse(OzoneQuota.isRemove("not remove"));
+    assertFalse(OzoneQuota.isRemove(null));
+  }
+}