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 9cfff8a1e587..8c39228bb35d 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 @@ -322,6 +322,7 @@ public static boolean isReadOnly( case RecoverLease: case SetTimes: case AbortExpiredMultiPartUploads: + case SetSnapshotProperty: case UnknownCommand: return false; case EchoRPC: diff --git a/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto b/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto index 77d2f237156b..fb3c6925fc02 100644 --- a/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto +++ b/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto @@ -143,7 +143,7 @@ enum Type { PrintCompactionLogDag = 125; ListKeysLight = 126; AbortExpiredMultiPartUploads = 127; - + SetSnapshotProperty = 128; } enum SafeMode { @@ -277,6 +277,7 @@ message OMRequest { optional PrintCompactionLogDagRequest PrintCompactionLogDagRequest = 125; optional MultipartUploadsExpiredAbortRequest multipartUploadsExpiredAbortRequest = 126; + optional SetSnapshotPropertyRequest SetSnapshotPropertyRequest = 127; } message OMResponse { @@ -395,6 +396,7 @@ message OMResponse { optional PrintCompactionLogDagResponse PrintCompactionLogDagResponse = 125; optional ListKeysLightResponse listKeysLightResponse = 126; optional MultipartUploadsExpiredAbortResponse multipartUploadsExpiredAbortResponse = 127; + optional SetSnapshotPropertyResponse SetSnapshotPropertyResponse = 128; } enum Status { @@ -1860,6 +1862,16 @@ message SnapshotPurgeRequest { repeated string updatedSnapshotDBKey = 2; } +message SetSnapshotPropertyRequest { + optional SnapshotProperty snapshotProperty = 1; +} + +message SnapshotProperty { + optional string snapshotKey = 1; + optional uint64 exclusiveSize = 2; + optional uint64 exclusiveReplicatedSize = 3; +} + message DeleteTenantRequest { optional string tenantId = 1; } @@ -1945,6 +1957,10 @@ message SnapshotPurgeResponse { } +message SetSnapshotPropertyResponse { + +} + message SnapshotDiffReportProto { optional string volumeName = 1; optional string bucketName = 2; 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 d6ebb44da07e..636c3af2092a 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 @@ -78,6 +78,7 @@ import org.apache.hadoop.ozone.om.request.snapshot.OMSnapshotDeleteRequest; import org.apache.hadoop.ozone.om.request.snapshot.OMSnapshotMoveDeletedKeysRequest; import org.apache.hadoop.ozone.om.request.snapshot.OMSnapshotPurgeRequest; +import org.apache.hadoop.ozone.om.request.snapshot.OMSnapshotSetPropertyRequest; import org.apache.hadoop.ozone.om.request.upgrade.OMCancelPrepareRequest; import org.apache.hadoop.ozone.om.request.upgrade.OMFinalizeUpgradeRequest; import org.apache.hadoop.ozone.om.request.upgrade.OMPrepareRequest; @@ -226,6 +227,8 @@ public static OMClientRequest createClientRequest(OMRequest omRequest, return new OMSnapshotMoveDeletedKeysRequest(omRequest); case SnapshotPurge: return new OMSnapshotPurgeRequest(omRequest); + case SetSnapshotProperty: + return new OMSnapshotSetPropertyRequest(omRequest); case DeleteOpenKeys: BucketLayout bktLayout = BucketLayout.DEFAULT; if (omRequest.getDeleteOpenKeysRequest().hasBucketLayout()) { diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotSetPropertyRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotSetPropertyRequest.java new file mode 100644 index 000000000000..c0b1b4f3ae81 --- /dev/null +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotSetPropertyRequest.java @@ -0,0 +1,101 @@ +/** + * 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.snapshot; + +import org.apache.hadoop.hdds.utils.db.cache.CacheKey; +import org.apache.hadoop.hdds.utils.db.cache.CacheValue; +import org.apache.hadoop.ozone.om.OMMetadataManager; +import org.apache.hadoop.ozone.om.OzoneManager; +import org.apache.hadoop.ozone.om.exceptions.OMException; +import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; +import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper; +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.snapshot.OMSnapshotSetPropertyResponse; +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.SnapshotProperty; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.INVALID_SNAPSHOT_ERROR; + +/** + * Updates the exclusive size of the snapshot. + */ +public class OMSnapshotSetPropertyRequest extends OMClientRequest { + private static final Logger LOG = + LoggerFactory.getLogger(OMSnapshotSetPropertyRequest.class); + + public OMSnapshotSetPropertyRequest(OMRequest omRequest) { + super(omRequest); + } + + @Override + public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, + long trxnLogIndex, OzoneManagerDoubleBufferHelper omDoubleBufferHelper) { + + OMClientResponse omClientResponse = null; + OMMetadataManager metadataManager = ozoneManager.getMetadataManager(); + + OzoneManagerProtocolProtos.OMResponse.Builder omResponse = + OmResponseUtil.getOMResponseBuilder(getOmRequest()); + OzoneManagerProtocolProtos.SetSnapshotPropertyRequest + setSnapshotPropertyRequest = getOmRequest() + .getSetSnapshotPropertyRequest(); + + SnapshotProperty snapshotProperty = setSnapshotPropertyRequest + .getSnapshotProperty(); + SnapshotInfo updatedSnapInfo = null; + + try { + String snapshotKey = snapshotProperty.getSnapshotKey(); + long exclusiveSize = snapshotProperty.getExclusiveSize(); + long exclusiveReplicatedSize = snapshotProperty + .getExclusiveReplicatedSize(); + updatedSnapInfo = metadataManager.getSnapshotInfoTable() + .get(snapshotKey); + + if (updatedSnapInfo == null) { + LOG.error("SnapshotInfo for Snapshot: {} is not found", snapshotKey); + throw new OMException("SnapshotInfo for Snapshot: " + snapshotKey + + " is not found", INVALID_SNAPSHOT_ERROR); + } + + // Set Exclusive size. + updatedSnapInfo.setExclusiveSize(exclusiveSize); + updatedSnapInfo.setExclusiveReplicatedSize(exclusiveReplicatedSize); + // Update Table Cache + metadataManager.getSnapshotInfoTable().addCacheEntry( + new CacheKey<>(snapshotKey), + CacheValue.get(trxnLogIndex, updatedSnapInfo)); + omClientResponse = new OMSnapshotSetPropertyResponse( + omResponse.build(), updatedSnapInfo); + } catch (IOException ex) { + omClientResponse = new OMSnapshotSetPropertyResponse( + createErrorOMResponse(omResponse, ex)); + } finally { + addResponseToDoubleBuffer(trxnLogIndex, omClientResponse, + omDoubleBufferHelper); + } + return omClientResponse; + } +} diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/snapshot/OMSnapshotSetPropertyResponse.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/snapshot/OMSnapshotSetPropertyResponse.java new file mode 100644 index 000000000000..ed2953b5415e --- /dev/null +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/snapshot/OMSnapshotSetPropertyResponse.java @@ -0,0 +1,60 @@ +/** + * 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.snapshot; + +import org.apache.hadoop.hdds.utils.db.BatchOperation; +import org.apache.hadoop.ozone.om.OMMetadataManager; +import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; +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; + +import javax.annotation.Nonnull; +import java.io.IOException; + +import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.SNAPSHOT_INFO_TABLE; + +/** + * Response for OMSnapshotSetPropertyRequest. + */ +@CleanupTableInfo(cleanupTables = {SNAPSHOT_INFO_TABLE}) +public class OMSnapshotSetPropertyResponse extends OMClientResponse { + private final SnapshotInfo updatedSnapInfo; + + public OMSnapshotSetPropertyResponse( + @Nonnull OMResponse omResponse, + @Nonnull SnapshotInfo updatedSnapInfo) { + super(omResponse); + this.updatedSnapInfo = updatedSnapInfo; + } + + public OMSnapshotSetPropertyResponse(@Nonnull OMResponse omResponse) { + super(omResponse); + checkStatusNotOK(); + this.updatedSnapInfo = null; + } + + @Override + protected void addToDBBatch(OMMetadataManager omMetadataManager, + BatchOperation batchOperation) + throws IOException { + omMetadataManager.getSnapshotInfoTable().putWithBatch(batchOperation, + updatedSnapInfo.getTableKey(), updatedSnapInfo); + } +} diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/KeyDeletingService.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/KeyDeletingService.java index 5a1f8336fc0c..f03484347e82 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/KeyDeletingService.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/KeyDeletingService.java @@ -19,7 +19,11 @@ import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; import java.util.List; +import java.util.Map; +import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; @@ -27,6 +31,7 @@ import com.google.protobuf.ServiceException; import org.apache.hadoop.hdds.conf.ConfigurationSource; import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol; +import org.apache.hadoop.hdds.utils.IOUtils; import org.apache.hadoop.hdds.utils.db.Table; import org.apache.hadoop.hdds.utils.db.TableIterator; import org.apache.hadoop.ozone.OzoneConsts; @@ -43,6 +48,8 @@ import org.apache.hadoop.ozone.om.snapshot.SnapshotCache; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotPurgeRequest; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotProperty; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SetSnapshotPropertyRequest; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type; import org.apache.hadoop.hdds.utils.BackgroundTask; import org.apache.hadoop.hdds.utils.BackgroundTaskQueue; @@ -84,10 +91,12 @@ public class KeyDeletingService extends AbstractKeyDeletingService { private static final int KEY_DELETING_CORE_POOL_SIZE = 1; private final KeyManager manager; - private static ClientId clientId = ClientId.randomId(); private final int keyLimitPerTask; private final AtomicLong deletedKeyCount; private final AtomicBoolean suspended; + private final Map exclusiveSizeMap; + private final Map exclusiveReplicatedSizeMap; + private final Set completedExclusiveSizeSet; public KeyDeletingService(OzoneManager ozoneManager, ScmBlockLocationProtocol scmClient, @@ -101,6 +110,9 @@ public KeyDeletingService(OzoneManager ozoneManager, OZONE_KEY_DELETING_LIMIT_PER_TASK_DEFAULT); this.deletedKeyCount = new AtomicLong(0); this.suspended = new AtomicBoolean(false); + this.exclusiveSizeMap = new HashMap<>(); + this.exclusiveReplicatedSizeMap = new HashMap<>(); + this.completedExclusiveSizeSet = new HashSet<>(); } /** @@ -212,6 +224,7 @@ public BackgroundTaskResult call() { return EmptyTaskResult.newResult(); } + @SuppressWarnings("checkstyle:MethodLength") private void processSnapshotDeepClean(int delCount) throws IOException { OmSnapshotManager omSnapshotManager = @@ -269,10 +282,17 @@ private void processSnapshotDeepClean(int delCount) String snapshotBucketKey = dbBucketKey + OzoneConsts.OM_KEY_PREFIX; SnapshotInfo previousSnapshot = getPreviousActiveSnapshot( currSnapInfo, snapChainManager, omSnapshotManager); + SnapshotInfo previousToPrevSnapshot = null; + + if (previousSnapshot != null) { + previousToPrevSnapshot = getPreviousActiveSnapshot( + previousSnapshot, snapChainManager, omSnapshotManager); + } + Table previousKeyTable = null; + Table prevRenamedTable = null; ReferenceCounted rcPrevOmSnapshot = null; - OmSnapshot omPreviousSnapshot = null; // Split RepeatedOmKeyInfo and update current snapshot // deletedKeyTable and next snapshot deletedKeyTable. @@ -281,10 +301,29 @@ private void processSnapshotDeepClean(int delCount) previousSnapshot.getVolumeName(), previousSnapshot.getBucketName(), getSnapshotPrefix(previousSnapshot.getName()), true); - omPreviousSnapshot = (OmSnapshot) rcPrevOmSnapshot.get(); + OmSnapshot omPreviousSnapshot = (OmSnapshot) + rcPrevOmSnapshot.get(); previousKeyTable = omPreviousSnapshot.getMetadataManager() .getKeyTable(bucketInfo.getBucketLayout()); + prevRenamedTable = omPreviousSnapshot + .getMetadataManager().getSnapshotRenamedTable(); + } + + Table previousToPrevKeyTable = null; + ReferenceCounted + rcPrevToPrevOmSnapshot = null; + if (previousToPrevSnapshot != null) { + rcPrevToPrevOmSnapshot = omSnapshotManager.checkForSnapshot( + previousToPrevSnapshot.getVolumeName(), + previousToPrevSnapshot.getBucketName(), + getSnapshotPrefix(previousToPrevSnapshot.getName()), true); + OmSnapshot omPreviousToPrevSnapshot = (OmSnapshot) + rcPrevToPrevOmSnapshot.get(); + + previousToPrevKeyTable = omPreviousToPrevSnapshot + .getMetadataManager() + .getKeyTable(bucketInfo.getBucketLayout()); } try (TableIterator blocksForKeyDelete = currOmSnapshot @@ -338,6 +386,14 @@ private void processSnapshotDeepClean(int delCount) if (delCount < keyLimitPerTask) { // Deep clean is completed, we can update the SnapInfo. deepCleanedSnapshots.add(currSnapInfo.getTableKey()); + // exclusiveSizeList contains check is used to prevent + // case where there is no entry in deletedTable, this + // will throw NPE when we submit request. + if (previousSnapshot != null && exclusiveSizeMap + .containsKey(previousSnapshot.getTableKey())) { + completedExclusiveSizeSet.add( + previousSnapshot.getTableKey()); + } } if (!keysToPurge.isEmpty()) { @@ -345,19 +401,144 @@ private void processSnapshotDeepClean(int delCount) keysToModify, currSnapInfo.getTableKey()); } } finally { - if (previousSnapshot != null) { - rcPrevOmSnapshot.close(); - } + IOUtils.closeQuietly(rcPrevOmSnapshot, rcPrevToPrevOmSnapshot); } } } } + + updateSnapshotExclusiveSize(); updateDeepCleanedSnapshots(deepCleanedSnapshots); } + /** + * To calculate Exclusive Size for current snapshot, Check + * the next snapshot deletedTable if the deleted key is + * referenced in current snapshot and not referenced in the + * previous snapshot then that key is exclusive to the current + * snapshot. Here since we are only iterating through + * deletedTable we can check the previous and previous to + * previous snapshot to achieve the same. + * previousSnapshot - Snapshot for which exclusive size is + * getting calculating. + * currSnapshot - Snapshot's deletedTable is used to calculate + * previousSnapshot snapshot's exclusive size. + * previousToPrevSnapshot - Snapshot which is used to check + * if key is exclusive to previousSnapshot. + */ + @SuppressWarnings("checkstyle:ParameterNumber") + private void calculateExclusiveSize( + SnapshotInfo previousSnapshot, + SnapshotInfo previousToPrevSnapshot, + OmKeyInfo keyInfo, + OmBucketInfo bucketInfo, long volumeId, + Table snapRenamedTable, + Table previousKeyTable, + Table prevRenamedTable, + Table previousToPrevKeyTable) throws IOException { + String prevSnapKey = previousSnapshot.getTableKey(); + long exclusiveReplicatedSize = + exclusiveReplicatedSizeMap.getOrDefault( + prevSnapKey, 0L) + keyInfo.getReplicatedSize(); + long exclusiveSize = exclusiveSizeMap.getOrDefault( + prevSnapKey, 0L) + keyInfo.getDataSize(); + + // If there is no previous to previous snapshot, then + // the previous snapshot is the first snapshot. + if (previousToPrevSnapshot == null) { + exclusiveSizeMap.put(prevSnapKey, exclusiveSize); + exclusiveReplicatedSizeMap.put(prevSnapKey, + exclusiveReplicatedSize); + } else { + OmKeyInfo keyInfoPrevSnapshot = getPreviousSnapshotKeyName( + keyInfo, bucketInfo, volumeId, + snapRenamedTable, previousKeyTable); + OmKeyInfo keyInfoPrevToPrevSnapshot = getPreviousSnapshotKeyName( + keyInfoPrevSnapshot, bucketInfo, volumeId, + prevRenamedTable, previousToPrevKeyTable); + // If the previous to previous snapshot doesn't + // have the key, then it is exclusive size for the + // previous snapshot. + if (keyInfoPrevToPrevSnapshot == null) { + exclusiveSizeMap.put(prevSnapKey, exclusiveSize); + exclusiveReplicatedSizeMap.put(prevSnapKey, + exclusiveReplicatedSize); + } + } + } + + private OmKeyInfo getPreviousSnapshotKeyName( + OmKeyInfo keyInfo, OmBucketInfo bucketInfo, long volumeId, + Table snapRenamedTable, + Table previousKeyTable) throws IOException { + + if (keyInfo == null) { + return null; + } + + String dbKeyPrevSnap; + if (bucketInfo.getBucketLayout().isFileSystemOptimized()) { + dbKeyPrevSnap = getOzoneManager().getMetadataManager().getOzonePathKey( + volumeId, + bucketInfo.getObjectID(), + keyInfo.getParentObjectID(), + keyInfo.getFileName()); + } else { + dbKeyPrevSnap = getOzoneManager().getMetadataManager().getOzoneKey( + keyInfo.getVolumeName(), + keyInfo.getBucketName(), + keyInfo.getKeyName()); + } + + String dbRenameKey = getOzoneManager().getMetadataManager().getRenameKey( + keyInfo.getVolumeName(), + keyInfo.getBucketName(), + keyInfo.getObjectID()); + + String renamedKey = snapRenamedTable.getIfExist(dbRenameKey); + dbKeyPrevSnap = renamedKey != null ? renamedKey : dbKeyPrevSnap; + + return previousKeyTable.get(dbKeyPrevSnap); + } + + private void updateSnapshotExclusiveSize() { + + if (completedExclusiveSizeSet.isEmpty()) { + return; + } + + Iterator completedSnapshotIterator = + completedExclusiveSizeSet.iterator(); + while (completedSnapshotIterator.hasNext()) { + ClientId clientId = ClientId.randomId(); + String dbKey = completedSnapshotIterator.next(); + SnapshotProperty snapshotProperty = SnapshotProperty.newBuilder() + .setSnapshotKey(dbKey) + .setExclusiveSize(exclusiveSizeMap.get(dbKey)) + .setExclusiveReplicatedSize( + exclusiveReplicatedSizeMap.get(dbKey)) + .build(); + SetSnapshotPropertyRequest setSnapshotPropertyRequest = + SetSnapshotPropertyRequest.newBuilder() + .setSnapshotProperty(snapshotProperty) + .build(); + + OMRequest omRequest = OMRequest.newBuilder() + .setCmdType(Type.SetSnapshotProperty) + .setSetSnapshotPropertyRequest(setSnapshotPropertyRequest) + .setClientId(clientId.toString()) + .build(); + submitRequest(omRequest, clientId); + exclusiveSizeMap.remove(dbKey); + exclusiveReplicatedSizeMap.remove(dbKey); + completedSnapshotIterator.remove(); + } + } + private void updateDeepCleanedSnapshots(List deepCleanedSnapshots) { if (!deepCleanedSnapshots.isEmpty()) { + ClientId clientId = ClientId.randomId(); SnapshotPurgeRequest snapshotPurgeRequest = SnapshotPurgeRequest .newBuilder() .addAllUpdatedSnapshotDBKey(deepCleanedSnapshots) @@ -369,11 +550,11 @@ private void updateDeepCleanedSnapshots(List deepCleanedSnapshots) { .setClientId(clientId.toString()) .build(); - submitRequest(omRequest); + submitRequest(omRequest, clientId); } } - public void submitRequest(OMRequest omRequest) { + public void submitRequest(OMRequest omRequest, ClientId clientId) { try { if (isRatisEnabled()) { OzoneManagerRatisServer server = getOzoneManager().getOmRatisServer(); diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/snapshot/TestOMSnapshotSetPropertyRequestAndResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/snapshot/TestOMSnapshotSetPropertyRequestAndResponse.java new file mode 100644 index 000000000000..6ab86609dafe --- /dev/null +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/snapshot/TestOMSnapshotSetPropertyRequestAndResponse.java @@ -0,0 +1,188 @@ +/* + * 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.snapshot; + +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.hdds.utils.db.TableIterator; +import org.apache.hadoop.hdds.utils.db.cache.CacheKey; +import org.apache.hadoop.hdds.utils.db.cache.CacheValue; +import org.apache.hadoop.ozone.OzoneConfigKeys; +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.SnapshotInfo; +import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper; +import org.apache.hadoop.ozone.om.request.OMRequestTestUtils; + +import org.apache.hadoop.ozone.om.response.snapshot.OMSnapshotSetPropertyResponse; +import org.apache.hadoop.ozone.om.upgrade.OMLayoutVersionManager; +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.SnapshotProperty; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SetSnapshotPropertyRequest; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.io.TempDir; +import org.mockito.Mockito; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +/** + * Tests TestOMSnapshotSetPropertyRequest + * TestOMSnapshotSetPropertyResponse class. + */ +public class TestOMSnapshotSetPropertyRequestAndResponse { + private BatchOperation batchOperation; + private OzoneManager ozoneManager; + private OMMetadataManager omMetadataManager; + + private String volumeName; + private String bucketName; + private String snapName; + private long exclusiveSize; + private long exclusiveSizeAfterRepl; + + // Just setting ozoneManagerDoubleBuffer which does nothing. + private static final OzoneManagerDoubleBufferHelper + DOUBLE_BUFFER_HELPER = ((response, transactionIndex) -> null); + + @BeforeEach + void setup(@TempDir File testDir) throws Exception { + ozoneManager = Mockito.mock(OzoneManager.class); + OMLayoutVersionManager lvm = mock(OMLayoutVersionManager.class); + when(lvm.isAllowed(anyString())).thenReturn(true); + when(ozoneManager.getVersionManager()).thenReturn(lvm); + when(ozoneManager.isRatisEnabled()).thenReturn(true); + OzoneConfiguration ozoneConfiguration = new OzoneConfiguration(); + ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS, + testDir.getAbsolutePath()); + ozoneConfiguration.set(OzoneConfigKeys.OZONE_METADATA_DIRS, + testDir.getAbsolutePath()); + omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration, + ozoneManager); + when(ozoneManager.getMetadataManager()).thenReturn(omMetadataManager); + + volumeName = UUID.randomUUID().toString(); + bucketName = UUID.randomUUID().toString(); + snapName = UUID.randomUUID().toString(); + exclusiveSize = 2000L; + exclusiveSizeAfterRepl = 6000L; + } + + @Test + public void testValidateAndUpdateCache() throws IOException { + createSnapshotDataForTest(); + assertFalse(omMetadataManager.getSnapshotInfoTable().isEmpty()); + List snapshotUpdateSizeRequests = + createSnapshotUpdateSizeRequest(); + + // Pre-Execute + for (OMRequest request: snapshotUpdateSizeRequests) { + OMSnapshotSetPropertyRequest omSnapshotSetPropertyRequest = new + OMSnapshotSetPropertyRequest(request); + OMRequest modifiedOmRequest = omSnapshotSetPropertyRequest + .preExecute(ozoneManager); + omSnapshotSetPropertyRequest = new + OMSnapshotSetPropertyRequest(modifiedOmRequest); + + // Validate and Update Cache + OMSnapshotSetPropertyResponse omSnapshotSetPropertyResponse = + (OMSnapshotSetPropertyResponse) omSnapshotSetPropertyRequest + .validateAndUpdateCache(ozoneManager, 200L, + DOUBLE_BUFFER_HELPER); + + // Commit to DB. + batchOperation = omMetadataManager.getStore().initBatchOperation(); + omSnapshotSetPropertyResponse.checkAndUpdateDB(omMetadataManager, + batchOperation); + omMetadataManager.getStore().commitBatchOperation(batchOperation); + } + + // Check if the exclusive size is set. + try (TableIterator> + iterator = omMetadataManager.getSnapshotInfoTable().iterator()) { + while (iterator.hasNext()) { + Table.KeyValue snapshotEntry = iterator.next(); + assertCacheValues(snapshotEntry.getKey()); + assertEquals(exclusiveSize, snapshotEntry.getValue(). + getExclusiveSize()); + assertEquals(exclusiveSizeAfterRepl, snapshotEntry.getValue() + .getExclusiveReplicatedSize()); + } + } + } + + private void assertCacheValues(String dbKey) { + CacheValue cacheValue = omMetadataManager + .getSnapshotInfoTable() + .getCacheValue(new CacheKey<>(dbKey)); + assertEquals(exclusiveSize, cacheValue.getCacheValue().getExclusiveSize()); + assertEquals(exclusiveSizeAfterRepl, cacheValue.getCacheValue() + .getExclusiveReplicatedSize()); + } + + private List createSnapshotUpdateSizeRequest() + throws IOException { + List omRequests = new ArrayList<>(); + try (TableIterator> + iterator = omMetadataManager.getSnapshotInfoTable().iterator()) { + while (iterator.hasNext()) { + String snapDbKey = iterator.next().getKey(); + SnapshotProperty snapshotSize = SnapshotProperty.newBuilder() + .setSnapshotKey(snapDbKey) + .setExclusiveSize(exclusiveSize) + .setExclusiveReplicatedSize(exclusiveSizeAfterRepl) + .build(); + SetSnapshotPropertyRequest snapshotUpdateSizeRequest = + SetSnapshotPropertyRequest.newBuilder() + .setSnapshotProperty(snapshotSize) + .build(); + + OMRequest omRequest = OMRequest.newBuilder() + .setCmdType(OzoneManagerProtocolProtos.Type.SetSnapshotProperty) + .setSetSnapshotPropertyRequest(snapshotUpdateSizeRequest) + .setClientId(UUID.randomUUID().toString()) + .build(); + omRequests.add(omRequest); + } + } + return omRequests; + } + + private void createSnapshotDataForTest() throws IOException { + // Create 10 Snapshots + for (int i = 0; i < 10; i++) { + OMRequestTestUtils.addSnapshotToTableCache(volumeName, bucketName, + snapName + i, omMetadataManager); + } + } +} diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestKeyDeletingService.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestKeyDeletingService.java index 5b1c1325d669..340ce3b6bdf2 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestKeyDeletingService.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestKeyDeletingService.java @@ -24,12 +24,15 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.LinkedList; import java.util.List; +import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; +import org.apache.hadoop.hdds.client.RatisReplicationConfig; import org.apache.hadoop.hdds.utils.db.Table; import org.apache.hadoop.hdds.utils.db.TableIterator; import org.apache.hadoop.ozone.om.IOmMetadataReader; @@ -71,6 +74,7 @@ import org.apache.commons.lang3.RandomStringUtils; import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_CONTAINER_REPORT_INTERVAL; +import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.THREE; import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL; import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETING_SERVICE_INTERVAL; import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix; @@ -550,15 +554,126 @@ volumeName, bucketName, getSnapshotPrefix("snap3"), true)) { // 5 keys can be deep cleaned as it was stuck previously assertTableRowCount(snap3deletedTable, 10, metadataManager); - checkSnapDeepCleanStatus(snapshotInfoTable, false); writeClient.deleteSnapshot(volumeName, bucketName, "snap2"); assertTableRowCount(snapshotInfoTable, 2, metadataManager); assertTableRowCount(snap3deletedTable, 0, metadataManager); assertTableRowCount(deletedTable, 0, metadataManager); + checkSnapDeepCleanStatus(snapshotInfoTable, false); + } + + } + + @Test + public void testSnapshotExclusiveSize() throws Exception { + OzoneConfiguration conf = createConfAndInitValues(); + OmTestManagers omTestManagers + = new OmTestManagers(conf); + KeyManager keyManager = omTestManagers.getKeyManager(); + writeClient = omTestManagers.getWriteClient(); + om = omTestManagers.getOzoneManager(); + OMMetadataManager metadataManager = omTestManagers.getMetadataManager(); + Table snapshotInfoTable = + om.getMetadataManager().getSnapshotInfoTable(); + Table deletedTable = + om.getMetadataManager().getDeletedTable(); + Table renamedTable = + om.getMetadataManager().getSnapshotRenamedTable(); + Table keyTable = + om.getMetadataManager().getKeyTable(BucketLayout.DEFAULT); + + KeyDeletingService keyDeletingService = keyManager.getDeletingService(); + // Supspend KDS + keyDeletingService.suspend(); + + String volumeName = "volume1"; + String bucketName = "bucket1"; + String keyName = "key"; + + // Create Volume and Buckets + createVolumeAndBucket(keyManager, volumeName, bucketName, false); + + // Create 3 keys + for (int i = 1; i <= 3; i++) { + createAndCommitKey(keyManager, volumeName, bucketName, keyName + i, 3); + } + assertTableRowCount(keyTable, 3, metadataManager); + + // Create Snapshot1 + writeClient.createSnapshot(volumeName, bucketName, "snap1"); + assertTableRowCount(snapshotInfoTable, 1, metadataManager); + assertTableRowCount(deletedTable, 0, metadataManager); + + // Create 2 keys + for (int i = 4; i <= 5; i++) { + createAndCommitKey(keyManager, volumeName, bucketName, keyName + i, 3); + } + // Delete a key, rename 2 keys. We will be using this to test + // how we handle renamed key for exclusive size calculation. + renameKey(volumeName, bucketName, keyName + 1, "renamedKey1"); + renameKey(volumeName, bucketName, keyName + 2, "renamedKey2"); + deleteKey(volumeName, bucketName, keyName + 3); + assertTableRowCount(deletedTable, 1, metadataManager); + assertTableRowCount(renamedTable, 2, metadataManager); + + // Create Snapshot2 + writeClient.createSnapshot(volumeName, bucketName, "snap2"); + assertTableRowCount(snapshotInfoTable, 2, metadataManager); + assertTableRowCount(deletedTable, 0, metadataManager); + + // Create 2 keys + for (int i = 6; i <= 7; i++) { + createAndCommitKey(keyManager, volumeName, bucketName, keyName + i, 3); } + deleteKey(volumeName, bucketName, "renamedKey1"); + deleteKey(volumeName, bucketName, "key4"); + // Do a second rename of already renamedKey2 + renameKey(volumeName, bucketName, "renamedKey2", "renamedKey22"); + assertTableRowCount(deletedTable, 2, metadataManager); + assertTableRowCount(renamedTable, 1, metadataManager); + + // Create Snapshot3 + writeClient.createSnapshot(volumeName, bucketName, "snap3"); + // Delete 4 keys + deleteKey(volumeName, bucketName, "renamedKey22"); + for (int i = 5; i <= 7; i++) { + deleteKey(volumeName, bucketName, keyName + i); + } + + // Create Snapshot4 + writeClient.createSnapshot(volumeName, bucketName, "snap4"); + createAndCommitKey(keyManager, volumeName, bucketName, "key8", 3); + keyDeletingService.resume(); + + Map expectedSize = new HashMap() {{ + put("snap1", 1000L); + put("snap2", 1000L); + put("snap3", 2000L); + put("snap4", 0L); + }}; + + long prevKdsRunCount = keyDeletingService.getRunCount().get(); + + // Let KeyDeletingService to run for some iterations + GenericTestUtils.waitFor( + () -> (keyDeletingService.getRunCount().get() > prevKdsRunCount + 5), + 100, 10000); + + // Check if the exclusive size is set. + try (TableIterator> + iterator = snapshotInfoTable.iterator()) { + while (iterator.hasNext()) { + Table.KeyValue snapshotEntry = iterator.next(); + String snapshotName = snapshotEntry.getValue().getName(); + assertEquals(expectedSize.get(snapshotName), snapshotEntry.getValue(). + getExclusiveSize()); + // Since for the test we are using RATIS/THREE + assertEquals(expectedSize.get(snapshotName) * 3, + snapshotEntry.getValue().getExclusiveReplicatedSize()); + } + } } private void checkSnapDeepCleanStatus(Table @@ -613,6 +728,37 @@ private void createVolumeAndBucket(KeyManager keyManager, String volumeName, .build()); } + private void deleteKey(String volumeName, + String bucketName, + String keyName) throws IOException { + OmKeyArgs keyArg = + new OmKeyArgs.Builder() + .setVolumeName(volumeName) + .setBucketName(bucketName) + .setKeyName(keyName) + .setAcls(Collections.emptyList()) + .setReplicationConfig(StandaloneReplicationConfig.getInstance( + HddsProtos.ReplicationFactor.THREE)) + .build(); + writeClient.deleteKey(keyArg); + } + + private void renameKey(String volumeName, + String bucketName, + String keyName, + String toKeyName) throws IOException { + OmKeyArgs keyArg = + new OmKeyArgs.Builder() + .setVolumeName(volumeName) + .setBucketName(bucketName) + .setKeyName(keyName) + .setAcls(Collections.emptyList()) + .setReplicationConfig(StandaloneReplicationConfig.getInstance( + HddsProtos.ReplicationFactor.THREE)) + .build(); + writeClient.renameKey(keyArg, toKeyName); + } + private OmKeyArgs createAndCommitKey(KeyManager keyManager, String volumeName, String bucketName, String keyName, int numBlocks) throws IOException { return createAndCommitKey(keyManager, volumeName, bucketName, keyName, @@ -630,8 +776,8 @@ private OmKeyArgs createAndCommitKey(KeyManager keyManager, String volumeName, .setBucketName(bucketName) .setKeyName(keyName) .setAcls(Collections.emptyList()) - .setReplicationConfig(StandaloneReplicationConfig.getInstance( - HddsProtos.ReplicationFactor.ONE)) + .setReplicationConfig(RatisReplicationConfig.getInstance(THREE)) + .setDataSize(1000L) .setLocationInfoList(new ArrayList<>()) .build(); //Open and Commit the Key in the Key Manager.