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

sammichen pushed a commit to branch HDDS-8342
in repository https://gitbox.apache.org/repos/asf/ozone.git


The following commit(s) were added to refs/heads/HDDS-8342 by this push:
     new 59a2b7e77d3 HDDS-12783. Implement S3Gateway api for 
LifeCycleConfiguration (#8557)
59a2b7e77d3 is described below

commit 59a2b7e77d32ac1ad7cf5fb0802c6023c27c6c1c
Author: XiChen <[email protected]>
AuthorDate: Thu Jun 26 16:34:49 2025 +0800

    HDDS-12783. Implement S3Gateway api for LifeCycleConfiguration (#8557)
    
    Co-contributed by Mohanad Elsafty ([email protected])
---
 .../apache/hadoop/ozone/client/OzoneBucket.java    |  34 ++
 .../ozone/client/OzoneLifecycleConfiguration.java  | 167 +++++++
 .../ozone/client/protocol/ClientProtocol.java      |  29 ++
 .../apache/hadoop/ozone/client/rpc/RpcClient.java  |  18 +
 .../hadoop/ozone/om/exceptions/OMException.java    |   3 +-
 .../apache/hadoop/ozone/om/helpers/OmLCRule.java   |   6 +
 .../hadoop/ozone/om/helpers/TestOmLCRule.java      |  16 +-
 .../om/helpers/TestOmLifeCycleConfiguration.java   |   5 +
 .../src/main/proto/OmClientProtocol.proto          |   1 +
 .../org/apache/hadoop/ozone/audit/S3GAction.java   |   3 +
 .../hadoop/ozone/s3/endpoint/BucketEndpoint.java   | 112 ++++-
 ...utBucketLifecycleConfigurationUnmarshaller.java |  80 ++++
 .../s3/endpoint/S3LifecycleConfiguration.java      | 484 ++++++++++++++++++++
 .../hadoop/ozone/s3/exception/S3ErrorTable.java    |   4 +
 .../hadoop/ozone/client/ClientProtocolStub.java    |  18 +
 .../hadoop/ozone/client/OzoneBucketStub.java       |  64 +++
 .../hadoop/ozone/client/OzoneVolumeStub.java       |   1 +
 .../hadoop/ozone/s3/endpoint/TestBucketAcl.java    |  24 +-
 .../hadoop/ozone/s3/endpoint/TestBucketDelete.java |   6 +-
 .../hadoop/ozone/s3/endpoint/TestBucketList.java   |  46 +-
 .../hadoop/ozone/s3/endpoint/TestBucketPut.java    |   8 +-
 .../ozone/s3/endpoint/TestPermissionCheck.java     |  10 +-
 ...ava => TestS3LifecycleConfigurationDelete.java} |  73 +--
 .../endpoint/TestS3LifecycleConfigurationGet.java  | 100 ++++
 .../endpoint/TestS3LifecycleConfigurationPut.java  | 502 +++++++++++++++++++++
 .../ozone/s3/metrics/TestS3GatewayMetrics.java     |  22 +-
 26 files changed, 1739 insertions(+), 97 deletions(-)

diff --git 
a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
 
b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
index 057ae61c10a..29eb9377e97 100644
--- 
a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
+++ 
b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
@@ -56,6 +56,7 @@
 import org.apache.hadoop.ozone.om.helpers.BucketLayout;
 import org.apache.hadoop.ozone.om.helpers.ErrorInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmLifecycleConfiguration;
 import org.apache.hadoop.ozone.om.helpers.OmMultipartInfo;
 import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadCompleteInfo;
 import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
@@ -1091,6 +1092,39 @@ public void deleteObjectTagging(String keyName) throws 
IOException {
     proxy.deleteObjectTagging(volumeName, name, keyName);
   }
 
+  /**
+   * Gets the lifecycle configuration information.
+   * @return OzoneLifecycleConfiguration or exception is thrown.
+   * @throws IOException
+   */
+  @JsonIgnore
+  public OzoneLifecycleConfiguration getLifecycleConfiguration()
+      throws IOException {
+    return proxy.getLifecycleConfiguration(volumeName, name);
+  }
+
+  /**
+   * Sets the lifecycle configuration for this bucket.
+   * This operation will completely overwrite any existing lifecycle 
configuration on the bucket.
+   * If the bucket already has a lifecycle configuration, it will be replaced 
with the new one.
+   * 
+   * @param lifecycleConfiguration - lifecycle configuration info to be set.
+   * @throws IOException if there is an error setting the lifecycle 
configuration.
+   */
+  public void setLifecycleConfiguration(OmLifecycleConfiguration 
lifecycleConfiguration)
+      throws IOException {
+    proxy.createLifecycleConfiguration(lifecycleConfiguration);
+  }
+
+  /**
+   * Deletes existing lifecycle configuration.
+   * @throws IOException
+   */
+  public void deleteLifecycleConfiguration()
+      throws IOException {
+    proxy.deleteLifecycleConfiguration(volumeName, name);
+  }
+
   public void setSourcePathExist(boolean b) {
     this.sourcePathExist = b;
   }
diff --git 
a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneLifecycleConfiguration.java
 
b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneLifecycleConfiguration.java
new file mode 100644
index 00000000000..5ab47f8524f
--- /dev/null
+++ 
b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneLifecycleConfiguration.java
@@ -0,0 +1,167 @@
+/*
+ * 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.client;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.commons.lang3.tuple.Pair;
+
+/**
+ * A class that encapsulates OzoneLifecycleConfiguration.
+ */
+public class OzoneLifecycleConfiguration {
+  private final String volume;
+  private final String bucket;
+  private final long creationTime;
+  private final List<OzoneLCRule> rules;
+
+  public OzoneLifecycleConfiguration(String volume, String bucket,
+                                     long creationTime, List<OzoneLCRule> 
rules) {
+    this.volume = volume;
+    this.bucket = bucket;
+    this.creationTime = creationTime;
+    this.rules = rules;
+  }
+
+  /**
+   * A class that encapsulates OzoneLCExpiration.
+   */
+  public static class OzoneLCExpiration {
+    private final Integer days;
+    private final String date;
+
+    public OzoneLCExpiration(Integer days, String date) {
+      this.days = days;
+      this.date = date;
+    }
+
+    public String getDate() {
+      return date;
+    }
+
+    public Integer getDays() {
+      return days;
+    }
+  }
+
+  /**
+   * A class that encapsulates {@link 
org.apache.hadoop.ozone.om.helpers.OmLifecycleRuleAndOperator}.
+   */
+  public static final class LifecycleAndOperator {
+    private final Map<String, String> tags;
+    private final String prefix;
+
+    public LifecycleAndOperator(Map<String, String> tags, String prefix) {
+      this.tags = tags;
+      this.prefix = prefix;
+    }
+
+    public Map<String, String> getTags() {
+      return tags;
+    }
+
+    public String getPrefix() {
+      return prefix;
+    }
+
+  }
+
+  /**
+   * A class that encapsulates OzoneLCFilter.
+   */
+  public static final class OzoneLCFilter {
+    private final String prefix;
+    private final Pair<String, String> tag;
+    private final LifecycleAndOperator andOperator;
+
+    public OzoneLCFilter(String prefix, Pair<String, String> tag,
+                         LifecycleAndOperator andOperator) {
+      this.prefix = prefix;
+      this.tag = tag;
+      this.andOperator = andOperator;
+    }
+
+    public String getPrefix() {
+      return prefix;
+    }
+
+    public Pair<String, String> getTag() {
+      return tag;
+    }
+
+    public LifecycleAndOperator getAndOperator() {
+      return andOperator;
+    }
+  }
+
+  /**
+   * A class that encapsulates a lifecycle configuration rule.
+   */
+  public static class OzoneLCRule {
+    private final String id;
+    private final String prefix;
+    private final String status;
+    private final OzoneLCExpiration expiration;
+    private final OzoneLCFilter filter;
+
+    public OzoneLCRule(String id, String prefix, String status,
+                       OzoneLCExpiration expiration, OzoneLCFilter filter) {
+      this.id = id;
+      this.prefix = prefix;
+      this.status = status;
+      this.expiration = expiration;
+      this.filter = filter;
+    }
+
+    public String getId() {
+      return id;
+    }
+
+    public String getPrefix() {
+      return prefix;
+    }
+
+    public String getStatus() {
+      return status;
+    }
+
+    public OzoneLCExpiration getExpiration() {
+      return expiration;
+    }
+
+    public OzoneLCFilter getFilter() {
+      return filter;
+    }
+  }
+
+  public String getVolume() {
+    return volume;
+  }
+
+  public String getBucket() {
+    return bucket;
+  }
+
+  public long getCreationTime() {
+    return creationTime;
+  }
+
+  public List<OzoneLCRule> getRules() {
+    return rules;
+  }
+}
diff --git 
a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
 
b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
index 7ef2c38eb32..c8f52a466b8 100644
--- 
a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
+++ 
b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
@@ -35,6 +35,7 @@
 import org.apache.hadoop.ozone.client.OzoneBucket;
 import org.apache.hadoop.ozone.client.OzoneKey;
 import org.apache.hadoop.ozone.client.OzoneKeyDetails;
+import org.apache.hadoop.ozone.client.OzoneLifecycleConfiguration;
 import org.apache.hadoop.ozone.client.OzoneMultipartUploadList;
 import org.apache.hadoop.ozone.client.OzoneMultipartUploadPartListParts;
 import org.apache.hadoop.ozone.client.OzoneSnapshot;
@@ -52,6 +53,7 @@
 import org.apache.hadoop.ozone.om.helpers.LeaseKeyInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmLifecycleConfiguration;
 import org.apache.hadoop.ozone.om.helpers.OmMultipartInfo;
 import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadCompleteInfo;
 import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
@@ -1343,4 +1345,31 @@ void putObjectTagging(String volumeName, String 
bucketName, String keyName,
   void deleteObjectTagging(String volumeName, String bucketName, String 
keyName)
       throws IOException;
 
+  /**
+   * Gets the lifecycle configuration information.
+   * @param volumeName - Volume name.
+   * @param bucketName - Bucket name.
+   * @return OzoneLifecycleConfiguration or exception is thrown.
+   * @throws IOException
+   */
+  OzoneLifecycleConfiguration getLifecycleConfiguration(String volumeName, 
String bucketName)
+      throws IOException;
+
+  /**
+   * Creates a new lifecycle configuration.
+   * @param lifecycleConfiguration - lifecycle configuration info.
+   * @throws IOException
+   */
+  void createLifecycleConfiguration(OmLifecycleConfiguration 
lifecycleConfiguration)
+      throws IOException;
+
+  /**
+   * Deletes existing lifecycle configuration.
+   * @param volumeName - Volume name.
+   * @param bucketName - Bucket name.
+   * @throws IOException
+   */
+  void deleteLifecycleConfiguration(String volumeName, String bucketName)
+      throws IOException;
+
 }
diff --git 
a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
 
b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
index 4c0e99a4469..f81d6ad2e4d 100644
--- 
a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
+++ 
b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
@@ -101,6 +101,7 @@
 import org.apache.hadoop.ozone.client.OzoneKey;
 import org.apache.hadoop.ozone.client.OzoneKeyDetails;
 import org.apache.hadoop.ozone.client.OzoneKeyLocation;
+import org.apache.hadoop.ozone.client.OzoneLifecycleConfiguration;
 import org.apache.hadoop.ozone.client.OzoneMultipartUpload;
 import org.apache.hadoop.ozone.client.OzoneMultipartUploadList;
 import org.apache.hadoop.ozone.client.OzoneMultipartUploadPartListParts;
@@ -139,6 +140,7 @@
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.apache.hadoop.ozone.om.helpers.OmLifecycleConfiguration;
 import org.apache.hadoop.ozone.om.helpers.OmMultipartInfo;
 import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadCompleteInfo;
 import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadCompleteList;
@@ -2785,6 +2787,22 @@ public void deleteObjectTagging(String volumeName, 
String bucketName,
     ozoneManagerClient.deleteObjectTagging(keyArgs);
   }
 
+  @Override
+  public OzoneLifecycleConfiguration getLifecycleConfiguration(String 
volumeName, String bucketName)
+      throws IOException {
+    throw new UnsupportedOperationException("Lifecycle configuration 
operations are not yet supported");
+  }
+
+  @Override
+  public void createLifecycleConfiguration(OmLifecycleConfiguration 
lifecycleConfiguration) throws IOException {
+    throw new UnsupportedOperationException("Lifecycle configuration 
operations are not yet supported");
+  }
+
+  @Override
+  public void deleteLifecycleConfiguration(String volumeName, String 
bucketName) throws IOException {
+    throw new UnsupportedOperationException("Lifecycle configuration 
operations are not yet supported");
+  }
+
   private static ExecutorService createThreadPoolExecutor(
        int corePoolSize, int maximumPoolSize, String threadNameFormat) {
     return new ThreadPoolExecutor(corePoolSize, maximumPoolSize,
diff --git 
a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java
 
b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java
index cc540c15e3b..4d25c28f146 100644
--- 
a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java
+++ 
b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java
@@ -271,6 +271,7 @@ public enum ResultCodes {
     TOO_MANY_BUCKETS,
     KEY_UNDER_LEASE_RECOVERY,
     KEY_ALREADY_CLOSED,
-    KEY_UNDER_LEASE_SOFT_LIMIT_PERIOD
+    KEY_UNDER_LEASE_SOFT_LIMIT_PERIOD,
+    LIFECYCLE_CONFIGURATION_NOT_FOUND
   }
 }
diff --git 
a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmLCRule.java
 
b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmLCRule.java
index e6f3af4b1de..1f99a07dee0 100644
--- 
a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmLCRule.java
+++ 
b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmLCRule.java
@@ -125,6 +125,7 @@ public boolean isTagEnable() {
    * - ID length should not exceed the allowed limit
    * - At least one action must be specified
    * - Filter and Prefix cannot be used together
+   * - Filter and prefix cannot both be null
    * - Prefix can be "", in which case the rule applies to all objects in the 
bucket.
    * - Actions must be valid
    * - Filter must be valid
@@ -161,6 +162,11 @@ public void valid() throws OMException {
           OMException.ResultCodes.INVALID_REQUEST);
     }
 
+    if (prefix == null && filter == null) {
+      throw new OMException("Filter and Prefix cannot both be null.",
+          OMException.ResultCodes.INVALID_REQUEST);
+    }
+
     if (filter != null) {
       filter.valid();
     }
diff --git 
a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmLCRule.java
 
b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmLCRule.java
index 287537bb440..56c543523fe 100644
--- 
a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmLCRule.java
+++ 
b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmLCRule.java
@@ -62,14 +62,9 @@ public void testCreateValidOmLCRule() throws OMException {
         .setEnabled(true)
         .setPrefix("")
         .setAction(exp);
-    assertDoesNotThrow(r2::build);
+    OmLCRule omLCRule = assertDoesNotThrow(r2::build);
 
     // Empty id should generate a 48 (default) bit one.
-    OmLCRule.Builder r3 = new OmLCRule.Builder()
-        .setEnabled(true)
-        .setAction(exp);
-
-    OmLCRule omLCRule = assertDoesNotThrow(r3::build);
     assertEquals(OmLCRule.LC_ID_LENGTH, omLCRule.getId().length(),
         "Expected a " + OmLCRule.LC_ID_LENGTH + " length generated ID");
   }
@@ -95,6 +90,13 @@ public void testCreateInValidOmLCRule() throws OMException {
         .setAction(null);
     assertOMException(r2::build, INVALID_REQUEST,
         "At least one action needs to be specified in a rule");
+
+    OmLCRule.Builder r3 = new OmLCRule.Builder()
+        .setEnabled(true)
+        .setAction(exp);
+
+    assertOMException(r3::build, INVALID_REQUEST,
+        "Filter and Prefix cannot both be null.");
   }
 
   @Test
@@ -157,11 +159,13 @@ public void testDuplicateRuleIDs() throws OMException {
 
     rules.add(new OmLCRule.Builder()
         .setId("duplicate-id")
+            .setPrefix("")
         .setAction(new OmLCExpiration.Builder().setDays(30).build())
         .build());
 
     rules.add(new OmLCRule.Builder()
         .setId("duplicate-id") // Same ID
+        .setPrefix("")
         .setAction(new OmLCExpiration.Builder().setDays(60).build())
         .build());
 
diff --git 
a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmLifeCycleConfiguration.java
 
b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmLifeCycleConfiguration.java
index 69188933555..c70325f3460 100644
--- 
a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmLifeCycleConfiguration.java
+++ 
b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/helpers/TestOmLifeCycleConfiguration.java
@@ -50,6 +50,7 @@ public void testCreateValidLCConfiguration() throws 
OMException {
         .setBucket("spark")
         .setRules(Collections.singletonList(new OmLCRule.Builder()
             .setId("spark logs")
+                .setPrefix("")
             .setAction(new OmLCExpiration.Builder()
                 .setDays(30)
                 .build())
@@ -63,6 +64,7 @@ public void testCreateValidLCConfiguration() throws 
OMException {
   public void testCreateInValidLCConfiguration() throws OMException {
     OmLCRule rule = new OmLCRule.Builder()
         .setId("spark logs")
+        .setPrefix("")
         .setAction(new OmLCExpiration.Builder().setDays(30).build())
         .build();
 
@@ -85,6 +87,7 @@ public void testCreateInValidLCConfiguration() throws 
OMException {
       OmLCRule r = new OmLCRule.Builder()
           .setId(Integer.toString(i))
           .setAction(new OmLCExpiration.Builder().setDays(30).build())
+          .setPrefix("")
           .build();
       rules4.add(r);
     }
@@ -104,10 +107,12 @@ public void testToBuilder() throws OMException {
     OmLCRule rule1 = new OmLCRule.Builder()
         .setId("test-rule1")
         .setAction(new OmLCExpiration.Builder().setDays(30).build())
+        .setPrefix("")
         .build();
 
     OmLCRule rule2 = new OmLCRule.Builder()
         .setId("test-rule2")
+        .setPrefix("")
         .setAction(new OmLCExpiration.Builder().setDays(60).build())
         .build();
 
diff --git 
a/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto 
b/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
index c52ea1af0c6..9300c928845 100644
--- a/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
+++ b/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
@@ -563,6 +563,7 @@ enum Status {
     KEY_UNDER_LEASE_RECOVERY = 95;
     KEY_ALREADY_CLOSED = 96;
     KEY_UNDER_LEASE_SOFT_LIMIT_PERIOD = 97;
+    LIFECYCLE_CONFIGURATION_NOT_FOUND = 98;
 
 }
 
diff --git 
a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/audit/S3GAction.java
 
b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/audit/S3GAction.java
index 6c295b7aafc..08bc2c2e1bd 100644
--- 
a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/audit/S3GAction.java
+++ 
b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/audit/S3GAction.java
@@ -31,6 +31,9 @@ public enum S3GAction implements AuditAction {
   PUT_ACL,
   LIST_MULTIPART_UPLOAD,
   MULTI_DELETE,
+  GET_BUCKET_LIFECYCLE,
+  PUT_BUCKET_LIFECYCLE,
+  DELETE_BUCKET_LIFECYCLE,
 
   //RootEndpoint
   LIST_S3_BUCKETS,
diff --git 
a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java
 
b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java
index 118900aa53d..1e4da61e881 100644
--- 
a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java
+++ 
b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java
@@ -48,6 +48,7 @@
 import javax.ws.rs.PathParam;
 import javax.ws.rs.Produces;
 import javax.ws.rs.QueryParam;
+import javax.ws.rs.WebApplicationException;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.HttpHeaders;
 import javax.ws.rs.core.MediaType;
@@ -59,11 +60,13 @@
 import org.apache.hadoop.ozone.audit.S3GAction;
 import org.apache.hadoop.ozone.client.OzoneBucket;
 import org.apache.hadoop.ozone.client.OzoneKey;
+import org.apache.hadoop.ozone.client.OzoneLifecycleConfiguration;
 import org.apache.hadoop.ozone.client.OzoneMultipartUploadList;
 import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
 import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
 import org.apache.hadoop.ozone.om.helpers.ErrorInfo;
+import org.apache.hadoop.ozone.om.helpers.OmLifecycleConfiguration;
 import org.apache.hadoop.ozone.om.helpers.OzoneAclUtil;
 import org.apache.hadoop.ozone.s3.commontypes.EncodingTypeObject;
 import org.apache.hadoop.ozone.s3.commontypes.KeyMetadata;
@@ -117,6 +120,7 @@ public Response get(
       @QueryParam("key-marker") String keyMarker,
       @QueryParam("upload-id-marker") String uploadIdMarker,
       @DefaultValue("1000") @QueryParam("max-uploads") int maxUploads,
+      @QueryParam("lifecycle") String lifecycleMarker,
       @Context HttpHeaders hh) throws OS3Exception, IOException {
     long startNanos = Time.monotonicNowNanos();
     S3GAction s3GAction = S3GAction.GET_BUCKET;
@@ -128,6 +132,11 @@ public Response get(
     OzoneBucket bucket = null;
 
     try {
+      if (lifecycleMarker != null) {
+        s3GAction = S3GAction.GET_BUCKET_LIFECYCLE;
+        return getBucketLifecycleConfiguration(bucketName, hh);
+      }
+
       if (aclMarker != null) {
         s3GAction = S3GAction.GET_ACL;
         S3BucketAcl result = getAcl(bucketName);
@@ -295,12 +304,17 @@ public Response get(
   @PUT
   public Response put(@PathParam("bucket") String bucketName,
                       @QueryParam("acl") String aclMarker,
+                      @QueryParam("lifecycle") String lifecycleMarker,
                       @Context HttpHeaders httpHeaders,
                       InputStream body) throws IOException, OS3Exception {
     long startNanos = Time.monotonicNowNanos();
     S3GAction s3GAction = S3GAction.CREATE_BUCKET;
 
     try {
+      if (lifecycleMarker != null) {
+        s3GAction = S3GAction.PUT_BUCKET_LIFECYCLE;
+        return putBucketLifecycleConfiguration(bucketName, httpHeaders, body);
+      }
       if (aclMarker != null) {
         s3GAction = S3GAction.PUT_ACL;
         Response response =  putAcl(bucketName, httpHeaders, body);
@@ -418,12 +432,17 @@ public Response head(@PathParam("bucket") String 
bucketName)
    * for more details.
    */
   @DELETE
-  public Response delete(@PathParam("bucket") String bucketName)
+  public Response delete(@PathParam("bucket") String bucketName,
+                         @QueryParam("lifecycle") String lifecycleMarker)
       throws IOException, OS3Exception {
     long startNanos = Time.monotonicNowNanos();
     S3GAction s3GAction = S3GAction.DELETE_BUCKET;
 
     try {
+      if (lifecycleMarker != null) {
+        s3GAction = S3GAction.DELETE_BUCKET_LIFECYCLE;
+        return deleteBucketLifecycleConfiguration(bucketName, null);
+      }
       deleteS3Bucket(bucketName);
     } catch (OMException ex) {
       AUDIT.logWriteFailure(
@@ -752,6 +771,97 @@ private void addKey(ListObjectResponse response, OzoneKey 
next) {
     response.addKey(keyMetadata);
   }
 
+  private void verifyBucketOwner(String bucketName, HttpHeaders httpHeaders)
+      throws OS3Exception {
+    if (httpHeaders == null) {
+      return;
+    }
+    String expectedBucketOwner = 
httpHeaders.getHeaderString("x-amz-expected-bucket-owner");
+    if (expectedBucketOwner == null || expectedBucketOwner.isEmpty()) {
+      return;
+    }
+
+    try {
+      String actualOwner = getBucket(bucketName).getOwner();
+      if (actualOwner != null && !actualOwner.equals(expectedBucketOwner)) {
+        LOG.debug("Bucket: {}, ExpectedBucketOwner: {}, ActualBucketOwner: {}",
+            bucketName, expectedBucketOwner, actualOwner);
+        throw S3ErrorTable.newError(S3ErrorTable.ACCESS_DENIED, bucketName);
+      }
+    } catch (Exception ex) {
+      LOG.error("Owner verification failed for bucket: {}", bucketName, ex);
+      throw S3ErrorTable.newError(S3ErrorTable.ACCESS_DENIED, bucketName);
+    }
+  }
+
+
+  public Response putBucketLifecycleConfiguration(
+      String bucketName, HttpHeaders httpHeaders, InputStream body)
+      throws IOException, OS3Exception {
+    verifyBucketOwner(bucketName, httpHeaders);
+    S3LifecycleConfiguration s3LifecycleConfiguration;
+    OzoneBucket ozoneBucket = getBucket(bucketName);
+    try {
+      s3LifecycleConfiguration = new 
PutBucketLifecycleConfigurationUnmarshaller().readFrom(null,
+          null, null, null, null, body);
+      OmLifecycleConfiguration lcc =
+          
s3LifecycleConfiguration.toOmLifecycleConfiguration(ozoneBucket.getVolumeName(),
 bucketName);
+      ozoneBucket.setLifecycleConfiguration(lcc);
+    } catch (WebApplicationException ex) {
+      throw S3ErrorTable.newError(S3ErrorTable.MALFORMED_XML, bucketName);
+    } catch (OMException ex) {
+      if (ex.getResult() == ResultCodes.ACCESS_DENIED) {
+        throw S3ErrorTable.newError(S3ErrorTable.ACCESS_DENIED, bucketName);
+      } else if (ex.getResult() == ResultCodes.INVALID_REQUEST) {
+        throw S3ErrorTable.newError(S3ErrorTable.INVALID_REQUEST, bucketName);
+      }
+    }
+    return Response.ok().build();
+  }
+
+  public Response getBucketLifecycleConfiguration(String bucketName, 
HttpHeaders httpHeaders)
+      throws IOException, OS3Exception {
+    verifyBucketOwner(bucketName, httpHeaders);
+    OzoneLifecycleConfiguration ozoneLifecycleConfiguration =
+        getLifecycleConfiguration(bucketName);
+    return 
Response.ok(S3LifecycleConfiguration.fromOzoneLifecycleConfiguration(
+        ozoneLifecycleConfiguration), MediaType.APPLICATION_XML_TYPE).build();
+  }
+
+  protected OzoneLifecycleConfiguration getLifecycleConfiguration(
+      String bucketName) throws IOException, OS3Exception {
+    try {
+      OzoneBucket ozoneBucket = getBucket(bucketName);
+      return ozoneBucket.getLifecycleConfiguration();
+    } catch (OMException ex) {
+      if (ex.getResult() == ResultCodes.LIFECYCLE_CONFIGURATION_NOT_FOUND) {
+        throw S3ErrorTable.newError(
+            S3ErrorTable.NO_SUCH_LIFECYCLE_CONFIGURATION, bucketName);
+      }
+      throw ex;
+    }
+  }
+
+  public Response deleteBucketLifecycleConfiguration(String bucketName, 
HttpHeaders httpHeaders)
+      throws IOException, OS3Exception {
+    verifyBucketOwner(bucketName, httpHeaders);
+    deleteLifecycleConfiguration(bucketName);
+    return Response.noContent().build();
+  }
+
+  protected void deleteLifecycleConfiguration(String bucketName)
+      throws IOException, OS3Exception {
+    try {
+      getBucket(bucketName).deleteLifecycleConfiguration();
+    } catch (OMException ex) {
+      if (ex.getResult() == ResultCodes.LIFECYCLE_CONFIGURATION_NOT_FOUND) {
+        throw S3ErrorTable.newError(
+            S3ErrorTable.NO_SUCH_LIFECYCLE_CONFIGURATION, bucketName);
+      }
+      throw ex;
+    }
+  }
+
   @Override
   public void init() {
     listKeysShallowEnabled = ozoneConfiguration.getBoolean(
diff --git 
a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/PutBucketLifecycleConfigurationUnmarshaller.java
 
b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/PutBucketLifecycleConfigurationUnmarshaller.java
new file mode 100644
index 00000000000..6cf28a7c12f
--- /dev/null
+++ 
b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/PutBucketLifecycleConfigurationUnmarshaller.java
@@ -0,0 +1,80 @@
+/*
+ * 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.s3.endpoint;
+
+import static org.apache.hadoop.ozone.s3.util.S3Consts.S3_XML_NAMESPACE;
+
+import java.io.InputStream;
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Type;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.ext.MessageBodyReader;
+import javax.xml.XMLConstants;
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.UnmarshallerHandler;
+import javax.xml.parsers.SAXParserFactory;
+import org.xml.sax.InputSource;
+import org.xml.sax.XMLReader;
+
+/**
+ * Custom unmarshaller to read Lifecycle configuration namespace.
+ */
+public class PutBucketLifecycleConfigurationUnmarshaller
+    implements MessageBodyReader<S3LifecycleConfiguration>  {
+
+  private final JAXBContext context;
+  private final XMLReader xmlReader;
+
+  public PutBucketLifecycleConfigurationUnmarshaller() {
+    try {
+      context = JAXBContext.newInstance(S3LifecycleConfiguration.class);
+      SAXParserFactory saxParserFactory = SAXParserFactory.newInstance();
+      saxParserFactory.setFeature(XMLConstants.FEATURE_SECURE_PROCESSING, 
true);
+      xmlReader = saxParserFactory.newSAXParser().getXMLReader();
+    } catch (Exception ex) {
+      throw new AssertionError("Can not instantiate " +
+          "PutBucketLifecycleConfiguration parser", ex);
+    }
+  }
+
+  @Override
+  public boolean isReadable(Class<?> type, Type genericType,
+      Annotation[] annotations, MediaType mediaType) {
+    return type.equals(S3LifecycleConfiguration.class);
+  }
+
+  @Override
+  public S3LifecycleConfiguration readFrom(Class<S3LifecycleConfiguration> 
type,
+      Type genericType, Annotation[] annotations, MediaType mediaType,
+      MultivaluedMap<String, String> httpHeaders, InputStream inputStream)
+      throws WebApplicationException {
+    try {
+      UnmarshallerHandler unmarshallerHandler =
+          context.createUnmarshaller().getUnmarshallerHandler();
+      XmlNamespaceFilter filter = new XmlNamespaceFilter(S3_XML_NAMESPACE);
+      filter.setContentHandler(unmarshallerHandler);
+      filter.setParent(xmlReader);
+      filter.parse(new InputSource(inputStream));
+      return (S3LifecycleConfiguration)(unmarshallerHandler.getResult());
+    } catch (Exception e) {
+      throw new WebApplicationException("Can't parse request body to XML.", e);
+    }
+  }
+}
diff --git 
a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/S3LifecycleConfiguration.java
 
b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/S3LifecycleConfiguration.java
new file mode 100644
index 00000000000..ae5e5b4a66d
--- /dev/null
+++ 
b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/S3LifecycleConfiguration.java
@@ -0,0 +1,484 @@
+/*
+ * 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.s3.endpoint;
+
+import java.time.format.DateTimeParseException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+import org.apache.hadoop.ozone.client.OzoneLifecycleConfiguration;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmLCExpiration;
+import org.apache.hadoop.ozone.om.helpers.OmLCFilter;
+import org.apache.hadoop.ozone.om.helpers.OmLCRule;
+import org.apache.hadoop.ozone.om.helpers.OmLifecycleConfiguration;
+import org.apache.hadoop.ozone.om.helpers.OmLifecycleRuleAndOperator;
+import org.apache.hadoop.ozone.s3.exception.OS3Exception;
+import org.apache.hadoop.ozone.s3.exception.S3ErrorTable;
+
+/**
+ * Request for put bucket lifecycle configuration.
+ */
+@XmlAccessorType(XmlAccessType.FIELD)
+@XmlRootElement(name = "LifecycleConfiguration",
+    namespace = "http://s3.amazonaws.com/doc/2006-03-01/";)
+public class S3LifecycleConfiguration {
+  @XmlElement(name = "Rule")
+  private List<Rule> rules = new ArrayList<>();
+
+  public List<Rule> getRules() {
+    return rules;
+  }
+
+  public void setRules(List<Rule> rules) {
+    this.rules = rules;
+  }
+
+  /**
+   * Rule entity for lifecycle configuration.
+   */
+  @XmlAccessorType(XmlAccessType.FIELD)
+  @XmlRootElement(name = "Rule")
+  public static class Rule {
+    @XmlElement(name = "ID")
+    private String id;
+
+    @XmlElement(name = "Status")
+    private String status;
+
+    @XmlElement(name = "Prefix")
+    private String prefix;
+
+    @XmlElement(name = "Expiration")
+    private Expiration expiration;
+
+    @XmlElement(name = "Filter")
+    private Filter filter;
+
+    public String getId() {
+      return id;
+    }
+
+    public void setId(String id) {
+      this.id = id;
+    }
+
+    public String getStatus() {
+      return status;
+    }
+
+    public void setStatus(String status) {
+      this.status = status;
+    }
+
+    public String getPrefix() {
+      return prefix;
+    }
+
+    public void setPrefix(String prefix) {
+      this.prefix = prefix;
+    }
+
+    public Expiration getExpiration() {
+      return expiration;
+    }
+
+    public void setExpiration(Expiration expiration) {
+      this.expiration = expiration;
+    }
+
+    public Filter getFilter() {
+      return filter;
+    }
+
+    public void setFilter(Filter filter) {
+      this.filter = filter;
+    }
+  }
+
+  /**
+   * Expiration entity for lifecycle rule.
+   */
+  @XmlAccessorType(XmlAccessType.FIELD)
+  @XmlRootElement(name = "Expiration")
+  public static class Expiration {
+    @XmlElement(name = "Days")
+    private Integer days;
+
+    @XmlElement(name = "Date")
+    private String date;
+
+    public Integer getDays() {
+      return days;
+    }
+
+    public void setDays(Integer days) {
+      this.days = days;
+    }
+
+    public String getDate() {
+      return date;
+    }
+
+    public void setDate(String date) {
+      this.date = date;
+    }
+  }
+
+  /**
+   * Tag entity for filter criteria.
+   */
+  @XmlAccessorType(XmlAccessType.FIELD)
+  @XmlRootElement(name = "Tag")
+  public static class Tag {
+    @XmlElement(name = "Key")
+    private String key;
+
+    @XmlElement(name = "Value")
+    private String value;
+
+    public Tag() {
+    }
+
+    public Tag(String key, String value) {
+      this.key = key;
+      this.value = value;
+    }
+
+    public String getKey() {
+      return key;
+    }
+
+    public String getValue() {
+      return value;
+    }
+
+    public void setKey(String key) {
+      this.key = key;
+    }
+
+    public void setValue(String value) {
+      this.value = value;
+    }
+  }
+
+  /**
+   * And operator entity for combining multiple filter criteria.
+   */
+  @XmlAccessorType(XmlAccessType.FIELD)
+  @XmlRootElement(name = "And")
+  public static class AndOperator {
+    @XmlElement(name = "Prefix")
+    private String prefix;
+
+    @XmlElement(name = "Tag")
+    private List<Tag> tags = null;
+
+    public List<Tag> getTags() {
+      return tags;
+    }
+
+    public String getPrefix() {
+      return prefix;
+    }
+
+    public void setPrefix(String prefix) {
+      this.prefix = prefix;
+    }
+
+    public void setTags(List<Tag> tags) {
+      this.tags = tags;
+    }
+  }
+
+  /**
+   * Filter entity for lifecycle rule.
+   */
+  @XmlAccessorType(XmlAccessType.FIELD)
+  @XmlRootElement(name = "Filter")
+  public static class Filter {
+    @XmlElement(name = "Prefix")
+    private String prefix;
+
+    @XmlElement(name = "Tag")
+    private Tag tag = null;
+
+    @XmlElement(name = "And")
+    private AndOperator andOperator;
+
+    public String getPrefix() {
+      return prefix;
+    }
+
+    public void setPrefix(String prefix) {
+      this.prefix = prefix;
+    }
+
+    public Tag getTag() {
+      return tag;
+    }
+
+    public void setTag(Tag tag) {
+      this.tag = tag;
+    }
+
+    public AndOperator getAndOperator() {
+      return andOperator;
+    }
+
+    public void setAndOperator(AndOperator andOperator) {
+      this.andOperator = andOperator;
+    }
+  }
+
+  /**
+   * Converts S3 lifecycle XML representation to Ozone internal representation.
+   *
+   * @param volumeName the Volume name
+   * @param bucketName the Bucket name
+   * @return OmLifecycleConfiguration internal representation
+   * @throws DateTimeParseException if the expiration date format is invalid
+   * @throws IllegalArgumentException if the configuration is invalid
+   * @throws OS3Exception if lifecycle is invalid
+   */
+  public OmLifecycleConfiguration toOmLifecycleConfiguration(String 
volumeName, String bucketName)
+      throws OS3Exception, OMException {
+    try {
+      OmLifecycleConfiguration.Builder builder = new 
OmLifecycleConfiguration.Builder()
+          .setVolume(volumeName)
+          .setBucket(bucketName);
+
+      for (Rule rule : getRules()) {
+        builder.addRule(convertToOmRule(rule));
+      }
+
+      return builder.build();
+    } catch (Exception ex) {
+      if (ex instanceof IllegalStateException) {
+        throw S3ErrorTable.newError(S3ErrorTable.INVALID_REQUEST, bucketName, 
ex);
+      }
+      throw ex;
+    }
+  }
+
+  /**
+   * Converts a single S3 lifecycle rule to Ozone internal rule representation.
+   *
+   * @param rule the S3 lifecycle rule
+   * @return OmLCRule internal rule representation
+   */
+  private OmLCRule convertToOmRule(Rule rule) throws OMException {
+    OmLCRule.Builder builder = new OmLCRule.Builder()
+        .setEnabled("Enabled".equals(rule.getStatus()))
+        .setId(rule.getId())
+        .setPrefix(rule.getPrefix());
+
+    if (rule.getExpiration() != null) {
+      builder.setAction(convertToOmExpiration(rule.getExpiration()));
+    }
+    if (rule.getFilter() != null) {
+      builder.setFilter(convertToOmFilter(rule.getFilter()));
+    }
+
+    return builder.build();
+  }
+
+  /**
+   * Converts S3 expiration to internal expiration.
+   *
+   * @param expiration the S3 expiration
+   * @return OmLCExpiration internal expiration
+   */
+  private OmLCExpiration convertToOmExpiration(Expiration expiration) throws 
OMException {
+    OmLCExpiration.Builder builder = new OmLCExpiration.Builder();
+
+    if (expiration.getDays() != null) {
+      builder.setDays(expiration.getDays());
+    }
+    if (expiration.getDate() != null) {
+      builder.setDate(expiration.getDate());
+    }
+
+    return builder.build();
+  }
+
+  /**
+   * Converts S3 filter to internal filter.
+   *
+   * @param filter the S3 filter
+   * @return OmLCFilter internal filter
+   */
+  private OmLCFilter convertToOmFilter(Filter filter) throws OMException {
+    OmLCFilter.Builder builder = new OmLCFilter.Builder();
+
+    if (filter.getPrefix() != null) {
+      builder.setPrefix(filter.getPrefix());
+    }
+    if (filter.getTag() != null) {
+      builder.setTag(filter.getTag().getKey(), filter.getTag().getValue());
+    }
+    if (filter.getAndOperator() != null) {
+      builder.setAndOperator(convertToOmAndOperator(filter.getAndOperator()));
+    }
+
+    return builder.build();
+  }
+
+  /**
+   * Converts S3 AND operator to internal AND operator.
+   *
+   * @param andOperator the S3 AND operator
+   * @return OmLifecycleRuleAndOperator internal AND operator
+   */
+  private OmLifecycleRuleAndOperator convertToOmAndOperator(AndOperator 
andOperator) throws OMException {
+    OmLifecycleRuleAndOperator.Builder builder = new 
OmLifecycleRuleAndOperator.Builder();
+
+    if (andOperator.getPrefix() != null) {
+      builder.setPrefix(andOperator.getPrefix());
+    }
+    if (andOperator.getTags() != null) {
+      Map<String, String> tags = andOperator.getTags().stream()
+          .collect(Collectors.toMap(Tag::getKey, Tag::getValue));
+      builder.setTags(tags);
+    }
+
+    return builder.build();
+  }
+
+  /**
+   * Creates a LifecycleConfiguration instance (XML representation) from an
+   * Ozone internal lifecycle configuration.
+   *
+   * @param ozoneLifecycleConfiguration internal lifecycle configuration
+   * @return LifecycleConfiguration XML representation
+   */
+  public static S3LifecycleConfiguration fromOzoneLifecycleConfiguration(
+      OzoneLifecycleConfiguration ozoneLifecycleConfiguration) {
+
+    S3LifecycleConfiguration s3LifecycleConfiguration = new 
S3LifecycleConfiguration();
+    List<Rule> rules = new ArrayList<>();
+
+    for (OzoneLifecycleConfiguration.OzoneLCRule ozoneRule : 
ozoneLifecycleConfiguration.getRules()) {
+      rules.add(convertFromOzoneRule(ozoneRule));
+    }
+
+    s3LifecycleConfiguration.setRules(rules);
+    return s3LifecycleConfiguration;
+  }
+
+  /**
+   * Converts an Ozone internal rule to S3 lifecycle rule.
+   *
+   * @param ozoneRule internal lifecycle rule
+   * @return Rule S3 lifecycle rule
+   */
+  private static Rule 
convertFromOzoneRule(OzoneLifecycleConfiguration.OzoneLCRule ozoneRule) {
+    Rule rule = new Rule();
+
+    rule.setId(ozoneRule.getId());
+    rule.setStatus(ozoneRule.getStatus());
+    if (ozoneRule.getPrefix() != null) {
+      rule.setPrefix(ozoneRule.getPrefix());
+    }
+    if (ozoneRule.getExpiration() != null) {
+      
rule.setExpiration(convertFromOzoneExpiration(ozoneRule.getExpiration()));
+    }
+    if (ozoneRule.getFilter() != null) {
+      rule.setFilter(convertFromOzoneFilter(ozoneRule.getFilter()));
+    }
+
+    return rule;
+  }
+
+  /**
+   * Converts an Ozone internal expiration to S3 expiration.
+   *
+   * @param ozoneExpiration internal expiration
+   * @return Expiration S3 expiration
+   */
+  private static Expiration convertFromOzoneExpiration(
+      OzoneLifecycleConfiguration.OzoneLCExpiration ozoneExpiration) {
+
+    Expiration expiration = new Expiration();
+
+    String date = ozoneExpiration.getDate();
+    if (date != null && !date.isEmpty()) {
+      expiration.setDate(date);
+    }
+    if (ozoneExpiration.getDays() > 0) {
+      expiration.setDays(ozoneExpiration.getDays());
+    }
+
+    return expiration;
+  }
+
+  /**
+   * Converts an Ozone internal filter to S3 filter.
+   *
+   * @param ozoneFilter internal filter
+   * @return Filter S3 filter
+   */
+  private static Filter convertFromOzoneFilter(
+      OzoneLifecycleConfiguration.OzoneLCFilter ozoneFilter) {
+
+    Filter filter = new Filter();
+
+    filter.setPrefix(ozoneFilter.getPrefix());
+
+    if (ozoneFilter.getTag() != null) {
+      filter.setTag(new Tag(
+          ozoneFilter.getTag().getKey(),
+          ozoneFilter.getTag().getValue()
+      ));
+    }
+
+    if (ozoneFilter.getAndOperator() != null) {
+      
filter.setAndOperator(convertFromOzoneAndOperator(ozoneFilter.getAndOperator()));
+    }
+
+    return filter;
+  }
+
+  /**
+   * Converts an Ozone internal AND operator to S3 AND operator.
+   *
+   * @param ozoneAndOperator internal AND operator
+   * @return AndOperator S3 AND operator
+   */
+  private static AndOperator convertFromOzoneAndOperator(
+      OzoneLifecycleConfiguration.LifecycleAndOperator ozoneAndOperator) {
+
+    AndOperator andOperator = new AndOperator();
+
+    andOperator.setPrefix(ozoneAndOperator.getPrefix());
+
+    if (ozoneAndOperator.getTags() != null) {
+      List<Tag> tags = ozoneAndOperator.getTags().entrySet().stream()
+          .map(entry -> new Tag(entry.getKey(), entry.getValue()))
+          .collect(Collectors.toList());
+      andOperator.setTags(tags);
+    }
+
+    return andOperator;
+  }
+}
diff --git 
a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/exception/S3ErrorTable.java
 
b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/exception/S3ErrorTable.java
index ed10f2894f0..1cb0188e06f 100644
--- 
a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/exception/S3ErrorTable.java
+++ 
b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/exception/S3ErrorTable.java
@@ -154,6 +154,10 @@ private S3ErrorTable() {
       HTTP_FORBIDDEN
   );
 
+  public static final OS3Exception NO_SUCH_LIFECYCLE_CONFIGURATION =
+      new OS3Exception("NoSuchLifecycleConfiguration",
+      "The specified lifecycle configurations does not exist", HTTP_NOT_FOUND);
+
   public static OS3Exception newError(OS3Exception e, String resource) {
     return newError(e, resource, null);
   }
diff --git 
a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/ClientProtocolStub.java
 
b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/ClientProtocolStub.java
index c15a85dc2e3..e1853f305b7 100644
--- 
a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/ClientProtocolStub.java
+++ 
b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/ClientProtocolStub.java
@@ -42,6 +42,7 @@
 import org.apache.hadoop.ozone.om.helpers.LeaseKeyInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
 import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmLifecycleConfiguration;
 import org.apache.hadoop.ozone.om.helpers.OmMultipartInfo;
 import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadCompleteInfo;
 import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
@@ -789,4 +790,21 @@ public void deleteObjectTagging(String volumeName, String 
bucketName, String key
     getBucket(volumeName, bucketName).deleteObjectTagging(keyName);
   }
 
+  @Override
+  public OzoneLifecycleConfiguration getLifecycleConfiguration(String 
volumeName, String bucketName)
+      throws IOException {
+    return getBucket(volumeName, bucketName).getLifecycleConfiguration();
+  }
+
+  @Override
+  public void createLifecycleConfiguration(OmLifecycleConfiguration 
lifecycleConfiguration) throws IOException {
+    getBucket(lifecycleConfiguration.getVolume(), 
lifecycleConfiguration.getBucket())
+        .setLifecycleConfiguration(lifecycleConfiguration);
+  }
+
+  @Override
+  public void deleteLifecycleConfiguration(String volumeName, String 
bucketName) throws IOException {
+
+  }
+
 }
diff --git 
a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneBucketStub.java
 
b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneBucketStub.java
index 5d060c228c8..56e134faa21 100644
--- 
a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneBucketStub.java
+++ 
b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneBucketStub.java
@@ -20,6 +20,7 @@
 import static org.apache.hadoop.ozone.OzoneConsts.ETAG;
 import static org.apache.hadoop.ozone.OzoneConsts.MD5_HASH;
 import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER;
+import static 
org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.LIFECYCLE_CONFIGURATION_NOT_FOUND;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -54,6 +55,8 @@
 import org.apache.hadoop.ozone.om.exceptions.OMException;
 import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
 import org.apache.hadoop.ozone.om.helpers.ErrorInfo;
+import org.apache.hadoop.ozone.om.helpers.OmLCRule;
+import org.apache.hadoop.ozone.om.helpers.OmLifecycleConfiguration;
 import org.apache.hadoop.ozone.om.helpers.OmMultipartInfo;
 import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadCompleteInfo;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -78,6 +81,8 @@ public final class OzoneBucketStub extends OzoneBucket {
 
   private ArrayList<OzoneAcl> aclList = new ArrayList<>();
   private ReplicationConfig replicationConfig;
+  private Map<String, OzoneLifecycleConfiguration> lifecyclesMap = new 
HashMap<>();
+
 
   public static Builder newBuilder() {
     return new Builder();
@@ -688,6 +693,65 @@ private void assertDoesNotExist(String keyName) throws 
OMException {
     }
   }
 
+  @Override
+  public void setLifecycleConfiguration(
+      OmLifecycleConfiguration lifecycleConfiguration) throws IOException {
+    lifecyclesMap.put(lifecycleConfiguration.getBucket(),
+        toOzoneLifecycleConfiguration(lifecycleConfiguration));
+  }
+
+  @Override
+  public OzoneLifecycleConfiguration getLifecycleConfiguration() throws 
IOException {
+    OzoneLifecycleConfiguration lcc = lifecyclesMap.get(getName());
+    if (lcc == null) {
+      throw new OMException("Lifecycle configuration not found",
+          LIFECYCLE_CONFIGURATION_NOT_FOUND);
+    }
+    return lcc;
+  }
+
+  @Override
+  public void deleteLifecycleConfiguration()
+      throws IOException {
+    if (!lifecyclesMap.containsKey(getName())) {
+      throw new OMException("Lifecycle configurations does not exist",
+          OMException.ResultCodes.LIFECYCLE_CONFIGURATION_NOT_FOUND);
+    }
+    lifecyclesMap.remove(getName());
+  }
+
+  private static OzoneLifecycleConfiguration toOzoneLifecycleConfiguration(
+      OmLifecycleConfiguration omLifecycleConfiguration) {
+    List<OzoneLifecycleConfiguration.OzoneLCRule> rules = new ArrayList<>();
+
+    for (OmLCRule r: omLifecycleConfiguration.getRules()) {
+      OzoneLifecycleConfiguration.OzoneLCExpiration e = null;
+      OzoneLifecycleConfiguration.OzoneLCFilter f = null;
+
+      if (r.getExpiration() != null) {
+        e = new OzoneLifecycleConfiguration.OzoneLCExpiration(
+            r.getExpiration().getDays(), r.getExpiration().getDate());
+      }
+      if (r.getFilter() != null) {
+        OzoneLifecycleConfiguration.LifecycleAndOperator andOperator = null;
+        if (r.getFilter().getAndOperator() != null) {
+          andOperator = new 
OzoneLifecycleConfiguration.LifecycleAndOperator(r.getFilter().getAndOperator()
+              .getTags(), r.getFilter().getAndOperator().getPrefix());
+        }
+        f = new OzoneLifecycleConfiguration.OzoneLCFilter(
+            r.getFilter().getPrefix(), r.getFilter().getTag(), andOperator);
+      }
+
+      rules.add(new OzoneLifecycleConfiguration.OzoneLCRule(r.getId(),
+          r.getPrefix(), (r.isEnabled() ? "Enabled" : "Disabled"), e, f));
+    }
+
+    return new OzoneLifecycleConfiguration(
+        omLifecycleConfiguration.getVolume(),
+        omLifecycleConfiguration.getBucket(),
+        omLifecycleConfiguration.getCreationTime(), rules);
+  }
+
   /**
    * ByteArrayOutputStream stub with metadata.
    */
diff --git 
a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneVolumeStub.java
 
b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneVolumeStub.java
index d1358cec593..f5cceba4475 100644
--- 
a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneVolumeStub.java
+++ 
b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneVolumeStub.java
@@ -124,6 +124,7 @@ public void createBucket(String bucketName, BucketArgs 
bucketArgs) {
         .setBucketLayout(bucketArgs.getBucketLayout())
         .setStorageType(bucketArgs.getStorageType())
         .setVersioning(bucketArgs.getVersioning())
+        .setOwner(bucketArgs.getOwner())
         .setCreationTime(Time.now())
         .build());
   }
diff --git 
a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketAcl.java
 
b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketAcl.java
index 1099fbea98c..4b07f7e0182 100644
--- 
a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketAcl.java
+++ 
b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketAcl.java
@@ -82,7 +82,7 @@ public void testGetAcl() throws Exception {
     when(parameterMap.containsKey(ACL_MARKER)).thenReturn(true);
     Response response =
         bucketEndpoint.get(BUCKET_NAME, null, null, null, 0, null,
-            null, null, null, ACL_MARKER, null, null, 0, headers);
+            null, null, null, ACL_MARKER, null, null, 0, null, headers);
     assertEquals(HTTP_OK, response.getStatus());
     System.out.println(response.getEntity());
   }
@@ -93,7 +93,7 @@ public void testSetAclWithNotSupportedGranteeType() throws 
Exception {
         .thenReturn(S3Acl.ACLIdentityType.GROUP.getHeaderType() + "=root");
     when(parameterMap.containsKey(ACL_MARKER)).thenReturn(true);
     OS3Exception e = assertThrows(OS3Exception.class, () ->
-        bucketEndpoint.put(BUCKET_NAME, ACL_MARKER, headers, null));
+        bucketEndpoint.put(BUCKET_NAME, ACL_MARKER, null, headers, null));
     assertEquals(e.getHttpCode(), HTTP_NOT_IMPLEMENTED);
   }
 
@@ -103,7 +103,7 @@ public void testRead() throws Exception {
     when(headers.getHeaderString(S3Acl.GRANT_READ))
         .thenReturn(S3Acl.ACLIdentityType.USER.getHeaderType() + "=root");
     Response response =
-        bucketEndpoint.put(BUCKET_NAME, ACL_MARKER, headers, null);
+        bucketEndpoint.put(BUCKET_NAME, ACL_MARKER, null, headers, null);
     assertEquals(HTTP_OK, response.getStatus());
     S3BucketAcl getResponse = bucketEndpoint.getAcl(BUCKET_NAME);
     assertEquals(1, getResponse.getAclList().getGrantList().size());
@@ -117,7 +117,7 @@ public void testWrite() throws Exception {
     when(headers.getHeaderString(S3Acl.GRANT_WRITE))
         .thenReturn(S3Acl.ACLIdentityType.USER.getHeaderType() + "=root");
     Response response =
-        bucketEndpoint.put(BUCKET_NAME, ACL_MARKER, headers, null);
+        bucketEndpoint.put(BUCKET_NAME, ACL_MARKER, null, headers, null);
     assertEquals(HTTP_OK, response.getStatus());
     S3BucketAcl getResponse = bucketEndpoint.getAcl(BUCKET_NAME);
     assertEquals(1, getResponse.getAclList().getGrantList().size());
@@ -131,7 +131,7 @@ public void testReadACP() throws Exception {
     when(headers.getHeaderString(S3Acl.GRANT_READ_CAP))
         .thenReturn(S3Acl.ACLIdentityType.USER.getHeaderType() + "=root");
     Response response =
-        bucketEndpoint.put(BUCKET_NAME, ACL_MARKER, headers, null);
+        bucketEndpoint.put(BUCKET_NAME, ACL_MARKER, null, headers, null);
     assertEquals(HTTP_OK, response.getStatus());
     S3BucketAcl getResponse =
         bucketEndpoint.getAcl(BUCKET_NAME);
@@ -146,7 +146,7 @@ public void testWriteACP() throws Exception {
     when(headers.getHeaderString(S3Acl.GRANT_WRITE_CAP))
         .thenReturn(S3Acl.ACLIdentityType.USER.getHeaderType() + "=root");
     Response response =
-        bucketEndpoint.put(BUCKET_NAME, ACL_MARKER, headers, null);
+        bucketEndpoint.put(BUCKET_NAME, ACL_MARKER, null, headers, null);
     assertEquals(HTTP_OK, response.getStatus());
     S3BucketAcl getResponse = bucketEndpoint.getAcl(BUCKET_NAME);
     assertEquals(1, getResponse.getAclList().getGrantList().size());
@@ -160,7 +160,7 @@ public void testFullControl() throws Exception {
     when(headers.getHeaderString(S3Acl.GRANT_FULL_CONTROL))
         .thenReturn(S3Acl.ACLIdentityType.USER.getHeaderType() + "=root");
     Response response =
-        bucketEndpoint.put(BUCKET_NAME, ACL_MARKER, headers, null);
+        bucketEndpoint.put(BUCKET_NAME, ACL_MARKER, null, headers, null);
     assertEquals(HTTP_OK, response.getStatus());
     S3BucketAcl getResponse = bucketEndpoint.getAcl(BUCKET_NAME);
     assertEquals(1, getResponse.getAclList().getGrantList().size());
@@ -182,7 +182,7 @@ public void testCombination() throws Exception {
     when(headers.getHeaderString(S3Acl.GRANT_FULL_CONTROL))
         .thenReturn(S3Acl.ACLIdentityType.USER.getHeaderType() + "=root");
     Response response =
-        bucketEndpoint.put(BUCKET_NAME, ACL_MARKER, headers, null);
+        bucketEndpoint.put(BUCKET_NAME, ACL_MARKER, null, headers, null);
     assertEquals(HTTP_OK, response.getStatus());
     S3BucketAcl getResponse = bucketEndpoint.getAcl(BUCKET_NAME);
     assertEquals(5, getResponse.getAclList().getGrantList().size());
@@ -195,7 +195,7 @@ public void testPutClearOldAcls() throws Exception {
         .thenReturn(S3Acl.ACLIdentityType.USER.getHeaderType() + "=root");
     // Put READ
     Response response =
-        bucketEndpoint.put(BUCKET_NAME, ACL_MARKER, headers, null);
+        bucketEndpoint.put(BUCKET_NAME, ACL_MARKER, null, headers, null);
     assertEquals(HTTP_OK, response.getStatus());
     S3BucketAcl getResponse = bucketEndpoint.getAcl(BUCKET_NAME);
     assertEquals(1, getResponse.getAclList().getGrantList().size());
@@ -212,7 +212,7 @@ public void testPutClearOldAcls() throws Exception {
         .thenReturn(S3Acl.ACLIdentityType.USER.getHeaderType() + "=root");
     //Put WRITE
     response =
-        bucketEndpoint.put(BUCKET_NAME, ACL_MARKER, headers, null);
+        bucketEndpoint.put(BUCKET_NAME, ACL_MARKER, null, headers, null);
     assertEquals(HTTP_OK, response.getStatus());
     getResponse = bucketEndpoint.getAcl(BUCKET_NAME);
     assertEquals(1, getResponse.getAclList().getGrantList().size());
@@ -230,7 +230,7 @@ public void testAclInBodyWithGroupUser() {
         .getResourceAsStream("groupAccessControlList.xml");
     when(parameterMap.containsKey(ACL_MARKER)).thenReturn(true);
     assertThrows(OS3Exception.class, () -> bucketEndpoint.put(
-        BUCKET_NAME, ACL_MARKER, headers, inputBody));
+        BUCKET_NAME, ACL_MARKER, null, headers, inputBody));
   }
 
   @Test
@@ -239,7 +239,7 @@ public void testAclInBody() throws Exception {
         .getResourceAsStream("userAccessControlList.xml");
     when(parameterMap.containsKey(ACL_MARKER)).thenReturn(true);
     Response response =
-        bucketEndpoint.put(BUCKET_NAME, ACL_MARKER, headers, inputBody);
+        bucketEndpoint.put(BUCKET_NAME, ACL_MARKER, null, headers, inputBody);
     assertEquals(HTTP_OK, response.getStatus());
     S3BucketAcl getResponse = bucketEndpoint.getAcl(BUCKET_NAME);
     assertEquals(2, getResponse.getAclList().getGrantList().size());
diff --git 
a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketDelete.java
 
b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketDelete.java
index b06e714d03d..39e0ad42453 100644
--- 
a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketDelete.java
+++ 
b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketDelete.java
@@ -61,7 +61,7 @@ public void setup() throws Exception {
 
   @Test
   public void testBucketEndpoint() throws Exception {
-    Response response = bucketEndpoint.delete(bucketName);
+    Response response = bucketEndpoint.delete(bucketName, null);
     assertEquals(HttpStatus.SC_NO_CONTENT, response.getStatus());
 
   }
@@ -69,7 +69,7 @@ public void testBucketEndpoint() throws Exception {
   @Test
   public void testDeleteWithNoSuchBucket() throws Exception {
     try {
-      bucketEndpoint.delete("unknownbucket");
+      bucketEndpoint.delete("unknownbucket", null);
     } catch (OS3Exception ex) {
       assertEquals(S3ErrorTable.NO_SUCH_BUCKET.getCode(), ex.getCode());
       assertEquals(S3ErrorTable.NO_SUCH_BUCKET.getErrorMessage(),
@@ -85,7 +85,7 @@ public void testDeleteWithBucketNotEmpty() throws Exception {
     try {
       ObjectStoreStub stub = (ObjectStoreStub) objectStoreStub;
       stub.setBucketEmptyStatus(bucketName, false);
-      bucketEndpoint.delete(bucketName);
+      bucketEndpoint.delete(bucketName, null);
     } catch (OS3Exception ex) {
       assertEquals(S3ErrorTable.BUCKET_NOT_EMPTY.getCode(), ex.getCode());
       assertEquals(S3ErrorTable.BUCKET_NOT_EMPTY.getErrorMessage(),
diff --git 
a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketList.java
 
b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketList.java
index b3589afbfd4..32c19b67c21 100644
--- 
a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketList.java
+++ 
b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketList.java
@@ -54,7 +54,7 @@ public void listRoot() throws OS3Exception, IOException {
 
     ListObjectResponse getBucketResponse =
         (ListObjectResponse) getBucket.get("b1", "/", null, null, 100, "",
-                null, null, null, null, null, null, 0, null)
+                null, null, null, null, null, null, 0, null, null)
             .getEntity();
 
     assertEquals(1, getBucketResponse.getCommonPrefixes().size());
@@ -79,7 +79,7 @@ public void listDir() throws OS3Exception, IOException {
 
     ListObjectResponse getBucketResponse =
         (ListObjectResponse) getBucket.get("b1", "/", null, null, 100,
-            "dir1", null, null, null, null, null, null, 0, null).getEntity();
+            "dir1", null, null, null, null, null, null, 0, null, 
null).getEntity();
 
     assertEquals(1, getBucketResponse.getCommonPrefixes().size());
     assertEquals("dir1/",
@@ -103,7 +103,7 @@ public void listSubDir() throws OS3Exception, IOException {
     ListObjectResponse getBucketResponse =
         (ListObjectResponse) getBucket
             .get("b1", "/", null, null, 100, "dir1/", null,
-                null, null, null, null, null, 0, null)
+                null, null, null, null, null, 0, null, null)
             .getEntity();
 
     assertEquals(1, getBucketResponse.getCommonPrefixes().size());
@@ -139,7 +139,7 @@ public void listObjectOwner() throws OS3Exception, 
IOException {
     getBucket.setRequestIdentifier(new RequestIdentifier());
     ListObjectResponse getBucketResponse =
         (ListObjectResponse) getBucket.get("b1", "/", null, null, 100,
-            "key", null, null, null, null, null, null, 0, null).getEntity();
+            "key", null, null, null, null, null, null, 0, null, 
null).getEntity();
 
     assertEquals(2, getBucketResponse.getContents().size());
     assertEquals(user1.getShortUserName(),
@@ -163,7 +163,7 @@ public void listWithPrefixAndDelimiter() throws 
OS3Exception, IOException {
 
     ListObjectResponse getBucketResponse =
         (ListObjectResponse) getBucket.get("b1", "/", null, null, 100,
-            "dir1", null, null, null, null, null, null, 0, null).getEntity();
+            "dir1", null, null, null, null, null, null, 0, null, 
null).getEntity();
 
     assertEquals(3, getBucketResponse.getCommonPrefixes().size());
 
@@ -183,7 +183,7 @@ public void listWithPrefixAndDelimiter1() throws 
OS3Exception, IOException {
 
     ListObjectResponse getBucketResponse =
         (ListObjectResponse) getBucket.get("b1", "/", null, null, 100,
-            "", null, null, null, null, null, null, 0, null).getEntity();
+            "", null, null, null, null, null, null, 0, null, null).getEntity();
 
     assertEquals(3, getBucketResponse.getCommonPrefixes().size());
     assertEquals("file2", getBucketResponse.getContents().get(0)
@@ -204,7 +204,7 @@ public void listWithPrefixAndDelimiter2() throws 
OS3Exception, IOException {
     getBucket.setRequestIdentifier(new RequestIdentifier());
     ListObjectResponse getBucketResponse =
         (ListObjectResponse) getBucket.get("b1", "/", null, null, 100, 
"dir1bh",
-            null, "dir1/dir2/file2", null, null, null, null, 0, 
null).getEntity();
+            null, "dir1/dir2/file2", null, null, null, null, 0, null, 
null).getEntity();
 
     assertEquals(2, getBucketResponse.getCommonPrefixes().size());
 
@@ -224,7 +224,7 @@ public void listWithPrefixAndEmptyStrDelimiter()
     // Should behave the same if delimiter is null
     ListObjectResponse getBucketResponse =
         (ListObjectResponse) getBucket.get("b1", "", null, null, 100, "dir1/",
-          null, null, null, null, null, null, 0, null).getEntity();
+          null, null, null, null, null, null, 0, null, null).getEntity();
 
     assertEquals(0, getBucketResponse.getCommonPrefixes().size());
     assertEquals(4, getBucketResponse.getContents().size());
@@ -256,7 +256,7 @@ public void listWithContinuationToken() throws 
OS3Exception, IOException {
     // First time
     ListObjectResponse getBucketResponse =
         (ListObjectResponse) getBucket.get("b1", null, null, null, maxKeys,
-            "", null, null, null, null, null, null, 0, null).getEntity();
+            "", null, null, null, null, null, null, 0, null, null).getEntity();
 
     assertTrue(getBucketResponse.isTruncated());
     assertEquals(2, getBucketResponse.getContents().size());
@@ -265,7 +265,7 @@ public void listWithContinuationToken() throws 
OS3Exception, IOException {
     String continueToken = getBucketResponse.getNextToken();
     getBucketResponse =
         (ListObjectResponse) getBucket.get("b1", null, null, null, maxKeys,
-            "", continueToken, null, null, null, null, null, 0, 
null).getEntity();
+            "", continueToken, null, null, null, null, null, 0, null, 
null).getEntity();
     assertTrue(getBucketResponse.isTruncated());
     assertEquals(2, getBucketResponse.getContents().size());
 
@@ -275,7 +275,7 @@ public void listWithContinuationToken() throws 
OS3Exception, IOException {
     //3rd time
     getBucketResponse =
         (ListObjectResponse) getBucket.get("b1", null, null, null, maxKeys,
-            "", continueToken, null, null, null, null, null, 0, 
null).getEntity();
+            "", continueToken, null, null, null, null, null, 0, null, 
null).getEntity();
 
     assertFalse(getBucketResponse.isTruncated());
     assertEquals(1, getBucketResponse.getContents().size());
@@ -308,7 +308,7 @@ public void listWithContinuationTokenDirBreak()
 
     getBucketResponse =
         (ListObjectResponse) getBucket.get("b1", "/", null, null, maxKeys,
-            "test/", null, null, null, null, null, null, 0, null).getEntity();
+            "test/", null, null, null, null, null, null, 0, null, 
null).getEntity();
 
     assertEquals(0, getBucketResponse.getContents().size());
     assertEquals(2, getBucketResponse.getCommonPrefixes().size());
@@ -320,7 +320,7 @@ public void listWithContinuationTokenDirBreak()
     getBucketResponse =
         (ListObjectResponse) getBucket.get("b1", "/", null, null, maxKeys,
             "test/", getBucketResponse.getNextToken(), null, null, null,
-            null, null, 0, null).getEntity();
+            null, null, 0, null, null).getEntity();
     assertEquals(1, getBucketResponse.getContents().size());
     assertEquals(1, getBucketResponse.getCommonPrefixes().size());
     assertEquals("test/dir3/",
@@ -352,7 +352,7 @@ public void listWithContinuationToken1() throws 
OS3Exception, IOException {
     // First time
     ListObjectResponse getBucketResponse =
         (ListObjectResponse) getBucket.get("b1", "/", null, null, maxKeys,
-            "dir", null, null, null, null, null, null, 0, null).getEntity();
+            "dir", null, null, null, null, null, null, 0, null, 
null).getEntity();
 
     assertTrue(getBucketResponse.isTruncated());
     assertEquals(2, getBucketResponse.getCommonPrefixes().size());
@@ -361,7 +361,7 @@ public void listWithContinuationToken1() throws 
OS3Exception, IOException {
     String continueToken = getBucketResponse.getNextToken();
     getBucketResponse =
         (ListObjectResponse) getBucket.get("b1", "/", null, null, maxKeys,
-            "dir", continueToken, null, null, null, null, null, 0, 
null).getEntity();
+            "dir", continueToken, null, null, null, null, null, 0, null, 
null).getEntity();
     assertTrue(getBucketResponse.isTruncated());
     assertEquals(2, getBucketResponse.getCommonPrefixes().size());
 
@@ -369,7 +369,7 @@ public void listWithContinuationToken1() throws 
OS3Exception, IOException {
     continueToken = getBucketResponse.getNextToken();
     getBucketResponse =
         (ListObjectResponse) getBucket.get("b1", "/", null, null, maxKeys,
-            "dir", continueToken, null, null, null, null, null, 0, 
null).getEntity();
+            "dir", continueToken, null, null, null, null, null, 0, null, 
null).getEntity();
 
     assertFalse(getBucketResponse.isTruncated());
     assertEquals(1, getBucketResponse.getCommonPrefixes().size());
@@ -389,7 +389,7 @@ public void listWithContinuationTokenFail() throws 
IOException {
     getBucket.setRequestIdentifier(new RequestIdentifier());
 
     OS3Exception e = assertThrows(OS3Exception.class, () -> getBucket.get("b1",
-            "/", null, null, 2, "dir", "random", null, null, null, null, null, 
1000, null)
+            "/", null, null, 2, "dir", "random", null, null, null, null, null, 
1000, null, null)
         .getEntity(), "listWithContinuationTokenFail");
     assertEquals("random", e.getResource());
     assertEquals("Invalid Argument", e.getErrorMessage());
@@ -409,7 +409,7 @@ public void testStartAfter() throws IOException, 
OS3Exception {
 
     ListObjectResponse getBucketResponse =
         (ListObjectResponse) getBucket.get("b1", null, null, null, 1000,
-            null, null, null, null, null, null, null, 0, null).getEntity();
+            null, null, null, null, null, null, null, 0, null, 
null).getEntity();
 
     assertFalse(getBucketResponse.isTruncated());
     assertEquals(5, getBucketResponse.getContents().size());
@@ -420,14 +420,14 @@ public void testStartAfter() throws IOException, 
OS3Exception {
 
     getBucketResponse =
         (ListObjectResponse) getBucket.get("b1", null, null, null,
-            1000, null, null, startAfter, null, null, null, null, 0, 
null).getEntity();
+            1000, null, null, startAfter, null, null, null, null, 0, null, 
null).getEntity();
 
     assertFalse(getBucketResponse.isTruncated());
     assertEquals(4, getBucketResponse.getContents().size());
 
     getBucketResponse =
         (ListObjectResponse) getBucket.get("b1", null, null, null,
-            1000, null, null, "random", null, null, null, null, 0, 
null).getEntity();
+            1000, null, null, "random", null, null, null, null, 0, null, 
null).getEntity();
 
     assertFalse(getBucketResponse.isTruncated());
     assertEquals(0, getBucketResponse.getContents().size());
@@ -474,7 +474,7 @@ public void testEncodingType() throws IOException, 
OS3Exception {
 
     ListObjectResponse response = (ListObjectResponse) getBucket.get(
         "b1", delimiter, encodingType, null, 1000, prefix,
-        null, startAfter, null, null, null, null, 0, null).getEntity();
+        null, startAfter, null, null, null, null, 0, null, null).getEntity();
 
     // Assert encodingType == url.
     // The Object name will be encoded by ObjectKeyNameAdapter
@@ -492,7 +492,7 @@ public void testEncodingType() throws IOException, 
OS3Exception {
 
     response = (ListObjectResponse) getBucket.get(
         "b1", delimiter, null, null, 1000, prefix,
-        null, startAfter, null, null, null, null, 0, null).getEntity();
+        null, startAfter, null, null, null, null, 0, null, null).getEntity();
 
     // Assert encodingType == null.
     // The Object name will not be encoded by ObjectKeyNameAdapter
@@ -517,7 +517,7 @@ public void testEncodingTypeException() throws IOException {
     getBucket.setRequestIdentifier(new RequestIdentifier());
     OS3Exception e = assertThrows(OS3Exception.class, () -> getBucket.get(
         "b1", null, "unSupportType", null, 1000, null,
-        null, null, null, null, null, null, 0, null).getEntity());
+        null, null, null, null, null, null, 0, null, null).getEntity());
     assertEquals(S3ErrorTable.INVALID_ARGUMENT.getCode(), e.getCode());
   }
 
diff --git 
a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketPut.java
 
b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketPut.java
index 6f5d4d5d13b..5300718a64a 100644
--- 
a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketPut.java
+++ 
b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketPut.java
@@ -57,7 +57,7 @@ public void setup() throws Exception {
   @Test
   public void testBucketFailWithAuthHeaderMissing() throws Exception {
     try {
-      bucketEndpoint.put(bucketName, null, null, null);
+      bucketEndpoint.put(bucketName, null, null, null, null);
     } catch (OS3Exception ex) {
       assertEquals(HTTP_NOT_FOUND, ex.getHttpCode());
       assertEquals(MALFORMED_HEADER.getCode(), ex.getCode());
@@ -66,13 +66,13 @@ public void testBucketFailWithAuthHeaderMissing() throws 
Exception {
 
   @Test
   public void testBucketPut() throws Exception {
-    Response response = bucketEndpoint.put(bucketName, null, null, null);
+    Response response = bucketEndpoint.put(bucketName, null, null, null, null);
     assertEquals(200, response.getStatus());
     assertNotNull(response.getLocation());
 
     // Create-bucket on an existing bucket fails
     OS3Exception e = assertThrows(OS3Exception.class, () -> bucketEndpoint.put(
-        bucketName, null, null, null));
+        bucketName, null, null, null, null));
     assertEquals(HTTP_CONFLICT, e.getHttpCode());
     assertEquals(BUCKET_ALREADY_EXISTS.getCode(), e.getCode());
   }
@@ -80,7 +80,7 @@ public void testBucketPut() throws Exception {
   @Test
   public void testBucketFailWithInvalidHeader() throws Exception {
     try {
-      bucketEndpoint.put(bucketName, null, null, null);
+      bucketEndpoint.put(bucketName, null, null, null, null);
     } catch (OS3Exception ex) {
       assertEquals(HTTP_NOT_FOUND, ex.getHttpCode());
       assertEquals(MALFORMED_HEADER.getCode(), ex.getCode());
diff --git 
a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestPermissionCheck.java
 
b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestPermissionCheck.java
index 3804e412fe5..0518fbc1344 100644
--- 
a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestPermissionCheck.java
+++ 
b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestPermissionCheck.java
@@ -126,7 +126,7 @@ public void testCreateBucket() throws IOException {
         .setClient(client)
         .build();
     OS3Exception e = assertThrows(OS3Exception.class, () ->
-        bucketEndpoint.put("bucketName", null, null, null));
+        bucketEndpoint.put("bucketName", null, null, null, null));
     assertEquals(HTTP_FORBIDDEN, e.getHttpCode());
   }
 
@@ -137,7 +137,7 @@ public void testDeleteBucket() throws IOException {
         .setClient(client)
         .build();
     OS3Exception e = assertThrows(OS3Exception.class, () ->
-        bucketEndpoint.delete("bucketName"));
+        bucketEndpoint.delete("bucketName", null));
     assertEquals(HTTP_FORBIDDEN, e.getHttpCode());
   }
   @Test
@@ -163,7 +163,7 @@ public void testListKey() throws IOException {
         .build();
     OS3Exception e = assertThrows(OS3Exception.class, () -> bucketEndpoint.get(
         "bucketName", null, null, null, 1000,
-        null, null, null, null, null, null, null, 0, null));
+        null, null, null, null, null, null, null, 0, null, null));
     assertEquals(HTTP_FORBIDDEN, e.getHttpCode());
   }
 
@@ -208,7 +208,7 @@ public void testGetAcl() throws Exception {
         .build();
     OS3Exception e = assertThrows(OS3Exception.class, () -> bucketEndpoint.get(
             "bucketName", null, null, null, 1000, null, null, null, null, 
"acl",
-            null, null, 0, null), "Expected OS3Exception with FORBIDDEN http 
code.");
+            null, null, 0, null, null), "Expected OS3Exception with FORBIDDEN 
http code.");
     assertEquals(HTTP_FORBIDDEN, e.getHttpCode());
   }
 
@@ -229,7 +229,7 @@ public void testSetAcl() throws Exception {
         .setClient(client)
         .build();
     try {
-      bucketEndpoint.put("bucketName", "acl", headers, null);
+      bucketEndpoint.put("bucketName", "acl", null, headers, null);
     } catch (Exception e) {
       assertTrue(e instanceof OS3Exception &&
           ((OS3Exception)e).getHttpCode() == HTTP_FORBIDDEN);
diff --git 
a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketPut.java
 
b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestS3LifecycleConfigurationDelete.java
similarity index 51%
copy from 
hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketPut.java
copy to 
hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestS3LifecycleConfigurationDelete.java
index 6f5d4d5d13b..507c6041272 100644
--- 
a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketPut.java
+++ 
b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestS3LifecycleConfigurationDelete.java
@@ -17,16 +17,17 @@
 
 package org.apache.hadoop.ozone.s3.endpoint;
 
-import static java.net.HttpURLConnection.HTTP_CONFLICT;
 import static java.net.HttpURLConnection.HTTP_NOT_FOUND;
-import static 
org.apache.hadoop.ozone.s3.exception.S3ErrorTable.BUCKET_ALREADY_EXISTS;
-import static 
org.apache.hadoop.ozone.s3.exception.S3ErrorTable.MALFORMED_HEADER;
+import static java.net.HttpURLConnection.HTTP_NO_CONTENT;
+import static 
org.apache.hadoop.ozone.s3.exception.S3ErrorTable.NO_SUCH_LIFECYCLE_CONFIGURATION;
 import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNotNull;
-import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.fail;
 
+import java.io.ByteArrayInputStream;
+import java.io.InputStream;
+import java.nio.charset.StandardCharsets;
 import javax.ws.rs.core.Response;
-import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.client.ObjectStore;
 import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.client.OzoneClientStub;
 import org.apache.hadoop.ozone.s3.exception.OS3Exception;
@@ -34,56 +35,66 @@
 import org.junit.jupiter.api.Test;
 
 /**
- * This class test Create Bucket functionality.
+ * Testing for DeleteBucketLifecycleConfiguration.
  */
-public class TestBucketPut {
-
-  private String bucketName = OzoneConsts.BUCKET;
+public class TestS3LifecycleConfigurationDelete {
   private OzoneClient clientStub;
   private BucketEndpoint bucketEndpoint;
 
   @BeforeEach
   public void setup() throws Exception {
-
-    //Create client stub and object store stub.
     clientStub = new OzoneClientStub();
-
-    // Create HeadBucket and setClient to OzoneClientStub
     bucketEndpoint = EndpointBuilder.newBucketEndpointBuilder()
         .setClient(clientStub)
         .build();
+    ObjectStore objectStore = clientStub.getObjectStore();
+    objectStore.createS3Bucket("bucket1");
   }
 
   @Test
-  public void testBucketFailWithAuthHeaderMissing() throws Exception {
+  public void testDeleteNonExistentLifecycleConfiguration()
+      throws Exception {
     try {
-      bucketEndpoint.put(bucketName, null, null, null);
+      bucketEndpoint.delete("bucket1", "");
+      fail();
     } catch (OS3Exception ex) {
       assertEquals(HTTP_NOT_FOUND, ex.getHttpCode());
-      assertEquals(MALFORMED_HEADER.getCode(), ex.getCode());
+      assertEquals(NO_SUCH_LIFECYCLE_CONFIGURATION.getCode(),
+              ex.getCode());
     }
   }
 
   @Test
-  public void testBucketPut() throws Exception {
-    Response response = bucketEndpoint.put(bucketName, null, null, null);
-    assertEquals(200, response.getStatus());
-    assertNotNull(response.getLocation());
+  public void testDeleteLifecycleConfiguration() throws Exception {
+    String bucketName = "bucket1";
+    bucketEndpoint.put(bucketName, null, "", null, getBody());
+    Response r = bucketEndpoint.delete(bucketName, "");
 
-    // Create-bucket on an existing bucket fails
-    OS3Exception e = assertThrows(OS3Exception.class, () -> bucketEndpoint.put(
-        bucketName, null, null, null));
-    assertEquals(HTTP_CONFLICT, e.getHttpCode());
-    assertEquals(BUCKET_ALREADY_EXISTS.getCode(), e.getCode());
-  }
+    assertEquals(HTTP_NO_CONTENT, r.getStatus());
 
-  @Test
-  public void testBucketFailWithInvalidHeader() throws Exception {
     try {
-      bucketEndpoint.put(bucketName, null, null, null);
+      // Make sure it was deleted.
+      bucketEndpoint.get(bucketName, null, null, null, 0, null, null,
+          null, null, null, null, null, 0, "", null);
+      fail();
     } catch (OS3Exception ex) {
       assertEquals(HTTP_NOT_FOUND, ex.getHttpCode());
-      assertEquals(MALFORMED_HEADER.getCode(), ex.getCode());
+      assertEquals(NO_SUCH_LIFECYCLE_CONFIGURATION.getCode(),
+          ex.getCode());
     }
   }
+
+  private static InputStream getBody() {
+    String xml = ("<LifecycleConfiguration xmlns=\"http://s3.amazonaws"; +
+        ".com/doc/2006-03-01/\">" +
+        "<Rule>" +
+        "<ID>remove logs after 30 days</ID>" +
+        "<Prefix>prefix/</Prefix>" +
+        "<Expiration><Days>30</Days></Expiration>" +
+        "<Status>Enabled</Status>" +
+        "</Rule>" +
+        "</LifecycleConfiguration>");
+
+    return new ByteArrayInputStream(xml.getBytes(StandardCharsets.UTF_8));
+  }
 }
diff --git 
a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestS3LifecycleConfigurationGet.java
 
b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestS3LifecycleConfigurationGet.java
new file mode 100644
index 00000000000..4411ce8e5e1
--- /dev/null
+++ 
b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestS3LifecycleConfigurationGet.java
@@ -0,0 +1,100 @@
+/*
+ * 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.s3.endpoint;
+
+import static java.net.HttpURLConnection.HTTP_NOT_FOUND;
+import static java.net.HttpURLConnection.HTTP_OK;
+import static 
org.apache.hadoop.ozone.s3.exception.S3ErrorTable.NO_SUCH_LIFECYCLE_CONFIGURATION;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.fail;
+
+import java.io.ByteArrayInputStream;
+import java.io.InputStream;
+import java.nio.charset.StandardCharsets;
+import javax.ws.rs.core.Response;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientStub;
+import org.apache.hadoop.ozone.s3.exception.OS3Exception;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Testing for GetBucketLifecycleConfiguration.
+ */
+public class TestS3LifecycleConfigurationGet {
+
+  private OzoneClient clientStub;
+  private BucketEndpoint bucketEndpoint;
+
+  @BeforeEach
+  public void setup() throws Exception {
+    clientStub = new OzoneClientStub();
+    bucketEndpoint = EndpointBuilder.newBucketEndpointBuilder()
+        .setClient(clientStub)
+        .build();
+    ObjectStore objectStore = clientStub.getObjectStore();
+    objectStore.createS3Bucket("bucket1");
+  }
+
+  @Test
+  public void testGetNonExistentLifecycleConfiguration()
+      throws Exception {
+    try {
+      bucketEndpoint.get("bucket1", null, null, null, 0, null, null,
+          null, null, null, null, null, 0, "", null);
+      fail();
+    } catch (OS3Exception ex) {
+      assertEquals(HTTP_NOT_FOUND, ex.getHttpCode());
+      assertEquals(NO_SUCH_LIFECYCLE_CONFIGURATION.getCode(),
+          ex.getCode());
+    }
+  }
+
+  @Test
+  public void testGetLifecycleConfiguration() throws Exception {
+    String bucketName = "bucket1";
+    bucketEndpoint.put(bucketName, null, "", null, getBody());
+    Response r = bucketEndpoint.get(bucketName, null, null, null, 0, null, 
null,
+        null, null, null, null, null, 0, "", null);
+
+    assertEquals(HTTP_OK, r.getStatus());
+    S3LifecycleConfiguration lcc =
+        (S3LifecycleConfiguration) r.getEntity();
+    assertEquals("remove logs after 30 days",
+        lcc.getRules().get(0).getId());
+    assertEquals("prefix/", lcc.getRules().get(0).getPrefix());
+    assertEquals("Enabled", lcc.getRules().get(0).getStatus());
+    assertEquals(30,
+        lcc.getRules().get(0).getExpiration().getDays().intValue());
+  }
+
+  private static InputStream getBody() {
+    String xml = ("<LifecycleConfiguration xmlns=\"http://s3.amazonaws"; +
+        ".com/doc/2006-03-01/\">" +
+        "<Rule>" +
+        "<ID>remove logs after 30 days</ID>" +
+        "<Prefix>prefix/</Prefix>" +
+        "<Expiration><Days>30</Days></Expiration>" +
+        "<Status>Enabled</Status>" +
+        "</Rule>" +
+        "</LifecycleConfiguration>");
+
+    return new ByteArrayInputStream(xml.getBytes(StandardCharsets.UTF_8));
+  }
+}
diff --git 
a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestS3LifecycleConfigurationPut.java
 
b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestS3LifecycleConfigurationPut.java
new file mode 100644
index 00000000000..ac0c533bd57
--- /dev/null
+++ 
b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestS3LifecycleConfigurationPut.java
@@ -0,0 +1,502 @@
+/*
+ * 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.s3.endpoint;
+
+import static java.net.HttpURLConnection.HTTP_BAD_REQUEST;
+import static java.net.HttpURLConnection.HTTP_FORBIDDEN;
+import static java.net.HttpURLConnection.HTTP_NOT_FOUND;
+import static java.net.HttpURLConnection.HTTP_OK;
+import static org.apache.hadoop.ozone.s3.exception.S3ErrorTable.ACCESS_DENIED;
+import static 
org.apache.hadoop.ozone.s3.exception.S3ErrorTable.INVALID_REQUEST;
+import static org.apache.hadoop.ozone.s3.exception.S3ErrorTable.MALFORMED_XML;
+import static org.apache.hadoop.ozone.s3.exception.S3ErrorTable.NO_SUCH_BUCKET;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.fail;
+import static org.mockito.Mockito.when;
+
+import java.io.ByteArrayInputStream;
+import java.io.InputStream;
+import java.nio.charset.StandardCharsets;
+import java.util.function.Supplier;
+import javax.ws.rs.core.HttpHeaders;
+import org.apache.hadoop.ozone.client.BucketArgs;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientStub;
+import org.apache.hadoop.ozone.s3.exception.OS3Exception;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+/**
+ * Testing for PutBucketLifecycleConfiguration.
+ */
+public class TestS3LifecycleConfigurationPut {
+
+  private OzoneClient clientStub;
+  private BucketEndpoint bucketEndpoint;
+
+  @BeforeEach
+  public void setup() throws Exception {
+    clientStub = new OzoneClientStub();
+    bucketEndpoint = EndpointBuilder.newBucketEndpointBuilder()
+        .setClient(clientStub)
+        .build();
+    ObjectStore objectStore = clientStub.getObjectStore();
+    BucketArgs bucketArgs = BucketArgs.newBuilder().setOwner("owner").build();
+    objectStore.createVolume("s3v");
+    objectStore.getS3Volume().createBucket("bucket1", bucketArgs);
+  }
+
+  @Test
+  public void testLifecycleConfigurationFailWithEmptyBody() throws Exception {
+    try {
+      bucketEndpoint.put("bucket1", null, "", null, null);
+      fail();
+    } catch (OS3Exception ex) {
+      assertEquals(HTTP_BAD_REQUEST, ex.getHttpCode());
+      assertEquals(MALFORMED_XML.getCode(), ex.getCode());
+    }
+  }
+
+  @Test
+  public void testLifecycleConfigurationFailWithNonExistentBucket()
+      throws Exception {
+    try {
+      bucketEndpoint.put("nonexistentbucket", null, "", null, onePrefix());
+      fail();
+    } catch (OS3Exception ex) {
+      assertEquals(HTTP_NOT_FOUND, ex.getHttpCode());
+      assertEquals(NO_SUCH_BUCKET.getCode(), ex.getCode());
+    }
+  }
+
+  @Test
+  public void testPutInvalidLifecycleConfiguration() throws Exception {
+    
testInvalidLifecycleConfiguration(TestS3LifecycleConfigurationPut::withoutAction,
+        HTTP_BAD_REQUEST, INVALID_REQUEST.getCode());
+    
testInvalidLifecycleConfiguration(TestS3LifecycleConfigurationPut::withoutFilter,
+        HTTP_BAD_REQUEST, INVALID_REQUEST.getCode());
+    testInvalidLifecycleConfiguration(this::useDuplicateTagInAndOperator,
+        HTTP_BAD_REQUEST, INVALID_REQUEST.getCode());
+    
testInvalidLifecycleConfiguration(this::usePrefixTagWithoutAndOperatorInFilter,
+        HTTP_BAD_REQUEST, INVALID_REQUEST.getCode());
+    
testInvalidLifecycleConfiguration(this::usePrefixAndOperatorCoExistInFilter,
+        HTTP_BAD_REQUEST, INVALID_REQUEST.getCode());
+    testInvalidLifecycleConfiguration(this::usePrefixFilterCoExist,
+        HTTP_BAD_REQUEST, INVALID_REQUEST.getCode());
+    testInvalidLifecycleConfiguration(this::useAndOperatorOnlyOnePrefix,
+        HTTP_BAD_REQUEST, INVALID_REQUEST.getCode());
+    testInvalidLifecycleConfiguration(this::useAndOperatorOnlyOneTag,
+        HTTP_BAD_REQUEST, INVALID_REQUEST.getCode());
+    testInvalidLifecycleConfiguration(this::useEmptyAndOperator,
+        HTTP_BAD_REQUEST, INVALID_REQUEST.getCode());
+  }
+
+  private void testInvalidLifecycleConfiguration(Supplier<InputStream> 
inputStream,
+      int expectedHttpCode, String expectedErrorCode) throws Exception {
+    try {
+      bucketEndpoint.put("bucket1", null, "", null, inputStream.get());
+      fail("Expected an OS3Exception to be thrown");
+    } catch (OS3Exception ex) {
+      assertEquals(expectedHttpCode, ex.getHttpCode());
+      assertEquals(expectedErrorCode, ex.getCode());
+    }
+  }
+
+  @Test
+  public void testPutInvalidExpirationDateLCC() throws Exception {
+    try {
+      String xml = ("<LifecycleConfiguration xmlns=\"http://s3.amazonaws"; +
+          ".com/doc/2006-03-01/\">" +
+          "<Rule>" +
+          "<ID>remove logs after 30 days</ID>" +
+          "<Prefix>prefix/</Prefix>" +
+          "<Status>Enabled</Status>" +
+          "<Expiration><Date>2023-03-03</Date></Expiration>" +
+          "</Rule>" +
+          "</LifecycleConfiguration>");
+
+      bucketEndpoint.put("bucket1", null, "", null,
+          new ByteArrayInputStream(xml.getBytes(StandardCharsets.UTF_8)));
+      fail();
+    } catch (OS3Exception ex) {
+      assertEquals(HTTP_BAD_REQUEST, ex.getHttpCode());
+      assertEquals(INVALID_REQUEST.getCode(), ex.getCode());
+    }
+  }
+
+  @Test
+  public void testPutValidLifecycleConfiguration() throws Exception {
+    assertEquals(HTTP_OK, bucketEndpoint.put(
+        "bucket1", null, "", null, onePrefix()).getStatus());
+    assertEquals(HTTP_OK, bucketEndpoint.put(
+        "bucket1", null, "", null, emptyPrefix()).getStatus());
+    assertEquals(HTTP_OK, bucketEndpoint.put(
+        "bucket1", null, "", null, oneTag()).getStatus());
+    assertEquals(HTTP_OK, bucketEndpoint.put(
+        "bucket1", null, "", null, twoTagsInAndOperator()).getStatus());
+    assertEquals(HTTP_OK, bucketEndpoint.put(
+        "bucket1", null, "", null, 
onePrefixTwoTagsInAndOperator()).getStatus());
+    assertEquals(HTTP_OK, bucketEndpoint.put(
+        "bucket1", null, "", null, onePrefixTwoTags()).getStatus());
+  }
+
+  @Test
+  public void testPutLifecycleConfigurationFailsWithNonBucketOwner()
+      throws Exception {
+    HttpHeaders httpHeaders = Mockito.mock(HttpHeaders.class);
+    when(httpHeaders.getHeaderString("x-amz-expected-bucket-owner"))
+        .thenReturn("anotheruser");
+
+    try {
+      bucketEndpoint.put("bucket1", null, "", httpHeaders, onePrefix());
+      fail();
+    } catch (OS3Exception ex) {
+      assertEquals(HTTP_FORBIDDEN, ex.getHttpCode());
+      assertEquals(ACCESS_DENIED.getCode(), ex.getCode());
+    }
+  }
+
+  private static InputStream onePrefix() {
+    String xml = ("<LifecycleConfiguration xmlns=\"http://s3.amazonaws"; +
+        ".com/doc/2006-03-01/\">" +
+        "<Rule>" +
+        "<ID>remove logs after 30 days</ID>" +
+        "<Prefix>prefix/</Prefix>" +
+        "<Status>Enabled</Status>" +
+        "<Expiration><Days>30</Days></Expiration>" +
+        "</Rule>" +
+        "</LifecycleConfiguration>");
+
+    return new ByteArrayInputStream(xml.getBytes(StandardCharsets.UTF_8));
+  }
+
+  private static InputStream withoutAction() {
+    String xml = (
+        "<LifecycleConfiguration xmlns=\"http://s3.amazonaws"; +
+        ".com/doc/2006-03-01/\">" +
+        "<Rule>" +
+        "<ID>remove logs after 30 days</ID>" +
+        "<Prefix>prefix/</Prefix>" +
+        "<Status>Enabled</Status>" +
+        "</Rule>" +
+        "</LifecycleConfiguration>");
+    return new ByteArrayInputStream(xml.getBytes(StandardCharsets.UTF_8));
+  }
+
+
+  private static InputStream withoutFilter() {
+    String xml =
+            "<LifecycleConfiguration 
xmlns=\"http://s3.amazonaws.com/doc/2006-03-01/\";>" +
+            "     <Rule>" +
+            "         <Expiration>" +
+            "             <Date>2044-01-19T00:00:00+00:00</Date>" +
+            "         </Expiration>" +
+            "         <ID>12334</ID>" +
+            "         <Status>Enabled</Status>" +
+            "     </Rule>" +
+            "</LifecycleConfiguration>";
+
+    return new ByteArrayInputStream(xml.getBytes(StandardCharsets.UTF_8));
+  }
+
+  private static InputStream twoTagsInAndOperator() {
+    String xml =
+        "<LifecycleConfiguration 
xmlns=\"http://s3.amazonaws.com/doc/2006-03-01/\";>" +
+            "     <Rule>" +
+            "         <Expiration>" +
+            "             <Date>2044-01-19T00:00:00+00:00</Date>" +
+            "         </Expiration>" +
+            "         <ID>12334</ID>" +
+            "         <Filter>" +
+            "             <And>" +
+            "                 <Tag>" +
+            "                     <Key>key1</Key>" +
+            "                     <Value>value1</Value>" +
+            "                 </Tag>" +
+            "                 <Tag>" +
+            "                     <Key>key2</Key>" +
+            "                     <Value>value1</Value>" +
+            "                 </Tag>" +
+            "             </And>" +
+            "         </Filter>" +
+            "         <Status>Enabled</Status>" +
+            "     </Rule>" +
+            "</LifecycleConfiguration>";
+
+    return new ByteArrayInputStream(xml.getBytes(StandardCharsets.UTF_8));
+  }
+
+  private static InputStream onePrefixTwoTagsInAndOperator() {
+    String xml =
+        "<LifecycleConfiguration 
xmlns=\"http://s3.amazonaws.com/doc/2006-03-01/\";>" +
+            "     <Rule>" +
+            "         <Expiration>" +
+            "             <Date>2044-01-19T00:00:00+00:00</Date>" +
+            "         </Expiration>" +
+            "         <ID>12334</ID>" +
+            "         <Filter>" +
+            "             <And>" +
+            "                 <Prefix></Prefix>" +
+            "                 <Tag>" +
+            "                     <Key>key1</Key>" +
+            "                     <Value>value1</Value>" +
+            "                 </Tag>" +
+            "                 <Tag>" +
+            "                     <Key>key2</Key>" +
+            "                     <Value>value1</Value>" +
+            "                 </Tag>" +
+            "             </And>" +
+            "         </Filter>" +
+            "         <Status>Enabled</Status>" +
+            "     </Rule>" +
+            "</LifecycleConfiguration>";
+
+    return new ByteArrayInputStream(xml.getBytes(StandardCharsets.UTF_8));
+  }
+
+  private static InputStream onePrefixTwoTags() {
+    String xml =
+        "<LifecycleConfiguration 
xmlns=\"http://s3.amazonaws.com/doc/2006-03-01/\";>" +
+            "     <Rule>" +
+            "         <Expiration>" +
+            "             <Date>2044-01-19T00:00:00+00:00</Date>" +
+            "         </Expiration>" +
+            "         <ID>12334</ID>" +
+            "         <Filter>" +
+            "             <And>" +
+            "                 <Prefix>key-prefix</Prefix>" +
+            "                 <Tag>" +
+            "                     <Key>key1</Key>" +
+            "                     <Value>value1</Value>" +
+            "                 </Tag>" +
+            "                 <Tag>" +
+            "                     <Key>key2</Key>" +
+            "                     <Value>value1</Value>" +
+            "                 </Tag>" +
+            "             </And>" +
+            "         </Filter>" +
+            "         <Status>Enabled</Status>" +
+            "     </Rule>" +
+            "</LifecycleConfiguration>";
+
+    return new ByteArrayInputStream(xml.getBytes(StandardCharsets.UTF_8));
+  }
+
+  private static InputStream emptyPrefix() {
+    String xml =
+        "<LifecycleConfiguration 
xmlns=\"http://s3.amazonaws.com/doc/2006-03-01/\";>" +
+            "     <Rule>" +
+            "         <Expiration>" +
+            "             <Date>2044-01-19T00:00:00+00:00</Date>" +
+            "         </Expiration>" +
+            "         <ID>12334</ID>" +
+            "         <Filter>" +
+            "             <Prefix></Prefix>" +
+            "         </Filter>" +
+            "         <Status>Enabled</Status>" +
+            "     </Rule>" +
+            "</LifecycleConfiguration>";
+
+    return new ByteArrayInputStream(xml.getBytes(StandardCharsets.UTF_8));
+  }
+
+  private static InputStream oneTag() {
+    String xml =
+        "<LifecycleConfiguration 
xmlns=\"http://s3.amazonaws.com/doc/2006-03-01/\";>" +
+            "     <Rule>" +
+            "         <Expiration>" +
+            "             <Date>2044-01-19T00:00:00+00:00</Date>" +
+            "         </Expiration>" +
+            "         <ID>12334</ID>" +
+            "         <Filter>" +
+            "             <Tag>" +
+            "                 <Key>key1</Key>" +
+            "                 <Value>value1</Value>" +
+            "             </Tag>" +
+            "         </Filter>" +
+            "         <Status>Enabled</Status>" +
+            "     </Rule>" +
+            "</LifecycleConfiguration>";
+
+    return new ByteArrayInputStream(xml.getBytes(StandardCharsets.UTF_8));
+  }
+
+  private InputStream usePrefixTagWithoutAndOperatorInFilter() {
+    String xml =
+        "<LifecycleConfiguration 
xmlns=\"http://s3.amazonaws.com/doc/2006-03-01/\";>" +
+            "     <Rule>" +
+            "         <Expiration>" +
+            "             <Date>2044-01-19T00:00:00+00:00</Date>" +
+            "         </Expiration>" +
+            "         <ID>12334</ID>" +
+            "         <Filter>" +
+            "             <Prefix>key-prefix</Prefix>" +
+            "             <Tag>" +
+            "                 <Key>key2</Key>" +
+            "                 <Value>value1</Value>" +
+            "             </Tag>" +
+            "         </Filter>" +
+            "         <Status>Enabled</Status>" +
+            "     </Rule>" +
+            "</LifecycleConfiguration>";
+
+    return new ByteArrayInputStream(xml.getBytes(StandardCharsets.UTF_8));
+  }
+
+  private InputStream usePrefixAndOperatorCoExistInFilter() {
+    String xml =
+        "<LifecycleConfiguration 
xmlns=\"http://s3.amazonaws.com/doc/2006-03-01/\";>" +
+            "     <Rule>" +
+            "         <Expiration>" +
+            "             <Date>2044-01-19T00:00:00+00:00</Date>" +
+            "         </Expiration>" +
+            "         <ID>12334</ID>" +
+            "         <Filter>" +
+            "             <Prefix>key-prefix</Prefix>" +
+            "             <And>" +
+                "             <Prefix>key-prefix</Prefix>" +
+            "                 <Tag>" +
+            "                     <Key>key2</Key>" +
+            "                     <Value>value1</Value>" +
+            "                 </Tag>" +
+            "             </And>" +
+            "         </Filter>" +
+            "         <Status>Enabled</Status>" +
+            "     </Rule>" +
+            "</LifecycleConfiguration>";
+
+    return new ByteArrayInputStream(xml.getBytes(StandardCharsets.UTF_8));
+  }
+
+  private InputStream useAndOperatorOnlyOnePrefix() {
+    String xml =
+        "<LifecycleConfiguration 
xmlns=\"http://s3.amazonaws.com/doc/2006-03-01/\";>" +
+            "     <Rule>" +
+            "         <Expiration>" +
+            "             <Date>2044-01-19T00:00:00+00:00</Date>" +
+            "         </Expiration>" +
+            "         <ID>12334</ID>" +
+            "         <Filter>" +
+            "             <And>" +
+            "                 <Prefix>key-prefix</Prefix>" +
+            "             </And>" +
+            "         </Filter>" +
+            "         <Status>Enabled</Status>" +
+            "     </Rule>" +
+            "</LifecycleConfiguration>";
+
+    return new ByteArrayInputStream(xml.getBytes(StandardCharsets.UTF_8));
+  }
+
+  private InputStream useAndOperatorOnlyOneTag() {
+    String xml =
+        "<LifecycleConfiguration 
xmlns=\"http://s3.amazonaws.com/doc/2006-03-01/\";>" +
+            "     <Rule>" +
+            "         <Expiration>" +
+            "             <Date>2044-01-19T00:00:00+00:00</Date>" +
+            "         </Expiration>" +
+            "         <ID>12334</ID>" +
+            "         <Filter>" +
+            "             <And>" +
+            "                 <Tag>" +
+            "                     <Key>key2</Key>" +
+            "                     <Value>value1</Value>" +
+            "                 </Tag>" +
+            "             </And>" +
+            "         </Filter>" +
+            "         <Status>Enabled</Status>" +
+            "     </Rule>" +
+            "</LifecycleConfiguration>";
+
+    return new ByteArrayInputStream(xml.getBytes(StandardCharsets.UTF_8));
+  }
+
+  private InputStream useEmptyAndOperator() {
+    String xml =
+        "<LifecycleConfiguration 
xmlns=\"http://s3.amazonaws.com/doc/2006-03-01/\";>" +
+            "     <Rule>" +
+            "         <Expiration>" +
+            "             <Date>2044-01-19T00:00:00+00:00</Date>" +
+            "         </Expiration>" +
+            "         <ID>12334</ID>" +
+            "         <Filter>" +
+            "             <And>" +
+            "             </And>" +
+            "         </Filter>" +
+            "         <Status>Enabled</Status>" +
+            "     </Rule>" +
+            "</LifecycleConfiguration>";
+
+    return new ByteArrayInputStream(xml.getBytes(StandardCharsets.UTF_8));
+  }
+
+  private InputStream usePrefixFilterCoExist() {
+    String xml =
+        "<LifecycleConfiguration 
xmlns=\"http://s3.amazonaws.com/doc/2006-03-01/\";>" +
+            "     <Rule>" +
+            "         <Expiration>" +
+            "             <Date>2044-01-19T00:00:00+00:00</Date>" +
+            "         </Expiration>" +
+            "         <ID>12334</ID>" +
+            "         <Prefix>key-prefix</Prefix>" +
+            "         <Filter>" +
+            "             <Tag>" +
+            "                 <Key>key1</Key>" +
+            "                 <Value>value1</Value>" +
+            "             </Tag>" +
+            "         </Filter>" +
+            "         <Status>Enabled</Status>" +
+            "     </Rule>" +
+            "</LifecycleConfiguration>";
+
+    return new ByteArrayInputStream(xml.getBytes(StandardCharsets.UTF_8));
+  }
+
+  private InputStream useDuplicateTagInAndOperator() {
+    String xml =
+        "<LifecycleConfiguration 
xmlns=\"http://s3.amazonaws.com/doc/2006-03-01/\";>" +
+            "     <Rule>" +
+            "         <Expiration>" +
+            "             <Date>2044-01-19T00:00:00+00:00</Date>" +
+            "         </Expiration>" +
+            "         <ID>12334</ID>" +
+            "         <Filter>" +
+            "             <And>" +
+            "                 <Prefix>key-prefix</Prefix>" +
+            "                 <Tag>" +
+            "                     <Key>key1</Key>" +
+            "                     <Value>value1</Value>" +
+            "                 </Tag>" +
+            "                 <Tag>" +
+            "                     <Key>key1</Key>" +
+            "                     <Value>value2</Value>" +
+            "                 </Tag>" +
+            "             </And>" +
+            "         </Filter>" +
+            "         <Status>Enabled</Status>" +
+            "     </Rule>" +
+            "</LifecycleConfiguration>";
+
+    return new ByteArrayInputStream(xml.getBytes(StandardCharsets.UTF_8));
+  }
+
+
+
+}
diff --git 
a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/metrics/TestS3GatewayMetrics.java
 
b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/metrics/TestS3GatewayMetrics.java
index 0081d0d5abc..d5660121c7f 100644
--- 
a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/metrics/TestS3GatewayMetrics.java
+++ 
b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/metrics/TestS3GatewayMetrics.java
@@ -142,7 +142,7 @@ public void testGetBucketSuccess() throws Exception {
     bucketEndpoint.get(bucketName, null,
         null, null, 1000, null,
         null, "random", null,
-        null, null, null, 0, null).getEntity();
+        null, null, null, 0, null, null).getEntity();
 
     long curMetric = metrics.getGetBucketSuccess();
     assertEquals(1L, curMetric - oriMetric);
@@ -155,7 +155,7 @@ public void testGetBucketFailure() throws Exception {
     // Searching for a bucket that does not exist
     OS3Exception e = assertThrows(OS3Exception.class, () -> bucketEndpoint.get(
         "newBucket", null, null, null, 1000, null, null, "random", null,
-        null, null, null, 0, null));
+        null, null, null, 0, null, null));
     assertEquals(S3ErrorTable.NO_SUCH_BUCKET.getCode(), e.getCode());
     assertEquals(S3ErrorTable.NO_SUCH_BUCKET.getErrorMessage(),
         e.getErrorMessage());
@@ -167,7 +167,7 @@ public void testGetBucketFailure() throws Exception {
   public void testCreateBucketSuccess() throws Exception {
 
     long oriMetric = metrics.getCreateBucketSuccess();
-    assertDoesNotThrow(() -> bucketEndpoint.put("newBucket", null, null, 
null));
+    assertDoesNotThrow(() -> bucketEndpoint.put("newBucket", null, null, null, 
null));
     long curMetric = metrics.getCreateBucketSuccess();
     assertEquals(1L, curMetric - oriMetric);
   }
@@ -178,7 +178,7 @@ public void testCreateBucketFailure() throws Exception {
 
     // Creating an error by trying to create a bucket that already exists
     OS3Exception e = assertThrows(OS3Exception.class, () -> bucketEndpoint.put(
-        bucketName, null, null, null));
+        bucketName, null, null, null, null));
     assertEquals(HTTP_CONFLICT, e.getHttpCode());
     assertEquals(BUCKET_ALREADY_EXISTS.getCode(), e.getCode());
 
@@ -190,7 +190,7 @@ public void testCreateBucketFailure() throws Exception {
   public void testDeleteBucketSuccess() throws Exception {
     long oriMetric = metrics.getDeleteBucketSuccess();
 
-    bucketEndpoint.delete(bucketName);
+    bucketEndpoint.delete(bucketName, null);
 
     long curMetric = metrics.getDeleteBucketSuccess();
     assertEquals(1L, curMetric - oriMetric);
@@ -199,11 +199,11 @@ public void testDeleteBucketSuccess() throws Exception {
   @Test
   public void testDeleteBucketFailure() throws Exception {
     long oriMetric = metrics.getDeleteBucketFailure();
-    bucketEndpoint.delete(bucketName);
+    bucketEndpoint.delete(bucketName, null);
 
     // Deleting a bucket that does not exist will result in delete failure
     OS3Exception e = assertThrows(OS3Exception.class, () ->
-        bucketEndpoint.delete(bucketName));
+        bucketEndpoint.delete(bucketName, null));
     assertEquals(S3ErrorTable.NO_SUCH_BUCKET.getCode(), e.getCode());
     assertEquals(S3ErrorTable.NO_SUCH_BUCKET.getErrorMessage(),
         e.getErrorMessage());
@@ -219,7 +219,7 @@ public void testGetAclSuccess() throws Exception {
     Response response =
         bucketEndpoint.get(bucketName, null, null,
             null, 0, null, null,
-            null, null, "acl", null, null, 0, null);
+            null, null, "acl", null, null, 0, null, null);
     long curMetric = metrics.getGetAclSuccess();
     assertEquals(HTTP_OK, response.getStatus());
     assertEquals(1L, curMetric - oriMetric);
@@ -232,7 +232,7 @@ public void testGetAclFailure() throws Exception {
     // Failing the getACL endpoint by applying ACL on a non-Existent Bucket
     OS3Exception e = assertThrows(OS3Exception.class, () -> bucketEndpoint.get(
         "random_bucket", null, null, null, 0, null,
-        null, null, null, "acl", null, null, 0, null));
+        null, null, null, "acl", null, null, 0, null, null));
     assertEquals(S3ErrorTable.NO_SUCH_BUCKET.getCode(), e.getCode());
     assertEquals(S3ErrorTable.NO_SUCH_BUCKET.getErrorMessage(),
         e.getErrorMessage());
@@ -248,7 +248,7 @@ public void testPutAclSuccess() throws Exception {
     InputStream inputBody = TestBucketAcl.class.getClassLoader()
         .getResourceAsStream("userAccessControlList.xml");
 
-    bucketEndpoint.put("b1", ACL_MARKER, headers, inputBody);
+    bucketEndpoint.put("b1", ACL_MARKER, null, headers, inputBody);
     inputBody.close();
     long curMetric = metrics.getPutAclSuccess();
     assertEquals(1L, curMetric - oriMetric);
@@ -262,7 +262,7 @@ public void testPutAclFailure() throws Exception {
     InputStream inputBody = TestBucketAcl.class.getClassLoader()
         .getResourceAsStream("userAccessControlList.xml");
     try {
-      assertThrows(OS3Exception.class, () -> 
bucketEndpoint.put("unknown_bucket", ACL_MARKER, headers,
+      assertThrows(OS3Exception.class, () -> 
bucketEndpoint.put("unknown_bucket", ACL_MARKER, null, headers,
           inputBody));
     } finally {
       inputBody.close();


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to