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 el...@apache.org on 2018/10/08 08:47:55 UTC

[32/50] [abbrv] hadoop git commit: HDDS-572. Support S3 buckets as first class objects in Ozone Manager - 1. Contributed by Anu Engineer.

HDDS-572. Support S3 buckets as first class objects in Ozone Manager - 1.
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/e6b77ad6
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e6b77ad6
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e6b77ad6

Branch: refs/heads/HEAD
Commit: e6b77ad65f923858fb67f5c2165fefe52d6f8c17
Parents: 7fb91b8
Author: Anu Engineer <ae...@apache.org>
Authored: Thu Oct 4 21:40:13 2018 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Thu Oct 4 21:40:13 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/ozone/OzoneConsts.java    |   1 +
 .../hadoop/ozone/om/OMMetadataManager.java      |   6 +
 .../hadoop/ozone/om/OmMetadataManagerImpl.java  |  14 ++
 .../hadoop/ozone/om/OzoneManagerLock.java       |  58 +++++-
 .../apache/hadoop/ozone/om/S3BucketManager.java |  59 ++++++
 .../hadoop/ozone/om/S3BucketManagerImpl.java    | 194 +++++++++++++++++++
 .../hadoop/ozone/om/exceptions/OMException.java |   5 +-
 .../hadoop/ozone/om/TestOzoneManagerLock.java   |   3 +-
 .../hadoop/ozone/om/TestS3BucketManager.java    | 115 +++++++++++
 9 files changed, 443 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6b77ad6/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
index 0a15ec8..923271c 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
@@ -139,6 +139,7 @@ public final class OzoneConsts {
 
   public static final String OM_KEY_PREFIX = "/";
   public static final String OM_USER_PREFIX = "$";
+  public static final String OM_S3_PREFIX ="S3:";
 
   /**
    * Max OM Quota size of 1024 PB.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6b77ad6/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java
index c8fb39c..5f490ec 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java
@@ -236,4 +236,10 @@ public interface OMMetadataManager {
    */
   Table getOpenKeyTable();
 
+  /**
+   * Gets the S3Bucket to Ozone Volume/bucket mapping table.
+   *
+   * @return Table.
+   */
+  Table getS3Table();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6b77ad6/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java
index a7e1bed..75bd712 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java
@@ -88,6 +88,8 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
    * |-------------------------------------------------------------------|
    * | openKey            | /volumeName/bucketName/keyName/id->KeyInfo   |
    * |-------------------------------------------------------------------|
+   * | s3Table            | s3BucketName -> /volumeName/bucketName       |
+   * |-------------------------------------------------------------------|
    */
 
   private static final String USER_TABLE = "userTable";
@@ -96,6 +98,7 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
   private static final String KEY_TABLE = "keyTable";
   private static final String DELETED_TABLE = "deletedTable";
   private static final String OPEN_KEY_TABLE = "openKeyTable";
+  private static final String S3_TABLE = "s3Table";
 
   private final DBStore store;
 
@@ -108,6 +111,7 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
   private final Table keyTable;
   private final Table deletedTable;
   private final Table openKeyTable;
+  private final Table s3Table;
 
   public OmMetadataManagerImpl(OzoneConfiguration conf) throws IOException {
     File metaDir = getOzoneMetaDirPath(conf);
@@ -125,6 +129,7 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
         .addTable(KEY_TABLE)
         .addTable(DELETED_TABLE)
         .addTable(OPEN_KEY_TABLE)
+        .addTable(S3_TABLE)
         .build();
 
     userTable = this.store.getTable(USER_TABLE);
@@ -145,6 +150,9 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
     openKeyTable = this.store.getTable(OPEN_KEY_TABLE);
     checkTableStatus(openKeyTable, OPEN_KEY_TABLE);
 
+    s3Table = this.store.getTable(S3_TABLE);
+    checkTableStatus(s3Table, S3_TABLE);
+
   }
 
   @Override
@@ -177,6 +185,12 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
     return openKeyTable;
   }
 
