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 ce01a02542 HDDS-12782. Implement create, delete, and get for bucket
lifecycle configurations in OM (#8875)
ce01a02542 is described below
commit ce01a02542cbed833c1ce13825393968eb961523
Author: XiChen <[email protected]>
AuthorDate: Thu Aug 7 11:02:57 2025 +0800
HDDS-12782. Implement create, delete, and get for bucket lifecycle
configurations in OM (#8875)
Co-authored-by: Mohanad Elsafty <[email protected]>
---
.../apache/hadoop/ozone/client/OzoneBucket.java | 2 +-
.../ozone/client/OzoneLifecycleConfiguration.java | 37 +++-
.../ozone/client/protocol/ClientProtocol.java | 4 +-
.../apache/hadoop/ozone/client/rpc/RpcClient.java | 24 ++-
.../main/java/org/apache/hadoop/ozone/OmUtils.java | 5 +-
.../apache/hadoop/ozone/om/helpers/OmLCRule.java | 4 +
.../ozone/om/helpers/OmLifecycleConfiguration.java | 5 +
.../ozone/om/protocol/OzoneManagerProtocol.java | 36 ++++
...OzoneManagerProtocolClientSideTranslatorPB.java | 60 +++++-
.../ozone/s3/awssdk/v1/AbstractS3SDKV1Tests.java | 162 ++++++++++++++
.../ozone/client/rpc/OzoneRpcClientTests.java | 197 +++++++++++++++++
.../src/main/proto/OmClientProtocol.proto | 37 ++++
.../apache/hadoop/ozone/om/OMMetadataManager.java | 10 +
.../org/apache/hadoop/ozone/audit/OMAction.java | 6 +-
.../hadoop/ozone/om/OmMetadataManagerImpl.java | 34 +++
.../org/apache/hadoop/ozone/om/OzoneManager.java | 46 ++++
.../om/ratis/utils/OzoneManagerRatisUtils.java | 6 +
.../om/request/bucket/OMBucketDeleteRequest.java | 12 +-
.../OMLifecycleConfigurationDeleteRequest.java | 172 +++++++++++++++
.../OMLifecycleConfigurationSetRequest.java | 191 ++++++++++++++++
.../ozone/om/request/lifecycle/package-info.java | 21 ++
.../om/response/bucket/OMBucketDeleteResponse.java | 8 +
.../OMLifecycleConfigurationDeleteResponse.java | 65 ++++++
.../OMLifecycleConfigurationSetResponse.java | 64 ++++++
.../ozone/om/response/lifecycle/package-info.java | 21 ++
.../protocolPB/OzoneManagerRequestHandler.java | 27 +++
.../TestOMLifecycleConfigurationDeleteRequest.java | 152 +++++++++++++
.../TestOMLifecycleConfigurationRequest.java | 152 +++++++++++++
.../TestOMLifecycleConfigurationSetRequest.java | 239 +++++++++++++++++++++
...TestOMLifecycleConfigurationDeleteResponse.java | 117 ++++++++++
.../TestOMLifecycleConfigurationSetResponse.java | 125 +++++++++++
.../hadoop/ozone/s3/endpoint/BucketEndpoint.java | 2 +-
.../s3/endpoint/S3LifecycleConfiguration.java | 21 +-
.../hadoop/ozone/client/ClientProtocolStub.java | 2 +-
34 files changed, 2033 insertions(+), 33 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 29eb9377e9..5d1b007c11 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
@@ -1113,7 +1113,7 @@ public OzoneLifecycleConfiguration
getLifecycleConfiguration()
*/
public void setLifecycleConfiguration(OmLifecycleConfiguration
lifecycleConfiguration)
throws IOException {
- proxy.createLifecycleConfiguration(lifecycleConfiguration);
+ proxy.setLifecycleConfiguration(lifecycleConfiguration);
}
/**
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
index 5ab47f8524..ab4b881e65 100644
---
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
@@ -17,9 +17,12 @@
package org.apache.hadoop.ozone.client;
+import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.ozone.om.helpers.OmLCRule;
+import org.apache.hadoop.ozone.om.helpers.OmLifecycleConfiguration;
/**
* A class that encapsulates OzoneLifecycleConfiguration.
@@ -120,7 +123,7 @@ public static class OzoneLCRule {
private final OzoneLCFilter filter;
public OzoneLCRule(String id, String prefix, String status,
- OzoneLCExpiration expiration, OzoneLCFilter filter) {
+ OzoneLCExpiration expiration, OzoneLCFilter filter) {
this.id = id;
this.prefix = prefix;
this.status = status;
@@ -164,4 +167,36 @@ public long getCreationTime() {
public List<OzoneLCRule> getRules() {
return rules;
}
+
+ public static OzoneLifecycleConfiguration fromOmLifecycleConfiguration(
+ OmLifecycleConfiguration lifecycleConfiguration) {
+ List<OmLCRule> omLCRules = lifecycleConfiguration.getRules();
+ List<OzoneLifecycleConfiguration.OzoneLCRule> rules = new ArrayList<>();
+
+ for (OmLCRule r: omLCRules) {
+
+ OzoneLifecycleConfiguration.OzoneLCExpiration e = null;
+ if (r.getExpiration() != null) {
+ e = new OzoneLifecycleConfiguration.OzoneLCExpiration(
+ r.getExpiration().getDays(), r.getExpiration().getDate());
+ }
+
+ OzoneLifecycleConfiguration.OzoneLCFilter f = null;
+ if (r.getFilter() != null) {
+ LifecycleAndOperator andOperator = null;
+ if (r.getFilter().getAndOperator() != null) {
+ andOperator = new
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(lifecycleConfiguration.getVolume(),
+ lifecycleConfiguration.getBucket(),
lifecycleConfiguration.getCreationTime(), 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 c8f52a466b..3bd0c10003 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
@@ -1357,10 +1357,12 @@ OzoneLifecycleConfiguration
getLifecycleConfiguration(String volumeName, String
/**
* Creates a new lifecycle configuration.
+ * 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.
* @throws IOException
*/
- void createLifecycleConfiguration(OmLifecycleConfiguration
lifecycleConfiguration)
+ void setLifecycleConfiguration(OmLifecycleConfiguration
lifecycleConfiguration)
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 f81d6ad2e4..223cc6ab23 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
@@ -2790,17 +2790,33 @@ public void deleteObjectTagging(String volumeName,
String bucketName,
@Override
public OzoneLifecycleConfiguration getLifecycleConfiguration(String
volumeName, String bucketName)
throws IOException {
- throw new UnsupportedOperationException("Lifecycle configuration
operations are not yet supported");
+ verifyVolumeName(volumeName);
+ verifyBucketName(bucketName);
+
+ OmLifecycleConfiguration lifecycleConfiguration =
+ ozoneManagerClient.getLifecycleConfiguration(volumeName, bucketName);
+ return OzoneLifecycleConfiguration.fromOmLifecycleConfiguration(
+ lifecycleConfiguration);
}
@Override
- public void createLifecycleConfiguration(OmLifecycleConfiguration
lifecycleConfiguration) throws IOException {
- throw new UnsupportedOperationException("Lifecycle configuration
operations are not yet supported");
+ public void setLifecycleConfiguration(OmLifecycleConfiguration
lifecycleConfiguration) throws IOException {
+ Preconditions.checkNotNull(lifecycleConfiguration);
+ verifyVolumeName(lifecycleConfiguration.getVolume());
+ verifyBucketName(lifecycleConfiguration.getBucket());
+
+ LOG.info("Creating lifecycle configuration for: {}/{}",
lifecycleConfiguration.getVolume(),
+ lifecycleConfiguration.getBucket());
+ ozoneManagerClient.setLifecycleConfiguration(lifecycleConfiguration);
}
@Override
public void deleteLifecycleConfiguration(String volumeName, String
bucketName) throws IOException {
- throw new UnsupportedOperationException("Lifecycle configuration
operations are not yet supported");
+ verifyVolumeName(volumeName);
+ verifyBucketName(bucketName);
+
+ LOG.info("Deleting lifecycle Configuration for : {}/{}", volumeName,
bucketName);
+ ozoneManagerClient.deleteLifecycleConfiguration(volumeName, bucketName);
}
private static ExecutorService createThreadPoolExecutor(
diff --git
a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java
b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java
index 100ff74ed5..9a945dfea8 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java
@@ -277,6 +277,7 @@ public static boolean isReadOnly(
case GetObjectTagging:
case GetQuotaRepairStatus:
case StartQuotaRepair:
+ case GetLifecycleConfiguration:
return true;
case CreateVolume:
case SetVolumeProperty:
@@ -338,6 +339,8 @@ public static boolean isReadOnly(
case QuotaRepair:
case PutObjectTagging:
case DeleteObjectTagging:
+ case SetLifecycleConfiguration:
+ case DeleteLifecycleConfiguration:
case UnknownCommand:
return false;
case EchoRPC:
@@ -916,7 +919,7 @@ public static String
getOMAddressListPrintString(List<OMNodeDetails> omList) {
public static boolean isBucketSnapshotIndicator(String key) {
return key.startsWith(OM_SNAPSHOT_INDICATOR) && key.split("/").length == 2;
}
-
+
public static List<List<String>> format(
List<ServiceInfo> nodes, int port, String leaderId, String
leaderReadiness) {
List<List<String>> omInfoList = new ArrayList<>();
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 9d71129792..b8a6386551 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
@@ -89,6 +89,10 @@ public String getId() {
return id;
}
+ public String getPrefix() {
+ return prefix;
+ }
+
@Nullable
public String getEffectivePrefix() {
return prefix != null ? prefix :
diff --git
a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmLifecycleConfiguration.java
b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmLifecycleConfiguration.java
index 74ee18c1fd..d711e3cb93 100644
---
a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmLifecycleConfiguration.java
+++
b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmLifecycleConfiguration.java
@@ -89,6 +89,10 @@ public long getCreationTime() {
return creationTime;
}
+ public BucketLayout getBucketLayout() {
+ return bucketLayout;
+ }
+
/**
* Validates the lifecycle configuration.
* - Volume and Bucket cannot be blank
@@ -141,6 +145,7 @@ public Builder toBuilder() {
return new Builder(this)
.setVolume(this.volume)
.setBucket(this.bucket)
+ .setBucketLayout(bucketLayout)
.setCreationTime(this.creationTime)
.setRules(this.rules);
}
diff --git
a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/OzoneManagerProtocol.java
b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/OzoneManagerProtocol.java
index 37cd50d10d..626bef6504 100644
---
a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/OzoneManagerProtocol.java
+++
b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/OzoneManagerProtocol.java
@@ -41,6 +41,7 @@
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmLifecycleConfiguration;
import org.apache.hadoop.ozone.om.helpers.OmMultipartCommitUploadPartInfo;
import org.apache.hadoop.ozone.om.helpers.OmMultipartInfo;
import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadCompleteInfo;
@@ -1190,4 +1191,39 @@ default void deleteObjectTagging(OmKeyArgs args) throws
IOException {
* @throws IOException
*/
void startQuotaRepair(List<String> buckets) throws IOException;
+
+ /**
+ * Gets the lifecycle configuration information.
+ * @param volumeName - Volume name.
+ * @param bucketName - Bucket name.
+ * @return OmLifecycleConfiguration or exception is thrown.
+ * @throws IOException
+ */
+ OmLifecycleConfiguration getLifecycleConfiguration(String volumeName,
+ String bucketName) throws IOException;
+
+ /**
+ * Creates a new lifecycle configuration.
+ * 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.
+ * @throws IOException
+ */
+ default void setLifecycleConfiguration(
+ OmLifecycleConfiguration lifecycleConfiguration) throws IOException {
+ throw new UnsupportedOperationException("OzoneManager does not require " +
+ "this to be implemented, as write requests use a new approach.");
+ }
+
+ /**
+ * Deletes existing lifecycle configuration.
+ * @param volumeName - Volume name.
+ * @param bucketName - Bucket name.
+ * @throws IOException
+ */
+ default void deleteLifecycleConfiguration(String volumeName,
+ String bucketName) throws IOException {
+ throw new UnsupportedOperationException("OzoneManager does not require " +
+ "this to be implemented, as write requests use a new approach.");
+ }
}
diff --git
a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java
b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java
index 9e2227525c..3a2d41c0ca 100644
---
a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java
+++
b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java
@@ -69,6 +69,7 @@
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmLifecycleConfiguration;
import org.apache.hadoop.ozone.om.helpers.OmMultipartCommitUploadPartInfo;
import org.apache.hadoop.ozone.om.helpers.OmMultipartInfo;
import org.apache.hadoop.ozone.om.helpers.OmMultipartUpload;
@@ -119,6 +120,7 @@
import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteKeyArgs;
import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteKeyRequest;
import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteKeysRequest;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteLifecycleConfigurationRequest;
import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteObjectTaggingRequest;
import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteSnapshotRequest;
import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteTenantRequest;
@@ -137,6 +139,8 @@
import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.GetFileStatusResponse;
import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.GetKeyInfoRequest;
import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.GetKeyInfoResponse;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.GetLifecycleConfigurationRequest;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.GetLifecycleConfigurationResponse;
import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.GetObjectTaggingRequest;
import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.GetObjectTaggingResponse;
import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.GetS3SecretRequest;
@@ -148,6 +152,7 @@
import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.InfoVolumeRequest;
import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.InfoVolumeResponse;
import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyArgs;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.LifecycleConfiguration;
import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ListBucketsRequest;
import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ListBucketsResponse;
import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ListKeysLightResponse;
@@ -213,6 +218,7 @@
import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SetAclResponse;
import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SetBucketPropertyRequest;
import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SetBucketPropertyResponse;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SetLifecycleConfigurationRequest;
import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SetS3SecretRequest;
import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SetS3SecretResponse;
import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SetSafeModeRequest;
@@ -1253,7 +1259,7 @@ public String createSnapshot(String volumeName,
if (!StringUtils.isBlank(snapshotName)) {
requestBuilder.setSnapshotName(snapshotName);
}
-
+
final OMRequest omRequest = createOMRequest(Type.CreateSnapshot)
.setCreateSnapshotRequest(requestBuilder)
.build();
@@ -2666,6 +2672,58 @@ public void deleteObjectTagging(OmKeyArgs args) throws
IOException {
handleError(omResponse);
}
+ @Override
+ public OmLifecycleConfiguration getLifecycleConfiguration(String volumeName,
+ String bucketName) throws IOException {
+ GetLifecycleConfigurationRequest.Builder req =
+ GetLifecycleConfigurationRequest.newBuilder();
+ req.setVolumeName(volumeName);
+ req.setBucketName(bucketName);
+
+ OMRequest omRequest = createOMRequest(Type.GetLifecycleConfiguration)
+ .setGetLifecycleConfigurationRequest(req)
+ .build();
+
+ GetLifecycleConfigurationResponse resp = handleError(submitRequest(
+ omRequest)).getGetLifecycleConfigurationResponse();
+
+ return OmLifecycleConfiguration.getFromProtobuf(
+ resp.getLifecycleConfiguration());
+ }
+
+ @Override
+ public void setLifecycleConfiguration(
+ OmLifecycleConfiguration omLifecycleConfiguration) throws IOException {
+ SetLifecycleConfigurationRequest.Builder req =
+ SetLifecycleConfigurationRequest.newBuilder();
+ LifecycleConfiguration lifecycleConfiguration =
+ omLifecycleConfiguration.getProtobuf();
+ req.setLifecycleConfiguration(lifecycleConfiguration);
+
+ OMRequest omRequest =
+ createOMRequest(Type.SetLifecycleConfiguration)
+ .setSetLifecycleConfigurationRequest(req)
+ .build();
+
+ OMResponse omResponse = submitRequest(omRequest);
+ handleError(omResponse);
+ }
+
+ @Override
+ public void deleteLifecycleConfiguration(String volumeName, String
bucketName)
+ throws IOException {
+ DeleteLifecycleConfigurationRequest.Builder req =
+ DeleteLifecycleConfigurationRequest.newBuilder();
+ req.setVolumeName(volumeName);
+ req.setBucketName(bucketName);
+
+ OMRequest omRequest = createOMRequest(Type.DeleteLifecycleConfiguration)
+ .setDeleteLifecycleConfigurationRequest(req)
+ .build();
+
+ handleError(submitRequest(omRequest));
+ }
+
private SafeMode toProtoBuf(SafeModeAction action) {
switch (action) {
case ENTER:
diff --git
a/hadoop-ozone/integration-test-s3/src/test/java/org/apache/hadoop/ozone/s3/awssdk/v1/AbstractS3SDKV1Tests.java
b/hadoop-ozone/integration-test-s3/src/test/java/org/apache/hadoop/ozone/s3/awssdk/v1/AbstractS3SDKV1Tests.java
index a4eefb81db..6e30b58276 100644
---
a/hadoop-ozone/integration-test-s3/src/test/java/org/apache/hadoop/ozone/s3/awssdk/v1/AbstractS3SDKV1Tests.java
+++
b/hadoop-ozone/integration-test-s3/src/test/java/org/apache/hadoop/ozone/s3/awssdk/v1/AbstractS3SDKV1Tests.java
@@ -33,10 +33,12 @@
import com.amazonaws.services.s3.model.AbortMultipartUploadRequest;
import com.amazonaws.services.s3.model.AccessControlList;
import com.amazonaws.services.s3.model.Bucket;
+import com.amazonaws.services.s3.model.BucketLifecycleConfiguration;
import com.amazonaws.services.s3.model.CanonicalGrantee;
import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest;
import com.amazonaws.services.s3.model.CompleteMultipartUploadResult;
import com.amazonaws.services.s3.model.CreateBucketRequest;
+import com.amazonaws.services.s3.model.GetBucketLifecycleConfigurationRequest;
import com.amazonaws.services.s3.model.GetObjectRequest;
import com.amazonaws.services.s3.model.Grantee;
import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
@@ -61,6 +63,7 @@
import com.amazonaws.services.s3.model.S3Object;
import com.amazonaws.services.s3.model.S3ObjectInputStream;
import com.amazonaws.services.s3.model.S3ObjectSummary;
+import com.amazonaws.services.s3.model.SetBucketLifecycleConfigurationRequest;
import com.amazonaws.services.s3.model.SetObjectAclRequest;
import com.amazonaws.services.s3.model.Tag;
import com.amazonaws.services.s3.model.UploadPartRequest;
@@ -74,6 +77,7 @@
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
+import java.net.HttpURLConnection;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.nio.file.Path;
@@ -926,6 +930,164 @@ public void testQuotaExceeded() throws IOException {
assertEquals("QuotaExceeded", ase.getErrorCode());
}
+ @Test
+ public void testS3LifecycleConfigurationCreateSuccessfully() {
+ final String bucketName = getBucketName();
+ s3Client.createBucket(bucketName);
+ BucketLifecycleConfiguration configuration = new
BucketLifecycleConfiguration();
+ List<BucketLifecycleConfiguration.Rule> rules = new ArrayList<>();
+ BucketLifecycleConfiguration.Rule rule1 = new
BucketLifecycleConfiguration.Rule()
+ .withId("expire-logs-after-365-days")
+ .withPrefix("logs/")
+ .withStatus(BucketLifecycleConfiguration.ENABLED)
+ .withExpirationInDays(365);
+ rules.add(rule1);
+
+ configuration.setRules(rules);
+
+ // Set lifecycle configuration
+ SetBucketLifecycleConfigurationRequest request =
+ new SetBucketLifecycleConfigurationRequest(bucketName, configuration);
+ s3Client.setBucketLifecycleConfiguration(request);
+
+ // Verify the configuration was set
+ BucketLifecycleConfiguration retrievedConfig =
+ s3Client.getBucketLifecycleConfiguration(bucketName);
+ assertEquals(1, retrievedConfig.getRules().size());
+
+ // Verify rule 1
+ BucketLifecycleConfiguration.Rule retrievedRule1 =
retrievedConfig.getRules().get(0);
+ assertEquals("expire-logs-after-365-days", retrievedRule1.getId());
+ assertEquals("logs/", retrievedRule1.getPrefix());
+ assertEquals(BucketLifecycleConfiguration.ENABLED,
retrievedRule1.getStatus());
+ assertEquals(365, retrievedRule1.getExpirationInDays());
+ }
+
+ @Test
+ public void testS3LifecycleConfigurationCreationFailed() {
+ final String bucketName = getBucketName();
+ s3Client.createBucket(bucketName);
+
+ // Test 1: Invalid configuration
+ BucketLifecycleConfiguration configuration = new
BucketLifecycleConfiguration();
+ List<BucketLifecycleConfiguration.Rule> rules = new ArrayList<>();
+
+ BucketLifecycleConfiguration.Rule rule = new
BucketLifecycleConfiguration.Rule()
+ .withId("invalid")
+ .withStatus(BucketLifecycleConfiguration.ENABLED);
+ rules.add(rule);
+ configuration.setRules(rules);
+ SetBucketLifecycleConfigurationRequest request =
+ new SetBucketLifecycleConfigurationRequest(bucketName, configuration);
+
+ AmazonServiceException ase = assertThrows(AmazonServiceException.class,
+ () -> s3Client.setBucketLifecycleConfiguration(request));
+ assertEquals(ErrorType.Client, ase.getErrorType());
+ assertEquals(HttpURLConnection.HTTP_BAD_REQUEST, ase.getStatusCode());
+
+ // Test 2: Non-existent bucket
+ final String nonExistentBucket = getBucketName("nonexistent");
+ BucketLifecycleConfiguration validConfig = new
BucketLifecycleConfiguration();
+ List<BucketLifecycleConfiguration.Rule> validRules = new ArrayList<>();
+ BucketLifecycleConfiguration.Rule validRule = new
BucketLifecycleConfiguration.Rule()
+ .withId("test-rule")
+ .withPrefix("test/")
+ .withStatus(BucketLifecycleConfiguration.ENABLED)
+ .withExpirationInDays(30);
+ validRules.add(validRule);
+ validConfig.setRules(validRules);
+
+ SetBucketLifecycleConfigurationRequest nonExistentRequest =
+ new SetBucketLifecycleConfigurationRequest(nonExistentBucket,
validConfig);
+
+ AmazonServiceException ase2 = assertThrows(AmazonServiceException.class,
+ () -> s3Client.setBucketLifecycleConfiguration(nonExistentRequest));
+ assertEquals(ErrorType.Client, ase2.getErrorType());
+ assertEquals(HttpURLConnection.HTTP_NOT_FOUND, ase2.getStatusCode());
+ assertEquals("NoSuchBucket", ase2.getErrorCode());
+ }
+
+ @Test
+ public void testS3LifecycleConfigurationDelete() {
+ final String bucketName = getBucketName();
+ s3Client.createBucket(bucketName);
+
+ // Test delete lifecycle for a bucket, while doesn't have lifecycle
+ assertNull(s3Client.getBucketLifecycleConfiguration(bucketName));
+ assertThrows(AmazonServiceException.class,
+ () -> s3Client.deleteBucketLifecycleConfiguration(bucketName));
+
+ // First create a lifecycle configuration
+ BucketLifecycleConfiguration configuration = new
BucketLifecycleConfiguration();
+ List<BucketLifecycleConfiguration.Rule> rules = new ArrayList<>();
+ BucketLifecycleConfiguration.Rule rule = new
BucketLifecycleConfiguration.Rule()
+ .withId("test-rule")
+ .withPrefix("test/")
+ .withStatus(BucketLifecycleConfiguration.ENABLED)
+ .withExpirationInDays(30);
+ rules.add(rule);
+ configuration.setRules(rules);
+
+ s3Client.setBucketLifecycleConfiguration(bucketName, configuration);
+ // Verify it exists
+ BucketLifecycleConfiguration retrievedConfig =
+ s3Client.getBucketLifecycleConfiguration(bucketName);
+ assertEquals(1, retrievedConfig.getRules().size());
+ // Delete the lifecycle configuration
+ s3Client.deleteBucketLifecycleConfiguration(bucketName);
+ assertNull(s3Client.getBucketLifecycleConfiguration(bucketName));
+ // Test delete on non-existent bucket
+ final String nonExistentBucket = getBucketName("nonexistent");
+ assertThrows(AmazonServiceException.class,
+ () -> s3Client.deleteBucketLifecycleConfiguration(nonExistentBucket));
+ }
+
+ @Test
+ public void testS3LifecycleConfigurationGet() {
+ final String bucketName = getBucketName();
+ s3Client.createBucket(bucketName);
+
+ // Test get on bucket without lifecycle configuration
+ assertNull(s3Client.getBucketLifecycleConfiguration(bucketName));
+
+ // Create a comprehensive lifecycle configuration
+ BucketLifecycleConfiguration configuration = new
BucketLifecycleConfiguration();
+ List<BucketLifecycleConfiguration.Rule> rules = new ArrayList<>();
+
+ // Rule with expiration and prefix
+ BucketLifecycleConfiguration.Rule rule1 = new
BucketLifecycleConfiguration.Rule()
+ .withId("expire-old-files")
+ .withPrefix("old/")
+ .withStatus(BucketLifecycleConfiguration.ENABLED)
+ .withExpirationInDays(365);
+
+ rules.add(rule1);
+ configuration.setRules(rules);
+
+ // Set the configuration
+ s3Client.setBucketLifecycleConfiguration(bucketName, configuration);
+
+ // Get and verify the configuration
+ BucketLifecycleConfiguration retrievedConfig =
+ s3Client.getBucketLifecycleConfiguration(bucketName);
+
+ assertEquals(1, retrievedConfig.getRules().size());
+
+ // Verify first rule
+ BucketLifecycleConfiguration.Rule retrievedRule1 =
retrievedConfig.getRules().get(0);
+ assertEquals("expire-old-files", retrievedRule1.getId());
+ assertEquals("old/", retrievedRule1.getPrefix());
+ assertEquals(BucketLifecycleConfiguration.ENABLED,
retrievedRule1.getStatus());
+ assertEquals(365, retrievedRule1.getExpirationInDays());
+
+ // Test getting configuration using GetBucketLifecycleConfigurationRequest
+ GetBucketLifecycleConfigurationRequest getRequest =
+ new GetBucketLifecycleConfigurationRequest(bucketName);
+ BucketLifecycleConfiguration configFromRequest =
+ s3Client.getBucketLifecycleConfiguration(getRequest);
+ assertEquals(retrievedConfig.getRules().size(),
configFromRequest.getRules().size());
+ }
+
private boolean isBucketEmpty(Bucket bucket) {
ObjectListing objectListing = s3Client.listObjects(bucket.getName());
return objectListing.getObjectSummaries().isEmpty();
diff --git
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/OzoneRpcClientTests.java
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/OzoneRpcClientTests.java
index 66279bae7e..947a5d7e80 100644
---
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/OzoneRpcClientTests.java
+++
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/OzoneRpcClientTests.java
@@ -147,6 +147,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.OzoneMultipartUploadPartListParts;
import org.apache.hadoop.ozone.client.OzoneSnapshot;
import org.apache.hadoop.ozone.client.OzoneVolume;
@@ -175,6 +176,9 @@
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.OmLCExpiration;
+import org.apache.hadoop.ozone.om.helpers.OmLCRule;
+import org.apache.hadoop.ozone.om.helpers.OmLifecycleConfiguration;
import org.apache.hadoop.ozone.om.helpers.OmMultipartCommitUploadPartInfo;
import org.apache.hadoop.ozone.om.helpers.OmMultipartInfo;
import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadCompleteInfo;
@@ -5293,4 +5297,197 @@ public void testGetObjectTagging(BucketLayout
bucketLayout) throws Exception {
assertEquals(tags.size(), tagsRetrieved.size());
assertThat(tagsRetrieved).containsAllEntriesOf(tags);
}
+
+ @ParameterizedTest
+ @MethodSource("bucketLayouts")
+ public void testSetLifecycleConfiguration(BucketLayout bucketLayout) throws
Exception {
+ String volumeName = UUID.randomUUID().toString();
+ String bucketName = UUID.randomUUID().toString();
+ store.createVolume(volumeName);
+ BucketArgs bucketArgs =
+ BucketArgs.newBuilder().setBucketLayout(bucketLayout).build();
+ store.getVolume(volumeName).createBucket(bucketName, bucketArgs);
+ ClientProtocol proxy = store.getClientProxy();
+
+ OmLifecycleConfiguration lcc1 = createOmLifecycleConfiguration(volumeName,
+ bucketName, true, bucketLayout);
+ proxy.setLifecycleConfiguration(lcc1);
+
+ // No such volume
+ OmLifecycleConfiguration lcc2 =
createOmLifecycleConfiguration("nonexistentvolume",
+ "nonexistentbucket", true, bucketLayout);
+ OzoneTestUtils.expectOmException(ResultCodes.VOLUME_NOT_FOUND,
+ () -> proxy.setLifecycleConfiguration(lcc2));
+
+ // No such bucket
+ OmLifecycleConfiguration lcc3 = createOmLifecycleConfiguration(volumeName,
+ "nonexistentbucket", true, bucketLayout);
+ OzoneTestUtils.expectOmException(ResultCodes.BUCKET_NOT_FOUND,
+ () -> proxy.setLifecycleConfiguration(lcc3));
+
+ // Invalid volumeName
+ OmLifecycleConfiguration lcc4 =
createOmLifecycleConfiguration("VOLUMENAME",
+ bucketName, true, bucketLayout);
+ OzoneTestUtils.expectOmException(ResultCodes.INVALID_VOLUME_NAME,
+ () -> proxy.setLifecycleConfiguration(lcc4));
+
+ // Invalid bucketName
+ OmLifecycleConfiguration lcc5 = createOmLifecycleConfiguration(volumeName,
+ "BUCKETNAME", true, bucketLayout);
+ OzoneTestUtils.expectOmException(ResultCodes.INVALID_BUCKET_NAME,
+ () -> proxy.setLifecycleConfiguration(lcc5));
+ }
+
+ @ParameterizedTest
+ @MethodSource("bucketLayouts")
+ public void testDeleteLifecycleConfiguration(BucketLayout bucketLayout)
throws Exception {
+ String volumeName = UUID.randomUUID().toString();
+ String bucketName = UUID.randomUUID().toString();
+ store.createVolume(volumeName);
+ BucketArgs bucketArgs =
+ BucketArgs.newBuilder().setBucketLayout(bucketLayout).build();
+ store.getVolume(volumeName).createBucket(bucketName, bucketArgs);
+ ClientProtocol proxy = store.getClientProxy();
+
+ // No such lifecycle configuration
+ OzoneTestUtils.expectOmException(
+ ResultCodes.LIFECYCLE_CONFIGURATION_NOT_FOUND,
+ () -> proxy.deleteLifecycleConfiguration(volumeName, bucketName));
+
+ OmLifecycleConfiguration lcc1 = createOmLifecycleConfiguration(volumeName,
+ bucketName, true, bucketLayout);
+ proxy.setLifecycleConfiguration(lcc1);
+ proxy.deleteLifecycleConfiguration(volumeName, bucketName);
+ }
+
+ @Test
+ public void testDeleteBucketWithAttachedLifecycleConfiguration()
+ throws Exception {
+ String bucketName = UUID.randomUUID().toString();
+ store.createS3Bucket(bucketName);
+ String volumeName = store.getS3Bucket(bucketName).getVolumeName();
+ ClientProtocol proxy = store.getClientProxy();
+
+ // Create a new lifecycle configuration and make sure verify it.
+ OmLifecycleConfiguration lcc1 = createOmLifecycleConfiguration(volumeName,
+ bucketName, true, BucketLayout.OBJECT_STORE);
+ proxy.setLifecycleConfiguration(lcc1);
+ OzoneLifecycleConfiguration lcc2 =
+ proxy.getLifecycleConfiguration(volumeName, bucketName);
+ assertEquals(lcc1.getVolume(), lcc2.getVolume());
+ assertEquals(lcc1.getBucket(), lcc2.getBucket());
+ assertEquals(lcc1.getRules().get(0).getId(), lcc2.getRules()
+ .get(0).getId());
+ // CreationTime is added when being created.
+ assertNotEquals(lcc1.getCreationTime(), lcc2.getCreationTime());
+
+ store.deleteS3Bucket(bucketName);
+
+ OzoneTestUtils.expectOmException(ResultCodes.BUCKET_NOT_FOUND,
+ () -> proxy.getLifecycleConfiguration(volumeName, bucketName));
+ }
+
+ @ParameterizedTest
+ @MethodSource("bucketLayouts")
+ public void testGetLifecycleConfiguration(BucketLayout bucketLayout) throws
Exception {
+ String volumeName = UUID.randomUUID().toString();
+ String bucketName = UUID.randomUUID().toString();
+ store.createVolume(volumeName);
+ BucketArgs bucketArgs =
+ BucketArgs.newBuilder().setBucketLayout(bucketLayout).build();
+ store.getVolume(volumeName).createBucket(bucketName, bucketArgs);
+ ClientProtocol proxy = store.getClientProxy();
+
+ // No such lifecycle configuration
+ OzoneTestUtils.expectOmException(
+ ResultCodes.LIFECYCLE_CONFIGURATION_NOT_FOUND,
+ () -> proxy.getLifecycleConfiguration(volumeName, bucketName));
+
+ OmLifecycleConfiguration lcc1 = createOmLifecycleConfiguration(volumeName,
+ bucketName, true, bucketLayout);
+ proxy.setLifecycleConfiguration(lcc1);
+
+ OzoneLifecycleConfiguration lcc2 =
+ proxy.getLifecycleConfiguration(volumeName, bucketName);
+ assertEquals(lcc1.getVolume(), lcc2.getVolume());
+ assertEquals(lcc1.getBucket(), lcc2.getBucket());
+ assertEquals(lcc1.getRules().get(0).getId(), lcc2.getRules()
+ .get(0).getId());
+ // CreationTime is added when being created.
+ assertNotEquals(lcc1.getCreationTime(), lcc2.getCreationTime());
+ }
+
+ @ParameterizedTest
+ @MethodSource("bucketLayouts")
+ public void testLifecycleConfigurationWithLinkedBucket(BucketLayout
bucketLayout) throws Exception {
+ String volumeName = UUID.randomUUID().toString();
+ String sourceBucketName = UUID.randomUUID().toString();
+ String linkedBucketName = UUID.randomUUID().toString();
+ store.createVolume(volumeName);
+ OzoneVolume volume = store.getVolume(volumeName);
+
+ // Create source bucket nand linked bucket
+ BucketArgs bucketArgs =
BucketArgs.newBuilder().setBucketLayout(bucketLayout).build();
+ volume.createBucket(sourceBucketName, bucketArgs);
+ OzoneBucket sourceBucket = volume.getBucket(sourceBucketName);
+ assertNotNull(sourceBucket);
+ volume.createBucket(linkedBucketName,
+ BucketArgs.newBuilder()
+ .setSourceBucket(sourceBucketName)
+ .setSourceVolume(volumeName)
+ .build());
+ OzoneBucket linkedBucket = volume.getBucket(linkedBucketName);
+ assertNotNull(linkedBucket);
+
+ ClientProtocol proxy = store.getClientProxy();
+
OzoneTestUtils.expectOmException(ResultCodes.LIFECYCLE_CONFIGURATION_NOT_FOUND,
+ () -> proxy.getLifecycleConfiguration(volumeName, sourceBucketName));
+
OzoneTestUtils.expectOmException(ResultCodes.LIFECYCLE_CONFIGURATION_NOT_FOUND,
+ () -> proxy.getLifecycleConfiguration(volumeName, linkedBucketName));
+
+ OmLifecycleConfiguration lccThroughLinked =
createOmLifecycleConfiguration(volumeName,
+ linkedBucketName, true, bucketLayout);
+ proxy.setLifecycleConfiguration(lccThroughLinked);
+
+ OzoneLifecycleConfiguration lccFromSource =
+ proxy.getLifecycleConfiguration(volumeName, sourceBucketName);
+ // The actual stored configuration should be for the source bucket
+ assertEquals(volumeName, lccFromSource.getVolume());
+ assertEquals(sourceBucketName, lccFromSource.getBucket());
+
+ // Delete lifecycle configuration through linked bucket
+ proxy.deleteLifecycleConfiguration(volumeName, linkedBucketName);
+
+ // Verify lifecycle configuration is deleted for both buckets
+ OzoneTestUtils.expectOmException(
+ ResultCodes.LIFECYCLE_CONFIGURATION_NOT_FOUND,
+ () -> proxy.getLifecycleConfiguration(volumeName, linkedBucketName));
+ OzoneTestUtils.expectOmException(
+ ResultCodes.LIFECYCLE_CONFIGURATION_NOT_FOUND,
+ () -> proxy.getLifecycleConfiguration(volumeName, sourceBucketName));
+
+ volume.deleteBucket(linkedBucketName);
+ volume.deleteBucket(sourceBucketName);
+ store.deleteVolume(volumeName);
+ }
+
+ private OmLifecycleConfiguration createOmLifecycleConfiguration(String
volume,
+ String bucket, boolean hasRules, BucketLayout bucketLayout) throws
OMException {
+
+ OmLifecycleConfiguration.Builder builder =
+ new OmLifecycleConfiguration.Builder()
+ .setVolume(volume)
+ .setBucket(bucket)
+ .setBucketLayout(bucketLayout);
+
+ if (hasRules) {
+ builder.setRules(Collections.singletonList(new OmLCRule.Builder()
+ .setEnabled(true)
+ .setPrefix("")
+ .addAction(new OmLCExpiration.Builder().setDays(30).build())
+ .build()));
+ }
+
+ return builder.build();
+ }
}
diff --git
a/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
b/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
index 1f1c102c4f..4823e81eaa 100644
--- a/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
+++ b/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
@@ -156,6 +156,9 @@ enum Type {
PutObjectTagging = 140;
GetObjectTagging = 141;
DeleteObjectTagging = 142;
+ SetLifecycleConfiguration = 150;
+ GetLifecycleConfiguration = 151;
+ DeleteLifecycleConfiguration = 152;
}
enum SafeMode {
@@ -303,6 +306,10 @@ message OMRequest {
optional GetObjectTaggingRequest getObjectTaggingRequest =
140;
optional PutObjectTaggingRequest putObjectTaggingRequest =
141;
optional DeleteObjectTaggingRequest deleteObjectTaggingRequest =
142;
+
+ optional SetLifecycleConfigurationRequest setLifecycleConfigurationRequest =
150;
+ optional GetLifecycleConfigurationRequest getLifecycleConfigurationRequest
= 151;
+ optional DeleteLifecycleConfigurationRequest
deleteLifecycleConfigurationRequest = 152;
}
message OMResponse {
@@ -436,6 +443,10 @@ message OMResponse {
optional GetObjectTaggingResponse getObjectTaggingResponse =
140;
optional PutObjectTaggingResponse putObjectTaggingResponse =
141;
optional DeleteObjectTaggingResponse deleteObjectTaggingResponse =
142;
+
+ optional SetLifecycleConfigurationResponse setLifecycleConfigurationResponse
= 150;
+ optional GetLifecycleConfigurationResponse
getLifecycleConfigurationResponse = 151;
+ optional DeleteLifecycleConfigurationResponse
deleteLifecycleConfigurationResponse = 152;
}
enum Status {
@@ -2385,6 +2396,32 @@ message LifecycleConfiguration {
optional uint64 updateID = 7;
}
+message SetLifecycleConfigurationRequest {
+ required LifecycleConfiguration lifecycleConfiguration = 1;
+}
+
+message SetLifecycleConfigurationResponse {
+
+}
+
+message GetLifecycleConfigurationRequest {
+ required string volumeName = 1;
+ required string bucketName = 2;
+}
+
+message GetLifecycleConfigurationResponse {
+ required LifecycleConfiguration lifecycleConfiguration = 1;
+}
+
+message DeleteLifecycleConfigurationRequest {
+ required string volumeName = 1;
+ required string bucketName = 2;
+}
+
+message DeleteLifecycleConfigurationResponse {
+
+}
+
/**
The OM service that takes care of Ozone namespace.
*/
diff --git
a/hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java
b/hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java
index f69d671c7c..e790d370f8 100644
---
a/hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java
+++
b/hadoop-ozone/interface-storage/src/main/java/org/apache/hadoop/ozone/om/OMMetadataManager.java
@@ -476,6 +476,16 @@ String getMultipartKeyFSO(String volume, String bucket,
String key, String
*/
List<OmLifecycleConfiguration> listLifecycleConfigurations();
+ /**
+ * Fetches the lifecycle configuration by bucketName.
+ *
+ * @param bucketName bucketName of the lifecycle configuration
+ * @return OmLifecycleConfiguration
+ * @throws IOException
+ */
+ OmLifecycleConfiguration getLifecycleConfiguration(String volumeName,
+ String bucketName) throws IOException;
+
/**
* Gets the OM Meta table.
* @return meta table reference.
diff --git
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/audit/OMAction.java
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/audit/OMAction.java
index d192c48e9c..10cd0b2129 100644
---
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/audit/OMAction.java
+++
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/audit/OMAction.java
@@ -113,7 +113,11 @@ public enum OMAction implements AuditAction {
GET_OBJECT_TAGGING,
PUT_OBJECT_TAGGING,
- DELETE_OBJECT_TAGGING;
+ DELETE_OBJECT_TAGGING,
+
+ GET_LIFECYCLE_CONFIGURATION,
+ SET_LIFECYCLE_CONFIGURATION,
+ DELETE_LIFECYCLE_CONFIGURATION;
@Override
public String getAction() {
diff --git
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java
index f15bbaee34..e9b7dc8455 100644
---
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java
+++
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java
@@ -33,12 +33,14 @@
import static
org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_SNAPSHOT_CHECKPOINT_DIR_CREATION_POLL_TIMEOUT_DEFAULT;
import static
org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.BUCKET_NOT_FOUND;
import static
org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.FILE_NOT_FOUND;
+import static
org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.LIFECYCLE_CONFIGURATION_NOT_FOUND;
import static
org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.NO_SUCH_MULTIPART_UPLOAD_ERROR;
import static
org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_NOT_FOUND;
import static
org.apache.hadoop.ozone.om.service.SnapshotDeletingService.isBlockLocationInfoSame;
import static
org.apache.hadoop.ozone.om.snapshot.SnapshotUtils.checkSnapshotDirExist;
import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
import com.google.common.base.Strings;
import com.google.common.collect.Lists;
import java.io.File;
@@ -2112,6 +2114,38 @@ public List<OmLifecycleConfiguration>
listLifecycleConfigurations() {
return result;
}
+
+ /**
+ * Fetches the lifecycle configuration by bucketName.
+ *
+ * @param bucketName bucketName of the lifecycle configuration
+ * @return OmLifecycleConfiguration
+ * @throws IOException
+ */
+ @Override
+ public OmLifecycleConfiguration getLifecycleConfiguration(String volumeName,
+ String bucketName) throws IOException {
+ Preconditions.checkNotNull(bucketName);
+ try {
+ String bucketKey = getBucketKey(volumeName, bucketName);
+ OmLifecycleConfiguration value =
getLifecycleConfigurationTable().get(bucketKey);
+
+ if (value == null) {
+ LOG.debug("lifecycle configuration of bucket /{}/{} not found.",
+ volumeName, bucketName);
+ throw new OMException("Lifecycle configuration not found",
+ LIFECYCLE_CONFIGURATION_NOT_FOUND);
+ }
+ return value;
+ } catch (IOException ex) {
+ if (!(ex instanceof OMException)) {
+ LOG.error("Exception while getting lifecycle configuration for " +
+ "bucket: /{}/{}", volumeName, bucketName, ex);
+ }
+ throw ex;
+ }
+ }
+
/**
* Get Snapshot Chain Manager.
*
diff --git
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
index f0a6682a54..b78368a441 100644
---
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
+++
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
@@ -252,6 +252,7 @@
import org.apache.hadoop.ozone.om.helpers.OmDBUserPrincipalInfo;
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmLifecycleConfiguration;
import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadList;
import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadListParts;
import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
@@ -3113,6 +3114,51 @@ public ListSnapshotResponse listSnapshot(
}
}
+ /**
+ * Gets the lifecycle configuration information.
+ * @param volumeName - Volume name.
+ * @param bucketName - Bucket name.
+ * @return OmLifecycleConfiguration or exception is thrown.
+ * @throws IOException
+ */
+ @Override
+ public OmLifecycleConfiguration getLifecycleConfiguration(String volumeName,
+ String bucketName) throws IOException {
+ Map<String, String> auditMap = buildAuditMap(volumeName);
+ auditMap.put(OzoneConsts.BUCKET, bucketName);
+ ResolvedBucket resolvedBucket = resolveBucketLink(Pair.of(volumeName,
bucketName));
+ auditMap = buildAuditMap(resolvedBucket.realVolume());
+ auditMap.put(OzoneConsts.BUCKET, resolvedBucket.realBucket());
+
+ if (isAclEnabled) {
+ omMetadataReader.checkAcls(ResourceType.BUCKET, StoreType.OZONE,
ACLType.READ,
+ resolvedBucket.realVolume(), resolvedBucket.realBucket(), null);
+ }
+
+ boolean auditSuccess = true;
+ OMLockDetails omLockDetails =
metadataManager.getLock().acquireReadLock(BUCKET_LOCK,
+ resolvedBucket.realVolume(), resolvedBucket.realBucket());
+ boolean lockAcquired = omLockDetails.isLockAcquired();
+ try {
+ return metadataManager.getLifecycleConfiguration(
+ resolvedBucket.realVolume(), resolvedBucket.realBucket());
+ } catch (Exception ex) {
+ auditSuccess = false;
+ AUDIT.logReadFailure(buildAuditMessageForFailure(
+ OMAction.GET_LIFECYCLE_CONFIGURATION, auditMap, ex));
+ throw ex;
+ } finally {
+ if (lockAcquired) {
+ metadataManager.getLock().releaseReadLock(BUCKET_LOCK,
+ resolvedBucket.realVolume(), resolvedBucket.realBucket());
+ }
+ if (auditSuccess) {
+ AUDIT.logReadSuccess(buildAuditMessageForSuccess(
+ OMAction.GET_LIFECYCLE_CONFIGURATION, auditMap));
+ }
+ }
+ }
+
private Map<String, String> buildAuditMap(String volume) {
Map<String, String> auditMap = new LinkedHashMap<>();
auditMap.put(OzoneConsts.VOLUME, volume);
diff --git
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/utils/OzoneManagerRatisUtils.java
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/utils/OzoneManagerRatisUtils.java
index 5548be7bd8..1f8f0e24a2 100644
---
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/utils/OzoneManagerRatisUtils.java
+++
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/utils/OzoneManagerRatisUtils.java
@@ -66,6 +66,8 @@
import org.apache.hadoop.ozone.om.request.key.acl.prefix.OMPrefixAddAclRequest;
import
org.apache.hadoop.ozone.om.request.key.acl.prefix.OMPrefixRemoveAclRequest;
import org.apache.hadoop.ozone.om.request.key.acl.prefix.OMPrefixSetAclRequest;
+import
org.apache.hadoop.ozone.om.request.lifecycle.OMLifecycleConfigurationDeleteRequest;
+import
org.apache.hadoop.ozone.om.request.lifecycle.OMLifecycleConfigurationSetRequest;
import
org.apache.hadoop.ozone.om.request.s3.multipart.S3ExpiredMultipartUploadsAbortRequest;
import org.apache.hadoop.ozone.om.request.s3.security.OMSetSecretRequest;
import org.apache.hadoop.ozone.om.request.s3.security.S3GetSecretRequest;
@@ -345,6 +347,10 @@ public static OMClientRequest
createClientRequest(OMRequest omRequest,
volumeName = keyArgs.getVolumeName();
bucketName = keyArgs.getBucketName();
break;
+ case SetLifecycleConfiguration:
+ return new OMLifecycleConfigurationSetRequest(omRequest);
+ case DeleteLifecycleConfiguration:
+ return new OMLifecycleConfigurationDeleteRequest(omRequest);
default:
throw new OMException("Unrecognized write command type request "
+ cmdType, OMException.ResultCodes.INVALID_REQUEST);
diff --git
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketDeleteRequest.java
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketDeleteRequest.java
index e4f4e8204a..4f24960118 100644
---
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketDeleteRequest.java
+++
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketDeleteRequest.java
@@ -146,8 +146,7 @@ public OMClientResponse validateAndUpdateCache(OzoneManager
ozoneManager, Execut
ResultCodes.BUCKET_NOT_EMPTY);
}
- // appending '/' to end to eliminate cases where 2 buckets start with
same
- // characters.
+ // appending '/' to end to eliminate cases where 2 buckets start with
same characters.
String snapshotBucketKey = bucketKey + OzoneConsts.OM_KEY_PREFIX;
if (bucketContainsSnapshot(omMetadataManager, snapshotBucketKey)) {
@@ -167,10 +166,11 @@ public OMClientResponse
validateAndUpdateCache(OzoneManager ozoneManager, Execut
omMetadataManager.getBucketTable().addCacheEntry(
new CacheKey<>(bucketKey),
CacheValue.get(transactionLogIndex));
-
- omResponse.setDeleteBucketResponse(
- DeleteBucketResponse.newBuilder().build());
-
+ // Update lifecycle configuration table as well.
+ omMetadataManager.getLifecycleConfigurationTable().addCacheEntry(
+ new CacheKey<>(bucketKey),
+ CacheValue.get(transactionLogIndex));
+
omResponse.setDeleteBucketResponse(DeleteBucketResponse.newBuilder().build());
// update used namespace for volume
String volumeKey = omMetadataManager.getVolumeKey(volumeName);
OmVolumeArgs omVolumeArgs =
diff --git
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/lifecycle/OMLifecycleConfigurationDeleteRequest.java
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/lifecycle/OMLifecycleConfigurationDeleteRequest.java
new file mode 100644
index 0000000000..371479c58f
--- /dev/null
+++
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/lifecycle/OMLifecycleConfigurationDeleteRequest.java
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.hadoop.ozone.om.request.lifecycle;
+
+import static
org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+import java.io.IOException;
+import java.util.Map;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.AuditLogger;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.ResolvedBucket;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.execution.flowcontrol.ExecutionContext;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import
org.apache.hadoop.ozone.om.response.lifecycle.OMLifecycleConfigurationDeleteResponse;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteLifecycleConfigurationRequest;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteLifecycleConfigurationResponse;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.UserInfo;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Handles DeleteLifecycleConfiguration Request.
+ */
+public class OMLifecycleConfigurationDeleteRequest extends OMClientRequest {
+ private static final Logger LOG =
+ LoggerFactory.getLogger(OMLifecycleConfigurationDeleteRequest.class);
+
+ public OMLifecycleConfigurationDeleteRequest(OMRequest omRequest) {
+ super(omRequest);
+ }
+
+ @Override
+ public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
+ OMRequest request = super.preExecute(ozoneManager);
+ DeleteLifecycleConfigurationRequest deleteLifecycleConfigurationRequest =
+ request.getDeleteLifecycleConfigurationRequest();
+
+ String volumeName = deleteLifecycleConfigurationRequest.getVolumeName();
+ String bucketName = deleteLifecycleConfigurationRequest.getBucketName();
+
+ // Resolve bucket link and check ACLs
+ ResolvedBucket resolvedBucket = ozoneManager.resolveBucketLink(
+ Pair.of(volumeName, bucketName), this);
+
+ if (ozoneManager.getAclsEnabled()) {
+ checkAcls(ozoneManager, OzoneObj.ResourceType.BUCKET,
OzoneObj.StoreType.OZONE,
+ IAccessAuthorizer.ACLType.ALL, resolvedBucket.realVolume(),
+ resolvedBucket.realBucket(), null);
+ }
+
+ // Update the request with resolved volume and bucket names
+ DeleteLifecycleConfigurationRequest.Builder newRequest =
+ deleteLifecycleConfigurationRequest.toBuilder()
+ .setVolumeName(resolvedBucket.realVolume())
+ .setBucketName(resolvedBucket.realBucket());
+
+ return request.toBuilder()
+ .setUserInfo(getUserInfo())
+ .setDeleteLifecycleConfigurationRequest(newRequest.build())
+ .build();
+ }
+
+ @Override
+ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
ExecutionContext context) {
+ final long transactionLogIndex = context.getIndex();
+ OMMetadataManager metadataManager = ozoneManager.getMetadataManager();
+
+ OMRequest omRequest = getOmRequest();
+ DeleteLifecycleConfigurationRequest deleteLifecycleConfigurationRequest =
+ omRequest.getDeleteLifecycleConfigurationRequest();
+
+ String volumeName = deleteLifecycleConfigurationRequest.getVolumeName();
+ String bucketName = deleteLifecycleConfigurationRequest.getBucketName();
+
+ OMResponse.Builder omResponse = OmResponseUtil.getOMResponseBuilder(
+ getOmRequest());
+
+ AuditLogger auditLogger = ozoneManager.getAuditLogger();
+ Map<String, String> auditMap = buildVolumeAuditMap(volumeName);
+ auditMap.put(OzoneConsts.BUCKET, bucketName);
+
+ UserInfo userInfo = getOmRequest().getUserInfo();
+ IOException exception = null;
+ boolean acquiredBucketLock = false;
+ boolean success = true;
+ OMClientResponse omClientResponse = null;
+
+ try {
+ mergeOmLockDetails(metadataManager.getLock()
+ .acquireWriteLock(BUCKET_LOCK, volumeName, bucketName));
+ acquiredBucketLock = getOmLockDetails().isLockAcquired();
+
+ String bucketKey = metadataManager.getBucketKey(volumeName, bucketName);
+
+ // Check existence.
+ if (!metadataManager.getLifecycleConfigurationTable().isExist(
+ bucketKey)) {
+ LOG.debug("lifecycle bucket: {} volume: {} not found ", bucketName,
+ volumeName);
+ throw new OMException("Lifecycle configurations does not exist",
+ OMException.ResultCodes.LIFECYCLE_CONFIGURATION_NOT_FOUND);
+ }
+
+ // Update table cache.
+ metadataManager.getLifecycleConfigurationTable().addCacheEntry(
+ new CacheKey<>(bucketKey), CacheValue.get(transactionLogIndex));
+
+ omResponse.setDeleteLifecycleConfigurationResponse(
+ DeleteLifecycleConfigurationResponse.newBuilder());
+
+ omClientResponse = new OMLifecycleConfigurationDeleteResponse(
+ omResponse.build(), volumeName, bucketName);
+
+ } catch (IOException ex) {
+ success = false;
+ exception = ex;
+ omClientResponse = new OMLifecycleConfigurationDeleteResponse(
+ createErrorOMResponse(omResponse, exception));
+ } finally {
+ if (acquiredBucketLock) {
+
mergeOmLockDetails(metadataManager.getLock().releaseWriteLock(BUCKET_LOCK,
volumeName,
+ bucketName));
+ }
+ }
+ if (omClientResponse != null) {
+ omClientResponse.setOmLockDetails(getOmLockDetails());
+ }
+
+ // Performing audit logging outside the lock.
+ markForAudit(auditLogger, buildAuditMessage(
+ OMAction.DELETE_LIFECYCLE_CONFIGURATION, auditMap, exception,
userInfo));
+
+ if (success) {
+ LOG.debug("Deleted lifecycle bucket:{} volume:{}", bucketName,
+ volumeName);
+ return omClientResponse;
+ } else {
+ LOG.error("Delete lifecycle failed for bucket:{} in volume:{}",
+ bucketName, volumeName, exception);
+ return omClientResponse;
+ }
+ }
+}
diff --git
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/lifecycle/OMLifecycleConfigurationSetRequest.java
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/lifecycle/OMLifecycleConfigurationSetRequest.java
new file mode 100644
index 0000000000..91f4567dcc
--- /dev/null
+++
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/lifecycle/OMLifecycleConfigurationSetRequest.java
@@ -0,0 +1,191 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.om.request.lifecycle;
+
+import static
org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.BUCKET_NOT_FOUND;
+import static
org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OmUtils;
+import org.apache.hadoop.ozone.audit.AuditLogger;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.ResolvedBucket;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.execution.flowcontrol.ExecutionContext;
+import org.apache.hadoop.ozone.om.helpers.OmLifecycleConfiguration;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import
org.apache.hadoop.ozone.om.response.lifecycle.OMLifecycleConfigurationSetResponse;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.LifecycleConfiguration;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SetLifecycleConfigurationRequest;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SetLifecycleConfigurationResponse;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.UserInfo;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Handles SetLifecycleConfiguration Request.
+ */
+public class OMLifecycleConfigurationSetRequest extends OMClientRequest {
+ private static final Logger LOG =
+ LoggerFactory.getLogger(OMLifecycleConfigurationSetRequest.class);
+
+ public OMLifecycleConfigurationSetRequest(OMRequest omRequest) {
+ super(omRequest);
+ }
+
+ @Override
+ public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
+ OMRequest omRequest = super.preExecute(ozoneManager);
+ SetLifecycleConfigurationRequest request =
+ omRequest.getSetLifecycleConfigurationRequest();
+ LifecycleConfiguration lifecycleConfiguration =
+ request.getLifecycleConfiguration();
+
+ OmUtils.validateVolumeName(lifecycleConfiguration.getVolume(),
ozoneManager.isStrictS3());
+ OmUtils.validateBucketName(lifecycleConfiguration.getBucket(),
ozoneManager.isStrictS3());
+
+ String volumeName = lifecycleConfiguration.getVolume();
+ String bucketName = lifecycleConfiguration.getBucket();
+
+ ResolvedBucket resolvedBucket = ozoneManager.resolveBucketLink(
+ Pair.of(volumeName, bucketName), this);
+
+ if (ozoneManager.getAclsEnabled()) {
+ checkAcls(ozoneManager, OzoneObj.ResourceType.BUCKET,
OzoneObj.StoreType.OZONE,
+ IAccessAuthorizer.ACLType.ALL, resolvedBucket.realVolume(),
+ resolvedBucket.realBucket(), null);
+ }
+
+ if (resolvedBucket.bucketLayout().toProto() !=
request.getLifecycleConfiguration().getBucketLayout()) {
+ throw new OMException("Bucket layout mismatch: requested lifecycle
configuration " +
+ "has bucket layout " +
request.getLifecycleConfiguration().getBucketLayout() +
+ " but the actual bucket has layout " +
resolvedBucket.bucketLayout().toProto(),
+ OMException.ResultCodes.INVALID_REQUEST);
+ }
+
+ SetLifecycleConfigurationRequest.Builder newCreateRequest =
+ request.toBuilder();
+
+ LifecycleConfiguration.Builder newLifecycleConfiguration =
+ lifecycleConfiguration.toBuilder()
+ .setVolume(resolvedBucket.realVolume())
+ .setBucket(resolvedBucket.realBucket());
+
+ newLifecycleConfiguration.setCreationTime(Time.now());
+ newCreateRequest.setLifecycleConfiguration(newLifecycleConfiguration);
+
+ return omRequest.toBuilder().setUserInfo(getUserInfo())
+ .setSetLifecycleConfigurationRequest(newCreateRequest.build())
+ .build();
+ }
+
+ @Override
+ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
ExecutionContext context) {
+ final long transactionLogIndex = context.getIndex();
+ OMMetadataManager metadataManager = ozoneManager.getMetadataManager();
+
+ SetLifecycleConfigurationRequest setLifecycleConfigurationRequest =
+ getOmRequest().getSetLifecycleConfigurationRequest();
+
+ LifecycleConfiguration lifecycleConfiguration =
+ setLifecycleConfigurationRequest.getLifecycleConfiguration();
+
+ String volumeName = lifecycleConfiguration.getVolume();
+ String bucketName = lifecycleConfiguration.getBucket();
+
+ OMResponse.Builder omResponse = OmResponseUtil.getOMResponseBuilder(
+ getOmRequest());
+
+ AuditLogger auditLogger = ozoneManager.getAuditLogger();
+ UserInfo userInfo = getOmRequest().getUserInfo();
+
+ String bucketKey = metadataManager.getBucketKey(volumeName, bucketName);
+ IOException exception = null;
+ boolean acquiredBucketLock = false;
+ OMClientResponse omClientResponse = null;
+ Map<String, String> auditMap = new HashMap<>();
+
+ try {
+ OmLifecycleConfiguration omLifecycleConfiguration =
+ OmLifecycleConfiguration.getFromProtobuf(lifecycleConfiguration);
+ auditMap = omLifecycleConfiguration.toAuditMap();
+
+ mergeOmLockDetails(metadataManager.getLock()
+ .acquireWriteLock(BUCKET_LOCK, volumeName, bucketName));
+ acquiredBucketLock = getOmLockDetails().isLockAcquired();
+ omLifecycleConfiguration.valid();
+
+ if (!metadataManager.getBucketTable().isExist(bucketKey)) {
+ LOG.debug("bucket: {} in volume: {} doesn't exist", bucketName,
+ volumeName);
+ throw new OMException("Bucket doesn't exist", BUCKET_NOT_FOUND);
+ }
+ omLifecycleConfiguration.setUpdateID(transactionLogIndex);
+
+ metadataManager.getLifecycleConfigurationTable().addCacheEntry(
+ new CacheKey<>(bucketKey),
+ CacheValue.get(transactionLogIndex, omLifecycleConfiguration));
+
+ omResponse.setSetLifecycleConfigurationResponse(
+ SetLifecycleConfigurationResponse.newBuilder().build());
+
+ omClientResponse = new OMLifecycleConfigurationSetResponse(
+ omResponse.build(), omLifecycleConfiguration);
+ } catch (IOException ex) {
+ exception = ex;
+ omClientResponse = new OMLifecycleConfigurationSetResponse(
+ createErrorOMResponse(omResponse, exception));
+ } finally {
+ if (acquiredBucketLock) {
+
mergeOmLockDetails(metadataManager.getLock().releaseWriteLock(BUCKET_LOCK,
volumeName,
+ bucketName));
+ }
+ }
+ if (omClientResponse != null) {
+ omClientResponse.setOmLockDetails(getOmLockDetails());
+ }
+
+ // Performing audit logging outside the lock.
+ markForAudit(auditLogger,
buildAuditMessage(OMAction.SET_LIFECYCLE_CONFIGURATION,
+ auditMap, exception, userInfo));
+
+ if (exception == null) {
+ LOG.debug("Created lifecycle configuration bucket: {} in volume: {}",
+ bucketName, volumeName);
+ return omClientResponse;
+ } else {
+ LOG.error("Lifecycle configuration creation failed for bucket:{} " +
+ "in volume:{}", bucketName, volumeName, exception);
+ return omClientResponse;
+ }
+ }
+}
diff --git
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/lifecycle/package-info.java
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/lifecycle/package-info.java
new file mode 100644
index 0000000000..7a4c9d9f0e
--- /dev/null
+++
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/lifecycle/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+
+/**
+ * This package contains classes for handling lifecycle create and delete.
+ */
+package org.apache.hadoop.ozone.om.request.lifecycle;
diff --git
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/bucket/OMBucketDeleteResponse.java
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/bucket/OMBucketDeleteResponse.java
index 8b8be7e1f7..b6bd1cf037 100644
---
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/bucket/OMBucketDeleteResponse.java
+++
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/bucket/OMBucketDeleteResponse.java
@@ -79,6 +79,14 @@ public void addToDBBatch(OMMetadataManager omMetadataManager,
omMetadataManager.getVolumeKey(omVolumeArgs.getVolume()),
omVolumeArgs);
}
+
+ // Delete lifecycle attached to that bucket.
+ try {
+ omMetadataManager.getLifecycleConfigurationTable().deleteWithBatch(
+ batchOperation, dbBucketKey);
+ } catch (IOException ex) {
+ // Do nothing if there is no lifecycle attached.
+ }
}
public String getVolumeName() {
diff --git
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/lifecycle/OMLifecycleConfigurationDeleteResponse.java
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/lifecycle/OMLifecycleConfigurationDeleteResponse.java
new file mode 100644
index 0000000000..6e351882af
--- /dev/null
+++
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/lifecycle/OMLifecycleConfigurationDeleteResponse.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.hadoop.ozone.om.response.lifecycle;
+
+import static
org.apache.hadoop.ozone.om.OmMetadataManagerImpl.LIFECYCLE_CONFIGURATION_TABLE;
+
+import jakarta.annotation.Nonnull;
+import java.io.IOException;
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.response.CleanupTableInfo;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+
+/**
+ * Response for SetLifecycleConfiguration request.
+ */
+@CleanupTableInfo(cleanupTables = {LIFECYCLE_CONFIGURATION_TABLE})
+public class OMLifecycleConfigurationDeleteResponse extends OMClientResponse {
+
+ private final String volumeName;
+ private final String bucketName;
+
+ public OMLifecycleConfigurationDeleteResponse(
+ @Nonnull OMResponse omResponse) {
+ super(omResponse);
+ checkStatusNotOK();
+ this.volumeName = null;
+ this.bucketName = null;
+ }
+
+ public OMLifecycleConfigurationDeleteResponse(@Nonnull OMResponse omResponse,
+ String volumeName, String bucketName) {
+ super(omResponse);
+ this.volumeName = volumeName;
+ this.bucketName = bucketName;
+ }
+
+ @Override
+ protected void addToDBBatch(OMMetadataManager omMetadataManager,
+ BatchOperation batchOperation) throws IOException {
+
+ String dbLifecycleKey = omMetadataManager.getBucketKey(volumeName,
+ bucketName);
+
+ omMetadataManager.getLifecycleConfigurationTable().deleteWithBatch(
+ batchOperation, dbLifecycleKey);
+ }
+}
diff --git
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/lifecycle/OMLifecycleConfigurationSetResponse.java
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/lifecycle/OMLifecycleConfigurationSetResponse.java
new file mode 100644
index 0000000000..fcfbaa761e
--- /dev/null
+++
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/lifecycle/OMLifecycleConfigurationSetResponse.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.hadoop.ozone.om.response.lifecycle;
+
+import static
org.apache.hadoop.ozone.om.OmMetadataManagerImpl.LIFECYCLE_CONFIGURATION_TABLE;
+
+import jakarta.annotation.Nonnull;
+import java.io.IOException;
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.helpers.OmLifecycleConfiguration;
+import org.apache.hadoop.ozone.om.response.CleanupTableInfo;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+
+/**
+ * Response for SetLifecycleConfiguration request.
+ */
+@CleanupTableInfo(cleanupTables = {LIFECYCLE_CONFIGURATION_TABLE})
+public class OMLifecycleConfigurationSetResponse extends OMClientResponse {
+
+ private final OmLifecycleConfiguration omLifecycleConfiguration;
+
+ public OMLifecycleConfigurationSetResponse(
+ @Nonnull OMResponse omResponse) {
+ super(omResponse);
+ checkStatusNotOK();
+ this.omLifecycleConfiguration = null;
+ }
+
+ public OMLifecycleConfigurationSetResponse(@Nonnull OMResponse omResponse,
+ @Nonnull OmLifecycleConfiguration omLifecycleConfiguration) {
+ super(omResponse);
+ this.omLifecycleConfiguration = omLifecycleConfiguration;
+ }
+
+ @Override
+ protected void addToDBBatch(OMMetadataManager omMetadataManager,
+ BatchOperation batchOperation) throws IOException {
+
+ String dbLifecycleKey = omMetadataManager.getBucketKey(
+ omLifecycleConfiguration.getVolume(),
+ omLifecycleConfiguration.getBucket());
+
+ omMetadataManager.getLifecycleConfigurationTable().putWithBatch(
+ batchOperation, dbLifecycleKey, omLifecycleConfiguration);
+ }
+}
diff --git
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/lifecycle/package-info.java
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/lifecycle/package-info.java
new file mode 100644
index 0000000000..c222ae5147
--- /dev/null
+++
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/lifecycle/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+
+/**
+ * This package contains classes for handling lifecycle create and delete.
+ */
+package org.apache.hadoop.ozone.om.response.lifecycle;
diff --git
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java
index 67190972ce..debbbebf87 100644
---
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java
+++
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java
@@ -77,6 +77,7 @@
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmLifecycleConfiguration;
import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadList;
import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadListParts;
import org.apache.hadoop.ozone.om.helpers.OmPartInfo;
@@ -113,6 +114,8 @@
import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.GetFileStatusResponse;
import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.GetKeyInfoRequest;
import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.GetKeyInfoResponse;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.GetLifecycleConfigurationRequest;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.GetLifecycleConfigurationResponse;
import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.GetObjectTaggingRequest;
import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.GetObjectTaggingResponse;
import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.GetS3VolumeContextResponse;
@@ -392,6 +395,13 @@ public OMResponse handleReadRequest(OMRequest request) {
getObjectTagging(request.getGetObjectTaggingRequest());
responseBuilder.setGetObjectTaggingResponse(getObjectTaggingResponse);
break;
+ case GetLifecycleConfiguration:
+ GetLifecycleConfigurationResponse getLifecycleConfigurationResponse =
+ infoLifecycleConfiguration(
+ request.getGetLifecycleConfigurationRequest());
+ responseBuilder.setGetLifecycleConfigurationResponse(
+ getLifecycleConfigurationResponse);
+ break;
default:
responseBuilder.setSuccess(false);
responseBuilder.setMessage("Unrecognized Command Type: " + cmdType);
@@ -1380,6 +1390,23 @@ private PrepareStatusResponse getPrepareStatus() {
.setCurrentTxnIndex(prepareState.getIndex()).build();
}
+ private GetLifecycleConfigurationResponse infoLifecycleConfiguration(
+ GetLifecycleConfigurationRequest request) throws IOException {
+
+ GetLifecycleConfigurationResponse.Builder resp =
+ GetLifecycleConfigurationResponse.newBuilder();
+
+ String volume = request.getVolumeName();
+ String bucket = request.getBucketName();
+
+ OmLifecycleConfiguration omLifecycleConfiguration =
+ impl.getLifecycleConfiguration(volume, bucket);
+
+ resp.setLifecycleConfiguration(omLifecycleConfiguration.getProtobuf());
+
+ return resp.build();
+ }
+
private GetS3VolumeContextResponse getS3VolumeContext()
throws IOException {
return impl.getS3VolumeContext().getProtobuf();
diff --git
a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/lifecycle/TestOMLifecycleConfigurationDeleteRequest.java
b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/lifecycle/TestOMLifecycleConfigurationDeleteRequest.java
new file mode 100644
index 0000000000..a820c597f7
--- /dev/null
+++
b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/lifecycle/TestOMLifecycleConfigurationDeleteRequest.java
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.hadoop.ozone.om.request.lifecycle;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.util.UUID;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.ozone.om.ResolvedBucket;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Test class for delete Lifecycle configuration request.
+ */
+public class TestOMLifecycleConfigurationDeleteRequest extends
+ TestOMLifecycleConfigurationRequest {
+ @Test
+ public void testPreExecute() throws Exception {
+ OMRequest omRequest = createDeleteLifecycleConfigurationRequest(
+ UUID.randomUUID().toString(), UUID.randomUUID().toString());
+
+ OMLifecycleConfigurationDeleteRequest request =
+ new OMLifecycleConfigurationDeleteRequest(omRequest);
+
+ // As user info gets added.
+ assertNotEquals(omRequest, request.preExecute(ozoneManager));
+ }
+
+ @Test
+ public void testPreExecuteWithLinkedBucket() throws Exception {
+ String volumeName = UUID.randomUUID().toString();
+ String bucketName = UUID.randomUUID().toString();
+ String resolvedBucketName = bucketName + "-resolved";
+ String resolvedVolumeName = volumeName + "-resolved";
+ // Mock the bucket link resolution
+ when(ozoneManager.resolveBucketLink(any(Pair.class),
any(OMClientRequest.class)))
+ .thenAnswer(i -> new ResolvedBucket(i.getArgument(0),
+ Pair.of(resolvedVolumeName, resolvedBucketName),
+ "owner", BucketLayout.FILE_SYSTEM_OPTIMIZED));
+
+ OMRequest omRequest =
createDeleteLifecycleConfigurationRequest(volumeName, bucketName);
+ OMLifecycleConfigurationDeleteRequest request =
+ new OMLifecycleConfigurationDeleteRequest(omRequest);
+ OMRequest modifiedRequest = request.preExecute(ozoneManager);
+
+ // Verify that the resolved volume and bucket names are used
+ assertEquals(resolvedVolumeName,
+
modifiedRequest.getDeleteLifecycleConfigurationRequest().getVolumeName());
+ assertEquals(resolvedBucketName,
+
modifiedRequest.getDeleteLifecycleConfigurationRequest().getBucketName());
+ }
+
+ @Test
+ public void testValidateAndUpdateCache() throws Exception {
+ String volumeName = UUID.randomUUID().toString();
+ String bucketName = UUID.randomUUID().toString();
+
+ // Create Volume and bucket entries in DB.
+ OMRequestTestUtils.addVolumeAndBucketToDB(volumeName, bucketName,
+ omMetadataManager);
+
+ addLifecycleConfigurationToDB(volumeName, bucketName, "ownername");
+ assertNotNull(omMetadataManager.getLifecycleConfigurationTable()
+ .get(omMetadataManager.getBucketKey(volumeName, bucketName)));
+
+ OMRequest omRequest =
+ createDeleteLifecycleConfigurationRequest(volumeName, bucketName);
+
+ OMLifecycleConfigurationDeleteRequest deleteRequest =
+ new OMLifecycleConfigurationDeleteRequest(omRequest);
+
+ deleteRequest.validateAndUpdateCache(ozoneManager, 1L);
+
+ assertNull(omMetadataManager.getLifecycleConfigurationTable().get(
+ omMetadataManager.getBucketKey(volumeName, bucketName)));
+ }
+
+ @Test
+ public void testValidateAndUpdateCacheFailure() throws Exception {
+ String volumeName = UUID.randomUUID().toString();
+ String bucketName = UUID.randomUUID().toString();
+
+ // Create Volume and bucket entries in DB.
+ OMRequestTestUtils.addVolumeAndBucketToDB(volumeName, bucketName,
+ omMetadataManager);
+
+ OMRequest omRequest =
+ createDeleteLifecycleConfigurationRequest(volumeName, bucketName);
+
+ OMLifecycleConfigurationDeleteRequest deleteRequest =
+ new OMLifecycleConfigurationDeleteRequest(omRequest);
+
+ OMClientResponse omClientResponse = deleteRequest.validateAndUpdateCache(
+ ozoneManager, 1L);
+
+ OMResponse omResponse = omClientResponse.getOMResponse();
+ assertEquals(
+ OzoneManagerProtocolProtos.Status.LIFECYCLE_CONFIGURATION_NOT_FOUND,
+ omResponse.getStatus());
+ }
+
+ private void addLifecycleConfigurationToDB(String volumeName,
+ String bucketName, String ownerName) throws IOException {
+ OMRequest originalRequest = setLifecycleConfigurationRequest(volumeName,
+ bucketName, ownerName);
+
+ OMLifecycleConfigurationSetRequest request =
+ new OMLifecycleConfigurationSetRequest(originalRequest);
+
+ OMRequest modifiedRequest = request.preExecute(ozoneManager);
+
+ String lifecycleKey = omMetadataManager.getBucketKey(volumeName,
+ bucketName);
+
+ assertNull(omMetadataManager.getLifecycleConfigurationTable().get(
+ lifecycleKey));
+
+ request = new OMLifecycleConfigurationSetRequest(modifiedRequest);
+ long txLogIndex = 1;
+
+ request.validateAndUpdateCache(ozoneManager, txLogIndex);
+ }
+}
diff --git
a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/lifecycle/TestOMLifecycleConfigurationRequest.java
b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/lifecycle/TestOMLifecycleConfigurationRequest.java
new file mode 100644
index 0000000000..03075091b9
--- /dev/null
+++
b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/lifecycle/TestOMLifecycleConfigurationRequest.java
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.hadoop.ozone.om.request.lifecycle;
+
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.File;
+import java.util.UUID;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.audit.AuditLogger;
+import org.apache.hadoop.ozone.audit.AuditMessage;
+import org.apache.hadoop.ozone.om.OMConfigKeys;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.ResolvedBucket;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.apache.hadoop.ozone.om.upgrade.OMLayoutVersionManager;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.BucketLayoutProto;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteLifecycleConfigurationRequest;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.LifecycleAction;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.LifecycleConfiguration;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.LifecycleExpiration;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.LifecycleFilter;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.LifecycleRule;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SetLifecycleConfigurationRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.io.TempDir;
+import org.mockito.Mockito;
+
+/**
+ * Base test class for Lifecycle configuration request.
+ */
+@SuppressWarnings("visibilitymodifier")
+public class TestOMLifecycleConfigurationRequest {
+
+ @TempDir
+ private File tempDir;
+
+ protected OzoneManager ozoneManager;
+ protected OMMetrics omMetrics;
+ protected OMMetadataManager omMetadataManager;
+ protected AuditLogger auditLogger;
+
+ @BeforeEach
+ public void setup() throws Exception {
+ ozoneManager = mock(OzoneManager.class);
+ omMetrics = OMMetrics.create();
+ OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
+ ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS,
tempDir.getAbsolutePath());
+ omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration,
ozoneManager);
+ when(ozoneManager.getMetrics()).thenReturn(omMetrics);
+ when(ozoneManager.getMetadataManager()).thenReturn(omMetadataManager);
+ when(ozoneManager.getMaxUserVolumeCount()).thenReturn(10L);
+ when(ozoneManager.resolveBucketLink(any(Pair.class),
any(OMClientRequest.class)))
+ .thenAnswer(i -> new ResolvedBucket(i.getArgument(0),
+ i.getArgument(0), "dummyBucketOwner", BucketLayout.OBJECT_STORE));
+ OMLayoutVersionManager lvm = mock(OMLayoutVersionManager.class);
+ when(lvm.getMetadataLayoutVersion()).thenReturn(0);
+ when(ozoneManager.getVersionManager()).thenReturn(lvm);
+ auditLogger = mock(AuditLogger.class);
+ when(ozoneManager.getAuditLogger()).thenReturn(auditLogger);
+ Mockito.doNothing().when(auditLogger).logWrite(any(AuditMessage.class));
+ }
+
+ @AfterEach
+ public void stop() {
+ omMetrics.unRegister();
+ Mockito.framework().clearInlineMocks();
+ }
+
+ public OMRequest createDeleteLifecycleConfigurationRequest(
+ String volumeName, String bucketName) {
+ return OMRequest.newBuilder().setDeleteLifecycleConfigurationRequest(
+ DeleteLifecycleConfigurationRequest.newBuilder()
+ .setVolumeName(volumeName)
+ .setBucketName(bucketName))
+ .setCmdType(Type.DeleteLifecycleConfiguration)
+ .setClientId(UUID.randomUUID().toString()).build();
+ }
+
+ public static void addVolumeAndBucketToTable(String volumeName,
+ String bucketName, String ownerName, OMMetadataManager omMetadataManager)
+ throws Exception {
+ OMRequestTestUtils.addVolumeToDB(volumeName, ownerName, omMetadataManager);
+ OMRequestTestUtils.addBucketToDB(volumeName, bucketName,
omMetadataManager);
+ }
+
+ public OMRequest setLifecycleConfigurationRequest(String volumeName,
+ String bucketName, String ownerName) {
+ return setLifecycleConfigurationRequest(volumeName, bucketName,
+ ownerName, true);
+ }
+
+ public OMRequest setLifecycleConfigurationRequest(String volumeName,
+ String bucketName, String ownerName, boolean addRules) {
+ String prefix = "prefix/";
+ LifecycleConfiguration.Builder builder =
LifecycleConfiguration.newBuilder()
+ .setBucketLayout(BucketLayoutProto.OBJECT_STORE)
+ .setVolume(volumeName)
+ .setBucket(bucketName);
+
+ if (addRules) {
+ builder.addRules(LifecycleRule.newBuilder()
+ .setId(RandomStringUtils.randomAlphabetic(32))
+ .setEnabled(true)
+ .addAction(LifecycleAction.newBuilder()
+
.setExpiration(LifecycleExpiration.newBuilder().setDays(3).build()))
+ .setFilter(LifecycleFilter.newBuilder().setPrefix(prefix))
+ );
+ }
+
+ LifecycleConfiguration lcc = builder.build();
+
+ SetLifecycleConfigurationRequest setLifecycleConfigurationRequest =
+ SetLifecycleConfigurationRequest.newBuilder()
+ .setLifecycleConfiguration(lcc)
+ .build();
+
+ return OMRequest.newBuilder().setSetLifecycleConfigurationRequest(
+ setLifecycleConfigurationRequest)
+ .setCmdType(Type.SetLifecycleConfiguration)
+ .setClientId(UUID.randomUUID().toString())
+ .build();
+ }
+}
diff --git
a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/lifecycle/TestOMLifecycleConfigurationSetRequest.java
b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/lifecycle/TestOMLifecycleConfigurationSetRequest.java
new file mode 100644
index 0000000000..75de1eee84
--- /dev/null
+++
b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/lifecycle/TestOMLifecycleConfigurationSetRequest.java
@@ -0,0 +1,239 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.hadoop.ozone.om.request.lifecycle;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.when;
+
+import java.util.UUID;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.ozone.om.ResolvedBucket;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmLifecycleConfiguration;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.LifecycleConfiguration;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Test class for create Lifecycle configuration request.
+ */
+public class TestOMLifecycleConfigurationSetRequest extends
+ TestOMLifecycleConfigurationRequest {
+ @Test
+ public void testPreExecute() throws Exception {
+ String volumeName = UUID.randomUUID().toString();
+ String bucketName = UUID.randomUUID().toString();
+ doPreExecute(volumeName, bucketName);
+
+ // Volume name and bucket name length should be greater than
OZONE_MIN_BUCKET_NAME_LENGTH
+ assertThrows(OMException.class, () -> doPreExecute("v1", "bucket1"));
+ assertThrows(OMException.class, () -> doPreExecute("volume1", "b1"));
+ }
+
+ @Test
+ public void testPreExecuteWithLinkedBucket() throws Exception {
+ String volumeName = UUID.randomUUID().toString();
+ String bucketName = UUID.randomUUID().toString();
+ String resolvedBucketName = bucketName + "-resolved";
+ String resolvedVolumeName = volumeName + "-resolved";
+ // Mock the bucket link resolution
+ when(ozoneManager.resolveBucketLink(any(Pair.class),
any(OMClientRequest.class)))
+ .thenAnswer(i -> new ResolvedBucket(i.getArgument(0),
+ Pair.of(resolvedVolumeName, resolvedBucketName),
+ "owner", BucketLayout.OBJECT_STORE));
+ OMRequest originalRequest = setLifecycleConfigurationRequest(volumeName,
bucketName, "ownername");
+ OMLifecycleConfigurationSetRequest request = new
OMLifecycleConfigurationSetRequest(originalRequest);
+ OMRequest modifiedRequest = request.preExecute(ozoneManager);
+
+ // Verify that the resolved volume and bucket names are used in the
lifecycle configuration
+ LifecycleConfiguration lifecycleConfig =
+
modifiedRequest.getSetLifecycleConfigurationRequest().getLifecycleConfiguration();
+ assertEquals(resolvedVolumeName, lifecycleConfig.getVolume());
+ assertEquals(resolvedBucketName, lifecycleConfig.getBucket());
+ }
+
+ @Test
+ public void testValidateAndUpdateCacheSuccess() throws Exception {
+ String volumeName = UUID.randomUUID().toString();
+ String bucketName = UUID.randomUUID().toString();
+ String ownerName = "ownerName";
+
+ addVolumeAndBucketToTable(volumeName, bucketName, ownerName,
+ omMetadataManager);
+
+ OMRequest originalRequest = setLifecycleConfigurationRequest(volumeName,
+ bucketName, ownerName);
+
+ OMLifecycleConfigurationSetRequest request =
+ new OMLifecycleConfigurationSetRequest(originalRequest);
+
+ OMRequest modifiedRequest = request.preExecute(ozoneManager);
+
+ String lifecycleKey = omMetadataManager.getBucketKey(volumeName,
+ bucketName);
+
+ assertNull(omMetadataManager.getLifecycleConfigurationTable().get(
+ lifecycleKey));
+
+ request = new OMLifecycleConfigurationSetRequest(modifiedRequest);
+ long txLogIndex = 2;
+
+ OMClientResponse omClientResponse =
request.validateAndUpdateCache(ozoneManager, txLogIndex);
+ OMResponse omResponse = omClientResponse.getOMResponse();
+
+ assertNotNull(omResponse.getSetLifecycleConfigurationResponse());
+ assertEquals(OzoneManagerProtocolProtos.Status.OK,
+ omResponse.getStatus());
+ assertEquals(Type.SetLifecycleConfiguration,
+ omResponse.getCmdType());
+
+ LifecycleConfiguration lifecycleConfigurationRequestProto =
+ request.getOmRequest()
+ .getSetLifecycleConfigurationRequest()
+ .getLifecycleConfiguration();
+
+ OmLifecycleConfiguration lifecycleConfigurationRequest =
+ OmLifecycleConfiguration.getFromProtobuf(
+ lifecycleConfigurationRequestProto);
+
+ OmLifecycleConfiguration lifecycleConfiguration = omMetadataManager
+ .getLifecycleConfigurationTable().get(lifecycleKey);
+
+ assertNotNull(lifecycleConfiguration);
+ assertEquals(lifecycleConfigurationRequest.getVolume(),
+ lifecycleConfiguration.getVolume());
+ assertEquals(lifecycleConfigurationRequest.getBucket(),
+ lifecycleConfiguration.getBucket());
+ assertEquals(lifecycleConfigurationRequest.getBucket(),
+ lifecycleConfiguration.getBucket());
+ assertEquals(lifecycleConfigurationRequest.getCreationTime(),
+ lifecycleConfiguration.getCreationTime());
+ }
+
+ @Test
+ public void testValidateAndUpdateNoBucket() throws Exception {
+ String volumeName = UUID.randomUUID().toString();
+ String bucketName = UUID.randomUUID().toString();
+ String ownerName = "ownerName";
+
+ OMRequestTestUtils.addVolumeToDB(volumeName, ownerName, omMetadataManager);
+
+ OMRequest originalRequest = setLifecycleConfigurationRequest(volumeName,
+ bucketName, ownerName);
+
+ OMLifecycleConfigurationSetRequest request =
+ new OMLifecycleConfigurationSetRequest(originalRequest);
+
+ OMRequest modifiedRequest = request.preExecute(ozoneManager);
+
+ String lifecycleKey = omMetadataManager.getBucketKey(volumeName,
+ bucketName);
+
+ assertNull(omMetadataManager.getLifecycleConfigurationTable().get(
+ lifecycleKey));
+
+ request = new OMLifecycleConfigurationSetRequest(modifiedRequest);
+ long txLogIndex = 2;
+
+ OMClientResponse omClientResponse =
request.validateAndUpdateCache(ozoneManager, txLogIndex);
+ OMResponse omResponse = omClientResponse.getOMResponse();
+
+ assertEquals(OzoneManagerProtocolProtos.Status.BUCKET_NOT_FOUND,
+ omResponse.getStatus());
+ }
+
+ @Test
+ public void testValidateAndUpdateInvalidLCC() throws Exception {
+ String volumeName = UUID.randomUUID().toString();
+ String bucketName = UUID.randomUUID().toString();
+ String ownerName = "ownerName";
+
+ OMRequestTestUtils.addVolumeToDB(volumeName, ownerName, omMetadataManager);
+
+ OMRequest originalRequest = setLifecycleConfigurationRequest(volumeName,
+ bucketName, ownerName, false);
+
+ OMLifecycleConfigurationSetRequest request =
+ new OMLifecycleConfigurationSetRequest(originalRequest);
+
+ OMRequest modifiedRequest = request.preExecute(ozoneManager);
+
+ String lifecycleKey = omMetadataManager.getBucketKey(volumeName,
+ bucketName);
+
+ assertNull(omMetadataManager.getLifecycleConfigurationTable().get(
+ lifecycleKey));
+
+ request = new OMLifecycleConfigurationSetRequest(modifiedRequest);
+ long txLogIndex = 2;
+
+ OMClientResponse omClientResponse =
request.validateAndUpdateCache(ozoneManager, txLogIndex);
+ OMResponse omResponse = omClientResponse.getOMResponse();
+
+ assertEquals(OzoneManagerProtocolProtos.Status.INVALID_REQUEST,
+ omResponse.getStatus());
+ }
+
+ private void doPreExecute(String volumeName, String bucketName)
+ throws Exception {
+
+ OMRequest originalRequest = setLifecycleConfigurationRequest(volumeName,
+ bucketName, "ownername");
+
+ OMLifecycleConfigurationSetRequest request =
+ new OMLifecycleConfigurationSetRequest(originalRequest);
+
+ OMRequest modifiedRequest = request.preExecute(ozoneManager);
+ verifyRequest(modifiedRequest, originalRequest);
+ }
+
+ /**
+ * Verify modifiedOmRequest and originalRequest.
+ * @param modifiedRequest
+ * @param originalRequest
+ */
+ private void verifyRequest(OMRequest modifiedRequest,
+ OMRequest originalRequest) {
+
+ LifecycleConfiguration original =
+ originalRequest.getSetLifecycleConfigurationRequest()
+ .getLifecycleConfiguration();
+
+ LifecycleConfiguration updated =
+ modifiedRequest.getSetLifecycleConfigurationRequest()
+ .getLifecycleConfiguration();
+
+ assertEquals(original.getVolume(), updated.getVolume());
+ assertEquals(original.getBucket(), updated.getBucket());
+ assertNotEquals(original.getCreationTime(), updated.getCreationTime());
+ assertEquals(original.getRulesList(), updated.getRulesList());
+ }
+}
diff --git
a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/lifecycle/TestOMLifecycleConfigurationDeleteResponse.java
b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/lifecycle/TestOMLifecycleConfigurationDeleteResponse.java
new file mode 100644
index 0000000000..7c6a879985
--- /dev/null
+++
b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/lifecycle/TestOMLifecycleConfigurationDeleteResponse.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.om.response.lifecycle;
+
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.mockito.Mockito.mock;
+
+import java.io.File;
+import java.util.UUID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.ozone.om.OMConfigKeys;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmLCExpiration;
+import org.apache.hadoop.ozone.om.helpers.OmLCRule;
+import org.apache.hadoop.ozone.om.helpers.OmLifecycleConfiguration;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteLifecycleConfigurationResponse;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SetLifecycleConfigurationResponse;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Status;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.hadoop.util.Time;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+/**
+ * This class tests OMLifecycleConfigurationDeleteResponse.
+ */
+public class TestOMLifecycleConfigurationDeleteResponse {
+ @TempDir
+ private File tempDir;
+
+ private OMMetadataManager omMetadataManager;
+ private BatchOperation batchOperation;
+
+ @BeforeEach
+ public void setup() throws Exception {
+ OzoneManager ozoneManager = mock(OzoneManager.class);
+ OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
+ ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS,
tempDir.getAbsolutePath());
+ omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration,
ozoneManager);
+ batchOperation = omMetadataManager.getStore().initBatchOperation();
+ }
+
+ @AfterEach
+ public void tearDown() {
+ if (batchOperation != null) {
+ batchOperation.close();
+ }
+ }
+
+ @Test
+ public void testAddToDBBatch() throws Exception {
+ String volumeName = UUID.randomUUID().toString();
+ String bucketName = UUID.randomUUID().toString();
+
+ OmLifecycleConfiguration omLifecycleConfiguration =
+ new OmLifecycleConfiguration.Builder()
+ .setVolume(volumeName)
+ .setBucket(bucketName)
+ .setBucketLayout(BucketLayout.OBJECT_STORE)
+ .addRule(new OmLCRule.Builder().setPrefix("")
+ .addAction(new OmLCExpiration.Builder().setDays(30).build())
+ .build())
+ .setCreationTime(Time.now())
+ .build();
+
+ SetLifecycleConfigurationResponse setLifecycleConfigurationResponse =
+ SetLifecycleConfigurationResponse.newBuilder()
+ .build();
+
+ OMLifecycleConfigurationSetResponse createResponse =
+ new OMLifecycleConfigurationSetResponse(OMResponse.newBuilder()
+ .setCmdType(Type.SetLifecycleConfiguration)
+ .setStatus(Status.OK)
+ .setSetLifecycleConfigurationResponse(
+ setLifecycleConfigurationResponse).build(),
+ omLifecycleConfiguration);
+
+ OMLifecycleConfigurationDeleteResponse deleteResponse =
+ new OMLifecycleConfigurationDeleteResponse(OMResponse.newBuilder()
+ .setCmdType(Type.DeleteLifecycleConfiguration)
+ .setStatus(Status.OK)
+ .setDeleteLifecycleConfigurationResponse(
+ DeleteLifecycleConfigurationResponse.getDefaultInstance()
+ .newBuilderForType()).build(), volumeName, bucketName);
+
+ createResponse.addToDBBatch(omMetadataManager, batchOperation);
+ deleteResponse.addToDBBatch(omMetadataManager, batchOperation);
+
+ // Do manual commit and see whether addToBatch is successful or not.
+ omMetadataManager.getStore().commitBatchOperation(batchOperation);
+
+ assertNull(omMetadataManager.getLifecycleConfigurationTable().get(
+ omMetadataManager.getBucketKey(volumeName, bucketName)));
+ }
+}
diff --git
a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/lifecycle/TestOMLifecycleConfigurationSetResponse.java
b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/lifecycle/TestOMLifecycleConfigurationSetResponse.java
new file mode 100644
index 0000000000..6e91a8d1a2
--- /dev/null
+++
b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/lifecycle/TestOMLifecycleConfigurationSetResponse.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.hadoop.ozone.om.response.lifecycle;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.mock;
+
+import java.io.File;
+import java.util.UUID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.ozone.om.OMConfigKeys;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmLCExpiration;
+import org.apache.hadoop.ozone.om.helpers.OmLCRule;
+import org.apache.hadoop.ozone.om.helpers.OmLifecycleConfiguration;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SetLifecycleConfigurationResponse;
+import
org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Status;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.hadoop.util.Time;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+/**
+ * This class tests OMLifecycleConfigurationCreateResponse.
+ */
+public class TestOMLifecycleConfigurationSetResponse {
+
+ private OMMetadataManager omMetadataManager;
+ private BatchOperation batchOperation;
+
+ @TempDir
+ private File tempDir;
+
+ @BeforeEach
+ public void setup() throws Exception {
+ OzoneManager ozoneManager = mock(OzoneManager.class);
+ OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
+ ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS,
+ tempDir.getAbsolutePath());
+ omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration,
ozoneManager);
+ batchOperation = omMetadataManager.getStore().initBatchOperation();
+ }
+
+ @AfterEach
+ public void tearDown() {
+ if (batchOperation != null) {
+ batchOperation.close();
+ }
+ }
+
+ @Test
+ public void testAddToDBBatch() throws Exception {
+ String volumeName = UUID.randomUUID().toString();
+ String bucketName = UUID.randomUUID().toString();
+
+ OmLifecycleConfiguration omLifecycleConfiguration =
+ new OmLifecycleConfiguration.Builder()
+ .setVolume(volumeName)
+ .setBucket(bucketName)
+ .setBucketLayout(BucketLayout.OBJECT_STORE)
+ .addRule(new OmLCRule.Builder().setPrefix("")
+ .addAction(new OmLCExpiration.Builder().setDays(30).build())
+ .build())
+ .setCreationTime(Time.now())
+ .build();
+
+ assertEquals(0, omMetadataManager.countRowsInTable(
+ omMetadataManager.getLifecycleConfigurationTable()));
+
+ SetLifecycleConfigurationResponse setLifecycleConfigurationResponse =
+ SetLifecycleConfigurationResponse.newBuilder()
+ .build();
+
+ OMLifecycleConfigurationSetResponse response =
+ new OMLifecycleConfigurationSetResponse(OMResponse.newBuilder()
+ .setCmdType(Type.SetLifecycleConfiguration)
+ .setStatus(Status.OK)
+ .setSetLifecycleConfigurationResponse(
+ setLifecycleConfigurationResponse).build(),
+ omLifecycleConfiguration);
+
+ response.addToDBBatch(omMetadataManager, batchOperation);
+
+ // Do manual commit and see whether addToBatch is successful or not.
+ omMetadataManager.getStore().commitBatchOperation(batchOperation);
+
+ assertEquals(1, omMetadataManager.countRowsInTable(
+ omMetadataManager.getLifecycleConfigurationTable()));
+
+ Table.KeyValue<String, OmLifecycleConfiguration> keyValue =
+ omMetadataManager.getLifecycleConfigurationTable()
+ .iterator()
+ .next();
+
+ // Lifecycle configuration keys follow bucket key format.
+ assertEquals(omMetadataManager.getBucketKey(volumeName, bucketName),
+ keyValue.getKey());
+
+ assertEquals(omLifecycleConfiguration.getProtobuf(),
keyValue.getValue().getProtobuf());
+ }
+}
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 1e4da61e88..cb9b79c858 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
@@ -805,7 +805,7 @@ public Response putBucketLifecycleConfiguration(
s3LifecycleConfiguration = new
PutBucketLifecycleConfigurationUnmarshaller().readFrom(null,
null, null, null, null, body);
OmLifecycleConfiguration lcc =
-
s3LifecycleConfiguration.toOmLifecycleConfiguration(ozoneBucket.getVolumeName(),
bucketName);
+ s3LifecycleConfiguration.toOmLifecycleConfiguration(ozoneBucket);
ozoneBucket.setLifecycleConfiguration(lcc);
} catch (WebApplicationException ex) {
throw S3ErrorTable.newError(S3ErrorTable.MALFORMED_XML, bucketName);
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
index ae5e5b4a66..54009f2849 100644
---
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
@@ -17,7 +17,6 @@
package org.apache.hadoop.ozone.s3.endpoint;
-import java.time.format.DateTimeParseException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
@@ -26,6 +25,7 @@
import javax.xml.bind.annotation.XmlAccessorType;
import javax.xml.bind.annotation.XmlElement;
import javax.xml.bind.annotation.XmlRootElement;
+import org.apache.hadoop.ozone.client.OzoneBucket;
import org.apache.hadoop.ozone.client.OzoneLifecycleConfiguration;
import org.apache.hadoop.ozone.om.exceptions.OMException;
import org.apache.hadoop.ozone.om.helpers.OmLCExpiration;
@@ -251,22 +251,13 @@ public void setAndOperator(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)
+ public OmLifecycleConfiguration toOmLifecycleConfiguration(OzoneBucket
ozoneBucket)
throws OS3Exception, OMException {
try {
OmLifecycleConfiguration.Builder builder = new
OmLifecycleConfiguration.Builder()
- .setVolume(volumeName)
- .setBucket(bucketName);
+ .setVolume(ozoneBucket.getVolumeName())
+ .setBucketLayout(ozoneBucket.getBucketLayout())
+ .setBucket(ozoneBucket.getName());
for (Rule rule : getRules()) {
builder.addRule(convertToOmRule(rule));
@@ -275,7 +266,7 @@ public OmLifecycleConfiguration
toOmLifecycleConfiguration(String volumeName, St
return builder.build();
} catch (Exception ex) {
if (ex instanceof IllegalStateException) {
- throw S3ErrorTable.newError(S3ErrorTable.INVALID_REQUEST, bucketName,
ex);
+ throw S3ErrorTable.newError(S3ErrorTable.INVALID_REQUEST,
ozoneBucket.getName(), ex);
}
throw ex;
}
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 e1853f305b..feedd0167d 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
@@ -797,7 +797,7 @@ public OzoneLifecycleConfiguration
getLifecycleConfiguration(String volumeName,
}
@Override
- public void createLifecycleConfiguration(OmLifecycleConfiguration
lifecycleConfiguration) throws IOException {
+ public void setLifecycleConfiguration(OmLifecycleConfiguration
lifecycleConfiguration) throws IOException {
getBucket(lifecycleConfiguration.getVolume(),
lifecycleConfiguration.getBucket())
.setLifecycleConfiguration(lifecycleConfiguration);
}
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]