Pārlūkot izejas kodu

HDDS-1689. Implement S3 Create Bucket request to use Cache and DoubleBuffer. (#1088)

Bharat Viswanadham 6 gadi atpakaļ
vecāks
revīzija
3dc256ef81
23 mainītis faili ar 1044 papildinājumiem un 39 dzēšanām
  1. 5 0
      hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
  2. 3 0
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/audit/OMAction.java
  3. 2 0
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java
  4. 13 0
      hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto
  5. 24 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetrics.java
  6. 1 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerDoubleBuffer.java
  7. 3 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/utils/OzoneManagerRatisUtils.java
  8. 361 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/bucket/S3BucketCreateRequest.java
  9. 23 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/bucket/package-info.java
  10. 12 24
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeCreateRequest.java
  11. 1 3
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeDeleteRequest.java
  12. 40 5
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeRequest.java
  13. 1 3
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeSetOwnerRequest.java
  14. 1 2
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeSetQuotaRequest.java
  15. 73 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/bucket/S3BucketCreateResponse.java
  16. 24 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/bucket/package-info.java
  17. 1 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerHARequestHandlerImpl.java
  18. 20 0
      hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/TestOMRequestUtils.java
  19. 246 0
      hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/s3/bucket/TestS3BucketCreateRequest.java
  20. 23 0
      hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/s3/bucket/package-info.java
  21. 25 2
      hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/volume/TestOMVolumeCreateRequest.java
  22. 119 0
      hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/s3/bucket/TestS3BucketCreateResponse.java
  23. 23 0
      hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/s3/bucket/package-info.java

+ 5 - 0
hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java

@@ -264,6 +264,7 @@ public final class OzoneConsts {
   public static final String UPLOAD_ID = "uploadID";
   public static final String PART_NUMBER_MARKER = "partNumberMarker";
   public static final String MAX_PARTS = "maxParts";
+  public static final String S3_BUCKET = "s3Bucket";
 
 
 
@@ -303,4 +304,8 @@ public final class OzoneConsts {
   public static final String JAVA_TMP_DIR = "java.io.tmpdir";
   public static final String LOCALHOST = "localhost";
 
+
+  public static final int S3_BUCKET_MIN_LENGTH = 3;
+  public static final int S3_BUCKET_MAX_LENGTH = 64;
+
 }

+ 3 - 0
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/audit/OMAction.java

@@ -39,6 +39,9 @@ public enum OMAction implements AuditAction {
   UPDATE_KEY,
   PURGE_KEYS,
 
+  // S3 Bucket
+  CREATE_S3_BUCKET,
+
   // READ Actions
   CHECK_VOLUME_ACCESS,
   LIST_BUCKETS,

+ 2 - 0
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java

@@ -203,5 +203,7 @@ public class OMException extends IOException {
 
     PREFIX_NOT_FOUND,
 
+    S3_BUCKET_INVALID_LENGTH
+
   }
 }

+ 13 - 0
hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto

@@ -279,6 +279,8 @@ enum Status {
     PERMISSION_DENIED = 48;
     TIMEOUT = 49;
     PREFIX_NOT_FOUND=50;
+
+    S3_BUCKET_INVALID_LENGTH = 51; // s3 bucket invalid length.
 }
 
 
@@ -860,6 +862,17 @@ message ServiceInfo {
 message S3CreateBucketRequest {
     required string userName = 1;
     required string s3bucketname = 2;
+    // This will be set during OM HA by one of the OM node. In future if more
+    // data fields are required to create volume/bucket we can add them to
+    // this. This is the reason for creating a new message type for this.
+    // S3CreateBucket means create volume from userName and create bucket
+    // with s3BucketName.
+    optional S3CreateVolumeInfo s3CreateVolumeInfo = 3;
+}
+
+message S3CreateVolumeInfo {
+    // Creation time set in preExecute on one of the OM node.
+    required uint64 creationTime = 1;
 }
 
 message S3CreateBucketResponse {

+ 24 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetrics.java

@@ -113,17 +113,24 @@ public class OMMetrics {
 
   private @Metric MutableCounterLong numVolumes;
   private @Metric MutableCounterLong numBuckets;
+  private @Metric MutableCounterLong numS3Buckets;
 
   //TODO: This metric is an estimate and it may be inaccurate on restart if the
   // OM process was not shutdown cleanly. Key creations/deletions in the last
   // few minutes before restart may not be included in this count.
   private @Metric MutableCounterLong numKeys;
 
+
+
   // Metrics to track checkpointing statistics from last run.
   private @Metric MutableGaugeLong lastCheckpointCreationTimeTaken;
   private @Metric MutableGaugeLong lastCheckpointTarOperationTimeTaken;
   private @Metric MutableGaugeLong lastCheckpointStreamingTimeTaken;
 
+  private @Metric MutableCounterLong numS3BucketCreates;
+  private @Metric MutableCounterLong numS3BucketCreateFails;
+
+
   public OMMetrics() {
   }
 
@@ -134,6 +141,23 @@ public class OMMetrics {
         new OMMetrics());
   }
 
+  public void incNumS3BucketCreates() {
+    numBucketOps.incr();
+    numS3BucketCreates.incr();
+  }
+
+  public void incNumS3BucketCreateFails() {
+    numS3BucketCreateFails.incr();
+  }
+
+  public void incNumS3Buckets() {
+    numS3Buckets.incr();
+  }
+
+  public void decNumS3Buckets() {
+    numS3Buckets.incr();
+  }
+
   public void incNumVolumes() {
     numVolumes.incr();
   }

+ 1 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerDoubleBuffer.java

@@ -175,6 +175,7 @@ public class OzoneManagerDoubleBuffer {
     omMetadataManager.getOpenKeyTable().cleanupCache(lastRatisTransactionIndex);
     omMetadataManager.getKeyTable().cleanupCache(lastRatisTransactionIndex);
     omMetadataManager.getDeletedTable().cleanupCache(lastRatisTransactionIndex);
+    omMetadataManager.getS3Table().cleanupCache(lastRatisTransactionIndex);
   }
 
   /**

+ 3 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/utils/OzoneManagerRatisUtils.java

@@ -31,6 +31,7 @@ import org.apache.hadoop.ozone.om.request.key.OMKeyCreateRequest;
 import org.apache.hadoop.ozone.om.request.key.OMKeyDeleteRequest;
 import org.apache.hadoop.ozone.om.request.key.OMKeyPurgeRequest;
 import org.apache.hadoop.ozone.om.request.key.OMKeyRenameRequest;
+import org.apache.hadoop.ozone.om.request.s3.bucket.S3BucketCreateRequest;
 import org.apache.hadoop.ozone.om.request.volume.OMVolumeCreateRequest;
 import org.apache.hadoop.ozone.om.request.volume.OMVolumeDeleteRequest;
 import org.apache.hadoop.ozone.om.request.volume.OMVolumeSetOwnerRequest;
@@ -99,6 +100,8 @@ public final class OzoneManagerRatisUtils {
       return new OMFileCreateRequest(omRequest);
     case PurgeKeys:
       return new OMKeyPurgeRequest(omRequest);
+    case CreateS3Bucket:
+      return new S3BucketCreateRequest(omRequest);
     default:
       // TODO: will update once all request types are implemented.
       return null;

+ 361 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/bucket/S3BucketCreateRequest.java

@@ -0,0 +1,361 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.request.s3.bucket;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hdds.protocol.StorageType;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.OzoneManager;
+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.hadoop.ozone.om.request.volume.OMVolumeRequest;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.bucket.OMBucketCreateResponse;
+import org.apache.hadoop.ozone.om.response.s3.bucket.S3BucketCreateResponse;
+import org.apache.hadoop.ozone.om.response.volume.OMVolumeCreateResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .S3CreateBucketRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .S3CreateBucketResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .S3CreateVolumeInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .VolumeList;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.util.Time;
+import org.apache.hadoop.utils.db.cache.CacheKey;
+import org.apache.hadoop.utils.db.cache.CacheValue;
+
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_S3_VOLUME_PREFIX;
+import static org.apache.hadoop.ozone.OzoneConsts.S3_BUCKET_MAX_LENGTH;
+import static org.apache.hadoop.ozone.OzoneConsts.S3_BUCKET_MIN_LENGTH;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.S3_BUCKET_LOCK;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.USER_LOCK;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.VOLUME_LOCK;
+
+/**
+ * Handles S3 Bucket create request.
+ */
+public class S3BucketCreateRequest extends OMVolumeRequest {
+
+  private static final String S3_ADMIN_NAME = "OzoneS3Manager";
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(S3CreateBucketRequest.class);
+
+  public S3BucketCreateRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
+    S3CreateBucketRequest s3CreateBucketRequest =
+        getOmRequest().getCreateS3BucketRequest();
+    Preconditions.checkNotNull(s3CreateBucketRequest);
+
+    S3CreateBucketRequest.Builder newS3CreateBucketRequest =
+        s3CreateBucketRequest.toBuilder().setS3CreateVolumeInfo(
+            S3CreateVolumeInfo.newBuilder().setCreationTime(Time.now()));
+
+    // TODO: Do we need to enforce the bucket rules in this code path?
+    // https://docs.aws.amazon.com/AmazonS3/latest/dev/BucketRestrictions.html
+
+    // For now only checked the length.
+    int bucketLength = s3CreateBucketRequest.getS3Bucketname().length();
+    if (bucketLength < S3_BUCKET_MIN_LENGTH ||
+        bucketLength >= S3_BUCKET_MAX_LENGTH) {
+      throw new OMException("S3BucketName must be at least 3 and not more " +
+          "than 63 characters long",
+          OMException.ResultCodes.S3_BUCKET_INVALID_LENGTH);
+    }
+
+    return getOmRequest().toBuilder()
+        .setCreateS3BucketRequest(newS3CreateBucketRequest)
+        .setUserInfo(getUserInfo()).build();
+  }
+
+  @Override
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long transactionLogIndex) {
+
+    S3CreateBucketRequest s3CreateBucketRequest =
+        getOmRequest().getCreateS3BucketRequest();
+
+    String userName = s3CreateBucketRequest.getUserName();
+    String s3BucketName = s3CreateBucketRequest.getS3Bucketname();
+
+    OMResponse.Builder omResponse = OMResponse.newBuilder().setCmdType(
+        OzoneManagerProtocolProtos.Type.CreateS3Bucket).setStatus(
+        OzoneManagerProtocolProtos.Status.OK).setSuccess(true);
+
+    OMMetrics omMetrics = ozoneManager.getMetrics();
+    omMetrics.incNumS3BucketCreates();
+
+    // When s3 Bucket is created, we internally create ozone volume/ozone
+    // bucket.
+
+    // ozone volume name is generated from userName by calling
+    // formatOzoneVolumeName.
+
+    // ozone bucket name is same as s3 bucket name.
+    // In S3 buckets are unique, so we create a mapping like s3BucketName ->
+    // ozoneVolume/ozoneBucket and add it to s3 mapping table. If
+    // s3BucketName exists in mapping table, bucket already exist or we go
+    // ahead and create a bucket.
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    IOException exception = null;
+    VolumeList volumeList = null;
+    OmVolumeArgs omVolumeArgs = null;
+    OmBucketInfo omBucketInfo = null;
+
+    boolean volumeCreated = false;
+    boolean acquiredVolumeLock = false;
+    boolean acquiredUserLock = false;
+    boolean acquiredS3Lock = false;
+    String volumeName = formatOzoneVolumeName(userName);
+    try {
+      // check Acl
+      if (ozoneManager.getAclsEnabled()) {
+        checkAcls(ozoneManager, OzoneObj.ResourceType.BUCKET,
+            OzoneObj.StoreType.S3, IAccessAuthorizer.ACLType.CREATE, null,
+            s3BucketName, null);
+      }
+
+      acquiredS3Lock = omMetadataManager.getLock().acquireLock(S3_BUCKET_LOCK,
+          s3BucketName);
+
+      // First check if this s3Bucket exists
+      if (omMetadataManager.getS3Table().isExist(s3BucketName)) {
+        throw new OMException("S3Bucket " + s3BucketName + " already exists",
+            OMException.ResultCodes.S3_BUCKET_ALREADY_EXISTS);
+      }
+
+      try {
+        acquiredVolumeLock =
+            omMetadataManager.getLock().acquireLock(VOLUME_LOCK, volumeName);
+        acquiredUserLock = omMetadataManager.getLock().acquireLock(USER_LOCK,
+            userName);
+        // Check if volume exists, if it does not exist create
+        // ozone volume.
+        String volumeKey = omMetadataManager.getVolumeKey(volumeName);
+        if (!omMetadataManager.getVolumeTable().isExist(volumeKey)) {
+          omVolumeArgs = createOmVolumeArgs(volumeName, userName,
+              s3CreateBucketRequest.getS3CreateVolumeInfo()
+                  .getCreationTime());
+          volumeList = omMetadataManager.getUserTable().get(
+              omMetadataManager.getUserKey(userName));
+          volumeList = addVolumeToOwnerList(volumeList,
+              volumeName, userName, ozoneManager.getMaxUserVolumeCount());
+          createVolume(omMetadataManager, omVolumeArgs, volumeList, volumeKey,
+              omMetadataManager.getUserKey(userName), transactionLogIndex);
+          volumeCreated = true;
+        }
+      } finally {
+        if (acquiredUserLock) {
+          omMetadataManager.getLock().releaseLock(USER_LOCK, userName);
+        }
+        if (acquiredVolumeLock) {
+          omMetadataManager.getLock().releaseLock(VOLUME_LOCK, volumeName);
+        }
+      }
+
+      // check if ozone bucket exists, if it does not exist create ozone
+      // bucket
+      omBucketInfo = createBucket(omMetadataManager, volumeName, s3BucketName,
+          s3CreateBucketRequest.getS3CreateVolumeInfo().getCreationTime(),
+          transactionLogIndex);
+
+      // Now finally add it to s3 table cache.
+      omMetadataManager.getS3Table().addCacheEntry(
+          new CacheKey<>(s3BucketName), new CacheValue<>(
+              Optional.of(formatS3MappingName(volumeName, s3BucketName)),
+              transactionLogIndex));
+    } catch (IOException ex) {
+      exception = ex;
+    } finally {
+      if (acquiredS3Lock) {
+        omMetadataManager.getLock().releaseLock(S3_BUCKET_LOCK, s3BucketName);
+      }
+    }
+
+    // Performing audit logging outside of the lock.
+    auditLog(ozoneManager.getAuditLogger(),
+        buildAuditMessage(OMAction.CREATE_S3_BUCKET,
+            buildAuditMap(userName, s3BucketName), exception,
+            getOmRequest().getUserInfo()));
+
+    if (exception == null) {
+      LOG.debug("S3Bucket is successfully created for userName: {}, " +
+          "s3BucketName {}, volumeName {}", userName, s3BucketName, volumeName);
+      OMVolumeCreateResponse omVolumeCreateResponse = null;
+      if (volumeCreated) {
+        omMetrics.incNumVolumes();
+        omVolumeCreateResponse = new OMVolumeCreateResponse(omVolumeArgs,
+            volumeList, omResponse.build());
+      }
+
+      omMetrics.incNumBuckets();
+      OMBucketCreateResponse omBucketCreateResponse =
+          new OMBucketCreateResponse(omBucketInfo, omResponse.build());
+      omMetrics.incNumS3Buckets();
+      return new S3BucketCreateResponse(omVolumeCreateResponse,
+          omBucketCreateResponse, s3BucketName,
+          formatS3MappingName(volumeName, s3BucketName),
+          omResponse.setCreateS3BucketResponse(
+              S3CreateBucketResponse.newBuilder()).build());
+    } else {
+      LOG.error("S3Bucket Creation Failed for userName: {}, s3BucketName {}, " +
+          "VolumeName {}", userName, s3BucketName, volumeName);
+      omMetrics.incNumS3BucketCreateFails();
+      return new S3BucketCreateResponse(null, null, null, null,
+          createErrorOMResponse(omResponse, exception));
+    }
+  }
+
+
+  private OmBucketInfo createBucket(OMMetadataManager omMetadataManager,
+      String volumeName, String s3BucketName, long creationTime,
+      long transactionLogIndex) throws IOException {
+    // check if ozone bucket exists, if it does not exist create ozone
+    // bucket
+    boolean acquireBucketLock = false;
+    OmBucketInfo omBucketInfo = null;
+    try {
+      acquireBucketLock =
+          omMetadataManager.getLock().acquireLock(BUCKET_LOCK, volumeName,
+              s3BucketName);
+      String bucketKey = omMetadataManager.getBucketKey(volumeName,
+          s3BucketName);
+      if (!omMetadataManager.getBucketTable().isExist(bucketKey)) {
+        omBucketInfo = createOmBucketInfo(volumeName, s3BucketName,
+            creationTime);
+        // Add to bucket table cache.
+        omMetadataManager.getBucketTable().addCacheEntry(
+            new CacheKey<>(bucketKey),
+            new CacheValue<>(Optional.of(omBucketInfo), transactionLogIndex));
+      } else {
+        // This can happen when a ozone bucket exists already in the
+        // volume, but this is not a s3 bucket.
+        throw new OMException("Bucket " + s3BucketName + " already exists",
+            OMException.ResultCodes.BUCKET_ALREADY_EXISTS);
+      }
+    } finally {
+      if (acquireBucketLock) {
+        omMetadataManager.getLock().releaseLock(BUCKET_LOCK, volumeName,
+            s3BucketName);
+      }
+    }
+    return omBucketInfo;
+  }
+
+  /**
+   * Generate Ozone volume name from userName.
+   * @param userName
+   * @return volume name
+   */
+  @VisibleForTesting
+  public static String formatOzoneVolumeName(String userName) {
+    return String.format(OM_S3_VOLUME_PREFIX + "%s", userName);
+  }
+
+  /**
+   * Generate S3Mapping for provided volume and bucket. This information will
+   * be persisted in s3 table in OM DB.
+   * @param volumeName
+   * @param bucketName
+   * @return s3Mapping
+   */
+  @VisibleForTesting
+  public static String formatS3MappingName(String volumeName,
+      String bucketName) {
+    return String.format("%s" + OzoneConsts.OM_KEY_PREFIX + "%s", volumeName,
+        bucketName);
+  }
+
+  /**
+   * Create {@link OmVolumeArgs} which needs to be persisted in volume table
+   * in OM DB.
+   * @param volumeName
+   * @param userName
+   * @param creationTime
+   * @return {@link OmVolumeArgs}
+   */
+  private OmVolumeArgs createOmVolumeArgs(String volumeName, String userName,
+      long creationTime) {
+    return OmVolumeArgs.newBuilder()
+        .setAdminName(S3_ADMIN_NAME).setVolume(volumeName)
+        .setQuotaInBytes(OzoneConsts.MAX_QUOTA_IN_BYTES)
+        .setOwnerName(userName)
+        .setCreationTime(creationTime).build();
+  }
+
+  /**
+   * Create {@link OmBucketInfo} which needs to be persisted in to bucket table
+   * in OM DB.
+   * @param volumeName
+   * @param s3BucketName
+   * @param creationTime
+   * @return {@link OmBucketInfo}
+   */
+  private OmBucketInfo createOmBucketInfo(String volumeName,
+      String s3BucketName, long creationTime) {
+    //TODO: Now S3Bucket API takes only bucketName as param. In future if we
+    // support some configurable options we need to fix this.
+    return OmBucketInfo.newBuilder().setVolumeName(volumeName)
+        .setBucketName(s3BucketName).setIsVersionEnabled(Boolean.FALSE)
+        .setStorageType(StorageType.DEFAULT).setCreationTime(creationTime)
+        .build();
+  }
+
+  /**
+   * Build auditMap.
+   * @param userName
+   * @param s3BucketName
+   * @return auditMap
+   */
+  private Map<String, String> buildAuditMap(String userName,
+      String s3BucketName) {
+    Map<String, String> auditMap = new HashMap<>();
+    auditMap.put(userName, OzoneConsts.USERNAME);
+    auditMap.put(s3BucketName, OzoneConsts.S3_BUCKET);
+    return auditMap;
+  }
+}
+

+ 23 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/bucket/package-info.java

@@ -0,0 +1,23 @@
+/*
+ * 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 contains classes related to s3 bucket requests.
+ */
+package org.apache.hadoop.ozone.om.request.s3.bucket;

+ 12 - 24
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeCreateRequest.java

@@ -20,7 +20,6 @@ package org.apache.hadoop.ozone.om.request.volume;
 
 import java.io.IOException;
 
-import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -29,14 +28,11 @@ import org.apache.hadoop.ozone.audit.AuditLogger;
 import org.apache.hadoop.ozone.audit.OMAction;
 import org.apache.hadoop.ozone.om.response.volume.OMVolumeCreateResponse;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
-import org.apache.hadoop.utils.db.cache.CacheKey;
-import org.apache.hadoop.utils.db.cache.CacheValue;
 import org.apache.hadoop.ozone.om.OMMetrics;
 import org.apache.hadoop.ozone.om.OMMetadataManager;
 import org.apache.hadoop.ozone.om.OzoneManager;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
 import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
-import org.apache.hadoop.ozone.om.request.OMClientRequest;
 import org.apache.hadoop.ozone.om.response.OMClientResponse;
 import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
 import org.apache.hadoop.ozone.security.acl.OzoneObj;
@@ -60,8 +56,7 @@ import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.USER_LOC
 /**
  * Handles volume create request.
  */
-public class OMVolumeCreateRequest extends OMClientRequest
-    implements OMVolumeRequest {
+public class OMVolumeCreateRequest extends OMVolumeRequest {
   private static final Logger LOG =
       LoggerFactory.getLogger(OMVolumeCreateRequest.class);
 
@@ -132,10 +127,6 @@ public class OMVolumeCreateRequest extends OMClientRequest
           createErrorOMResponse(omResponse, ex));
     }
 
-
-
-    String dbUserKey = omMetadataManager.getUserKey(owner);
-    String dbVolumeKey = omMetadataManager.getVolumeKey(volume);
     VolumeList volumeList = null;
     boolean acquiredUserLock = false;
     IOException exception = null;
@@ -145,28 +136,25 @@ public class OMVolumeCreateRequest extends OMClientRequest
     try {
       acquiredUserLock = omMetadataManager.getLock().acquireLock(USER_LOCK,
           owner);
+      String dbVolumeKey = omMetadataManager.getVolumeKey(volume);
+
       OmVolumeArgs dbVolumeArgs =
           omMetadataManager.getVolumeTable().get(dbVolumeKey);
 
-      // Validation: Check if volume already exists
-      if (dbVolumeArgs != null) {
+      if (dbVolumeArgs == null) {
+        String dbUserKey = omMetadataManager.getUserKey(owner);
+        volumeList = omMetadataManager.getUserTable().get(dbUserKey);
+        volumeList = addVolumeToOwnerList(volumeList, volume, owner,
+            ozoneManager.getMaxUserVolumeCount());
+        createVolume(omMetadataManager, omVolumeArgs, volumeList, dbVolumeKey,
+              dbUserKey, transactionLogIndex);
+        LOG.debug("volume:{} successfully created", omVolumeArgs.getVolume());
+      } else {
         LOG.debug("volume:{} already exists", omVolumeArgs.getVolume());
         throw new OMException("Volume already exists",
             OMException.ResultCodes.VOLUME_ALREADY_EXISTS);
       }
 
-      volumeList = omMetadataManager.getUserTable().get(dbUserKey);
-      volumeList = addVolumeToOwnerList(volumeList,
-          volume, owner, ozoneManager.getMaxUserVolumeCount());
-
-      // Update cache: Update user and volume cache.
-      omMetadataManager.getUserTable().addCacheEntry(new CacheKey<>(dbUserKey),
-          new CacheValue<>(Optional.of(volumeList), transactionLogIndex));
-
-      omMetadataManager.getVolumeTable().addCacheEntry(
-          new CacheKey<>(dbVolumeKey),
-          new CacheValue<>(Optional.of(omVolumeArgs), transactionLogIndex));
-
     } catch (IOException ex) {
       exception = ex;
     } finally {

+ 1 - 3
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeDeleteRequest.java

@@ -35,7 +35,6 @@ import org.apache.hadoop.ozone.om.OMMetrics;
 import org.apache.hadoop.ozone.om.response.volume.OMVolumeCreateResponse;
 import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
 import org.apache.hadoop.ozone.security.acl.OzoneObj;
-import org.apache.hadoop.ozone.om.request.OMClientRequest;
 import org.apache.hadoop.ozone.om.response.OMClientResponse;
 import org.apache.hadoop.ozone.om.response.volume.OMVolumeDeleteResponse;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
@@ -55,8 +54,7 @@ import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.USER_LOC
 /**
  * Handles volume delete request.
  */
-public class OMVolumeDeleteRequest extends OMClientRequest
-    implements OMVolumeRequest {
+public class OMVolumeDeleteRequest extends OMVolumeRequest {
 
   private static final Logger LOG =
       LoggerFactory.getLogger(OMVolumeDeleteRequest.class);

+ 40 - 5
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeRequest.java

@@ -18,9 +18,17 @@
 
 package org.apache.hadoop.ozone.om.request.volume;
 
+import com.google.common.base.Optional;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .OMRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
     .VolumeList;
+import org.apache.hadoop.utils.db.cache.CacheKey;
+import org.apache.hadoop.utils.db.cache.CacheValue;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -29,7 +37,11 @@ import java.util.List;
 /**
  * Defines common methods required for volume requests.
  */
-public interface OMVolumeRequest {
+public abstract class OMVolumeRequest extends OMClientRequest {
+
+  public OMVolumeRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
 
   /**
    * Delete volume from user volume list. This method should be called after
@@ -40,7 +52,7 @@ public interface OMVolumeRequest {
    * @return VolumeList - updated volume list for the user.
    * @throws IOException
    */
-  default VolumeList delVolumeFromOwnerList(VolumeList volumeList,
+  protected VolumeList delVolumeFromOwnerList(VolumeList volumeList,
       String volume, String owner) throws IOException {
 
     List<String> prevVolList = new ArrayList<>();
@@ -72,9 +84,8 @@ public interface OMVolumeRequest {
    * @throws OMException - if user has volumes greater than
    * maxUserVolumeCount, an exception is thrown.
    */
-  default VolumeList addVolumeToOwnerList(
-      VolumeList volumeList, String volume, String owner,
-      long maxUserVolumeCount) throws IOException {
+  protected VolumeList addVolumeToOwnerList(VolumeList volumeList,
+      String volume, String owner, long maxUserVolumeCount) throws IOException {
 
     // Check the volume count
     if (volumeList != null &&
@@ -95,4 +106,28 @@ public interface OMVolumeRequest {
 
     return newVolList;
   }
+
+  /**
+   * Create Ozone Volume. This method should be called after acquiring user
+   * and volume Lock.
+   * @param omMetadataManager
+   * @param omVolumeArgs
+   * @param volumeList
+   * @param dbVolumeKey
+   * @param dbUserKey
+   * @param transactionLogIndex
+   * @throws IOException
+   */
+  protected void createVolume(final OMMetadataManager omMetadataManager,
+      OmVolumeArgs omVolumeArgs, VolumeList volumeList, String dbVolumeKey,
+      String dbUserKey, long transactionLogIndex) {
+    // Update cache: Update user and volume cache.
+    omMetadataManager.getUserTable().addCacheEntry(new CacheKey<>(dbUserKey),
+        new CacheValue<>(Optional.of(volumeList), transactionLogIndex));
+
+    omMetadataManager.getVolumeTable().addCacheEntry(
+        new CacheKey<>(dbVolumeKey),
+        new CacheValue<>(Optional.of(omVolumeArgs), transactionLogIndex));
+  }
+
 }

+ 1 - 3
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeSetOwnerRequest.java

@@ -35,7 +35,6 @@ import org.apache.hadoop.ozone.om.OMMetrics;
 import org.apache.hadoop.ozone.om.OzoneManager;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
 import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
-import org.apache.hadoop.ozone.om.request.OMClientRequest;
 import org.apache.hadoop.ozone.om.response.OMClientResponse;
 import org.apache.hadoop.ozone.om.response.volume.OMVolumeCreateResponse;
 import org.apache.hadoop.ozone.om.response.volume.OMVolumeSetOwnerResponse;
@@ -58,8 +57,7 @@ import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.VOLUME_L
 /**
  * Handle set owner request for volume.
  */
-public class OMVolumeSetOwnerRequest extends OMClientRequest
-    implements OMVolumeRequest {
+public class OMVolumeSetOwnerRequest extends OMVolumeRequest {
   private static final Logger LOG =
       LoggerFactory.getLogger(OMVolumeSetOwnerRequest.class);
 

+ 1 - 2
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeSetQuotaRequest.java

@@ -34,7 +34,6 @@ import org.apache.hadoop.ozone.om.OzoneManager;
 import org.apache.hadoop.ozone.om.OMMetrics;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
 import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
-import org.apache.hadoop.ozone.om.request.OMClientRequest;
 import org.apache.hadoop.ozone.om.response.OMClientResponse;
 import org.apache.hadoop.ozone.om.response.volume.OMVolumeSetQuotaResponse;
 import org.apache.hadoop.ozone.om.response.volume.OMVolumeCreateResponse;
@@ -58,7 +57,7 @@ import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.VOLUME_L
 /**
  * Handles set Quota request for volume.
  */
-public class OMVolumeSetQuotaRequest extends OMClientRequest {
+public class OMVolumeSetQuotaRequest extends OMVolumeRequest {
   private static final Logger LOG =
       LoggerFactory.getLogger(OMVolumeSetQuotaRequest.class);
 

+ 73 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/bucket/S3BucketCreateResponse.java

@@ -0,0 +1,73 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.response.s3.bucket;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import java.io.IOException;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.bucket.OMBucketCreateResponse;
+import org.apache.hadoop.ozone.om.response.volume.OMVolumeCreateResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .OMResponse;
+import org.apache.hadoop.utils.db.BatchOperation;
+
+/**
+ * Response for S3Bucket create request.
+ */
+public class S3BucketCreateResponse extends OMClientResponse {
+
+  private OMVolumeCreateResponse omVolumeCreateResponse;
+  private OMBucketCreateResponse omBucketCreateResponse;
+  private String s3Bucket;
+  private String s3Mapping;
+
+  public S3BucketCreateResponse(
+      @Nullable OMVolumeCreateResponse omVolumeCreateResponse,
+      @Nullable OMBucketCreateResponse omBucketCreateResponse,
+      @Nullable String s3BucketName,
+      @Nullable String s3Mapping, @Nonnull OMResponse omResponse) {
+    super(omResponse);
+    this.omVolumeCreateResponse = omVolumeCreateResponse;
+    this.omBucketCreateResponse = omBucketCreateResponse;
+    this.s3Bucket = s3BucketName;
+    this.s3Mapping = s3Mapping;
+  }
+
+  @Override
+  public void addToDBBatch(OMMetadataManager omMetadataManager,
+      BatchOperation batchOperation) throws IOException {
+
+    if (getOMResponse().getStatus() == OzoneManagerProtocolProtos.Status.OK) {
+      if (omVolumeCreateResponse != null) {
+        omVolumeCreateResponse.addToDBBatch(omMetadataManager, batchOperation);
+      }
+
+      Preconditions.checkState(omBucketCreateResponse != null);
+      omBucketCreateResponse.addToDBBatch(omMetadataManager, batchOperation);
+
+      omMetadataManager.getS3Table().putWithBatch(batchOperation, s3Bucket,
+          s3Mapping);
+    }
+  }
+}
+

+ 24 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/bucket/package-info.java

@@ -0,0 +1,24 @@
+/*
+ * 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 contains classes related to s3 bucket responses.
+ */
+package org.apache.hadoop.ozone.om.response.s3.bucket;
+

+ 1 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerHARequestHandlerImpl.java

@@ -67,6 +67,7 @@ public class OzoneManagerHARequestHandlerImpl
     case CreateDirectory:
     case CreateFile:
     case PurgeKeys:
+    case CreateS3Bucket:
       //TODO: We don't need to pass transactionID, this will be removed when
       // complete write requests is changed to new model. And also we can
       // return OMClientResponse, then adding to doubleBuffer can be taken

+ 20 - 0
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/TestOMRequestUtils.java

@@ -33,6 +33,7 @@ import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
 import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.request.s3.bucket.S3BucketCreateRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
     .OMRequest;
@@ -146,6 +147,12 @@ public final class TestOMRequestUtils {
     addVolumeToDB(volumeName, UUID.randomUUID().toString(), omMetadataManager);
   }
 
+  public static void addS3BucketToDB(String volumeName, String s3BucketName,
+      OMMetadataManager omMetadataManager) throws Exception {
+    omMetadataManager.getS3Table().put(s3BucketName,
+        S3BucketCreateRequest.formatS3MappingName(volumeName, s3BucketName));
+  }
+
   /**
    * Add volume creation entry to OM DB.
    * @param volumeName
@@ -183,6 +190,19 @@ public final class TestOMRequestUtils {
         .setClientId(UUID.randomUUID().toString()).build();
   }
 
+  public static OzoneManagerProtocolProtos.OMRequest createS3BucketRequest(
+      String userName, String s3BucketName) {
+    OzoneManagerProtocolProtos.S3CreateBucketRequest request =
+        OzoneManagerProtocolProtos.S3CreateBucketRequest.newBuilder()
+            .setUserName(userName)
+            .setS3Bucketname(s3BucketName).build();
+
+    return OzoneManagerProtocolProtos.OMRequest.newBuilder()
+        .setCreateS3BucketRequest(request)
+        .setCmdType(OzoneManagerProtocolProtos.Type.CreateS3Bucket)
+        .setClientId(UUID.randomUUID().toString()).build();
+  }
+
   public static List< HddsProtos.KeyValue> getMetadataList() {
     List<HddsProtos.KeyValue> metadataList = new ArrayList<>();
     metadataList.add(HddsProtos.KeyValue.newBuilder().setKey("key1").setValue(

+ 246 - 0
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/s3/bucket/TestS3BucketCreateRequest.java

@@ -0,0 +1,246 @@
+/*
+ * 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.request.s3.bucket;
+
+import java.util.UUID;
+
+import org.apache.commons.lang.RandomStringUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.mockito.Mockito;
+
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.audit.AuditLogger;
+import org.apache.hadoop.ozone.audit.AuditMessage;
+import org.apache.hadoop.ozone.om.OMConfigKeys;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
+    .OMResponse;
+import org.apache.hadoop.ozone.om.request.TestOMRequestUtils;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+
+import static org.junit.Assert.fail;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests S3BucketCreateRequest class, which handles S3 CreateBucket request.
+ */
+public class TestS3BucketCreateRequest {
+
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+
+  private OzoneManager ozoneManager;
+  private OMMetrics omMetrics;
+  private OMMetadataManager omMetadataManager;
+  private AuditLogger auditLogger;
+
+
+  @Before
+  public void setup() throws Exception {
+
+    ozoneManager = Mockito.mock(OzoneManager.class);
+    omMetrics = OMMetrics.create();
+    OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
+    ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS,
+        folder.newFolder().getAbsolutePath());
+    omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration);
+    when(ozoneManager.getMetrics()).thenReturn(omMetrics);
+    when(ozoneManager.getMetadataManager()).thenReturn(omMetadataManager);
+    auditLogger = Mockito.mock(AuditLogger.class);
+    when(ozoneManager.getAuditLogger()).thenReturn(auditLogger);
+    Mockito.doNothing().when(auditLogger).logWrite(any(AuditMessage.class));
+  }
+
+  @After
+  public void stop() {
+    omMetrics.unRegister();
+    Mockito.framework().clearInlineMocks();
+  }
+
+
+  @Test
+  public void testPreExecute() throws Exception {
+    String userName = UUID.randomUUID().toString();
+    String s3BucketName = UUID.randomUUID().toString();
+    doPreExecute(userName, s3BucketName);
+  }
+
+  @Test
+  public void testPreExecuteInvalidBucketLength() throws Exception {
+    String userName = UUID.randomUUID().toString();
+
+    // set bucket name which is less than 3 characters length
+    String s3BucketName = RandomStringUtils.randomAlphabetic(2);
+
+    try {
+      doPreExecute(userName, s3BucketName);
+      fail("testPreExecuteInvalidBucketLength failed");
+    } catch (OMException ex) {
+      GenericTestUtils.assertExceptionContains("S3_BUCKET_INVALID_LENGTH", ex);
+    }
+
+    // set bucket name which is greater than 63 characters length
+    s3BucketName = RandomStringUtils.randomAlphabetic(64);
+
+    try {
+      doPreExecute(userName, s3BucketName);
+      fail("testPreExecuteInvalidBucketLength failed");
+    } catch (OMException ex) {
+      GenericTestUtils.assertExceptionContains("S3_BUCKET_INVALID_LENGTH", ex);
+    }
+  }
+
+
+  @Test
+  public void testValidateAndUpdateCache() throws Exception {
+    String userName = UUID.randomUUID().toString();
+    String s3BucketName = UUID.randomUUID().toString();
+
+    S3BucketCreateRequest s3BucketCreateRequest = doPreExecute(userName,
+        s3BucketName);
+
+    doValidateAndUpdateCache(userName, s3BucketName,
+        s3BucketCreateRequest.getOmRequest());
+
+  }
+
+
+  @Test
+  public void testValidateAndUpdateCacheWithS3BucketAlreadyExists()
+      throws Exception {
+    String userName = UUID.randomUUID().toString();
+    String s3BucketName = UUID.randomUUID().toString();
+
+    TestOMRequestUtils.addS3BucketToDB(
+        S3BucketCreateRequest.formatOzoneVolumeName(userName), s3BucketName,
+        omMetadataManager);
+
+    S3BucketCreateRequest s3BucketCreateRequest =
+        doPreExecute(userName, s3BucketName);
+
+
+    // Try create same bucket again
+    OMClientResponse omClientResponse =
+        s3BucketCreateRequest.validateAndUpdateCache(ozoneManager, 2);
+
+    OMResponse omResponse = omClientResponse.getOMResponse();
+    Assert.assertNotNull(omResponse.getCreateBucketResponse());
+    Assert.assertEquals(
+        OzoneManagerProtocolProtos.Status.S3_BUCKET_ALREADY_EXISTS,
+        omResponse.getStatus());
+  }
+
+  @Test
+  public void testValidateAndUpdateCacheWithBucketAlreadyExists()
+      throws Exception {
+    String userName = UUID.randomUUID().toString();
+    String s3BucketName = UUID.randomUUID().toString();
+
+    S3BucketCreateRequest s3BucketCreateRequest =
+        doPreExecute(userName, s3BucketName);
+
+    TestOMRequestUtils.addVolumeAndBucketToDB(
+        s3BucketCreateRequest.formatOzoneVolumeName(userName),
+        s3BucketName, omMetadataManager);
+
+
+    // Try create same bucket again
+    OMClientResponse omClientResponse =
+        s3BucketCreateRequest.validateAndUpdateCache(ozoneManager, 2);
+
+    OMResponse omResponse = omClientResponse.getOMResponse();
+    Assert.assertNotNull(omResponse.getCreateBucketResponse());
+    Assert.assertEquals(OzoneManagerProtocolProtos.Status.BUCKET_ALREADY_EXISTS,
+        omResponse.getStatus());
+  }
+
+
+
+  private S3BucketCreateRequest doPreExecute(String userName,
+      String s3BucketName) throws Exception {
+    OMRequest originalRequest =
+        TestOMRequestUtils.createS3BucketRequest(userName, s3BucketName);
+
+    S3BucketCreateRequest s3BucketCreateRequest =
+        new S3BucketCreateRequest(originalRequest);
+
+    OMRequest modifiedRequest = s3BucketCreateRequest.preExecute(ozoneManager);
+    // Modification time will be set, so requests should not be equal.
+    Assert.assertNotEquals(originalRequest, modifiedRequest);
+    return new S3BucketCreateRequest(modifiedRequest);
+  }
+
+  private void doValidateAndUpdateCache(String userName, String s3BucketName,
+      OMRequest modifiedRequest) throws Exception {
+
+    // As we have not still called validateAndUpdateCache, get() should
+    // return null.
+
+    Assert.assertNull(omMetadataManager.getS3Table().get(s3BucketName));
+    S3BucketCreateRequest s3BucketCreateRequest =
+        new S3BucketCreateRequest(modifiedRequest);
+
+
+    OMClientResponse omClientResponse =
+        s3BucketCreateRequest.validateAndUpdateCache(ozoneManager, 1);
+
+    // As now after validateAndUpdateCache it should add entry to cache, get
+    // should return non null value.
+
+    Assert.assertNotNull(omMetadataManager.getS3Table().get(s3BucketName));
+
+    String bucketKey =
+        omMetadataManager.getBucketKey(
+            s3BucketCreateRequest.formatOzoneVolumeName(userName),
+            s3BucketName);
+
+    // check ozone bucket entry is created or not.
+    Assert.assertNotNull(omMetadataManager.getBucketTable().get(bucketKey));
+
+    String volumeKey = omMetadataManager.getVolumeKey(
+        s3BucketCreateRequest.formatOzoneVolumeName(userName));
+
+    // Check volume entry is created or not.
+    Assert.assertNotNull(omMetadataManager.getVolumeTable().get(volumeKey));
+
+    // check om response.
+    Assert.assertEquals(OzoneManagerProtocolProtos.Status.OK,
+        omClientResponse.getOMResponse().getStatus());
+    Assert.assertEquals(OzoneManagerProtocolProtos.Type.CreateS3Bucket,
+        omClientResponse.getOMResponse().getCmdType());
+
+  }
+
+}
+

+ 23 - 0
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/s3/bucket/package-info.java

@@ -0,0 +1,23 @@
+/*
+ * 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 contains test classes for s3 bucket requests.
+ */
+package org.apache.hadoop.ozone.om.request.s3.bucket;

+ 25 - 2
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/volume/TestOMVolumeCreateRequest.java

@@ -133,7 +133,7 @@ public class TestOMVolumeCreateRequest {
     OMVolumeCreateRequest omVolumeCreateRequest =
         new OMVolumeCreateRequest(originalRequest);
 
-    omVolumeCreateRequest.preExecute(ozoneManager);
+    OMRequest modifiedRequest = omVolumeCreateRequest.preExecute(ozoneManager);
 
     String volumeKey = omMetadataManager.getVolumeKey(volumeName);
     String ownerKey = omMetadataManager.getUserKey(ownerName);
@@ -144,6 +144,8 @@ public class TestOMVolumeCreateRequest {
     Assert.assertNull(omMetadataManager.getVolumeTable().get(volumeKey));
     Assert.assertNull(omMetadataManager.getUserTable().get(ownerKey));
 
+    omVolumeCreateRequest = new OMVolumeCreateRequest(modifiedRequest);
+
     OMClientResponse omClientResponse =
         omVolumeCreateRequest.validateAndUpdateCache(ozoneManager, 1);
 
@@ -175,6 +177,25 @@ public class TestOMVolumeCreateRequest {
     Assert.assertNotNull(volumeList);
     Assert.assertEquals(volumeName, volumeList.getVolumeNames(0));
 
+    // Create another volume for the user.
+    originalRequest = createVolumeRequest("vol1", adminName,
+        ownerName);
+
+    omVolumeCreateRequest =
+        new OMVolumeCreateRequest(originalRequest);
+
+    modifiedRequest = omVolumeCreateRequest.preExecute(ozoneManager);
+
+    omClientResponse =
+        omVolumeCreateRequest.validateAndUpdateCache(ozoneManager, 2L);
+
+    Assert.assertEquals(OzoneManagerProtocolProtos.Status.OK,
+        omClientResponse.getOMResponse().getStatus());
+
+    Assert.assertTrue(omMetadataManager
+        .getUserTable().get(ownerKey).getVolumeNamesList().size() == 2);
+
+
   }
 
 
@@ -193,7 +214,9 @@ public class TestOMVolumeCreateRequest {
     OMVolumeCreateRequest omVolumeCreateRequest =
         new OMVolumeCreateRequest(originalRequest);
 
-    omVolumeCreateRequest.preExecute(ozoneManager);
+    OMRequest modifiedRequest = omVolumeCreateRequest.preExecute(ozoneManager);
+
+    omVolumeCreateRequest = new OMVolumeCreateRequest(modifiedRequest);
 
     OMClientResponse omClientResponse =
         omVolumeCreateRequest.validateAndUpdateCache(ozoneManager, 1);

+ 119 - 0
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/s3/bucket/TestS3BucketCreateResponse.java

@@ -0,0 +1,119 @@
+/*
+ * 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.response.s3.bucket;
+
+import java.util.UUID;
+
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.om.OMConfigKeys;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.request.s3.bucket.S3BucketCreateRequest;
+import org.apache.hadoop.ozone.om.response.TestOMResponseUtils;
+import org.apache.hadoop.ozone.om.response.bucket.OMBucketCreateResponse;
+import org.apache.hadoop.ozone.om.response.volume.OMVolumeCreateResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.util.Time;
+import org.apache.hadoop.utils.db.BatchOperation;
+
+/**
+ * Class to test S3BucketCreateResponse.
+ */
+public class TestS3BucketCreateResponse {
+
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+
+  private OMMetadataManager omMetadataManager;
+  private BatchOperation batchOperation;
+
+  @Before
+  public void setup() throws Exception {
+    OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
+    ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS,
+        folder.newFolder().getAbsolutePath());
+    omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration);
+    batchOperation = omMetadataManager.getStore().initBatchOperation();
+  }
+
+
+  @Test
+  public void testAddToDBBatch() throws Exception {
+    String userName = UUID.randomUUID().toString();
+    String s3BucketName = UUID.randomUUID().toString();
+
+    OzoneManagerProtocolProtos.OMResponse omResponse =
+        OzoneManagerProtocolProtos.OMResponse.newBuilder()
+            .setCmdType(OzoneManagerProtocolProtos.Type.CreateS3Bucket)
+            .setStatus(OzoneManagerProtocolProtos.Status.OK)
+            .setSuccess(true)
+            .setCreateS3BucketResponse(
+                OzoneManagerProtocolProtos.S3CreateBucketResponse
+                    .getDefaultInstance())
+            .build();
+
+    String volumeName = S3BucketCreateRequest.formatOzoneVolumeName(userName);
+    OzoneManagerProtocolProtos.VolumeList volumeList =
+        OzoneManagerProtocolProtos.VolumeList.newBuilder()
+            .addVolumeNames(volumeName).build();
+
+    OmVolumeArgs omVolumeArgs = OmVolumeArgs.newBuilder()
+        .setOwnerName(userName).setAdminName(userName)
+        .setVolume(volumeName).setCreationTime(Time.now()).build();
+
+    OMVolumeCreateResponse omVolumeCreateResponse =
+        new OMVolumeCreateResponse(omVolumeArgs, volumeList, omResponse);
+
+
+    OmBucketInfo omBucketInfo = TestOMResponseUtils.createBucket(
+        volumeName, s3BucketName);
+    OMBucketCreateResponse omBucketCreateResponse =
+        new OMBucketCreateResponse(omBucketInfo, omResponse);
+
+    String s3Mapping = S3BucketCreateRequest.formatS3MappingName(volumeName,
+        s3BucketName);
+    S3BucketCreateResponse s3BucketCreateResponse =
+        new S3BucketCreateResponse(omVolumeCreateResponse,
+            omBucketCreateResponse, s3BucketName, s3Mapping, omResponse);
+
+    s3BucketCreateResponse.addToDBBatch(omMetadataManager, batchOperation);
+
+    // Do manual commit and see whether addToBatch is successful or not.
+    omMetadataManager.getStore().commitBatchOperation(batchOperation);
+
+    Assert.assertNotNull(omMetadataManager.getS3Table().get(s3BucketName));
+    Assert.assertEquals(s3Mapping,
+        omMetadataManager.getS3Table().get(s3BucketName));
+    Assert.assertNotNull(omMetadataManager.getVolumeTable().get(
+        omMetadataManager.getVolumeKey(volumeName)));
+    Assert.assertNotNull(omMetadataManager.getBucketTable().get(
+        omMetadataManager.getBucketKey(volumeName, s3BucketName)));
+
+  }
+}
+

+ 23 - 0
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/s3/bucket/package-info.java

@@ -0,0 +1,23 @@
+/*
+ * 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 contains test classes for s3 bucket responses.
+ */
+package org.apache.hadoop.ozone.om.response.s3.bucket;