+  @Override
+  public Table getS3Table() {
+    return s3Table;
+  }
+
+
   private void checkTableStatus(Table table, String name) throws IOException {
     String logMessage = "Unable to get a reference to %s table. Cannot " +
         "continue.";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6b77ad6/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerLock.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerLock.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerLock.java
index e9ea2df..6203ddf 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerLock.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerLock.java
@@ -25,6 +25,7 @@ import java.util.Map;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_S3_PREFIX;
 import static org.apache.hadoop.ozone.OzoneConsts.OM_USER_PREFIX;
 
 /**
@@ -57,24 +58,26 @@ import static org.apache.hadoop.ozone.OzoneConsts.OM_USER_PREFIX;
  *     +--> acquireUserLock (will throw Exception)<br>
  * </p>
  * <br>
- *
  * To acquire a user lock you should not hold any Volume/Bucket lock. Similarly
  * to acquire a Volume lock you should not hold any Bucket lock.
- *
  */
 public final class OzoneManagerLock {
 
   private static final String VOLUME_LOCK = "volumeLock";
   private static final String BUCKET_LOCK = "bucketLock";
-
+  private static final String S3_BUCKET_LOCK = "s3BucketLock";
 
   private final LockManager<String> manager;
 
   // To maintain locks held by current thread.
   private final ThreadLocal<Map<String, AtomicInteger>> myLocks =
-      ThreadLocal.withInitial(() -> ImmutableMap.of(
-          VOLUME_LOCK, new AtomicInteger(0),
-          BUCKET_LOCK, new AtomicInteger(0)));
+      ThreadLocal.withInitial(
+          () -> ImmutableMap.of(
+              VOLUME_LOCK, new AtomicInteger(0),
+              BUCKET_LOCK, new AtomicInteger(0),
+              S3_BUCKET_LOCK, new AtomicInteger(0)
+          )
+      );
 
   /**
    * Creates new OzoneManagerLock instance.
@@ -95,11 +98,11 @@ public final class OzoneManagerLock {
    */
   public void acquireUserLock(String user) {
     // Calling thread should not hold any volume or bucket lock.
-    if (hasAnyVolumeLock() || hasAnyBucketLock()) {
+    if (hasAnyVolumeLock() || hasAnyBucketLock() || hasAnyS3Lock()) {
       throw new RuntimeException(
           "Thread '" + Thread.currentThread().getName() +
               "' cannot acquire user lock" +
-              " while holding volume/bucket lock(s).");
+              " while holding volume, bucket or S3 bucket lock(s).");
     }
     manager.lock(OM_USER_PREFIX + user);
   }
@@ -122,6 +125,9 @@ public final class OzoneManagerLock {
    */
   public void acquireVolumeLock(String volume) {
     // Calling thread should not hold any bucket lock.
+    // You can take an Volume while holding S3 bucket lock, since
+    // semantically an S3 bucket maps to the ozone volume. So we check here
+    // only if ozone bucket lock is taken.
     if (hasAnyBucketLock()) {
       throw new RuntimeException(
           "Thread '" + Thread.currentThread().getName() +
@@ -140,6 +146,38 @@ public final class OzoneManagerLock {
   }
 
   /**
+   * Acquires S3 Bucket lock on the given resource.
+   *
+   * <p>If the lock is not available then the current thread becomes
+   * disabled for thread scheduling purposes and lies dormant until the lock has
+   * been acquired.
+   *
+   * @param s3BucketName S3Bucket Name on which the lock has to be acquired
+   */
+  public void acquireS3Lock(String s3BucketName) {
+    // Calling thread should not hold any bucket lock.
+    // You can take an Volume while holding S3 bucket lock, since
+    // semantically an S3 bucket maps to the ozone volume. So we check here
+    // only if ozone bucket lock is taken.
+    if (hasAnyBucketLock()) {
+      throw new RuntimeException(
+          "Thread '" + Thread.currentThread().getName() +
+              "' cannot acquire S3 bucket lock while holding Ozone bucket " +
+              "lock(s).");
+    }
+    manager.lock(OM_S3_PREFIX + s3BucketName);
+    myLocks.get().get(S3_BUCKET_LOCK).incrementAndGet();
+  }
+
+  /**
+   * Releases the volume lock on given resource.
+   */
+  public void releaseS3Lock(String s3BucketName) {
+    manager.unlock(OM_S3_PREFIX + s3BucketName);
+    myLocks.get().get(S3_BUCKET_LOCK).decrementAndGet();
+  }
+
+  /**
    * Acquires bucket lock on the given resource.
    *
    * <p>If the lock is not available then the current thread becomes
@@ -153,7 +191,6 @@ public final class OzoneManagerLock {
     myLocks.get().get(BUCKET_LOCK).incrementAndGet();
   }
 
-
   /**
    * Releases the bucket lock on given resource.
    */
@@ -178,4 +215,7 @@ public final class OzoneManagerLock {
     return myLocks.get().get(BUCKET_LOCK).get() != 0;
   }
 
+  private boolean hasAnyS3Lock() {
+    return myLocks.get().get(S3_BUCKET_LOCK).get() != 0;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6b77ad6/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/S3BucketManager.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/S3BucketManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/S3BucketManager.java
new file mode 100644
index 0000000..5fb4c60
--- /dev/null
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/S3BucketManager.java
@@ -0,0 +1,59 @@
+/*
+ * 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.om;
+
+import java.io.IOException;
+
+/**
+ * An interface that maps S3 buckets to Ozone
+ * volume/bucket.
+ */
+public interface S3BucketManager {
+  /**
+   * Creates an s3 bucket and maps it to Ozone volume/bucket.
+   * @param  userName - Name of the user who owns the bucket.
+   * @param bucketName - S3 Bucket Name.
+   * @throws  IOException in case the bucket cannot be created.
+   */
+  void createS3Bucket(String userName, String bucketName) throws IOException;
+
+  /**
+   * Returns the Ozone volume/bucket where the S3 Bucket points to.
+   * @param s3BucketName - S3 Bucket Name
+   * @return String - Ozone volume/bucket
+   * @throws IOException in case of failure to retrieve mapping.
+   */
+  String getOzoneBucketMapping(String s3BucketName) throws IOException;
+
+  /**
+   * Returns Ozone volume name for a given S3Bucket.
+   * @param s3BucketName - S3 bucket name.
+   * @return String - Ozone volume name where is s3bucket resides.
+   * @throws IOException - in case of failure to retrieve mapping.
+   */
+  String getOzoneVolumeName(String s3BucketName) throws IOException;
+
+  /**
+   * Returns Ozone bucket name for a given s3Bucket.
+   * @param s3BucketName  - S3 bucket Name.
+   * @return  Ozone bucket name for this given S3 bucket
+   * @throws IOException - in case of failure to retrieve mapping.
+   */
+  String getOzoneBucketName(String s3BucketName) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6b77ad6/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/S3BucketManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/S3BucketManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/S3BucketManagerImpl.java
new file mode 100644
index 0000000..907f054
--- /dev/null
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/S3BucketManagerImpl.java
@@ -0,0 +1,194 @@
+/*
+ * 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.om;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.logging.log4j.util.Strings;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.FAILED_VOLUME_ALREADY_EXISTS;
+
+/**
+ * S3 Bucket Manager, this class maintains a mapping between S3 Bucket and Ozone
+ * Volume/bucket.
+ */
+public class S3BucketManagerImpl implements S3BucketManager {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(S3BucketManagerImpl.class);
+
+  private static final String S3_ADMIN_NAME = "OzoneS3Manager";
+  private final OzoneConfiguration configuration;
+  private final OMMetadataManager omMetadataManager;
+  private final VolumeManager volumeManager;
+  private final BucketManager bucketManager;
+
+  /**
+   * Construct an S3 Bucket Manager Object.
+   *
+   * @param configuration - Ozone Configuration.
+   * @param omMetadataManager - Ozone Metadata Manager.
+   */
+  public S3BucketManagerImpl(
+      OzoneConfiguration configuration,
+      OMMetadataManager omMetadataManager,
+      VolumeManager volumeManager,
+      BucketManager bucketManager) {
+    this.configuration = configuration;
+    this.omMetadataManager = omMetadataManager;
+    this.volumeManager = volumeManager;
+    this.bucketManager = bucketManager;
+  }
+
+  @Override
+  public void createS3Bucket(String userName, String bucketName)
+      throws IOException {
+    Preconditions.checkArgument(
+        Strings.isNotBlank(bucketName), "Bucket name cannot be null or empty.");
+
+    Preconditions.checkArgument(Strings.isNotBlank(userName), "User name " +
+        "cannot be null or empty.");
+
+    // TODO: Decide if we want to enforce S3 Bucket Creation Rules in this
+    // code path?
+    // https://docs.aws.amazon.com/AmazonS3/latest/dev/BucketRestrictions.html
+
+    // Generate an Ozone volume name. For the time being, we are going to use
+    // s3userName as the Ozone volume name. Since S3 advices 100 buckets max
+    // for a user and we have no limit to the number of Ozone buckets under a
+    // volume we will stick to very simple model.
+    //
+    // s3Bucket -> ozoneVolume/OzoneBucket name
+    // s3BucketName ->s3userName/s3Bucketname
+    //
+    // You might wonder if all names map to this pattern, why we need to
+    // store the S3 bucketName in a table at all. This is to support
+    // anonymous access to bucket where the user name is absent.
+
+    // About Locking:
+    // We need to do this before we take the S3Bucket Lock since createVolume
+    // takes the userLock. So an attempt to take the user lock while holding
+    // S3Bucket lock will throw, so we need to create the volume if needed
+    // before we execute the bucket mapping functions.
+    String ozoneVolumeName = formatOzoneVolumeName(userName);
+    createOzoneVolumeIfNeeded(userName, ozoneVolumeName);
+
+    omMetadataManager.getLock().acquireS3Lock(bucketName);
+    try {
+      byte[] bucket =
+          omMetadataManager.getS3Table().get(
+              bucketName.getBytes(StandardCharsets.UTF_8));
+
+      if (bucket != null) {
+        throw new OMException(
+            "Unable to create S3 bucket. " + bucketName + " already exists.",
+            OMException.ResultCodes.S3_BUCKET_ALREADY_EXISTS);
+      }
+      String ozoneBucketName = bucketName;
+      createOzoneBucket(ozoneVolumeName, ozoneBucketName);
+      String finalName = String.format("%s/%s", ozoneVolumeName,
+          ozoneBucketName);
+
+      omMetadataManager.getS3Table().put(
+              bucketName.getBytes(StandardCharsets.UTF_8),
+              finalName.getBytes(StandardCharsets.UTF_8));
+    } finally {
+      omMetadataManager.getLock().releaseS3Lock(bucketName);
+    }
+  }
+
+  private String formatOzoneVolumeName(String userName) {
+    return String.format("s3%s", userName);
+  }
+
+  private void createOzoneVolumeIfNeeded(String userName, String volumeName)
+      throws IOException {
+    // We don't have to time of check. time of use problem here because
+    // this call is invoked while holding the s3Bucket lock.
+    try {
+      OmVolumeArgs args =
+          OmVolumeArgs.newBuilder()
+              .setAdminName(S3_ADMIN_NAME)
+              .setOwnerName(userName)
+              .setVolume(volumeName)
+              .setQuotaInBytes(OzoneConsts.MAX_QUOTA_IN_BYTES)
+              .build();
+      volumeManager.createVolume(args);
+    } catch (OMException exp) {
+      if (exp.getResult().compareTo(FAILED_VOLUME_ALREADY_EXISTS) == 0) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Volume already exists. {}", exp.getMessage());
+        }
+      } else {
+        throw exp;
+      }
+    }
+  }
+
+  private void createOzoneBucket(String volumeName, String bucketName)
+      throws IOException {
+    OmBucketInfo.Builder builder = OmBucketInfo.newBuilder();
+    OmBucketInfo bucketInfo =
+        builder
+            .setVolumeName(volumeName)
+            .setBucketName(bucketName)
+            .setIsVersionEnabled(Boolean.FALSE)
+            .setStorageType(StorageType.DEFAULT)
+            .build();
+    bucketManager.createBucket(bucketInfo);
+  }
+
+  @Override
+  public String getOzoneBucketMapping(String s3BucketName) throws IOException {
+    omMetadataManager.getLock().acquireS3Lock(s3BucketName);
+    try {
+      byte[] mapping =
+          omMetadataManager.getS3Table().get(
+              s3BucketName.getBytes(StandardCharsets.UTF_8));
+      if (mapping != null) {
+        return new String(mapping, StandardCharsets.UTF_8);
+      }
+      throw new OMException("No such S3 bucket.",
+          OMException.ResultCodes.S3_BUCKET_NOT_FOUND);
+    } finally {
+      omMetadataManager.getLock().releaseS3Lock(s3BucketName);
+    }
+  }
+
+  @Override
+  public String getOzoneVolumeName(String s3BucketName) throws IOException {
+    String mapping = getOzoneBucketMapping(s3BucketName);
+    return mapping.split("/")[0];
+  }
+
+  @Override
+  public String getOzoneBucketName(String s3BucketName) throws IOException {
+    String mapping = getOzoneBucketMapping(s3BucketName);
+    return mapping.split("/")[1];
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6b77ad6/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java
index 393ac91..3292adf 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java
@@ -111,9 +111,10 @@ public class OMException extends IOException {
     FAILED_KEY_RENAME,
     FAILED_INVALID_KEY_NAME,
     FAILED_METADATA_ERROR,
-    FAILED_INTERNAL_ERROR,
     OM_NOT_INITIALIZED,
     SCM_VERSION_MISMATCH_ERROR,
-    SCM_IN_CHILL_MODE
+    SCM_IN_CHILL_MODE,
+    S3_BUCKET_ALREADY_EXISTS,
+    S3_BUCKET_NOT_FOUND
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6b77ad6/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerLock.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerLock.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerLock.java
index d8d3096..2def5c4 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerLock.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerLock.java
@@ -165,7 +165,8 @@ public class TestOzoneManagerLock {
       Assert.fail();
     } catch (RuntimeException ex) {
       String msg =
-          "cannot acquire user lock while holding volume/bucket lock(s).";
+          "cannot acquire user lock while holding " +
+              "volume, bucket or S3 bucket lock(s).";
       Assert.assertTrue(ex.getMessage().contains(msg));
     }
     lock.releaseVolumeLock("volOne");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e6b77ad6/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestS3BucketManager.java
----------------------------------------------------------------------
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestS3BucketManager.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestS3BucketManager.java
new file mode 100644
index 0000000..4837f9a
--- /dev/null
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestS3BucketManager.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.om;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+
+import static org.junit.Assert.*;
+
+/**
+ * Tests for S3 Bucket Manager.
+ */
+public class TestS3BucketManager {
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  private OzoneConfiguration conf;
+  private OmMetadataManagerImpl metaMgr;
+  private BucketManager bucketManager;
+  private VolumeManager volumeManager;
+
+  @Before
+  public void init() throws IOException {
+    conf = new OzoneConfiguration();
+    File newFolder = folder.newFolder();
+    if (!newFolder.exists()) {
+      Assert.assertTrue(newFolder.mkdirs());
+    }
+    ServerUtils.setOzoneMetaDirPath(conf, newFolder.toString());
+    metaMgr = new OmMetadataManagerImpl(conf);
+    volumeManager = new VolumeManagerImpl(metaMgr, conf);
+    bucketManager = new BucketManagerImpl(metaMgr);
+  }
+
+  @Test
+  public void testCreateS3Bucket() throws IOException {
+    S3BucketManager s3BucketManager = new S3BucketManagerImpl(conf, metaMgr,
+        volumeManager, bucketManager);
+    s3BucketManager.createS3Bucket("bilbo", "bucket");
+
+    // This call should have created a ozone volume called s3bilbo and bucket
+    // called s3bilbo/bucket.
+    Assert.assertNotNull(volumeManager.getVolumeInfo("s3bilbo"));
+    Assert.assertNotNull(bucketManager.getBucketInfo("s3bilbo", "bucket"));
+
+    // recreating the same bucket should throw.
+    thrown.expect(IOException.class);
+    s3BucketManager.createS3Bucket("bilbo", "bucket");
+
+  }
+
+  @Test
+  public void testGetS3BucketMapping() throws IOException {
+    S3BucketManager s3BucketManager = new S3BucketManagerImpl(conf, metaMgr,
+        volumeManager, bucketManager);
+    s3BucketManager.createS3Bucket("bilbo", "newBucket");
+    String mapping = s3BucketManager.getOzoneBucketMapping("newBucket");
+    Assert.assertTrue(mapping.startsWith("s3bilbo/"));
+    Assert.assertTrue(mapping.endsWith("/newBucket"));
+  }
+
+  @Test
+  public void testGetOzoneNames() throws IOException {
+    S3BucketManager s3BucketManager = new S3BucketManagerImpl(conf, metaMgr,
+        volumeManager, bucketManager);
+    s3BucketManager.createS3Bucket("batman", "gotham");
+    String volumeName = s3BucketManager.getOzoneVolumeName("gotham");
+    Assert.assertTrue(volumeName.equalsIgnoreCase("s3batman"));
+    String bucketName =s3BucketManager.getOzoneBucketName("gotham");
+    Assert.assertTrue(bucketName.equalsIgnoreCase("gotham"));
+    // try to get a bucket that does not exist.
+    thrown.expectMessage("No such S3 bucket.");
+    s3BucketManager.getOzoneBucketMapping("raven");
+
+  }
+
+  @Test
+  /**
+   * This tests makes sure bucket names are unique across users.
+   */
+  public void testBucketNameAreUnique() throws IOException {
+    S3BucketManager s3BucketManager = new S3BucketManagerImpl(conf, metaMgr,
+        volumeManager, bucketManager);
+    s3BucketManager.createS3Bucket("superman", "metropolis");
+    // recreating the same bucket  even with a different user will throw.
+    thrown.expectMessage("Unable to create S3 bucket.");
+    s3BucketManager.createS3Bucket("luthor", "metropolis");
+  }
+}
\ 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