diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/exceptions/SCMException.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/exceptions/SCMException.java index b1dffa5e5135..6a4b20a4553d 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/exceptions/SCMException.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/exceptions/SCMException.java @@ -112,6 +112,9 @@ public enum ResultCodes { FAILED_TO_FIND_CONTAINER, FAILED_TO_FIND_CONTAINER_WITH_SPACE, BLOCK_EXISTS, + //OM can send a deleteBlockRequest for key delete more than once. Thus, it shouldn't + // fail the request since SCM already deleted the block in the previous run. We should never use this enum. + @Deprecated FAILED_TO_FIND_BLOCK, IO_EXCEPTION, UNEXPECTED_CONTAINER_STATE, diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/CheckedExceptionOperation.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/CheckedExceptionOperation.java new file mode 100644 index 000000000000..c6a962c107c9 --- /dev/null +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/CheckedExceptionOperation.java @@ -0,0 +1,17 @@ +package org.apache.hadoop.ozone.util; + +/** + * + * Represents a function that accepts one argument and produces a result. + * This is a functional interface whose functional method is apply(Object). + * Type parameters: + * – the type of the input to the function – the type of the result of the function + * - the type of exception thrown. + */ +public interface CheckedExceptionOperation { + R apply(T t) throws E; + + default CheckedExceptionOperation andThen(CheckedExceptionOperation operation) throws E { + return (T t) -> operation.apply(this.apply(t)); + } +} diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java index 7ed23b110c78..9f7f0b0a93b3 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java @@ -282,18 +282,11 @@ public List deleteKeyBlocks( e = ioe; perfMetrics.updateDeleteKeyFailureStats(startNanos); LOG.warn("Fail to delete {} keys", keyBlocksInfoList.size(), ioe); - switch (ioe instanceof SCMException ? ((SCMException) ioe).getResult() : - IO_EXCEPTION) { - case SAFE_MODE_EXCEPTION: + if ((ioe instanceof SCMException ? ((SCMException) ioe).getResult() : + IO_EXCEPTION) == SCMException.ResultCodes.SAFE_MODE_EXCEPTION) { resultCode = ScmBlockLocationProtocolProtos.DeleteScmBlockResult.Result.safeMode; - break; - case FAILED_TO_FIND_BLOCK: - resultCode = - ScmBlockLocationProtocolProtos.DeleteScmBlockResult.Result. - errorNotFound; - break; - default: + } else { resultCode = ScmBlockLocationProtocolProtos.DeleteScmBlockResult.Result. unknownFailure; diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java index a77bc4f53048..161b1c555ff2 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java @@ -404,7 +404,7 @@ private OMConfigKeys() { * Configuration properties for Snapshot Directory Service. */ public static final String OZONE_SNAPSHOT_DEEP_CLEANING_ENABLED = "ozone.snapshot.deep.cleaning.enabled"; - public static final boolean OZONE_SNAPSHOT_DEEP_CLEANING_ENABLED_DEFAULT = false; + public static final boolean OZONE_SNAPSHOT_DEEP_CLEANING_ENABLED_DEFAULT = true; public static final String OZONE_SNAPSHOT_DIRECTORY_SERVICE_INTERVAL = "ozone.snapshot.directory.service.interval"; public static final String OZONE_SNAPSHOT_DIRECTORY_SERVICE_INTERVAL_DEFAULT diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/lock/OzoneManagerLock.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/lock/OzoneManagerLock.java index aadb6c02c763..a0a13b984b4b 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/lock/OzoneManagerLock.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/lock/OzoneManagerLock.java @@ -457,7 +457,8 @@ public enum Resource { S3_SECRET_LOCK((byte) 4, "S3_SECRET_LOCK"), // 31 KEY_PATH_LOCK((byte) 5, "KEY_PATH_LOCK"), //63 PREFIX_LOCK((byte) 6, "PREFIX_LOCK"), //127 - SNAPSHOT_LOCK((byte) 7, "SNAPSHOT_LOCK"); // = 255 + SNAPSHOT_LOCK((byte) 7, "SNAPSHOT_LOCK"), // = 255 + SNAPSHOT_GC_LOCK((byte) 8, "SNAPSHOT_GC_LOCK"); // level of the resource private byte lockLevel; diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyPurging.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyPurging.java index e3bb5b5bccb8..29efc1caeb10 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyPurging.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyPurging.java @@ -140,7 +140,7 @@ public void testKeysPurgingByKeyDeletingService() throws Exception { GenericTestUtils.waitFor( () -> { try { - return keyManager.getPendingDeletionKeys(Integer.MAX_VALUE) + return keyManager.getPendingDeletionKeys(null, Integer.MAX_VALUE) .getKeyBlocksList().size() == 0; } catch (IOException e) { return false; diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestSnapshotDeletingServiceIntegrationTest.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestSnapshotDeletingServiceIntegrationTest.java index 254de072e05b..b4a3676820f0 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestSnapshotDeletingServiceIntegrationTest.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestSnapshotDeletingServiceIntegrationTest.java @@ -632,9 +632,9 @@ public void testParallelExcecutionOfKeyDeletionAndSnapshotDeletion() throws Exce try (ReferenceCounted snapshot = om.getOmSnapshotManager().getSnapshot(testBucket.getVolumeName(), testBucket.getName(), testBucket.getName() + "snap2")) { renamesKeyEntries = snapshot.get().getKeyManager().getRenamesKeyEntries(testBucket.getVolumeName(), - testBucket.getName(), "", 1000); + testBucket.getName(), "", (kv) -> true, 1000); deletedKeyEntries = snapshot.get().getKeyManager().getDeletedKeyEntries(testBucket.getVolumeName(), - testBucket.getName(), "", 1000); + testBucket.getName(), "", (kv) -> true, 1000); deletedDirEntries = snapshot.get().getKeyManager().getDeletedDirEntries(testBucket.getVolumeName(), testBucket.getName(), 1000); } @@ -669,20 +669,20 @@ public void testParallelExcecutionOfKeyDeletionAndSnapshotDeletion() throws Exce testBucket.getName(), testBucket.getName() + "snap2")) { Assertions.assertEquals(Collections.emptyList(), snapshot.get().getKeyManager().getRenamesKeyEntries(testBucket.getVolumeName(), - testBucket.getName(), "", 1000)); + testBucket.getName(), "", (kv) -> true, 1000)); Assertions.assertEquals(Collections.emptyList(), snapshot.get().getKeyManager().getDeletedKeyEntries(testBucket.getVolumeName(), - testBucket.getName(), "", 1000)); + testBucket.getName(), "", (kv) -> true, 1000)); Assertions.assertEquals(Collections.emptyList(), snapshot.get().getKeyManager().getDeletedDirEntries(testBucket.getVolumeName(), testBucket.getName(), 1000)); } List> aosRenamesKeyEntries = om.getKeyManager().getRenamesKeyEntries(testBucket.getVolumeName(), - testBucket.getName(), "", 1000); + testBucket.getName(), "", (kv) -> true, 1000); List>> aosDeletedKeyEntries = om.getKeyManager().getDeletedKeyEntries(testBucket.getVolumeName(), - testBucket.getName(), "", 1000); + testBucket.getName(), "", (kv) -> true, 1000); List> aosDeletedDirEntries = om.getKeyManager().getDeletedDirEntries(testBucket.getVolumeName(), testBucket.getName(), 1000); diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestSnapshotDirectoryCleaningService.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestSnapshotDirectoryCleaningService.java deleted file mode 100644 index 3be0725a0093..000000000000 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestSnapshotDirectoryCleaningService.java +++ /dev/null @@ -1,272 +0,0 @@ -/* - * 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.snapshot; - -import org.apache.hadoop.fs.CommonConfigurationKeysPublic; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hdds.conf.OzoneConfiguration; -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.MiniOzoneCluster; -import org.apache.hadoop.ozone.OzoneConsts; -import org.apache.hadoop.ozone.TestDataUtil; -import org.apache.hadoop.ozone.client.OzoneBucket; -import org.apache.hadoop.ozone.client.OzoneClient; -import org.apache.hadoop.ozone.om.OMConfigKeys; -import org.apache.hadoop.ozone.om.helpers.BucketLayout; -import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo; -import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; -import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo; -import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; -import org.apache.hadoop.ozone.om.service.SnapshotDirectoryCleaningService; -import org.apache.ozone.test.GenericTestUtils; -import org.apache.ozone.test.tag.Flaky; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.Timeout; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicLong; - -import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ACL_ENABLED; -import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL; -import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_FS_ITERATE_BATCH_SIZE; -import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_SNAPSHOT_DEEP_CLEANING_ENABLED; -import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; -import static org.junit.jupiter.api.Assertions.assertEquals; - -/** - * Test Snapshot Directory Service. - */ -@Timeout(300) -public class TestSnapshotDirectoryCleaningService { - - private static final Logger LOG = - LoggerFactory.getLogger(TestSnapshotDirectoryCleaningService.class); - - private static MiniOzoneCluster cluster; - private static FileSystem fs; - private static String volumeName; - private static String bucketName; - private static OzoneClient client; - - @BeforeAll - public static void init() throws Exception { - OzoneConfiguration conf = new OzoneConfiguration(); - conf.setInt(OMConfigKeys.OZONE_SNAPSHOT_DIRECTORY_SERVICE_INTERVAL, 2500); - conf.setBoolean(OZONE_SNAPSHOT_DEEP_CLEANING_ENABLED, true); - conf.setTimeDuration(OZONE_BLOCK_DELETING_SERVICE_INTERVAL, 2500, - TimeUnit.MILLISECONDS); - conf.setBoolean(OZONE_ACL_ENABLED, true); - cluster = MiniOzoneCluster.newBuilder(conf) - .setNumDatanodes(3) - .build(); - cluster.waitForClusterToBeReady(); - client = cluster.newClient(); - - // create a volume and a bucket to be used by OzoneFileSystem - OzoneBucket bucket = TestDataUtil.createVolumeAndBucket(client, - BucketLayout.FILE_SYSTEM_OPTIMIZED); - volumeName = bucket.getVolumeName(); - bucketName = bucket.getName(); - - String rootPath = String.format("%s://%s.%s/", - OzoneConsts.OZONE_URI_SCHEME, bucketName, volumeName); - - // Set the fs.defaultFS and start the filesystem - conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, rootPath); - // Set the number of keys to be processed during batch operate. - conf.setInt(OZONE_FS_ITERATE_BATCH_SIZE, 5); - - fs = FileSystem.get(conf); - } - - @AfterAll - public static void teardown() { - IOUtils.closeQuietly(client); - if (cluster != null) { - cluster.shutdown(); - } - IOUtils.closeQuietly(fs); - } - - @AfterEach - public void cleanup() { - assertDoesNotThrow(() -> { - Path root = new Path("/"); - FileStatus[] fileStatuses = fs.listStatus(root); - for (FileStatus fileStatus : fileStatuses) { - fs.delete(fileStatus.getPath(), true); - } - }); - } - - @SuppressWarnings("checkstyle:LineLength") - @Test - @Flaky("HDDS-11129") - public void testExclusiveSizeWithDirectoryDeepClean() throws Exception { - - Table deletedDirTable = - cluster.getOzoneManager().getMetadataManager().getDeletedDirTable(); - Table keyTable = - cluster.getOzoneManager().getMetadataManager() - .getKeyTable(BucketLayout.FILE_SYSTEM_OPTIMIZED); - Table dirTable = - cluster.getOzoneManager().getMetadataManager().getDirectoryTable(); - Table deletedKeyTable = - cluster.getOzoneManager().getMetadataManager().getDeletedTable(); - Table snapshotInfoTable = - cluster.getOzoneManager().getMetadataManager().getSnapshotInfoTable(); - SnapshotDirectoryCleaningService snapshotDirectoryCleaningService = - cluster.getOzoneManager().getKeyManager().getSnapshotDirectoryService(); - - /* DirTable - /v/b/snapDir - /v/b/snapDir/appRoot0-2/ - /v/b/snapDir/appRoot0-2/parentDir0-2/ - FileTable - /v/b/snapDir/testKey0 - testKey4 = 5 keys - /v/b/snapDir/appRoot0-2/parentDir0-2/childFile = 9 keys - /v/b/snapDir/appRoot0/parentDir0-2/childFile0-4 = 15 keys - */ - - Path root = new Path("/snapDir"); - // Create parent dir from root. - fs.mkdirs(root); - - // Add 5 files inside root dir - // Creates /v/b/snapDir/testKey0 - testKey4 - for (int i = 0; i < 5; i++) { - Path path = new Path(root, "testKey" + i); - try (FSDataOutputStream stream = fs.create(path)) { - stream.write(1); - } - } - - // Creates /v/b/snapDir/appRoot0-2/parentDir0-2/childFile - for (int i = 0; i < 3; i++) { - for (int j = 0; j < 3; j++) { - Path appRoot = new Path(root, "appRoot" + j); - Path parent = new Path(appRoot, "parentDir" + i); - Path child = new Path(parent, "childFile"); - try (FSDataOutputStream stream = fs.create(child)) { - stream.write(1); - } - } - } - - assertTableRowCount(keyTable, 14); - assertTableRowCount(dirTable, 13); - // Create snapshot - client.getObjectStore().createSnapshot(volumeName, bucketName, "snap1"); - - // Creates /v/b/snapDir/appRoot0/parentDir0-2/childFile0-4 - for (int i = 0; i < 3; i++) { - Path appRoot = new Path(root, "appRoot0"); - Path parent = new Path(appRoot, "parentDir" + i); - for (int j = 0; j < 5; j++) { - Path child = new Path(parent, "childFile" + j); - try (FSDataOutputStream stream = fs.create(child)) { - stream.write(1); - } - } - } - - for (int i = 5; i < 10; i++) { - Path path = new Path(root, "testKey" + i); - try (FSDataOutputStream stream = fs.create(path)) { - stream.write(1); - } - } - - assertTableRowCount(deletedDirTable, 0); - assertTableRowCount(keyTable, 34); - assertTableRowCount(dirTable, 13); - Path appRoot0 = new Path(root, "appRoot0"); - // Only parentDir0-2/childFile under appRoot0 is exclusive for snap1 - fs.delete(appRoot0, true); - assertTableRowCount(deletedDirTable, 1); - client.getObjectStore().createSnapshot(volumeName, bucketName, "snap2"); - - // Delete testKey0-9 - for (int i = 0; i < 10; i++) { - Path testKey = new Path(root, "testKey" + i); - fs.delete(testKey, false); - } - - fs.delete(root, true); - assertTableRowCount(deletedKeyTable, 10); - client.getObjectStore().createSnapshot(volumeName, bucketName, "snap3"); - long prevRunCount = snapshotDirectoryCleaningService.getRunCount().get(); - GenericTestUtils.waitFor(() -> snapshotDirectoryCleaningService.getRunCount().get() - > prevRunCount + 1, 100, 10000); - - Thread.sleep(2000); - Map expectedSize = new HashMap() {{ - // /v/b/snapDir/appRoot0/parentDir0-2/childFile contribute - // exclusive size, /v/b/snapDir/appRoot0/parentDir0-2/childFile0-4 - // are deep cleaned and hence don't contribute to size. - put("snap1", 3L); - // Only testKey5-9 contribute to the exclusive size - put("snap2", 5L); - put("snap3", 0L); - }}; - Thread.sleep(500); - 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 assertTableRowCount(Table table, int count) - throws TimeoutException, InterruptedException { - GenericTestUtils.waitFor(() -> assertTableRowCount(count, table), 1000, - 120000); // 2 minutes - } - - private boolean assertTableRowCount(int expectedCount, - Table table) { - AtomicLong count = new AtomicLong(0L); - assertDoesNotThrow(() -> { - count.set(cluster.getOzoneManager().getMetadataManager().countRowsInTable(table)); - LOG.info("{} actual row count={}, expectedCount={}", table.getName(), - count.get(), expectedCount); - }); - return count.get() == expectedCount; - } -} diff --git a/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto b/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto index f71dc44fec51..a0def0f27d98 100644 --- a/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto +++ b/hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto @@ -273,7 +273,7 @@ message OMRequest { optional ListSnapshotRequest ListSnapshotRequest = 113; optional SnapshotDiffRequest snapshotDiffRequest = 114; optional DeleteSnapshotRequest DeleteSnapshotRequest = 115; - optional SnapshotMoveDeletedKeysRequest SnapshotMoveDeletedKeysRequest = 116; + optional SnapshotMoveDeletedKeysRequest SnapshotMoveDeletedKeysRequest = 116 [deprecated = true]; optional hdds.TransferLeadershipRequestProto TransferOmLeadershipRequest = 117; optional SnapshotPurgeRequest SnapshotPurgeRequest = 118; @@ -295,6 +295,7 @@ message OMRequest { optional GetQuotaRepairStatusRequest GetQuotaRepairStatusRequest = 133; optional StartQuotaRepairRequest StartQuotaRepairRequest = 134; optional SnapshotMoveTableKeysRequest SnapshotMoveTableKeysRequest = 135; + repeated SetSnapshotPropertyRequest SetSnapshotPropertyRequests = 136; } message OMResponse { @@ -1378,8 +1379,9 @@ message PurgeKeysRequest { // if set, will purge keys in a snapshot DB instead of active DB optional string snapshotTableKey = 2; repeated SnapshotMoveKeyInfos keysToUpdate = 3; - // previous snapshotID can also be null & this field would be absent in older requests. - optional NullableUUID expectedPreviousSnapshotID = 4; + // previous snapshotID can also be null & this field would be absent in older requests. + optional NullableUUID expectedPreviousSnapshotID = 4; + repeated string renamedKeys = 5; } message PurgeKeysResponse { @@ -2012,6 +2014,7 @@ message SetSnapshotPropertyRequest { optional SnapshotSize snapshotSize = 3; optional bool deepCleanedDeletedDir = 4; optional bool deepCleanedDeletedKey = 5; + optional SnapshotSize snapshotDirSize = 6; } // SnapshotProperty in entirely deprecated, Keeping it here for proto.lock compatibility 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 ae57c18354d2..76bdd82ee362 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 @@ -605,6 +605,11 @@ default String getOpenFileName(long volumeId, long bucketId, long parentObjectId */ String getRenameKey(String volume, String bucket, long objectID); + /** + * Given renameKey, return the volume, bucket & objectID from the key. + */ + String[] splitRenameKey(String renameKey); + /** * Returns the DB key name of a multipart upload key in OM metadata store * for FSO-enabled buckets. diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManager.java index a0f3053d731c..9cfddcbeebe5 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManager.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManager.java @@ -29,11 +29,12 @@ import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadListParts; import org.apache.hadoop.ozone.om.fs.OzoneManagerFS; import org.apache.hadoop.hdds.utils.BackgroundService; +import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo; import org.apache.hadoop.ozone.om.service.DirectoryDeletingService; import org.apache.hadoop.ozone.om.service.KeyDeletingService; import org.apache.hadoop.ozone.om.service.SnapshotDeletingService; -import org.apache.hadoop.ozone.om.service.SnapshotDirectoryCleaningService; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ExpiredMultipartUploadsBucket; +import org.apache.hadoop.ozone.util.CheckedExceptionOperation; import java.io.IOException; import java.time.Duration; @@ -119,30 +120,52 @@ ListKeysResult listKeys(String volumeName, String bucketName, String startKey, * and a hashmap for key-value pair to be updated in the deletedTable. * @throws IOException */ - PendingKeysDeletion getPendingDeletionKeys(int count) throws IOException; + PendingKeysDeletion getPendingDeletionKeys( + CheckedExceptionOperation, Boolean, IOException> filter, int count) + throws IOException; + /** + * Returns a PendingKeysDeletion. It has a list of pending deletion key info + * that ups to the given count.Each entry is a {@link BlockGroup}, which + * contains the info about the key name and all its associated block IDs. + * Second is a Mapping of Key-Value pair which is updated in the deletedTable. + * + * @param count max number of keys to return. + * @return a Pair of list of {@link BlockGroup} representing keys and blocks, + * and a hashmap for key-value pair to be updated in the deletedTable. + * @throws IOException + */ + PendingKeysDeletion getPendingDeletionKeys( + String volume, String bucket, String startKey, + CheckedExceptionOperation, Boolean, IOException> filter, int count) + throws IOException; /** - * Returns a list rename entries from the snapshotRenamedTable. + * Returns a list renamed entries from the snapshotRenamedTable. * - * @param size max number of keys to return. + * @param count max number of keys to return. + * @param filter filter to apply on the entries. * @return a Pair of list of {@link org.apache.hadoop.hdds.utils.db.Table.KeyValue} representing the keys in the * underlying metadataManager. * @throws IOException */ List> getRenamesKeyEntries( - String volume, String bucket, String startKey, int size) throws IOException; - + String volume, String bucket, String startKey, + CheckedExceptionOperation, Boolean, IOException> filter, int size) + throws IOException; /** - * Returns a list deleted entries from the deletedTable. + * Returns a list rename entries from the snapshotRenamedTable. * - * @param size max number of keys to return. + * @param count max number of keys to return. + * @param filter filter to apply on the entries. * @return a Pair of list of {@link org.apache.hadoop.hdds.utils.db.Table.KeyValue} representing the keys in the * underlying metadataManager. * @throws IOException */ List>> getDeletedKeyEntries( - String volume, String bucket, String startKey, int size) throws IOException; + String volume, String bucket, String startKey, + CheckedExceptionOperation, Boolean, IOException> filter, int count) + throws IOException; /** * Returns the names of up to {@code count} open keys whose age is @@ -241,6 +264,8 @@ OmMultipartUploadListParts listParts(String volumeName, String bucketName, */ Table.KeyValue getPendingDeletionDir() throws IOException; + TableIterator> getPendingDeletionDirs( + String volume, String bucket) throws IOException; /** * Returns an iterator for pending deleted directories. * @throws IOException @@ -269,8 +294,12 @@ default List> getDeletedDirEntries(String volu * @return list of dirs * @throws IOException */ - List getPendingDeletionSubDirs(long volumeId, long bucketId, - OmKeyInfo parentInfo, long numEntries) throws IOException; + List getPendingDeletionSubDirs( + long volumeId, long bucketId, OmKeyInfo parentInfo, + CheckedExceptionOperation, Boolean, IOException> filter, + long numEntries) throws IOException; + + /** * Returns all sub files under the given parent directory. @@ -280,9 +309,10 @@ List getPendingDeletionSubDirs(long volumeId, long bucketId, * @return list of files * @throws IOException */ - List getPendingDeletionSubFiles(long volumeId, - long bucketId, OmKeyInfo parentInfo, long numEntries) - throws IOException; + List getPendingDeletionSubFiles(long volumeId, long bucketId, OmKeyInfo parentInfo, + CheckedExceptionOperation, Boolean, + IOException> filter, + long numEntries) throws IOException; /** * Returns the instance of Directory Deleting Service. @@ -314,9 +344,4 @@ List getPendingDeletionSubFiles(long volumeId, */ SnapshotDeletingService getSnapshotDeletingService(); - /** - * Returns the instance of Snapshot Directory service. - * @return Background service. - */ - SnapshotDirectoryCleaningService getSnapshotDirectoryService(); } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java index e99bdea85ea4..570392da7985 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java @@ -46,12 +46,14 @@ import com.google.common.base.Strings; import jakarta.annotation.Nonnull; import org.apache.commons.lang3.StringUtils; +import com.google.common.collect.Lists; import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.conf.StorageUnit; import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension; import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion; import org.apache.hadoop.fs.FileEncryptionInfo; import org.apache.hadoop.hdds.DFSConfigKeysLegacy; +import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.client.ReplicationConfig; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.DatanodeDetails; @@ -73,6 +75,8 @@ import org.apache.hadoop.net.DNSToSwitchMapping; import org.apache.hadoop.net.TableMapping; import org.apache.hadoop.ozone.OmUtils; +import org.apache.hadoop.ozone.common.BlockGroup; +import org.apache.hadoop.ozone.util.CheckedExceptionOperation; import org.apache.hadoop.ozone.OzoneAcl; import org.apache.hadoop.ozone.om.exceptions.OMException; import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes; @@ -102,7 +106,6 @@ import org.apache.hadoop.ozone.om.service.MultipartUploadCleanupService; import org.apache.hadoop.ozone.om.service.OpenKeyCleanupService; import org.apache.hadoop.ozone.om.service.SnapshotDeletingService; -import org.apache.hadoop.ozone.om.service.SnapshotDirectoryCleaningService; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ExpiredMultipartUploadsBucket; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.PartKeyInfo; import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer; @@ -147,10 +150,6 @@ import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_OPEN_KEY_CLEANUP_SERVICE_TIMEOUT_DEFAULT; import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_SNAPSHOT_DEEP_CLEANING_ENABLED; import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_SNAPSHOT_DEEP_CLEANING_ENABLED_DEFAULT; -import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_SNAPSHOT_DIRECTORY_SERVICE_INTERVAL; -import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_SNAPSHOT_DIRECTORY_SERVICE_INTERVAL_DEFAULT; -import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_SNAPSHOT_DIRECTORY_SERVICE_TIMEOUT; -import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_SNAPSHOT_DIRECTORY_SERVICE_TIMEOUT_DEFAULT; import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_SNAPSHOT_SST_FILTERING_SERVICE_INTERVAL; import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_SNAPSHOT_SST_FILTERING_SERVICE_INTERVAL_DEFAULT; import static org.apache.hadoop.ozone.om.OzoneManagerUtils.getBucketLayout; @@ -196,7 +195,6 @@ public class KeyManagerImpl implements KeyManager { private BackgroundService openKeyCleanupService; private BackgroundService multipartUploadCleanupService; - private SnapshotDirectoryCleaningService snapshotDirectoryCleaningService; private DNSToSwitchMapping dnsToSwitchMapping; public KeyManagerImpl(OzoneManager om, ScmClient scmClient, @@ -316,22 +314,6 @@ public void start(OzoneConfiguration configuration) { } } - if (isSnapshotDeepCleaningEnabled && snapshotDirectoryCleaningService == null && - ozoneManager.isFilesystemSnapshotEnabled()) { - long dirDeleteInterval = configuration.getTimeDuration( - OZONE_SNAPSHOT_DIRECTORY_SERVICE_INTERVAL, - OZONE_SNAPSHOT_DIRECTORY_SERVICE_INTERVAL_DEFAULT, - TimeUnit.MILLISECONDS); - long serviceTimeout = configuration.getTimeDuration( - OZONE_SNAPSHOT_DIRECTORY_SERVICE_TIMEOUT, - OZONE_SNAPSHOT_DIRECTORY_SERVICE_TIMEOUT_DEFAULT, - TimeUnit.MILLISECONDS); - snapshotDirectoryCleaningService = new SnapshotDirectoryCleaningService( - dirDeleteInterval, TimeUnit.MILLISECONDS, serviceTimeout, - ozoneManager, scmClient.getBlockClient()); - snapshotDirectoryCleaningService.start(); - } - if (multipartUploadCleanupService == null) { long serviceInterval = configuration.getTimeDuration( OZONE_OM_MPU_CLEANUP_SERVICE_INTERVAL, @@ -388,10 +370,6 @@ public void stop() throws IOException { multipartUploadCleanupService.shutdown(); multipartUploadCleanupService = null; } - if (snapshotDirectoryCleaningService != null) { - snapshotDirectoryCleaningService.shutdown(); - snapshotDirectoryCleaningService = null; - } } private OmBucketInfo getBucketInfo(String volumeName, String bucketName) @@ -658,25 +636,83 @@ public ListKeysResult listKeys(String volumeName, String bucketName, } @Override - public PendingKeysDeletion getPendingDeletionKeys(final int count) + public PendingKeysDeletion getPendingDeletionKeys( + CheckedExceptionOperation, Boolean, IOException> filter, int count) throws IOException { - OmMetadataManagerImpl omMetadataManager = - (OmMetadataManagerImpl) metadataManager; - return omMetadataManager - .getPendingDeletionKeys(count, ozoneManager.getOmSnapshotManager()); + return getPendingDeletionKeys(null, null, null, filter, count); + } + + @Override + public PendingKeysDeletion getPendingDeletionKeys( + String volume, String bucket, String startKey, + CheckedExceptionOperation, Boolean, IOException> filter, + int count) throws IOException { + List keyBlocksList = Lists.newArrayList(); + Map keysToModify = new HashMap<>(); + final String nextPageStartKey; + // Bucket prefix would be empty if volume is empty i.e. either null or "". + Optional bucketPrefix = getBucketPrefix(volume, bucket, false); + try (TableIterator> + delKeyIter = metadataManager.getDeletedTable().iterator(bucketPrefix.orElse(""))) { + + /* Seeking to the start key if it not null. The next key picked up would be ensured to start with the bucket + prefix, {@link org.apache.hadoop.hdds.utils.db.Table#iterator(bucketPrefix)} would ensure this. + */ + if (startKey != null) { + delKeyIter.seek(startKey); + } + int currentCount = 0; + while (delKeyIter.hasNext() && currentCount < count) { + RepeatedOmKeyInfo notReclaimableKeyInfo = new RepeatedOmKeyInfo(); + Table.KeyValue kv = delKeyIter.next(); + if (kv != null) { + List blockGroupList = Lists.newArrayList(); + // Multiple keys with the same path can be queued in one DB entry + RepeatedOmKeyInfo infoList = kv.getValue(); + for (OmKeyInfo info : infoList.cloneOmKeyInfoList()) { + + // Skip the key if the filter doesn't allow the file to be deleted. + if (filter == null || filter.apply(Table.newKeyValue(kv.getKey(), info))) { + List blockIDS = info.getKeyLocationVersions().stream() + .flatMap(versionLocations -> versionLocations.getLocationList().stream() + .map(b -> new BlockID(b.getContainerID(), b.getLocalID()))).collect(Collectors.toList()); + BlockGroup keyBlocks = BlockGroup.newBuilder().setKeyName(kv.getKey()) + .addAllBlockIDs(blockIDS).build(); + blockGroupList.add(keyBlocks); + currentCount++; + } else { + notReclaimableKeyInfo.addOmKeyInfo(info); + } + } + + List notReclaimableKeyInfoList = notReclaimableKeyInfo.getOmKeyInfoList(); + + // If all the versions are not reclaimable, then modify key by just purging the key that can be purged. + if (notReclaimableKeyInfoList.size() > 0 && + notReclaimableKeyInfoList.size() != infoList.getOmKeyInfoList().size()) { + keysToModify.put(kv.getKey(), notReclaimableKeyInfo); + } + keyBlocksList.addAll(blockGroupList); + } + } + nextPageStartKey = delKeyIter.hasNext() ? delKeyIter.next().getKey() : null; + } + return new PendingKeysDeletion(keyBlocksList, keysToModify, nextPageStartKey); } private List> getTableEntries(String startKey, TableIterator> tableIterator, - Function valueFunction, int size) throws IOException { + Function valueFunction, + CheckedExceptionOperation, Boolean, IOException> filter, + int size) throws IOException { List> entries = new ArrayList<>(); /* Seek to the start key if it not null. The next key in queue is ensured to start with the bucket prefix, {@link org.apache.hadoop.hdds.utils.db.Table#iterator(bucketPrefix)} would ensure this. */ if (startKey != null) { tableIterator.seek(startKey); - tableIterator.seekToFirst(); } + int currentCount = 0; while (tableIterator.hasNext() && currentCount < size) { Table.KeyValue kv = tableIterator.next(); @@ -702,21 +738,25 @@ private Optional getBucketPrefix(String volumeName, String bucketName, b @Override public List> getRenamesKeyEntries( - String volume, String bucket, String startKey, int size) throws IOException { + String volume, String bucket, String startKey, + CheckedExceptionOperation, Boolean, IOException> filter, + int size) throws IOException { Optional bucketPrefix = getBucketPrefix(volume, bucket, false); try (TableIterator> renamedKeyIter = metadataManager.getSnapshotRenamedTable().iterator(bucketPrefix.orElse(""))) { - return getTableEntries(startKey, renamedKeyIter, Function.identity(), size); + return getTableEntries(startKey, renamedKeyIter, Function.identity(), filter, size); } } @Override public List>> getDeletedKeyEntries( - String volume, String bucket, String startKey, int size) throws IOException { + String volume, String bucket, String startKey, + CheckedExceptionOperation, Boolean, IOException> filter, + int size) throws IOException { Optional bucketPrefix = getBucketPrefix(volume, bucket, false); try (TableIterator> delKeyIter = metadataManager.getDeletedTable().iterator(bucketPrefix.orElse(""))) { - return getTableEntries(startKey, delKeyIter, RepeatedOmKeyInfo::cloneOmKeyInfoList, size); + return getTableEntries(startKey, delKeyIter, RepeatedOmKeyInfo::cloneOmKeyInfoList, filter, size); } } @@ -770,10 +810,6 @@ public SnapshotDeletingService getSnapshotDeletingService() { return snapshotDeletingService; } - public SnapshotDirectoryCleaningService getSnapshotDirectoryService() { - return snapshotDirectoryCleaningService; - } - public boolean isSstFilteringSvcEnabled() { long serviceInterval = ozoneManager.getConfiguration() .getTimeDuration(OZONE_SNAPSHOT_SST_FILTERING_SERVICE_INTERVAL, @@ -781,7 +817,7 @@ public boolean isSstFilteringSvcEnabled() { TimeUnit.MILLISECONDS); return serviceInterval != DISABLE_VALUE; } - + @Override public OmMultipartUploadList listMultipartUploads(String volumeName, String bucketName, String prefix) throws OMException { @@ -2036,27 +2072,39 @@ public Table.KeyValue getPendingDeletionDir() } @Override - public List getPendingDeletionSubDirs(long volumeId, long bucketId, - OmKeyInfo parentInfo, long numEntries) throws IOException { + public TableIterator> getPendingDeletionDirs( + String volume, String bucket) throws IOException { + // Either both volume & bucket should be null or none of them should be null. + if (!StringUtils.isBlank(volume) && StringUtils.isBlank(bucket) || + StringUtils.isBlank(volume) && !StringUtils.isBlank(bucket)) { + throw new IOException("One of volume : " + volume + ", bucket: " + bucket + " is empty. Either both should be " + + "empty or none of the arguments should be empty"); + } + return StringUtils.isBlank(volume) ? metadataManager.getDeletedDirTable().iterator() : + metadataManager.getDeletedDirTable().iterator(metadataManager.getBucketKeyPrefixFSO(volume, bucket)); + } + + @Override + public List getPendingDeletionSubDirs( + long volumeId, long bucketId, OmKeyInfo parentInfo, + CheckedExceptionOperation, Boolean, IOException> filter, + long numEntries) throws IOException { String seekDirInDB = metadataManager.getOzonePathKey(volumeId, bucketId, parentInfo.getObjectID(), ""); long countEntries = 0; Table dirTable = metadataManager.getDirectoryTable(); - try (TableIterator> + try (TableIterator> iterator = dirTable.iterator()) { return gatherSubDirsWithIterator(parentInfo, numEntries, - seekDirInDB, countEntries, iterator); + seekDirInDB, filter, countEntries, iterator); } - } private List gatherSubDirsWithIterator(OmKeyInfo parentInfo, long numEntries, String seekDirInDB, - long countEntries, - TableIterator> iterator) + CheckedExceptionOperation, Boolean, IOException> filter, + long countEntries, TableIterator> iterator) throws IOException { List directories = new ArrayList<>(); iterator.seek(seekDirInDB); @@ -2071,34 +2119,34 @@ private List gatherSubDirsWithIterator(OmKeyInfo parentInfo, if (!metadataManager.getDirectoryTable().isExist(entry.getKey())) { continue; } - String dirName = OMFileRequest.getAbsolutePath(parentInfo.getKeyName(), - dirInfo.getName()); - OmKeyInfo omKeyInfo = OMFileRequest.getOmKeyInfo( - parentInfo.getVolumeName(), parentInfo.getBucketName(), dirInfo, - dirName); - directories.add(omKeyInfo); - countEntries++; + OmKeyInfo omKeyInfo = OMFileRequest.getOmKeyInfo(parentInfo, dirInfo); + if (filter.apply(Table.newKeyValue(metadataManager.getOzoneDeletePathKey(omKeyInfo.getObjectID(), entry.getKey()), + omKeyInfo))) { + directories.add(omKeyInfo); + countEntries++; + } + } return directories; } @Override - public List getPendingDeletionSubFiles(long volumeId, - long bucketId, OmKeyInfo parentInfo, long numEntries) - throws IOException { + public List getPendingDeletionSubFiles( + long volumeId, long bucketId, OmKeyInfo parentInfo, + CheckedExceptionOperation, Boolean, IOException> filter, long numEntries) + throws IOException { List files = new ArrayList<>(); String seekFileInDB = metadataManager.getOzonePathKey(volumeId, bucketId, parentInfo.getObjectID(), ""); - long countEntries = 0; - Table fileTable = metadataManager.getFileTable(); + Table fileTable = metadataManager.getFileTable(); try (TableIterator> iterator = fileTable.iterator()) { iterator.seek(seekFileInDB); - while (iterator.hasNext() && numEntries - countEntries > 0) { + while (iterator.hasNext() && numEntries > 0) { Table.KeyValue entry = iterator.next(); OmKeyInfo fileInfo = entry.getValue(); if (!OMFileRequest.isImmediateChild(fileInfo.getParentObjectID(), @@ -2108,13 +2156,12 @@ public List getPendingDeletionSubFiles(long volumeId, if (!metadataManager.getFileTable().isExist(entry.getKey())) { continue; } - fileInfo.setFileName(fileInfo.getKeyName()); - String fullKeyPath = OMFileRequest.getAbsolutePath( - parentInfo.getKeyName(), fileInfo.getKeyName()); - fileInfo.setKeyName(fullKeyPath); - - files.add(fileInfo); - countEntries++; + OMFileRequest.setKeyNameAndFileName(parentInfo, fileInfo); + if (filter.apply(Table.newKeyValue(metadataManager.getOzoneDeletePathKey(fileInfo.getObjectID(), + entry.getKey()), fileInfo))) { + files.add(fileInfo); + numEntries--; + } } } 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 4873a7db4916..6dcab308bfdc 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 @@ -87,7 +87,6 @@ import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolClientSideTranslatorPB; import org.apache.hadoop.ozone.om.request.file.OMFileRequest; import org.apache.hadoop.ozone.om.request.util.OMMultipartUploadUtils; -import org.apache.hadoop.ozone.om.snapshot.ReferenceCounted; import org.apache.hadoop.ozone.om.snapshot.SnapshotUtils; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ExpiredMultipartUploadInfo; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ExpiredMultipartUploadsBucket; @@ -114,7 +113,6 @@ import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.FILE_NOT_FOUND; import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_NOT_FOUND; import static org.apache.hadoop.ozone.OzoneConsts.OM_SNAPSHOT_CHECKPOINT_DIR; -import static org.apache.hadoop.ozone.om.service.SnapshotDeletingService.isBlockLocationInfoSame; import static org.apache.hadoop.ozone.om.snapshot.SnapshotUtils.checkSnapshotDirExist; import org.apache.hadoop.util.Time; @@ -1597,159 +1595,6 @@ private PersistedUserVolumeInfo getVolumesByUser(String userNameKey) } } - /** - * Returns a list of pending deletion key info up to the limit. - * Each entry is a {@link BlockGroup}, which contains the info about the key - * name and all its associated block IDs. - * - * @param keyCount max number of keys to return. - * @param omSnapshotManager SnapshotManager - * @return a list of {@link BlockGroup} represent keys and blocks. - * @throws IOException - */ - public PendingKeysDeletion getPendingDeletionKeys(final int keyCount, - OmSnapshotManager omSnapshotManager) - throws IOException { - List keyBlocksList = Lists.newArrayList(); - HashMap keysToModify = new HashMap<>(); - try (TableIterator> - keyIter = getDeletedTable().iterator()) { - int currentCount = 0; - while (keyIter.hasNext() && currentCount < keyCount) { - RepeatedOmKeyInfo notReclaimableKeyInfo = new RepeatedOmKeyInfo(); - KeyValue kv = keyIter.next(); - if (kv != null) { - List blockGroupList = Lists.newArrayList(); - // Get volume name and bucket name - String[] keySplit = kv.getKey().split(OM_KEY_PREFIX); - String bucketKey = getBucketKey(keySplit[1], keySplit[2]); - OmBucketInfo bucketInfo = getBucketTable().get(bucketKey); - // If Bucket deleted bucketInfo would be null, thus making previous snapshot also null. - SnapshotInfo previousSnapshotInfo = bucketInfo == null ? null : - SnapshotUtils.getLatestSnapshotInfo(bucketInfo.getVolumeName(), - bucketInfo.getBucketName(), ozoneManager, snapshotChainManager); - // previous snapshot is not active or it has not been flushed to disk then don't process the key in this - // iteration. - if (previousSnapshotInfo != null && - (previousSnapshotInfo.getSnapshotStatus() != SnapshotInfo.SnapshotStatus.SNAPSHOT_ACTIVE || - !OmSnapshotManager.areSnapshotChangesFlushedToDB(ozoneManager.getMetadataManager(), - previousSnapshotInfo))) { - continue; - } - // Get the latest snapshot in snapshot path. - try (ReferenceCounted rcLatestSnapshot = previousSnapshotInfo == null ? null : - omSnapshotManager.getSnapshot(previousSnapshotInfo.getVolumeName(), - previousSnapshotInfo.getBucketName(), previousSnapshotInfo.getName())) { - - // Multiple keys with the same path can be queued in one DB entry - RepeatedOmKeyInfo infoList = kv.getValue(); - for (OmKeyInfo info : infoList.cloneOmKeyInfoList()) { - // Skip the key if it exists in the previous snapshot (of the same - // scope) as in this case its blocks should not be reclaimed - - // If the last snapshot is deleted and the keys renamed in between - // the snapshots will be cleaned up by KDS. So we need to check - // in the renamedTable as well. - String dbRenameKey = getRenameKey(info.getVolumeName(), - info.getBucketName(), info.getObjectID()); - - if (rcLatestSnapshot != null) { - Table prevKeyTable = - rcLatestSnapshot.get() - .getMetadataManager() - .getKeyTable(bucketInfo.getBucketLayout()); - - Table prevDeletedTable = - rcLatestSnapshot.get().getMetadataManager().getDeletedTable(); - String prevKeyTableDBKey = getSnapshotRenamedTable() - .get(dbRenameKey); - String prevDelTableDBKey = getOzoneKey(info.getVolumeName(), - info.getBucketName(), info.getKeyName()); - // format: /volName/bucketName/keyName/objId - prevDelTableDBKey = getOzoneDeletePathKey(info.getObjectID(), - prevDelTableDBKey); - - if (prevKeyTableDBKey == null && - bucketInfo.getBucketLayout().isFileSystemOptimized()) { - long volumeId = getVolumeId(info.getVolumeName()); - prevKeyTableDBKey = getOzonePathKey(volumeId, - bucketInfo.getObjectID(), - info.getParentObjectID(), - info.getFileName()); - } else if (prevKeyTableDBKey == null) { - prevKeyTableDBKey = getOzoneKey(info.getVolumeName(), - info.getBucketName(), - info.getKeyName()); - } - - OmKeyInfo omKeyInfo = prevKeyTable.get(prevKeyTableDBKey); - // When key is deleted it is no longer in keyTable, we also - // have to check deletedTable of previous snapshot - RepeatedOmKeyInfo delOmKeyInfo = - prevDeletedTable.get(prevDelTableDBKey); - if (versionExistsInPreviousSnapshot(omKeyInfo, - info, delOmKeyInfo)) { - // If the infoList size is 1, there is nothing to split. - // We either delete it or skip it. - if (!(infoList.getOmKeyInfoList().size() == 1)) { - notReclaimableKeyInfo.addOmKeyInfo(info); - } - continue; - } - } - - // Add all blocks from all versions of the key to the deletion - // list - for (OmKeyLocationInfoGroup keyLocations : - info.getKeyLocationVersions()) { - List item = keyLocations.getLocationList().stream() - .map(b -> new BlockID(b.getContainerID(), b.getLocalID())) - .collect(Collectors.toList()); - BlockGroup keyBlocks = BlockGroup.newBuilder() - .setKeyName(kv.getKey()) - .addAllBlockIDs(item) - .build(); - blockGroupList.add(keyBlocks); - } - currentCount++; - } - - List notReclaimableKeyInfoList = - notReclaimableKeyInfo.getOmKeyInfoList(); - // If Bucket deleted bucketInfo would be null, thus making previous snapshot also null. - SnapshotInfo newPreviousSnapshotInfo = bucketInfo == null ? null : - SnapshotUtils.getLatestSnapshotInfo(bucketInfo.getVolumeName(), - bucketInfo.getBucketName(), ozoneManager, snapshotChainManager); - // Check if the previous snapshot in the chain hasn't changed. - if (Objects.equals(Optional.ofNullable(newPreviousSnapshotInfo).map(SnapshotInfo::getSnapshotId), - Optional.ofNullable(previousSnapshotInfo).map(SnapshotInfo::getSnapshotId))) { - // If all the versions are not reclaimable, then do nothing. - if (notReclaimableKeyInfoList.size() > 0 && - notReclaimableKeyInfoList.size() != - infoList.getOmKeyInfoList().size()) { - keysToModify.put(kv.getKey(), notReclaimableKeyInfo); - } - - if (notReclaimableKeyInfoList.size() != - infoList.getOmKeyInfoList().size()) { - keyBlocksList.addAll(blockGroupList); - } - } - } - } - } - } - return new PendingKeysDeletion(keyBlocksList, keysToModify); - } - - private boolean versionExistsInPreviousSnapshot(OmKeyInfo omKeyInfo, - OmKeyInfo info, RepeatedOmKeyInfo delOmKeyInfo) { - return (omKeyInfo != null && - info.getObjectID() == omKeyInfo.getObjectID() && - isBlockLocationInfoSame(omKeyInfo, info)) || - delOmKeyInfo != null; - } - /** * Decide whether the open key is a multipart upload related key. * @param openKeyInfo open key related to multipart upload @@ -2143,6 +1988,13 @@ public String getRenameKey(String volumeName, String bucketName, renameKey.append(OM_KEY_PREFIX).append(objectID); return renameKey.toString(); } + + @Override + public String[] splitRenameKey(String renameKey) { + String[] splitVals = renameKey.split(OM_KEY_PREFIX); + return new String[]{splitVals[1], splitVals[2], splitVals[3]}; + } + @Override public String getMultipartKey(long volumeId, long bucketId, long parentID, String fileName, diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/PendingKeysDeletion.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/PendingKeysDeletion.java index f2d73aaf411b..9e4acd1d7466 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/PendingKeysDeletion.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/PendingKeysDeletion.java @@ -21,28 +21,36 @@ import org.apache.hadoop.ozone.common.BlockGroup; import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo; -import java.util.HashMap; import java.util.List; +import java.util.Map; /** * Return class for OMMetadataManager#getPendingDeletionKeys. */ public class PendingKeysDeletion { - private HashMap keysToModify; + private Map keysToModify; private List keyBlocksList; + private String nextPageStartKey; + public PendingKeysDeletion(List keyBlocksList, - HashMap keysToModify) { + Map keysToModify, + String nextPageStartKey) { this.keysToModify = keysToModify; this.keyBlocksList = keyBlocksList; + this.nextPageStartKey = nextPageStartKey; } - public HashMap getKeysToModify() { + public Map getKeysToModify() { return keysToModify; } public List getKeyBlocksList() { return keyBlocksList; } + + public String getNextPageStartKey() { + return nextPageStartKey; + } } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/lock/MultiLocks.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/lock/MultiLocks.java new file mode 100644 index 000000000000..646a827439d7 --- /dev/null +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/lock/MultiLocks.java @@ -0,0 +1,54 @@ +package org.apache.hadoop.ozone.om.lock; + +import org.apache.hadoop.ozone.om.exceptions.OMException; + +import java.util.Collection; +import java.util.LinkedList; +import java.util.Queue; + +/** + * Class to take multiple locks on a resource. + */ +public class MultiLocks { + private final Queue objectLocks; + private final IOzoneManagerLock lock; + private final OzoneManagerLock.Resource resource; + private final boolean writeLock; + + public MultiLocks(IOzoneManagerLock lock, OzoneManagerLock.Resource resource, boolean writeLock) { + this.writeLock = writeLock; + this.resource = resource; + this.lock = lock; + this.objectLocks = new LinkedList<>(); + } + + public OMLockDetails acquireLock(Collection objects) throws OMException { + if (!objectLocks.isEmpty()) { + throw new OMException("More locks cannot be acquired when locks have been already acquired. Locks acquired : " + + objectLocks, OMException.ResultCodes.INTERNAL_ERROR); + } + OMLockDetails omLockDetails = OMLockDetails.EMPTY_DETAILS_LOCK_ACQUIRED; + for (T object : objects) { + if (object != null) { + omLockDetails = this.writeLock ? lock.acquireWriteLock(resource, object.toString()) + : lock.acquireReadLock(resource, object.toString()); + objectLocks.add(object); + if (!omLockDetails.isLockAcquired()) { + break; + } + } + } + if (!omLockDetails.isLockAcquired()) { + releaseLock(); + } + return omLockDetails; + } + + public void releaseLock() { + while (!objectLocks.isEmpty()) { + T object = objectLocks.poll(); + OMLockDetails lockDetails = this.writeLock ? lock.releaseWriteLock(resource, object.toString()) + : lock.releaseReadLock(resource, object.toString()); + } + } +} diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileRequest.java index 8f2a768c5257..ac7819962c40 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileRequest.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileRequest.java @@ -724,6 +724,14 @@ public static OzoneFileStatus getOMKeyInfoIfExists( return null; } + public static OmKeyInfo getOmKeyInfo(OmKeyInfo parentInfo, OmDirectoryInfo directoryInfo) { + String dirName = OMFileRequest.getAbsolutePath(parentInfo.getKeyName(), + directoryInfo.getName()); + return OMFileRequest.getOmKeyInfo( + parentInfo.getVolumeName(), parentInfo.getBucketName(), directoryInfo, + dirName); + } + /** * Prepare OmKeyInfo from OmDirectoryInfo. * @@ -773,6 +781,13 @@ public static String getAbsolutePath(String prefixName, String fileName) { return prefixName.concat(fileName); } + public static void setKeyNameAndFileName(OmKeyInfo parentInfo, OmKeyInfo omKeyInfo) { + omKeyInfo.setFileName(omKeyInfo.getKeyName()); + String fullKeyPath = OMFileRequest.getAbsolutePath( + parentInfo.getKeyName(), omKeyInfo.getKeyName()); + omKeyInfo.setKeyName(fullKeyPath); + } + /** * Build DirectoryInfo from OmKeyInfo. * diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyPurgeRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyPurgeRequest.java index a5e8cb145255..a01d01965b5a 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyPurgeRequest.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyPurgeRequest.java @@ -21,11 +21,11 @@ import java.io.IOException; import java.util.ArrayList; -import org.apache.hadoop.hdds.utils.TransactionInfo; import org.apache.hadoop.hdds.utils.db.cache.CacheKey; import org.apache.hadoop.hdds.utils.db.cache.CacheValue; import org.apache.hadoop.ozone.om.OmMetadataManagerImpl; import org.apache.hadoop.ozone.om.exceptions.OMException; +import org.apache.hadoop.hdds.utils.TransactionInfo; import org.apache.hadoop.ozone.om.snapshot.SnapshotUtils; import org.apache.ratis.server.protocol.TermIndex; import org.apache.hadoop.ozone.om.OzoneManager; @@ -70,8 +70,6 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, TermIn OMResponse.Builder omResponse = OmResponseUtil.getOMResponseBuilder( getOmRequest()); - - final SnapshotInfo fromSnapshotInfo; try { fromSnapshotInfo = fromSnapshot != null ? SnapshotUtils.getSnapshotInfo(ozoneManager, @@ -93,11 +91,12 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, TermIn List keysToBePurgedList = new ArrayList<>(); + List renamedKeysToBePurged = new ArrayList<>(purgeKeysRequest.getRenamedKeysList()); for (DeletedKeys bucketWithDeleteKeys : bucketDeletedKeysList) { keysToBePurgedList.addAll(bucketWithDeleteKeys.getKeysList()); } - if (keysToBePurgedList.isEmpty()) { + if (keysToBePurgedList.isEmpty() && renamedKeysToBePurged.isEmpty()) { return new OMKeyPurgeResponse(createErrorOMResponse(omResponse, new OMException("None of the keys can be purged be purged since a new snapshot was created for all the " + "buckets, making this request invalid", OMException.ResultCodes.KEY_DELETION_ERROR))); @@ -116,7 +115,7 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, TermIn } return new OMKeyPurgeResponse(omResponse.build(), - keysToBePurgedList, fromSnapshotInfo, keysToUpdateList); + keysToBePurgedList, renamedKeysToBePurged, fromSnapshotInfo, keysToUpdateList); } } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotCreateRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotCreateRequest.java index 2ded4f6a83e9..138046c27a94 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotCreateRequest.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotCreateRequest.java @@ -21,6 +21,7 @@ import org.apache.hadoop.hdds.client.DefaultReplicationConfig; import org.apache.hadoop.hdds.client.ReplicationConfig; import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.ozone.om.snapshot.SnapshotUtils; import org.apache.hadoop.hdds.utils.TransactionInfo; import org.apache.ratis.server.protocol.TermIndex; import org.apache.hadoop.hdds.utils.db.RDBStore; @@ -58,11 +59,13 @@ import java.io.IOException; import java.nio.file.InvalidPathException; +import java.util.Optional; import java.util.UUID; import static org.apache.hadoop.hdds.HddsUtils.fromProtobuf; import static org.apache.hadoop.hdds.HddsUtils.toProtobuf; import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.FILE_ALREADY_EXISTS; +import static org.apache.hadoop.ozone.om.helpers.SnapshotInfo.SnapshotStatus.SNAPSHOT_ACTIVE; import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK; import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.SNAPSHOT_LOCK; import static org.apache.hadoop.ozone.om.upgrade.OMLayoutFeature.FILESYSTEM_SNAPSHOT; @@ -183,8 +186,8 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, TermIn // TODO: [SNAPSHOT] Assign actual data size once we have the // pre-replicated key size counter in OmBucketInfo. snapshotInfo.setReferencedSize(estimateBucketDataSize(omBucketInfo)); - - addSnapshotInfoToSnapshotChainAndCache(omMetadataManager, termIndex.getIndex()); + addSnapshotInfoToSnapshotChainAndCache(ozoneManager, omMetadataManager, + termIndex.getIndex()); omResponse.setCreateSnapshotResponse( CreateSnapshotResponse.newBuilder() @@ -246,6 +249,7 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, TermIn * it was removed at T-5. */ private void addSnapshotInfoToSnapshotChainAndCache( + OzoneManager ozoneManager, OmMetadataManagerImpl omMetadataManager, long transactionLogIndex ) throws IOException { @@ -265,6 +269,21 @@ private void addSnapshotInfoToSnapshotChainAndCache( snapshotInfo.setPathPreviousSnapshotId(latestPathSnapshot); snapshotInfo.setGlobalPreviousSnapshotId(latestGlobalSnapshot); + Optional previousSnapshot = Optional.ofNullable(SnapshotUtils.getLatestSnapshotInfo( + snapshotInfo.getVolumeName(), snapshotInfo.getBucketName(), ozoneManager, snapshotChainManager)); + Optional previousPrevSnapshot = previousSnapshot.isPresent() ? + Optional.ofNullable(SnapshotUtils.getPreviousSnapshot(ozoneManager, + snapshotChainManager, previousSnapshot.get())) : Optional.empty(); + + // Reset the deep clean flag for the next active snapshot if and only if the last 2 snapshots in the + // chain are active, otherwise set it to prevent deep cleaning from running till the deleted snapshots don't + // get purged. + boolean deepCleanFlag = + !(previousSnapshot.map(SnapshotInfo::getSnapshotStatus).orElse(SNAPSHOT_ACTIVE) == SNAPSHOT_ACTIVE && + previousPrevSnapshot.map(SnapshotInfo::getSnapshotStatus).orElse(SNAPSHOT_ACTIVE) == SNAPSHOT_ACTIVE); + snapshotInfo.setDeepCleanedDeletedDir(deepCleanFlag); + snapshotInfo.setDeepClean(deepCleanFlag); + try { snapshotChainManager.addSnapshot(snapshotInfo); omMetadataManager.getSnapshotInfoTable() diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotMoveDeletedKeysRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotMoveDeletedKeysRequest.java index 18055bdda40c..7cc23f8e1afd 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotMoveDeletedKeysRequest.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotMoveDeletedKeysRequest.java @@ -20,9 +20,9 @@ package org.apache.hadoop.ozone.om.request.snapshot; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; -import org.apache.hadoop.hdds.utils.TransactionInfo; import org.apache.hadoop.hdds.utils.db.cache.CacheKey; import org.apache.hadoop.hdds.utils.db.cache.CacheValue; +import org.apache.hadoop.hdds.utils.TransactionInfo; import org.apache.ratis.server.protocol.TermIndex; import org.apache.hadoop.ozone.om.OmMetadataManagerImpl; import org.apache.hadoop.ozone.om.OzoneManager; @@ -91,16 +91,16 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, TermIn List movedDirs = moveDeletedKeysRequest.getDeletedDirsToMoveList(); // Update lastTransactionInfo for fromSnapshot and the nextSnapshot. - fromSnapshot.setLastTransactionInfo(TransactionInfo.valueOf(termIndex).toByteString()); + snapshotInfo.setLastTransactionInfo(TransactionInfo.valueOf(termIndex).toByteString()); omMetadataManager.getSnapshotInfoTable().addCacheEntry(new CacheKey<>(fromSnapshot.getTableKey()), - CacheValue.get(termIndex.getIndex(), fromSnapshot)); + CacheValue.get(termIndex.getIndex(), snapshotInfo)); if (nextSnapshot != null) { nextSnapshot.setLastTransactionInfo(TransactionInfo.valueOf(termIndex).toByteString()); omMetadataManager.getSnapshotInfoTable().addCacheEntry(new CacheKey<>(nextSnapshot.getTableKey()), CacheValue.get(termIndex.getIndex(), nextSnapshot)); } omClientResponse = new OMSnapshotMoveDeletedKeysResponse( - omResponse.build(), fromSnapshot, nextSnapshot, + omResponse.build(), snapshotInfo, nextSnapshot, nextDBKeysList, reclaimKeysList, renamedKeysList, movedDirs); } catch (IOException ex) { diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotPurgeRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotPurgeRequest.java index 38c51d4de5c0..867420d1c68e 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotPurgeRequest.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotPurgeRequest.java @@ -26,6 +26,7 @@ import org.apache.hadoop.hdds.utils.db.cache.CacheKey; import org.apache.hadoop.hdds.utils.db.cache.CacheValue; import org.apache.hadoop.ozone.om.OmMetadataManagerImpl; +import org.apache.hadoop.ozone.om.OmSnapshotManager; import org.apache.hadoop.ozone.om.OzoneManager; import org.apache.hadoop.ozone.om.SnapshotChainManager; import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; @@ -47,6 +48,8 @@ import java.util.NoSuchElementException; import java.util.UUID; +import static org.apache.hadoop.ozone.om.helpers.SnapshotInfo.SnapshotStatus.SNAPSHOT_ACTIVE; + /** * Handles OMSnapshotPurge Request. * This is an OM internal request. Does not need @RequireSnapshotFeatureState. @@ -76,6 +79,7 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, TermIn ozoneManager.getMetadataManager(); SnapshotChainManager snapshotChainManager = omMetadataManager.getSnapshotChainManager(); + OmSnapshotManager omSnapshotManager = ozoneManager.getOmSnapshotManager(); OMClientResponse omClientResponse = null; @@ -104,9 +108,19 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, TermIn continue; } SnapshotInfo nextSnapshot = SnapshotUtils.getNextSnapshot(ozoneManager, snapshotChainManager, fromSnapshot); + SnapshotInfo nextToNextSnapshot = nextSnapshot == null ? null + : SnapshotUtils.getNextSnapshot(ozoneManager, snapshotChainManager, nextSnapshot); + SnapshotInfo previousSnapshot = SnapshotUtils.getPreviousSnapshot(ozoneManager, + snapshotChainManager, fromSnapshot); + SnapshotInfo previousPrevSnapshot = previousSnapshot == null ? null + : SnapshotUtils.getPreviousSnapshot(ozoneManager, snapshotChainManager, previousSnapshot); + // Step 1: Reset the deep clean flag for the next active snapshot if and only if the last 2 snapshots in the + // chain are active, otherwise set it to prevent deep cleaning from running till the deleted snapshots don't + // get purged. + updateNextSnapshotInfoFields(nextSnapshot, nextToNextSnapshot, previousSnapshot, previousPrevSnapshot, + omMetadataManager, trxnLogIndex); + - // Step 1: Update the deep clean flag for the next active snapshot - updateSnapshotInfoAndCache(nextSnapshot, omMetadataManager, trxnLogIndex); // Step 2: Update the snapshot chain. updateSnapshotChainAndCache(omMetadataManager, fromSnapshot, trxnLogIndex); // Step 3: Purge the snapshot from SnapshotInfoTable cache and also remove from the map. @@ -136,19 +150,42 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, TermIn return omClientResponse; } - private void updateSnapshotInfoAndCache(SnapshotInfo snapInfo, OmMetadataManagerImpl omMetadataManager, - long trxnLogIndex) throws IOException { - if (snapInfo != null) { - // Setting next snapshot deep clean to false, Since the - // current snapshot is deleted. We can potentially - // reclaim more keys in the next snapshot. - snapInfo.setDeepClean(false); + private void updateNextSnapshotInfoFields(SnapshotInfo nextSnapshot, + SnapshotInfo nextToNextSnapshot, + SnapshotInfo previousSnapshot, + SnapshotInfo previousToPreviousSnapshot, + OmMetadataManagerImpl omMetadataManager, + long trxnLogIndex) throws IOException { + if (nextSnapshot != null) { + // Reset the deep clean flag for the next active snapshot if and only if the last 2 snapshots in the + // chain are active, otherwise set it to prevent deep cleaning from running till the deleted snapshots don't + // get purged. There could be potentially more keys to be reclaimed. + boolean deepCleanFlagValue = (previousSnapshot == null || previousSnapshot.getSnapshotStatus() == SNAPSHOT_ACTIVE) + && (previousToPreviousSnapshot == null || previousToPreviousSnapshot.getSnapshotStatus() == SNAPSHOT_ACTIVE); + nextSnapshot.setDeepClean(deepCleanFlagValue); + nextSnapshot.setDeepCleanedDeletedDir(deepCleanFlagValue); // Update table cache first - omMetadataManager.getSnapshotInfoTable().addCacheEntry(new CacheKey<>(snapInfo.getTableKey()), - CacheValue.get(trxnLogIndex, snapInfo)); - updatedSnapshotInfos.put(snapInfo.getTableKey(), snapInfo); + omMetadataManager.getSnapshotInfoTable().addCacheEntry(new CacheKey<>(nextSnapshot.getTableKey()), + CacheValue.get(trxnLogIndex, nextSnapshot)); + updatedSnapshotInfos.put(nextSnapshot.getTableKey(), nextSnapshot); } + + if (nextToNextSnapshot != null) { + // Reset the deep clean flag for the next active snapshot if and only if the last 2 snapshots in the + // chain are active, otherwise set it to prevent deep cleaning from running till the deleted snapshots don't + // get purged. + boolean deepCleanFlagValue = (previousSnapshot == null || previousSnapshot.getSnapshotStatus() == SNAPSHOT_ACTIVE) + && (nextSnapshot == null || nextSnapshot.getSnapshotStatus() == SNAPSHOT_ACTIVE); + nextToNextSnapshot.setDeepClean(deepCleanFlagValue); + nextToNextSnapshot.setDeepCleanedDeletedDir(deepCleanFlagValue); + + // Update table cache first + omMetadataManager.getSnapshotInfoTable().addCacheEntry(new CacheKey<>(nextToNextSnapshot.getTableKey()), + CacheValue.get(trxnLogIndex, nextSnapshot)); + updatedSnapshotInfos.put(nextToNextSnapshot.getTableKey(), nextSnapshot); + } + } /** 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 index 53047fd8026b..0e93a1b6229e 100644 --- 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 @@ -17,6 +17,7 @@ */ package org.apache.hadoop.ozone.om.request.snapshot; +import org.apache.commons.compress.utils.Lists; import org.apache.hadoop.ozone.om.OMMetrics; import org.apache.ratis.server.protocol.TermIndex; import org.apache.hadoop.hdds.utils.db.cache.CacheKey; @@ -36,8 +37,15 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.FILE_NOT_FOUND; +import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.INVALID_REQUEST; /** * Updates the exclusive size of the snapshot. @@ -50,6 +58,32 @@ public OMSnapshotSetPropertyRequest(OMRequest omRequest) { super(omRequest); } + private void updateSnapshotProperty( + SnapshotInfo snapInfo, OzoneManagerProtocolProtos.SetSnapshotPropertyRequest setSnapshotPropertyRequest) { + if (setSnapshotPropertyRequest.hasDeepCleanedDeletedDir()) { + snapInfo.setDeepCleanedDeletedDir(setSnapshotPropertyRequest + .getDeepCleanedDeletedDir()); + } + + if (setSnapshotPropertyRequest.hasDeepCleanedDeletedKey()) { + snapInfo.setDeepClean(setSnapshotPropertyRequest + .getDeepCleanedDeletedKey()); + } + + if (setSnapshotPropertyRequest.hasSnapshotSize()) { + SnapshotSize snapshotSize = setSnapshotPropertyRequest.getSnapshotSize(); + // Set Exclusive size. + snapInfo.setExclusiveSize(snapshotSize.getExclusiveSize()); + snapInfo.setExclusiveReplicatedSize(snapshotSize.getExclusiveReplicatedSize()); + } + if (setSnapshotPropertyRequest.hasSnapshotDirSize()) { + SnapshotSize snapshotSize = setSnapshotPropertyRequest.getSnapshotDirSize(); + // Set Exclusive size. + snapInfo.setExclusiveSize(snapshotSize.getExclusiveSize()); + snapInfo.setExclusiveReplicatedSize(snapshotSize.getExclusiveReplicatedSize()); + } + } + @Override public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, TermIndex termIndex) { OMMetrics omMetrics = ozoneManager.getMetrics(); @@ -59,55 +93,57 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, TermIn OzoneManagerProtocolProtos.OMResponse.Builder omResponse = OmResponseUtil.getOMResponseBuilder(getOmRequest()); - OzoneManagerProtocolProtos.SetSnapshotPropertyRequest - setSnapshotPropertyRequest = getOmRequest() - .getSetSnapshotPropertyRequest(); - - String snapshotKey = setSnapshotPropertyRequest.getSnapshotKey(); - + List setSnapshotPropertyRequests = Lists.newArrayList(); + if (getOmRequest().hasSetSnapshotPropertyRequest()) { + setSnapshotPropertyRequests.add(getOmRequest().getSetSnapshotPropertyRequest()); + } + setSnapshotPropertyRequests.addAll(getOmRequest().getSetSnapshotPropertyRequestsList()); + Set snapshotKeys = new HashSet<>(); + Map snapshotInfoMap = new HashMap<>(); try { - SnapshotInfo updatedSnapInfo = metadataManager.getSnapshotInfoTable().get(snapshotKey); - if (updatedSnapInfo == null) { - LOG.error("Snapshot: '{}' doesn't not exist in snapshot table.", snapshotKey); - throw new OMException("Snapshot: '{" + snapshotKey + "}' doesn't not exist in snapshot table.", FILE_NOT_FOUND); + for (OzoneManagerProtocolProtos.SetSnapshotPropertyRequest setSnapshotPropertyRequest : + setSnapshotPropertyRequests) { + String snapshotKey = setSnapshotPropertyRequest.getSnapshotKey(); + if (snapshotKeys.contains(snapshotKey)) { + throw new OMException("Snapshot with snapshot key: " + snapshotKey + " added multiple times in the request. " + + "Request: " + setSnapshotPropertyRequests, INVALID_REQUEST); + } + snapshotKeys.add(snapshotKey); + SnapshotInfo updatedSnapInfo = snapshotInfoMap.computeIfAbsent(snapshotKey, + (k) -> { + try { + return metadataManager.getSnapshotInfoTable().get(k); + } catch (IOException e) { + throw new UncheckedIOException("Exception while getting key " + k, e); + } + }); + if (updatedSnapInfo == null) { + LOG.error("Snapshot: '{}' doesn't not exist in snapshot table.", snapshotKey); + throw new OMException("Snapshot: '{}' doesn't not exist in snapshot table." + snapshotKey + + "Request: " + setSnapshotPropertyRequests, FILE_NOT_FOUND); + } + updateSnapshotProperty(updatedSnapInfo, setSnapshotPropertyRequest); } - - if (setSnapshotPropertyRequest.hasDeepCleanedDeletedDir()) { - updatedSnapInfo.setDeepCleanedDeletedDir(setSnapshotPropertyRequest - .getDeepCleanedDeletedDir()); + if (snapshotInfoMap.isEmpty()) { + throw new OMException("Snapshots: " + snapshotKeys + " don't not exist in snapshot table.", + FILE_NOT_FOUND); } - - if (setSnapshotPropertyRequest.hasDeepCleanedDeletedKey()) { - updatedSnapInfo.setDeepClean(setSnapshotPropertyRequest - .getDeepCleanedDeletedKey()); + // Update Table Cache + for (Map.Entry snapshot : snapshotInfoMap.entrySet()) { + metadataManager.getSnapshotInfoTable().addCacheEntry( + new CacheKey<>(snapshot.getKey()), + CacheValue.get(termIndex.getIndex(), snapshot.getValue())); + omMetrics.incNumSnapshotSetProperties(); } - if (setSnapshotPropertyRequest.hasSnapshotSize()) { - SnapshotSize snapshotSize = setSnapshotPropertyRequest - .getSnapshotSize(); - long exclusiveSize = updatedSnapInfo.getExclusiveSize() + - snapshotSize.getExclusiveSize(); - long exclusiveReplicatedSize = updatedSnapInfo - .getExclusiveReplicatedSize() + snapshotSize - .getExclusiveReplicatedSize(); - // Set Exclusive size. - updatedSnapInfo.setExclusiveSize(exclusiveSize); - updatedSnapInfo.setExclusiveReplicatedSize(exclusiveReplicatedSize); - } - // Update Table Cache - metadataManager.getSnapshotInfoTable().addCacheEntry( - new CacheKey<>(snapshotKey), - CacheValue.get(termIndex.getIndex(), updatedSnapInfo)); - omClientResponse = new OMSnapshotSetPropertyResponse( - omResponse.build(), updatedSnapInfo); - omMetrics.incNumSnapshotSetProperties(); - LOG.info("Successfully executed snapshotSetPropertyRequest: {{}}.", setSnapshotPropertyRequest); - } catch (IOException ex) { + omClientResponse = new OMSnapshotSetPropertyResponse(omResponse.build(), snapshotInfoMap.values()); + LOG.info("Successfully executed snapshotSetPropertyRequest: {{}}.", setSnapshotPropertyRequests); + } catch (UncheckedIOException | IOException ex) { omClientResponse = new OMSnapshotSetPropertyResponse( createErrorOMResponse(omResponse, ex)); omMetrics.incNumSnapshotSetPropertyFails(); - LOG.error("Failed to execute snapshotSetPropertyRequest: {{}}.", setSnapshotPropertyRequest, ex); + LOG.error("Failed to execute snapshotSetPropertyRequest: {{}}.", setSnapshotPropertyRequests, ex); } return omClientResponse; diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMDirectoriesPurgeResponseWithFSO.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMDirectoriesPurgeResponseWithFSO.java index 782063d32446..9ecdf9f25258 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMDirectoriesPurgeResponseWithFSO.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMDirectoriesPurgeResponseWithFSO.java @@ -98,13 +98,13 @@ public void addToDBBatch(OMMetadataManager metadataManager, // Init Batch Operation for snapshot db. try (BatchOperation writeBatch = fromSnapshotStore.initBatchOperation()) { - processPaths(fromSnapshot.getMetadataManager(), writeBatch); + processPaths(metadataManager, fromSnapshot.getMetadataManager(), batchOp, writeBatch); fromSnapshotStore.commitBatchOperation(writeBatch); } } metadataManager.getSnapshotInfoTable().putWithBatch(batchOp, fromSnapshotInfo.getTableKey(), fromSnapshotInfo); } else { - processPaths(metadataManager, batchOp); + processPaths(metadataManager, metadataManager, batchOp, batchOp); } // update bucket quota in active db @@ -115,8 +115,10 @@ public void addToDBBatch(OMMetadataManager metadataManager, } } - public void processPaths(OMMetadataManager omMetadataManager, - BatchOperation batchOperation) throws IOException { + public void processPaths(OMMetadataManager keySpaceOmMetadataManager, + OMMetadataManager deletedSpaceOmMetadataManager, + BatchOperation keySpaceBatchOperation, + BatchOperation deletedSpaceBatchOperation) throws IOException { for (OzoneManagerProtocolProtos.PurgePathRequest path : paths) { final long volumeId = path.getVolumeId(); final long bucketId = path.getBucketId(); @@ -129,14 +131,14 @@ public void processPaths(OMMetadataManager omMetadataManager, // Add all sub-directories to deleted directory table. for (OzoneManagerProtocolProtos.KeyInfo key : markDeletedSubDirsList) { OmKeyInfo keyInfo = OmKeyInfo.getFromProtobuf(key); - String ozoneDbKey = omMetadataManager.getOzonePathKey(volumeId, + String ozoneDbKey = keySpaceOmMetadataManager.getOzonePathKey(volumeId, bucketId, keyInfo.getParentObjectID(), keyInfo.getFileName()); - String ozoneDeleteKey = omMetadataManager.getOzoneDeletePathKey( + String ozoneDeleteKey = deletedSpaceOmMetadataManager.getOzoneDeletePathKey( key.getObjectID(), ozoneDbKey); - omMetadataManager.getDeletedDirTable().putWithBatch(batchOperation, + deletedSpaceOmMetadataManager.getDeletedDirTable().putWithBatch(deletedSpaceBatchOperation, ozoneDeleteKey, keyInfo); - omMetadataManager.getDirectoryTable().deleteWithBatch(batchOperation, + keySpaceOmMetadataManager.getDirectoryTable().deleteWithBatch(keySpaceBatchOperation, ozoneDbKey); if (LOG.isDebugEnabled()) { @@ -147,10 +149,10 @@ public void processPaths(OMMetadataManager omMetadataManager, for (OzoneManagerProtocolProtos.KeyInfo key : deletedSubFilesList) { OmKeyInfo keyInfo = OmKeyInfo.getFromProtobuf(key); - String ozoneDbKey = omMetadataManager.getOzonePathKey(volumeId, + String ozoneDbKey = keySpaceOmMetadataManager.getOzonePathKey(volumeId, bucketId, keyInfo.getParentObjectID(), keyInfo.getFileName()); - omMetadataManager.getKeyTable(getBucketLayout()) - .deleteWithBatch(batchOperation, ozoneDbKey); + keySpaceOmMetadataManager.getKeyTable(getBucketLayout()) + .deleteWithBatch(keySpaceBatchOperation, ozoneDbKey); if (LOG.isDebugEnabled()) { LOG.info("Move keyName:{} to DeletedTable DBKey: {}", @@ -160,26 +162,26 @@ public void processPaths(OMMetadataManager omMetadataManager, RepeatedOmKeyInfo repeatedOmKeyInfo = OmUtils.prepareKeyForDelete( keyInfo, keyInfo.getUpdateID(), isRatisEnabled); - String deletedKey = omMetadataManager + String deletedKey = keySpaceOmMetadataManager .getOzoneKey(keyInfo.getVolumeName(), keyInfo.getBucketName(), keyInfo.getKeyName()); - deletedKey = omMetadataManager.getOzoneDeletePathKey( + deletedKey = deletedSpaceOmMetadataManager.getOzoneDeletePathKey( keyInfo.getObjectID(), deletedKey); - omMetadataManager.getDeletedTable().putWithBatch(batchOperation, + deletedSpaceOmMetadataManager.getDeletedTable().putWithBatch(deletedSpaceBatchOperation, deletedKey, repeatedOmKeyInfo); } if (!openKeyInfoMap.isEmpty()) { for (Map.Entry entry : openKeyInfoMap.entrySet()) { - omMetadataManager.getOpenKeyTable(getBucketLayout()).putWithBatch( - batchOperation, entry.getKey(), entry.getValue()); + keySpaceOmMetadataManager.getOpenKeyTable(getBucketLayout()).putWithBatch( + keySpaceBatchOperation, entry.getKey(), entry.getValue()); } } // Delete the visited directory from deleted directory table if (path.hasDeletedDir()) { - omMetadataManager.getDeletedDirTable().deleteWithBatch(batchOperation, + deletedSpaceOmMetadataManager.getDeletedDirTable().deleteWithBatch(deletedSpaceBatchOperation, path.getDeletedDir()); if (LOG.isDebugEnabled()) { diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeyPurgeResponse.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeyPurgeResponse.java index cd2f7d190f45..4eb9785c28e1 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeyPurgeResponse.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMKeyPurgeResponse.java @@ -48,15 +48,18 @@ @CleanupTableInfo(cleanupTables = {DELETED_TABLE, SNAPSHOT_INFO_TABLE}) public class OMKeyPurgeResponse extends OmKeyResponse { private List purgeKeyList; + private List renamedList; private SnapshotInfo fromSnapshot; private List keysToUpdateList; public OMKeyPurgeResponse(@Nonnull OMResponse omResponse, @Nonnull List keyList, + @Nonnull List renamedList, SnapshotInfo fromSnapshot, List keysToUpdate) { super(omResponse); this.purgeKeyList = keyList; + this.renamedList = renamedList; this.fromSnapshot = fromSnapshot; this.keysToUpdateList = keysToUpdate; } @@ -106,8 +109,7 @@ private void processKeysToUpdate(BatchOperation batchOp, for (SnapshotMoveKeyInfos keyToUpdate : keysToUpdateList) { List keyInfosList = keyToUpdate.getKeyInfosList(); - RepeatedOmKeyInfo repeatedOmKeyInfo = - createRepeatedOmKeyInfo(keyInfosList); + RepeatedOmKeyInfo repeatedOmKeyInfo = createRepeatedOmKeyInfo(keyInfosList); metadataManager.getDeletedTable().putWithBatch(batchOp, keyToUpdate.getKey(), repeatedOmKeyInfo); } @@ -119,6 +121,10 @@ private void processKeys(BatchOperation batchOp, metadataManager.getDeletedTable().deleteWithBatch(batchOp, key); } + // Delete rename entries. + for (String key : renamedList) { + metadataManager.getSnapshotRenamedTable().deleteWithBatch(batchOp, key); + } } } 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 index c018de78cb88..7b0d4e8545b7 100644 --- 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 @@ -27,6 +27,7 @@ import jakarta.annotation.Nonnull; import java.io.IOException; +import java.util.Collection; import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.SNAPSHOT_INFO_TABLE; @@ -35,26 +36,29 @@ */ @CleanupTableInfo(cleanupTables = {SNAPSHOT_INFO_TABLE}) public class OMSnapshotSetPropertyResponse extends OMClientResponse { - private final SnapshotInfo updatedSnapInfo; + private final Collection updatedSnapInfos; public OMSnapshotSetPropertyResponse( @Nonnull OMResponse omResponse, - @Nonnull SnapshotInfo updatedSnapInfo) { + @Nonnull Collection updatedSnapInfos) { super(omResponse); - this.updatedSnapInfo = updatedSnapInfo; + this.updatedSnapInfos = updatedSnapInfos; } public OMSnapshotSetPropertyResponse(@Nonnull OMResponse omResponse) { super(omResponse); checkStatusNotOK(); - this.updatedSnapInfo = null; + this.updatedSnapInfos = null; } @Override protected void addToDBBatch(OMMetadataManager omMetadataManager, BatchOperation batchOperation) throws IOException { - omMetadataManager.getSnapshotInfoTable().putWithBatch(batchOperation, - updatedSnapInfo.getTableKey(), updatedSnapInfo); + for (SnapshotInfo updatedSnapInfo : updatedSnapInfos) { + omMetadataManager.getSnapshotInfoTable().putWithBatch(batchOperation, + updatedSnapInfo.getTableKey(), updatedSnapInfo); + } + } } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/AbstractKeyDeletingService.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/AbstractKeyDeletingService.java index 7559cf9a7291..1b7e427e504d 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/AbstractKeyDeletingService.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/AbstractKeyDeletingService.java @@ -20,7 +20,6 @@ import com.google.protobuf.ServiceException; import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.HddsUtils; -import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol; import org.apache.hadoop.hdds.utils.BackgroundService; import org.apache.hadoop.hdds.utils.db.BatchOperation; @@ -33,10 +32,8 @@ import org.apache.hadoop.ozone.om.KeyManager; import org.apache.hadoop.ozone.om.OMMetadataManager; import org.apache.hadoop.ozone.om.OzoneManager; -import org.apache.hadoop.ozone.om.helpers.OmBucketInfo; import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo; -import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeletedKeys; @@ -45,6 +42,7 @@ import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.PurgePathRequest; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveKeyInfos; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type; +import org.apache.hadoop.ozone.util.CheckedExceptionOperation; import org.apache.hadoop.util.Time; import org.apache.ratis.protocol.ClientId; import org.apache.ratis.util.Preconditions; @@ -54,14 +52,13 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.UUID; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; -import static org.apache.hadoop.ozone.OzoneConsts.OBJECT_ID_RECLAIM_BLOCKS; import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX; -import static org.apache.hadoop.ozone.om.service.SnapshotDeletingService.isBlockLocationInfoSame; /** * Abstracts common code from KeyDeletingService and DirectoryDeletingService @@ -93,13 +90,14 @@ public AbstractKeyDeletingService(String serviceName, long interval, this.runCount = new AtomicLong(0); } - protected int processKeyDeletes(List keyBlocksList, + protected Pair processKeyDeletes(List keyBlocksList, KeyManager manager, - HashMap keysToModify, + Map keysToModify, + List renameEntries, String snapTableKey, UUID expectedPreviousSnapshotId) throws IOException { long startTime = Time.monotonicNow(); - int delCount = 0; + Pair purgeResult = Pair.of(0, false); if (LOG.isDebugEnabled()) { LOG.debug("Send {} key(s) to SCM: {}", keyBlocksList.size(), keyBlocksList); @@ -118,18 +116,18 @@ protected int processKeyDeletes(List keyBlocksList, if (blockDeletionResults != null) { startTime = Time.monotonicNow(); if (isRatisEnabled()) { - delCount = submitPurgeKeysRequest(blockDeletionResults, - keysToModify, snapTableKey, expectedPreviousSnapshotId); + purgeResult = submitPurgeKeysRequest(blockDeletionResults, keysToModify, renameEntries, + snapTableKey, expectedPreviousSnapshotId); } else { // TODO: Once HA and non-HA paths are merged, we should have // only one code path here. Purge keys should go through an // OMRequest model. - delCount = deleteAllKeys(blockDeletionResults, manager); + purgeResult = deleteAllKeys(blockDeletionResults, manager); } LOG.info("Blocks for {} (out of {}) keys are deleted from DB in {} ms", - delCount, blockDeletionResults.size(), Time.monotonicNow() - startTime); + purgeResult, blockDeletionResults.size(), Time.monotonicNow() - startTime); } - return delCount; + return purgeResult; } /** @@ -138,12 +136,12 @@ protected int processKeyDeletes(List keyBlocksList, * @param results DeleteBlockGroups returned by SCM. * @throws IOException on Error */ - private int deleteAllKeys(List results, + private Pair deleteAllKeys(List results, KeyManager manager) throws IOException { Table deletedTable = manager.getMetadataManager().getDeletedTable(); DBStore store = manager.getMetadataManager().getStore(); - + boolean purgeSuccess = true; // Put all keys to delete in a single transaction and call for delete. int deletedCount = 0; try (BatchOperation writeBatch = store.initBatchOperation()) { @@ -156,12 +154,14 @@ private int deleteAllKeys(List results, LOG.debug("Key {} deleted from OM DB", result.getObjectKey()); } deletedCount++; + } else { + purgeSuccess = false; } } // Write a single transaction for delete. store.commitBatchOperation(writeBatch); } - return deletedCount; + return Pair.of(deletedCount, purgeSuccess); } /** @@ -170,13 +170,14 @@ private int deleteAllKeys(List results, * @param results DeleteBlockGroups returned by SCM. * @param keysToModify Updated list of RepeatedOmKeyInfo */ - private int submitPurgeKeysRequest(List results, - HashMap keysToModify, String snapTableKey, UUID expectedPreviousSnapshotId) { - Map, List> purgeKeysMapPerBucket = - new HashMap<>(); + private Pair submitPurgeKeysRequest(List results, + Map keysToModify, List renameEntriesToBeDeleted, + String snapTableKey, UUID expectedPreviousSnapshotId) { + Map, List> purgeKeysMapPerBucket = new HashMap<>(); // Put all keys to be purged in a list int deletedCount = 0; + boolean purgeSuccess = true; for (DeleteBlockGroupResult result : results) { if (result.isSuccess()) { // Add key to PurgeKeys list. @@ -195,6 +196,8 @@ private int submitPurgeKeysRequest(List results, } } deletedCount++; + } else { + purgeSuccess = false; } } @@ -210,8 +213,7 @@ private int submitPurgeKeysRequest(List results, purgeKeysRequest.setExpectedPreviousSnapshotID(expectedPreviousSnapshotNullableUUID.build()); // Add keys to PurgeKeysRequest bucket wise. - for (Map.Entry, List> entry : - purgeKeysMapPerBucket.entrySet()) { + for (Map.Entry, List> entry : purgeKeysMapPerBucket.entrySet()) { Pair volumeBucketPair = entry.getKey(); DeletedKeys deletedKeysInBucket = DeletedKeys.newBuilder() .setVolumeName(volumeBucketPair.getLeft()) @@ -220,6 +222,11 @@ private int submitPurgeKeysRequest(List results, .build(); purgeKeysRequest.addDeletedKeys(deletedKeysInBucket); } + // Adding rename entries to be purged. + if (renameEntriesToBeDeleted != null) { + purgeKeysRequest.addAllRenamedKeys(renameEntriesToBeDeleted); + } + List keysToUpdateList = new ArrayList<>(); if (keysToModify != null) { @@ -250,13 +257,17 @@ private int submitPurgeKeysRequest(List results, // Submit PurgeKeys request to OM try { - OzoneManagerRatisUtils.submitRequest(ozoneManager, omRequest, clientId, runCount.get()); + OzoneManagerProtocolProtos.OMResponse omResponse = OzoneManagerRatisUtils.submitRequest(ozoneManager, omRequest, + clientId, runCount.get()); + if (omResponse != null) { + purgeSuccess = purgeSuccess && omResponse.getSuccess(); + } } catch (ServiceException e) { LOG.error("PurgeKey request failed. Will retry at next run."); - return 0; + return Pair.of(0, false); } - return deletedCount; + return Pair.of(deletedCount, purgeSuccess); } /** @@ -278,9 +289,9 @@ private void addToMap(Map, List> map, String object map.get(volumeBucketPair).add(objectKey); } - protected void submitPurgePaths(List requests, - String snapTableKey, - UUID expectedPreviousSnapshotId) { + protected OzoneManagerProtocolProtos.OMResponse submitPurgePaths(List requests, + String snapTableKey, + UUID expectedPreviousSnapshotId) { OzoneManagerProtocolProtos.PurgeDirectoriesRequest.Builder purgeDirRequest = OzoneManagerProtocolProtos.PurgeDirectoriesRequest.newBuilder(); @@ -305,10 +316,11 @@ protected void submitPurgePaths(List requests, // Submit Purge paths request to OM try { - OzoneManagerRatisUtils.submitRequest(ozoneManager, omRequest, clientId, runCount.get()); + return OzoneManagerRatisUtils.submitRequest(ozoneManager, omRequest, clientId, runCount.get()); } catch (ServiceException e) { LOG.error("PurgePaths request failed. Will retry at next run."); } + return null; } private OzoneManagerProtocolProtos.PurgePathRequest wrapPurgeRequest( @@ -341,10 +353,12 @@ private OzoneManagerProtocolProtos.PurgePathRequest wrapPurgeRequest( return purgePathsRequest.build(); } - protected PurgePathRequest prepareDeleteDirRequest( + protected Optional prepareDeleteDirRequest( long remainNum, OmKeyInfo pendingDeletedDirInfo, String delDirName, List> subDirList, - KeyManager keyManager) throws IOException { + KeyManager keyManager, boolean deleteDir, + CheckedExceptionOperation, Boolean, IOException> fileDeletionChecker) + throws IOException { // step-0: Get one pending deleted directory if (LOG.isDebugEnabled()) { LOG.debug("Pending deleted dir name: {}", @@ -355,10 +369,11 @@ protected PurgePathRequest prepareDeleteDirRequest( final long volumeId = Long.parseLong(keys[1]); final long bucketId = Long.parseLong(keys[2]); - // step-1: get all sub directories under the deletedDir + // step-1: get all sub directories under the deletedDir. Always expand all sub directories irrespective of + // reference of sub-directory in previous snapshot. List subDirs = keyManager .getPendingDeletionSubDirs(volumeId, bucketId, - pendingDeletedDirInfo, remainNum); + pendingDeletedDirInfo, (keyInfo) -> true, remainNum); remainNum = remainNum - subDirs.size(); OMMetadataManager omMetadataManager = keyManager.getMetadataManager(); @@ -372,9 +387,13 @@ protected PurgePathRequest prepareDeleteDirRequest( } // step-2: get all sub files under the deletedDir - List subFiles = keyManager + // Only remove sub files if the parent directory is going to be deleted or can be reclaimed. + List subFiles = new ArrayList<>(); + for (OmKeyInfo omKeyInfo : keyManager .getPendingDeletionSubFiles(volumeId, bucketId, - pendingDeletedDirInfo, remainNum); + pendingDeletedDirInfo, (keyInfo) -> deleteDir || fileDeletionChecker.apply(keyInfo), remainNum)) { + subFiles.add(omKeyInfo); + } remainNum = remainNum - subFiles.size(); if (LOG.isDebugEnabled()) { @@ -388,18 +407,25 @@ protected PurgePathRequest prepareDeleteDirRequest( // limit. If count reached limit then there can be some more child // paths to be visited and will keep the parent deleted directory // for one more pass. - String purgeDeletedDir = remainNum > 0 ? delDirName : null; - return wrapPurgeRequest(volumeId, bucketId, - purgeDeletedDir, subFiles, subDirs); + // If there are no subpaths to expand and the directory itself cannot be reclaimed then skip purge processing for + // this dir, since this would be a noop. + String purgeDeletedDir = deleteDir && remainNum > 0 ? delDirName : null; + if (purgeDeletedDir == null && subFiles.isEmpty() && subDirs.isEmpty()) { + return Optional.empty(); + } + return Optional.of(wrapPurgeRequest(volumeId, bucketId, + purgeDeletedDir, subFiles, subDirs)); } @SuppressWarnings("checkstyle:ParameterNumber") - public long optimizeDirDeletesAndSubmitRequest(long remainNum, + public Pair> optimizeDirDeletesAndSubmitRequest(long remainNum, long dirNum, long subDirNum, long subFileNum, List> allSubDirList, List purgePathRequestList, String snapTableKey, long startTime, int remainingBufLimit, KeyManager keyManager, + CheckedExceptionOperation, Boolean, IOException> subDirPurgeChecker, + CheckedExceptionOperation, Boolean, IOException> fileDeletionChecker, UUID expectedPreviousSnapshotId) { // Optimization to handle delete sub-dir and keys to remove quickly @@ -409,30 +435,33 @@ public long optimizeDirDeletesAndSubmitRequest(long remainNum, int consumedSize = 0; while (remainNum > 0 && subDirRecursiveCnt < allSubDirList.size()) { try { - Pair stringOmKeyInfoPair - = allSubDirList.get(subDirRecursiveCnt); - PurgePathRequest request = prepareDeleteDirRequest( + Pair stringOmKeyInfoPair = allSubDirList.get(subDirRecursiveCnt); + Boolean result = subDirPurgeChecker.apply(Table.newKeyValue(stringOmKeyInfoPair.getKey(), + stringOmKeyInfoPair.getValue())); + Optional request = prepareDeleteDirRequest( remainNum, stringOmKeyInfoPair.getValue(), stringOmKeyInfoPair.getKey(), allSubDirList, - keyManager); + keyManager, result, fileDeletionChecker); + if (!request.isPresent()) { + continue; + } if (isBufferLimitCrossed(remainingBufLimit, consumedSize, - request.getSerializedSize())) { + request.get().getSerializedSize())) { // ignore further add request break; } - consumedSize += request.getSerializedSize(); - purgePathRequestList.add(request); - // reduce remain count for self, sub-files, and sub-directories - remainNum = remainNum - 1; - remainNum = remainNum - request.getDeletedSubFilesCount(); - remainNum = remainNum - request.getMarkDeletedSubDirsCount(); + PurgePathRequest requestVal = request.get(); + consumedSize += requestVal.getSerializedSize(); + purgePathRequestList.add(requestVal); + remainNum = remainNum - requestVal.getDeletedSubFilesCount(); + remainNum = remainNum - requestVal.getMarkDeletedSubDirsCount(); // Count up the purgeDeletedDir, subDirs and subFiles - if (request.getDeletedDir() != null - && !request.getDeletedDir().isEmpty()) { + if (requestVal.hasDeletedDir() && !requestVal.getDeletedDir().isEmpty()) { subdirDelNum++; + remainNum--; } - subDirNum += request.getMarkDeletedSubDirsCount(); - subFileNum += request.getDeletedSubFilesCount(); + subDirNum += requestVal.getMarkDeletedSubDirsCount(); + subFileNum += requestVal.getDeletedSubFilesCount(); subDirRecursiveCnt++; } catch (IOException e) { LOG.error("Error while running delete directories and files " + @@ -440,9 +469,9 @@ public long optimizeDirDeletesAndSubmitRequest(long remainNum, break; } } - + OzoneManagerProtocolProtos.OMResponse response = null; if (!purgePathRequestList.isEmpty()) { - submitPurgePaths(purgePathRequestList, snapTableKey, expectedPreviousSnapshotId); + response = submitPurgePaths(purgePathRequestList, snapTableKey, expectedPreviousSnapshotId); } if (dirNum != 0 || subDirNum != 0 || subFileNum != 0) { @@ -457,107 +486,7 @@ public long optimizeDirDeletesAndSubmitRequest(long remainNum, dirNum, subdirDelNum, subFileNum, (subDirNum - subdirDelNum), Time.monotonicNow() - startTime, getRunCount()); } - return remainNum; - } - - /** - * 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") - public void calculateExclusiveSize( - SnapshotInfo previousSnapshot, - SnapshotInfo previousToPrevSnapshot, - OmKeyInfo keyInfo, - OmBucketInfo bucketInfo, long volumeId, - Table snapRenamedTable, - Table previousKeyTable, - Table prevRenamedTable, - Table previousToPrevKeyTable, - Map exclusiveSizeMap, - Map exclusiveReplicatedSizeMap) 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); - OmKeyInfo prevKeyInfo = renamedKey != null ? - previousKeyTable.get(renamedKey) : - previousKeyTable.get(dbKeyPrevSnap); - - if (prevKeyInfo == null || - prevKeyInfo.getObjectID() != keyInfo.getObjectID()) { - return null; - } - - return isBlockLocationInfoSame(prevKeyInfo, keyInfo) ? - prevKeyInfo : null; + return Pair.of(remainNum, Optional.ofNullable(response)); } protected boolean isBufferLimitCrossed( @@ -565,72 +494,6 @@ protected boolean isBufferLimitCrossed( return cLimit + increment >= maxLimit; } - protected boolean isKeyReclaimable( - Table previousKeyTable, - Table renamedTable, - OmKeyInfo deletedKeyInfo, OmBucketInfo bucketInfo, - long volumeId, HddsProtos.KeyValue.Builder renamedKeyBuilder) - throws IOException { - - String dbKey; - // Handle case when the deleted snapshot is the first snapshot. - if (previousKeyTable == null) { - return true; - } - - // These are uncommitted blocks wrapped into a pseudo KeyInfo - if (deletedKeyInfo.getObjectID() == OBJECT_ID_RECLAIM_BLOCKS) { - return true; - } - - // Construct keyTable or fileTable DB key depending on the bucket type - if (bucketInfo.getBucketLayout().isFileSystemOptimized()) { - dbKey = ozoneManager.getMetadataManager().getOzonePathKey( - volumeId, - bucketInfo.getObjectID(), - deletedKeyInfo.getParentObjectID(), - deletedKeyInfo.getFileName()); - } else { - dbKey = ozoneManager.getMetadataManager().getOzoneKey( - deletedKeyInfo.getVolumeName(), - deletedKeyInfo.getBucketName(), - deletedKeyInfo.getKeyName()); - } - - /* - snapshotRenamedTable: - 1) /volumeName/bucketName/objectID -> - /volumeId/bucketId/parentId/fileName (FSO) - 2) /volumeName/bucketName/objectID -> - /volumeName/bucketName/keyName (non-FSO) - */ - String dbRenameKey = ozoneManager.getMetadataManager().getRenameKey( - deletedKeyInfo.getVolumeName(), deletedKeyInfo.getBucketName(), - deletedKeyInfo.getObjectID()); - - // Condition: key should not exist in snapshotRenamedTable - // of the current snapshot and keyTable of the previous snapshot. - // Check key exists in renamedTable of the Snapshot - String renamedKey = renamedTable.getIfExist(dbRenameKey); - - if (renamedKey != null && renamedKeyBuilder != null) { - renamedKeyBuilder.setKey(dbRenameKey).setValue(renamedKey); - } - // previousKeyTable is fileTable if the bucket is FSO, - // otherwise it is the keyTable. - OmKeyInfo prevKeyInfo = renamedKey != null ? previousKeyTable - .get(renamedKey) : previousKeyTable.get(dbKey); - - if (prevKeyInfo == null || - prevKeyInfo.getObjectID() != deletedKeyInfo.getObjectID()) { - return true; - } - - // For key overwrite the objectID will remain the same, In this - // case we need to check if OmKeyLocationInfo is also same. - return !isBlockLocationInfoSame(prevKeyInfo, deletedKeyInfo); - } - public boolean isRatisEnabled() { if (ozoneManager == null) { return false; @@ -690,4 +553,22 @@ public long getMovedFilesCount() { public BootstrapStateHandler.Lock getBootstrapStateLock() { return lock; } + + /** + * Submits SetSnapsnapshotPropertyRequest to OM. + * @param setSnapshotPropertyRequests request to be sent to OM + */ + protected void submitSetSnapshotRequest( + List setSnapshotPropertyRequests) { + OzoneManagerProtocolProtos.OMRequest omRequest = OzoneManagerProtocolProtos.OMRequest.newBuilder() + .setCmdType(OzoneManagerProtocolProtos.Type.SetSnapshotProperty) + .addAllSetSnapshotPropertyRequests(setSnapshotPropertyRequests) + .setClientId(clientId.toString()) + .build(); + try { + OzoneManagerRatisUtils.submitRequest(getOzoneManager(), omRequest, clientId, getRunCount().get()); + } catch (ServiceException e) { + LOG.error("Failed to submit set snapshot property request", e); + } + } } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/DirectoryDeletingService.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/DirectoryDeletingService.java index 09f4a8f8a3d7..4300e6269d1e 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/DirectoryDeletingService.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/DirectoryDeletingService.java @@ -26,16 +26,21 @@ import org.apache.hadoop.hdds.utils.db.Table; import org.apache.hadoop.hdds.utils.db.Table.KeyValue; import org.apache.hadoop.hdds.utils.db.TableIterator; +import org.apache.hadoop.ozone.om.KeyManager; import org.apache.hadoop.ozone.om.OMConfigKeys; import org.apache.hadoop.ozone.om.OmMetadataManagerImpl; import org.apache.hadoop.ozone.om.OmSnapshot; import org.apache.hadoop.ozone.om.OmSnapshotManager; import org.apache.hadoop.ozone.om.OzoneManager; -import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo; +import org.apache.hadoop.ozone.om.SnapshotChainManager; import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; +import org.apache.hadoop.ozone.om.lock.IOzoneManagerLock; import org.apache.hadoop.ozone.om.snapshot.ReferenceCounted; import org.apache.hadoop.ozone.om.snapshot.SnapshotUtils; +import org.apache.hadoop.ozone.om.snapshot.filter.ReclaimableDirFilter; +import org.apache.hadoop.ozone.om.snapshot.filter.ReclaimableKeyFilter; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.PurgePathRequest; import org.apache.hadoop.util.Time; import org.slf4j.Logger; @@ -43,12 +48,16 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; import java.util.List; -import java.util.Objects; +import java.util.Map; import java.util.Optional; import java.util.UUID; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_PATH_DELETING_LIMIT_PER_TASK; import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_PATH_DELETING_LIMIT_PER_TASK_DEFAULT; @@ -132,68 +141,98 @@ public void resume() { suspended.set(false); } + @Override public BackgroundTaskQueue getTasks() { BackgroundTaskQueue queue = new BackgroundTaskQueue(); - queue.add(new DirectoryDeletingService.DirDeletingTask(this)); + queue.add(new DirectoryDeletingService.DirDeletingTask()); return queue; } private final class DirDeletingTask implements BackgroundTask { - private final DirectoryDeletingService directoryDeletingService; - - private DirDeletingTask(DirectoryDeletingService service) { - this.directoryDeletingService = service; - } @Override public int getPriority() { return 0; } - @Override - public BackgroundTaskResult call() { - if (shouldRun()) { - if (LOG.isDebugEnabled()) { - LOG.debug("Running DirectoryDeletingService"); - } - isRunningOnAOS.set(true); - getRunCount().incrementAndGet(); - long dirNum = 0L; - long subDirNum = 0L; - long subFileNum = 0L; - long remainNum = pathLimitPerTask; - int consumedSize = 0; - List purgePathRequestList = new ArrayList<>(); - List> allSubDirList - = new ArrayList<>((int) remainNum); - - Table.KeyValue pendingDeletedDirInfo; - - try (TableIterator> - deleteTableIterator = getOzoneManager().getMetadataManager(). - getDeletedDirTable().iterator()) { - // This is to avoid race condition b/w purge request and snapshot chain updation. For AOS taking the global - // snapshotId since AOS could process multiple buckets in one iteration. - UUID expectedPreviousSnapshotId = - ((OmMetadataManagerImpl)getOzoneManager().getMetadataManager()).getSnapshotChainManager() - .getLatestGlobalSnapshotId(); + private OzoneManagerProtocolProtos.SetSnapshotPropertyRequest getSetSnapshotRequestUpdatingExclusiveSize( + Map exclusiveSizeMap, Map exclusiveReplicatedSizeMap, String prevSnapshotKeyTable) { + OzoneManagerProtocolProtos.SnapshotSize snapshotSize = OzoneManagerProtocolProtos.SnapshotSize.newBuilder() + .setExclusiveSize( + exclusiveSizeMap.getOrDefault(prevSnapshotKeyTable, 0L)) + .setExclusiveReplicatedSize( + exclusiveReplicatedSizeMap.getOrDefault( + prevSnapshotKeyTable, 0L)) + .build(); + exclusiveSizeMap.remove(prevSnapshotKeyTable); + exclusiveReplicatedSizeMap.remove(prevSnapshotKeyTable); - long startTime = Time.monotonicNow(); - while (remainNum > 0 && deleteTableIterator.hasNext()) { - pendingDeletedDirInfo = deleteTableIterator.next(); - // Do not reclaim if the directory is still being referenced by - // the previous snapshot. - if (previousSnapshotHasDir(pendingDeletedDirInfo)) { - continue; - } + return OzoneManagerProtocolProtos.SetSnapshotPropertyRequest.newBuilder() + .setSnapshotKey(prevSnapshotKeyTable) + .setSnapshotDirSize(snapshotSize) + .build(); + } - PurgePathRequest request = prepareDeleteDirRequest( + private OzoneManagerProtocolProtos.SetSnapshotPropertyRequest + getSetSnapshotPropertyRequestupdatingDeepCleanSnapshotDir(String snapshotKeyTable) { + return OzoneManagerProtocolProtos.SetSnapshotPropertyRequest.newBuilder() + .setSnapshotKey(snapshotKeyTable) + .setDeepCleanedDeletedDir(true) + .build(); + } + + + /** + * + * @param currentSnapshotInfo if null, deleted directories in AOS should be processed. + * @param keyManager KeyManager of the underlying store. + */ + private long processDeletedDirsForStore(SnapshotInfo currentSnapshotInfo, KeyManager keyManager, + long remainNum) throws IOException { + + long dirNum = 0L; + long subDirNum = 0L; + long subFileNum = 0L; + int consumedSize = 0; + long initialRemainNum = remainNum; + List purgePathRequestList = new ArrayList<>(); + List> allSubDirList + = new ArrayList<>((int) remainNum); + String volume = currentSnapshotInfo == null ? null : currentSnapshotInfo.getVolumeName(); + String bucket = currentSnapshotInfo == null ? null : currentSnapshotInfo.getBucketName(); + String snapshotTableKey = currentSnapshotInfo == null ? null : currentSnapshotInfo.getTableKey(); + Table.KeyValue pendingDeletedDirInfo; + + try (TableIterator> + deletedDirsIterator = keyManager.getPendingDeletionDirs(volume, bucket)) { + OmSnapshotManager omSnapshotManager = getOzoneManager().getOmSnapshotManager(); + SnapshotChainManager snapshotChainManager = ((OmMetadataManagerImpl)getOzoneManager().getMetadataManager()) + .getSnapshotChainManager(); + // This is to avoid race condition b/w purge request and snapshot chain updation. For AOS taking the global + // snapshotId since AOS could process multiple buckets in one iteration. While using path previous snapshot + // Id for a snapshot since it would process only one bucket. + UUID expectedPreviousSnapshotId = currentSnapshotInfo == null ? + snapshotChainManager.getLatestGlobalSnapshotId() : + SnapshotUtils.getPreviousSnapshotId(currentSnapshotInfo, snapshotChainManager); + IOzoneManagerLock lock = getOzoneManager().getMetadataManager().getLock(); + + try (ReclaimableDirFilter reclaimableDirFilter = new ReclaimableDirFilter(getOzoneManager(), + omSnapshotManager, snapshotChainManager, currentSnapshotInfo, keyManager.getMetadataManager(), lock); + ReclaimableKeyFilter reclaimableSubFileFilter = new ReclaimableKeyFilter(getOzoneManager(), + omSnapshotManager, snapshotChainManager, currentSnapshotInfo, keyManager.getMetadataManager(), lock)) { + long startTime = Time.monotonicNow(); + while (remainNum > 0 && deletedDirsIterator.hasNext()) { + pendingDeletedDirInfo = deletedDirsIterator.next(); + // Always perform listing on AOS. + Optional request = prepareDeleteDirRequest( remainNum, pendingDeletedDirInfo.getValue(), pendingDeletedDirInfo.getKey(), allSubDirList, - getOzoneManager().getKeyManager()); - if (isBufferLimitCrossed(ratisByteLimit, consumedSize, - request.getSerializedSize())) { + getOzoneManager().getKeyManager(), reclaimableDirFilter.apply(pendingDeletedDirInfo), + reclaimableSubFileFilter); + + if (request.isPresent() && isBufferLimitCrossed(ratisByteLimit, consumedSize, + request.get().getSerializedSize())) { if (purgePathRequestList.size() != 0) { // if message buffer reaches max limit, avoid sending further remainNum = 0; @@ -205,99 +244,122 @@ public BackgroundTaskResult call() { request = prepareDeleteDirRequest( remainNum, pendingDeletedDirInfo.getValue(), pendingDeletedDirInfo.getKey(), allSubDirList, - getOzoneManager().getKeyManager()); + getOzoneManager().getKeyManager(), reclaimableDirFilter.apply(pendingDeletedDirInfo), + reclaimableSubFileFilter); } - consumedSize += request.getSerializedSize(); - purgePathRequestList.add(request); - // reduce remain count for self, sub-files, and sub-directories - remainNum = remainNum - 1; - remainNum = remainNum - request.getDeletedSubFilesCount(); - remainNum = remainNum - request.getMarkDeletedSubDirsCount(); + if (!request.isPresent()) { + continue; + } + PurgePathRequest purgePathRequest = request.get(); + consumedSize += purgePathRequest.getSerializedSize(); + purgePathRequestList.add(purgePathRequest); + remainNum = remainNum - purgePathRequest.getDeletedSubFilesCount(); + remainNum = remainNum - purgePathRequest.getMarkDeletedSubDirsCount(); // Count up the purgeDeletedDir, subDirs and subFiles - if (request.getDeletedDir() != null - && !request.getDeletedDir().isEmpty()) { + if (purgePathRequest.getDeletedDir() != null + && !purgePathRequest.getDeletedDir().isEmpty()) { dirNum++; + remainNum--; } - subDirNum += request.getMarkDeletedSubDirsCount(); - subFileNum += request.getDeletedSubFilesCount(); + subDirNum += purgePathRequest.getMarkDeletedSubDirsCount(); + subFileNum += purgePathRequest.getDeletedSubFilesCount(); } - optimizeDirDeletesAndSubmitRequest( + Pair> retVal = optimizeDirDeletesAndSubmitRequest( remainNum, dirNum, subDirNum, subFileNum, - allSubDirList, purgePathRequestList, null, startTime, + allSubDirList, purgePathRequestList, snapshotTableKey, startTime, ratisByteLimit - consumedSize, - getOzoneManager().getKeyManager(), expectedPreviousSnapshotId); + getOzoneManager().getKeyManager(), reclaimableDirFilter, reclaimableSubFileFilter, + expectedPreviousSnapshotId); + remainNum = retVal.getKey(); - } catch (IOException e) { - LOG.error("Error while running delete directories and files " + - "background task. Will retry at next run.", e); - } - isRunningOnAOS.set(false); - synchronized (directoryDeletingService) { - this.directoryDeletingService.notify(); + if (remainNum == initialRemainNum && + retVal.getValue().map(OzoneManagerProtocolProtos.OMResponse::getSuccess).orElse(true)) { + List setSnapshotPropertyRequests = new ArrayList<>(); + Map exclusiveReplicatedSizeMap = reclaimableSubFileFilter.getExclusiveReplicatedSizeMap(); + Map exclusiveSizeMap = reclaimableSubFileFilter.getExclusiveSizeMap(); + for (String snapshot : Stream.of(exclusiveSizeMap.keySet(), exclusiveReplicatedSizeMap.keySet()) + .flatMap(Collection::stream).distinct().collect(Collectors.toList())) { + setSnapshotPropertyRequests.add(getSetSnapshotRequestUpdatingExclusiveSize(exclusiveSizeMap, + exclusiveReplicatedSizeMap, snapshot)); + } + //Updating directory deep clean flag of snapshot. + if (currentSnapshotInfo != null) { + setSnapshotPropertyRequests.add(getSetSnapshotPropertyRequestupdatingDeepCleanSnapshotDir( + currentSnapshotInfo.getTableKey())); + } + + submitSetSnapshotRequest(setSnapshotPropertyRequests); + } } + + } catch (IOException e) { + throw e; } - // place holder by returning empty results of this call back. - return BackgroundTaskResult.EmptyTaskResult.newResult(); + return remainNum; } - private boolean previousSnapshotHasDir( - KeyValue pendingDeletedDirInfo) throws IOException { - String key = pendingDeletedDirInfo.getKey(); - OmKeyInfo deletedDirInfo = pendingDeletedDirInfo.getValue(); - OmSnapshotManager omSnapshotManager = - getOzoneManager().getOmSnapshotManager(); - OmMetadataManagerImpl metadataManager = (OmMetadataManagerImpl) - getOzoneManager().getMetadataManager(); - SnapshotInfo previousSnapshotInfo = SnapshotUtils.getLatestSnapshotInfo(deletedDirInfo.getVolumeName(), - deletedDirInfo.getBucketName(), getOzoneManager(), metadataManager.getSnapshotChainManager()); - if (previousSnapshotInfo == null) { - return false; - } - // previous snapshot is not active or it has not been flushed to disk then don't process the key in this - // iteration. - if (previousSnapshotInfo.getSnapshotStatus() != SnapshotInfo.SnapshotStatus.SNAPSHOT_ACTIVE || - !OmSnapshotManager.areSnapshotChangesFlushedToDB(getOzoneManager().getMetadataManager(), - previousSnapshotInfo)) { - return true; - } - try (ReferenceCounted rcLatestSnapshot = - omSnapshotManager.getSnapshot( - deletedDirInfo.getVolumeName(), - deletedDirInfo.getBucketName(), - previousSnapshotInfo.getName())) { - - if (rcLatestSnapshot != null) { - String dbRenameKey = metadataManager - .getRenameKey(deletedDirInfo.getVolumeName(), - deletedDirInfo.getBucketName(), deletedDirInfo.getObjectID()); - Table prevDirTable = - rcLatestSnapshot.get().getMetadataManager().getDirectoryTable(); - Table prevDeletedDirTable = - rcLatestSnapshot.get().getMetadataManager().getDeletedDirTable(); - OmKeyInfo prevDeletedDirInfo = prevDeletedDirTable.get(key); - if (prevDeletedDirInfo != null) { - return true; + @Override + public BackgroundTaskResult call() { + if (shouldRun()) { + if (LOG.isDebugEnabled()) { + LOG.debug("Running DirectoryDeletingService"); + } + isRunningOnAOS.set(true); + getRunCount().incrementAndGet(); + long remainNum = pathLimitPerTask; + try { + remainNum = processDeletedDirsForStore(null, getOzoneManager().getKeyManager(), + remainNum); + } catch (IOException e) { + LOG.error("Error while running delete directories and files " + + "background task. Will retry at next run. on active object store", e); + } finally { + isRunningOnAOS.set(false); + } + + if (remainNum > 0) { + SnapshotChainManager chainManager = + ((OmMetadataManagerImpl)getOzoneManager().getMetadataManager()).getSnapshotChainManager(); + OmSnapshotManager omSnapshotManager = getOzoneManager().getOmSnapshotManager(); + Iterator iterator = null; + try { + iterator = chainManager.iterator(true); + + } catch (IOException e) { + LOG.error("Error while initializing snapshot chain iterator."); + return BackgroundTaskResult.EmptyTaskResult.newResult(); + } + + while (iterator.hasNext() && remainNum > 0) { + UUID snapshotId = iterator.next(); + try { + SnapshotInfo snapInfo = SnapshotUtils.getSnapshotInfo(getOzoneManager(), chainManager, snapshotId); + // Wait for snapshot changes to be flushed to disk. + if (!OmSnapshotManager.areSnapshotChangesFlushedToDB(getOzoneManager().getMetadataManager(), snapInfo)) { + LOG.info("Skipping snapshot processing since changes to snapshot {} have not been flushed to disk", + snapInfo); + continue; + } + // Check if snapshot has been directory deep cleaned. Return if deep cleaning already done. + if (snapInfo.getDeepCleanedDeletedDir()) { + LOG.debug("Snapshot {} has already been directory deep cleaned", snapInfo); + continue; + } + try (ReferenceCounted omSnapshot = omSnapshotManager.getSnapshot(snapInfo.getVolumeName(), + snapInfo.getBucketName(), snapInfo.getName())) { + remainNum = processDeletedDirsForStore(snapInfo, omSnapshot.get().getKeyManager(), remainNum); + } + + } catch (IOException e) { + LOG.error("Error while running delete directories and files " + + "background task for snapshot: {}. Will retry at next run. on active object store", snapshotId, e); + } } - String prevDirTableDBKey = metadataManager.getSnapshotRenamedTable() - .get(dbRenameKey); - // In OMKeyDeleteResponseWithFSO OzonePathKey is converted to - // OzoneDeletePathKey. Changing it back to check the previous DirTable - String prevDbKey = prevDirTableDBKey == null ? - metadataManager.getOzoneDeletePathDirKey(key) : prevDirTableDBKey; - OmDirectoryInfo prevDirInfo = prevDirTable.get(prevDbKey); - //Checking if the previous snapshot in the chain hasn't changed while checking if the deleted directory is - // present in the previous snapshot. If the chain has changed, the deleted directory could have been moved - // to the newly created snapshot. - SnapshotInfo newPreviousSnapshotInfo = SnapshotUtils.getLatestSnapshotInfo(deletedDirInfo.getVolumeName(), - deletedDirInfo.getBucketName(), getOzoneManager(), metadataManager.getSnapshotChainManager()); - return (!Objects.equals(Optional.ofNullable(newPreviousSnapshotInfo).map(SnapshotInfo::getSnapshotId), - Optional.ofNullable(previousSnapshotInfo).map(SnapshotInfo::getSnapshotId))) || (prevDirInfo != null && - prevDirInfo.getObjectID() == deletedDirInfo.getObjectID()); } } - - return false; + // place holder by returning empty results of this call back. + return BackgroundTaskResult.EmptyTaskResult.newResult(); } } 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 9a4f74eba59c..14c7c7c9d21b 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 @@ -17,40 +17,36 @@ package org.apache.hadoop.ozone.om.service; import java.io.IOException; +import java.io.UncheckedIOException; import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; +import java.util.Collection; import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Optional; -import java.util.Set; import java.util.UUID; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import java.util.stream.Stream; import com.google.common.base.Preconditions; -import com.google.protobuf.ServiceException; +import org.apache.commons.lang3.tuple.Pair; 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; import org.apache.hadoop.ozone.common.BlockGroup; import org.apache.hadoop.ozone.om.KeyManager; import org.apache.hadoop.ozone.om.OmMetadataManagerImpl; import org.apache.hadoop.ozone.om.OmSnapshot; import org.apache.hadoop.ozone.om.OmSnapshotManager; import org.apache.hadoop.ozone.om.OzoneManager; -import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils; +import org.apache.hadoop.ozone.om.lock.IOzoneManagerLock; import org.apache.hadoop.ozone.om.snapshot.ReferenceCounted; import org.apache.hadoop.ozone.om.snapshot.SnapshotUtils; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotSize; +import org.apache.hadoop.ozone.om.snapshot.filter.ReclaimableKeyFilter; +import org.apache.hadoop.ozone.om.snapshot.filter.ReclaimableRenameEntryFilter; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos; 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; import org.apache.hadoop.hdds.utils.BackgroundTaskResult; @@ -58,17 +54,12 @@ import com.google.common.annotations.VisibleForTesting; -import static org.apache.hadoop.ozone.om.helpers.SnapshotInfo.SnapshotStatus.SNAPSHOT_ACTIVE; import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_KEY_DELETING_LIMIT_PER_TASK; import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_KEY_DELETING_LIMIT_PER_TASK_DEFAULT; import org.apache.hadoop.ozone.om.PendingKeysDeletion; import org.apache.hadoop.ozone.om.SnapshotChainManager; -import org.apache.hadoop.ozone.om.helpers.OmBucketInfo; -import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; -import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo; import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; -import org.apache.ratis.protocol.ClientId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -88,16 +79,12 @@ public class KeyDeletingService extends AbstractKeyDeletingService { private static final int KEY_DELETING_CORE_POOL_SIZE = 1; private final KeyManager manager; + private int keyLimitPerTask; private final AtomicLong deletedKeyCount; private final AtomicBoolean suspended; - private final Map exclusiveSizeMap; - private final Map exclusiveReplicatedSizeMap; - private final Set completedExclusiveSizeSet; - private final Map snapshotSeekMap; - private AtomicBoolean isRunningOnAOS; private final boolean deepCleanSnapshots; - private final SnapshotChainManager snapshotChainManager; + private AtomicBoolean isRunningOnAOS; public KeyDeletingService(OzoneManager ozoneManager, ScmBlockLocationProtocol scmClient, @@ -114,13 +101,8 @@ public KeyDeletingService(OzoneManager ozoneManager, OZONE_KEY_DELETING_LIMIT_PER_TASK + " cannot be negative."); this.deletedKeyCount = new AtomicLong(0); this.suspended = new AtomicBoolean(false); - this.exclusiveSizeMap = new HashMap<>(); - this.exclusiveReplicatedSizeMap = new HashMap<>(); - this.completedExclusiveSizeSet = new HashSet<>(); - this.snapshotSeekMap = new HashMap<>(); - this.isRunningOnAOS = new AtomicBoolean(false); this.deepCleanSnapshots = deepCleanSnapshots; - this.snapshotChainManager = ((OmMetadataManagerImpl)manager.getMetadataManager()).getSnapshotChainManager(); + this.isRunningOnAOS = new AtomicBoolean(false); } /** @@ -140,7 +122,7 @@ public boolean isRunningOnAOS() { @Override public BackgroundTaskQueue getTasks() { BackgroundTaskQueue queue = new BackgroundTaskQueue(); - queue.add(new KeyDeletingTask(this)); + queue.add(new KeyDeletingTask()); return queue; } @@ -184,10 +166,123 @@ public void setKeyLimitPerTask(int keyLimitPerTask) { * DB. */ private final class KeyDeletingTask implements BackgroundTask { - private final KeyDeletingService deletingService; - private KeyDeletingTask(KeyDeletingService service) { - this.deletingService = service; + private OzoneManagerProtocolProtos.SetSnapshotPropertyRequest getSetSnapshotRequestUpdatingExclusiveSize( + Map exclusiveSizeMap, Map exclusiveReplicatedSizeMap, String prevSnapshotKeyTable) { + OzoneManagerProtocolProtos.SnapshotSize snapshotSize = OzoneManagerProtocolProtos.SnapshotSize.newBuilder() + .setExclusiveSize( + exclusiveSizeMap.getOrDefault(prevSnapshotKeyTable, 0L)) + .setExclusiveReplicatedSize( + exclusiveReplicatedSizeMap.getOrDefault( + prevSnapshotKeyTable, 0L)) + .build(); + exclusiveSizeMap.remove(prevSnapshotKeyTable); + exclusiveReplicatedSizeMap.remove(prevSnapshotKeyTable); + + return OzoneManagerProtocolProtos.SetSnapshotPropertyRequest.newBuilder() + .setSnapshotKey(prevSnapshotKeyTable) + .setSnapshotSize(snapshotSize) + .build(); + } + + /** + * + * @param currentSnapshotInfo if null, deleted directories in AOS should be processed. + * @param keyManager KeyManager of the underlying store. + */ + private int processDeletedKeysForStore(SnapshotInfo currentSnapshotInfo, KeyManager keyManager, + int remainNum) throws IOException { + String volume = currentSnapshotInfo == null ? null : currentSnapshotInfo.getVolumeName(); + String bucket = currentSnapshotInfo == null ? null : currentSnapshotInfo.getBucketName(); + String snapshotTableKey = currentSnapshotInfo == null ? null : currentSnapshotInfo.getTableKey(); + + String startKey = ""; + int initialRemainNum = remainNum; + boolean successStatus = true; + try { + // TODO: [SNAPSHOT] HDDS-7968. Reclaim eligible key blocks in + // snapshot's deletedTable when active DB's deletedTable + // doesn't have enough entries left. + // OM would have to keep track of which snapshot the key is coming + // from if the above would be done inside getPendingDeletionKeys(). + OmSnapshotManager omSnapshotManager = getOzoneManager().getOmSnapshotManager(); + SnapshotChainManager snapshotChainManager = ((OmMetadataManagerImpl)getOzoneManager().getMetadataManager()) + .getSnapshotChainManager(); + // This is to avoid race condition b/w purge request and snapshot chain update. For AOS taking the global + // snapshotId since AOS could process multiple buckets in one iteration. While using path + // previous snapshotId for a snapshot since it would process only one bucket. + UUID expectedPreviousSnapshotId = currentSnapshotInfo == null ? + snapshotChainManager.getLatestGlobalSnapshotId() : + SnapshotUtils.getPreviousSnapshotId(currentSnapshotInfo, snapshotChainManager); + + IOzoneManagerLock lock = getOzoneManager().getMetadataManager().getLock(); + + // Purge deleted Keys in the deletedTable && rename entries in the snapshotRenamedTable which doesn't have a + // reference in the previous snapshot. + try (ReclaimableKeyFilter reclaimableKeyFilter = new ReclaimableKeyFilter(getOzoneManager(), + omSnapshotManager, + snapshotChainManager, currentSnapshotInfo, keyManager.getMetadataManager(), lock); + ReclaimableRenameEntryFilter renameEntryFilter = new ReclaimableRenameEntryFilter( + getOzoneManager(), omSnapshotManager, snapshotChainManager, currentSnapshotInfo, + keyManager.getMetadataManager(), lock)) { + List renamedTableEntries = + keyManager.getRenamesKeyEntries(volume, bucket, startKey, renameEntryFilter, remainNum).stream() + .map(entry -> { + try { + return entry.getKey(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + }).collect(Collectors.toList()); + remainNum -= renamedTableEntries.size(); + + // Get pending keys that can be deleted + PendingKeysDeletion pendingKeysDeletion = keyManager.getPendingDeletionKeys(volume, bucket, startKey, + reclaimableKeyFilter, remainNum); + List keyBlocksList = pendingKeysDeletion.getKeyBlocksList(); + //submit purge requests if there are renamed entries to be purged or keys to be purged. + if (!renamedTableEntries.isEmpty() || keyBlocksList != null && !keyBlocksList.isEmpty()) { + // Validating if the previous snapshot is still the same before purging the blocks. + SnapshotUtils.validatePreviousSnapshotId(currentSnapshotInfo, snapshotChainManager, + expectedPreviousSnapshotId); + Pair purgeResult = processKeyDeletes(keyBlocksList, getOzoneManager().getKeyManager(), + pendingKeysDeletion.getKeysToModify(), renamedTableEntries, snapshotTableKey, + expectedPreviousSnapshotId); + remainNum -= purgeResult.getKey(); + successStatus = purgeResult.getValue(); + } + + // Checking remainNum is greater than zero and not equal to the initial value if there were some keys to + // reclaim. This is to check if + if (remainNum > 0 && successStatus) { + List setSnapshotPropertyRequests = new ArrayList<>(); + Map exclusiveReplicatedSizeMap = reclaimableKeyFilter.getExclusiveReplicatedSizeMap(); + Map exclusiveSizeMap = reclaimableKeyFilter.getExclusiveSizeMap(); + List previousPathSnapshotsInChain = + Stream.of(exclusiveSizeMap.keySet(), exclusiveReplicatedSizeMap.keySet()) + .flatMap(Collection::stream).distinct().collect(Collectors.toList()); + for (String snapshot : previousPathSnapshotsInChain) { + setSnapshotPropertyRequests.add(getSetSnapshotRequestUpdatingExclusiveSize(exclusiveSizeMap, + exclusiveReplicatedSizeMap, snapshot)); + } + + //Updating directory deep clean flag of snapshot. + if (currentSnapshotInfo != null) { + setSnapshotPropertyRequests.add(OzoneManagerProtocolProtos.SetSnapshotPropertyRequest.newBuilder() + .setSnapshotKey(snapshotTableKey) + .setDeepCleanedDeletedKey(true) + .build()); + } + submitSetSnapshotRequest(setSnapshotPropertyRequests); + } + } + + } catch (IOException e) { + throw e; + } catch (UncheckedIOException e) { + throw e.getCause(); + } + return remainNum; } @Override @@ -203,323 +298,66 @@ public BackgroundTaskResult call() { final long run = getRunCount().incrementAndGet(); LOG.debug("Running KeyDeletingService {}", run); isRunningOnAOS.set(true); - int delCount = 0; + int remainNum = keyLimitPerTask; try { - // TODO: [SNAPSHOT] HDDS-7968. Reclaim eligible key blocks in - // snapshot's deletedTable when active DB's deletedTable - // doesn't have enough entries left. - // OM would have to keep track of which snapshot the key is coming - // from if the above would be done inside getPendingDeletionKeys(). - // This is to avoid race condition b/w purge request and snapshot chain update. For AOS taking the global - // snapshotId since AOS could process multiple buckets in one iteration. - UUID expectedPreviousSnapshotId = snapshotChainManager.getLatestGlobalSnapshotId(); - PendingKeysDeletion pendingKeysDeletion = manager - .getPendingDeletionKeys(getKeyLimitPerTask()); - List keyBlocksList = pendingKeysDeletion - .getKeyBlocksList(); - if (keyBlocksList != null && !keyBlocksList.isEmpty()) { - delCount = processKeyDeletes(keyBlocksList, - getOzoneManager().getKeyManager(), - pendingKeysDeletion.getKeysToModify(), null, expectedPreviousSnapshotId); - deletedKeyCount.addAndGet(delCount); - } + remainNum = processDeletedKeysForStore(null, getOzoneManager().getKeyManager(), + remainNum); } catch (IOException e) { - LOG.error("Error while running delete keys background task. Will " + - "retry at next run.", e); + LOG.error("Error while running delete directories and files " + + "background task. Will retry at next run. on active object store", e); + } finally { + isRunningOnAOS.set(false); } - try { - if (deepCleanSnapshots && delCount < keyLimitPerTask) { - processSnapshotDeepClean(delCount); - } - } catch (Exception e) { - LOG.error("Error while running deep clean on snapshots. Will " + - "retry at next run.", e); - } - - } - isRunningOnAOS.set(false); - synchronized (deletingService) { - this.deletingService.notify(); - } - - // By design, no one cares about the results of this call back. - return EmptyTaskResult.newResult(); - } - - @SuppressWarnings("checkstyle:MethodLength") - private void processSnapshotDeepClean(int delCount) - throws IOException { - OmSnapshotManager omSnapshotManager = - getOzoneManager().getOmSnapshotManager(); - OmMetadataManagerImpl metadataManager = (OmMetadataManagerImpl) - getOzoneManager().getMetadataManager(); - SnapshotChainManager snapChainManager = metadataManager - .getSnapshotChainManager(); - Table snapshotInfoTable = - getOzoneManager().getMetadataManager().getSnapshotInfoTable(); - List deepCleanedSnapshots = new ArrayList<>(); - try (TableIterator> iterator = snapshotInfoTable.iterator()) { - - while (delCount < keyLimitPerTask && iterator.hasNext()) { - List keysToPurge = new ArrayList<>(); - HashMap keysToModify = new HashMap<>(); - SnapshotInfo currSnapInfo = snapshotInfoTable.get(iterator.next().getKey()); - // Deep clean only on active snapshot. Deleted Snapshots will be - // cleaned up by SnapshotDeletingService. - if (currSnapInfo == null || currSnapInfo.getSnapshotStatus() != SNAPSHOT_ACTIVE || - currSnapInfo.getDeepClean()) { - continue; + if (deepCleanSnapshots && remainNum > 0) { + SnapshotChainManager chainManager = + ((OmMetadataManagerImpl)getOzoneManager().getMetadataManager()).getSnapshotChainManager(); + OmSnapshotManager omSnapshotManager = getOzoneManager().getOmSnapshotManager(); + Iterator iterator = null; + try { + iterator = chainManager.iterator(true); + + } catch (IOException e) { + LOG.error("Error while initializing snapshot chain iterator."); + return BackgroundTaskResult.EmptyTaskResult.newResult(); } - SnapshotInfo prevSnapInfo = SnapshotUtils.getPreviousSnapshot(getOzoneManager(), snapChainManager, - currSnapInfo); - if (prevSnapInfo != null && - (prevSnapInfo.getSnapshotStatus() != SnapshotInfo.SnapshotStatus.SNAPSHOT_ACTIVE || - !OmSnapshotManager.areSnapshotChangesFlushedToDB(getOzoneManager().getMetadataManager(), - prevSnapInfo))) { - continue; - } - - try (ReferenceCounted - rcCurrOmSnapshot = omSnapshotManager.getSnapshot( - currSnapInfo.getVolumeName(), - currSnapInfo.getBucketName(), - currSnapInfo.getName())) { - OmSnapshot currOmSnapshot = rcCurrOmSnapshot.get(); - - Table snapDeletedTable = - currOmSnapshot.getMetadataManager().getDeletedTable(); - Table snapRenamedTable = - currOmSnapshot.getMetadataManager().getSnapshotRenamedTable(); - - long volumeId = metadataManager.getVolumeId( - currSnapInfo.getVolumeName()); - // Get bucketInfo for the snapshot bucket to get bucket layout. - String dbBucketKey = metadataManager.getBucketKey( - currSnapInfo.getVolumeName(), currSnapInfo.getBucketName()); - OmBucketInfo bucketInfo = metadataManager.getBucketTable() - .get(dbBucketKey); - - if (bucketInfo == null) { - throw new IllegalStateException("Bucket " + "/" + currSnapInfo - .getVolumeName() + "/" + currSnapInfo.getBucketName() + - " is not found. BucketInfo should not be null for" + - " snapshotted bucket. The OM is in unexpected state."); - } - - String snapshotBucketKey = dbBucketKey + OzoneConsts.OM_KEY_PREFIX; - SnapshotInfo previousSnapshot = SnapshotUtils.getPreviousSnapshot(getOzoneManager(), snapChainManager, - currSnapInfo); - SnapshotInfo previousToPrevSnapshot = null; - - if (previousSnapshot != null) { - previousToPrevSnapshot = SnapshotUtils.getPreviousSnapshot(getOzoneManager(), snapChainManager, - previousSnapshot); - } - - Table previousKeyTable = null; - Table prevRenamedTable = null; - ReferenceCounted rcPrevOmSnapshot = null; - - // Split RepeatedOmKeyInfo and update current snapshot - // deletedKeyTable and next snapshot deletedKeyTable. - if (previousSnapshot != null) { - rcPrevOmSnapshot = omSnapshotManager.getSnapshot( - previousSnapshot.getVolumeName(), - previousSnapshot.getBucketName(), - previousSnapshot.getName()); - OmSnapshot omPreviousSnapshot = rcPrevOmSnapshot.get(); - - previousKeyTable = omPreviousSnapshot.getMetadataManager() - .getKeyTable(bucketInfo.getBucketLayout()); - prevRenamedTable = omPreviousSnapshot - .getMetadataManager().getSnapshotRenamedTable(); - } - - Table previousToPrevKeyTable = null; - ReferenceCounted rcPrevToPrevOmSnapshot = null; - if (previousToPrevSnapshot != null) { - rcPrevToPrevOmSnapshot = omSnapshotManager.getSnapshot( - previousToPrevSnapshot.getVolumeName(), - previousToPrevSnapshot.getBucketName(), - previousToPrevSnapshot.getName()); - OmSnapshot omPreviousToPrevSnapshot = rcPrevToPrevOmSnapshot.get(); - - previousToPrevKeyTable = omPreviousToPrevSnapshot - .getMetadataManager() - .getKeyTable(bucketInfo.getBucketLayout()); - } - - try (TableIterator> deletedIterator = snapDeletedTable - .iterator()) { - - String lastKeyInCurrentRun = null; - String deletedTableSeek = snapshotSeekMap.getOrDefault( - currSnapInfo.getTableKey(), snapshotBucketKey); - deletedIterator.seek(deletedTableSeek); - // To avoid processing the last key from the previous - // run again. - if (!deletedTableSeek.equals(snapshotBucketKey) && - deletedIterator.hasNext()) { - deletedIterator.next(); + while (iterator.hasNext() && remainNum > 0) { + UUID snapshotId = iterator.next(); + try { + SnapshotInfo snapInfo = SnapshotUtils.getSnapshotInfo(getOzoneManager(), chainManager, snapshotId); + // Wait for snapshot changes to be flushed to disk. + if (!OmSnapshotManager.areSnapshotChangesFlushedToDB(getOzoneManager().getMetadataManager(), snapInfo)) { + LOG.info("Skipping snapshot processing since changes to snapshot {} have not been flushed to disk", + snapInfo); + continue; } - - while (deletedIterator.hasNext() && delCount < keyLimitPerTask) { - Table.KeyValue - deletedKeyValue = deletedIterator.next(); - String deletedKey = deletedKeyValue.getKey(); - lastKeyInCurrentRun = deletedKey; - - // Exit if it is out of the bucket scope. - if (!deletedKey.startsWith(snapshotBucketKey)) { - break; - } - - RepeatedOmKeyInfo repeatedOmKeyInfo = - deletedKeyValue.getValue(); - - List blockGroupList = new ArrayList<>(); - RepeatedOmKeyInfo newRepeatedOmKeyInfo = - new RepeatedOmKeyInfo(); - for (OmKeyInfo keyInfo : repeatedOmKeyInfo.getOmKeyInfoList()) { - if (previousSnapshot != null) { - // Calculates the exclusive size for the previous - // snapshot. See Java Doc for more info. - calculateExclusiveSize(previousSnapshot, - previousToPrevSnapshot, keyInfo, bucketInfo, volumeId, - snapRenamedTable, previousKeyTable, prevRenamedTable, - previousToPrevKeyTable, exclusiveSizeMap, - exclusiveReplicatedSizeMap); - } - - if (isKeyReclaimable(previousKeyTable, snapRenamedTable, - keyInfo, bucketInfo, volumeId, null)) { - List blocksForKeyDelete = currOmSnapshot - .getMetadataManager() - .getBlocksForKeyDelete(deletedKey); - if (blocksForKeyDelete != null) { - blockGroupList.addAll(blocksForKeyDelete); - } - delCount++; - } else { - newRepeatedOmKeyInfo.addOmKeyInfo(keyInfo); - } - } - - if (newRepeatedOmKeyInfo.getOmKeyInfoList().size() > 0 && - newRepeatedOmKeyInfo.getOmKeyInfoList().size() != - repeatedOmKeyInfo.getOmKeyInfoList().size()) { - keysToModify.put(deletedKey, newRepeatedOmKeyInfo); - } - - if (newRepeatedOmKeyInfo.getOmKeyInfoList().size() != - repeatedOmKeyInfo.getOmKeyInfoList().size()) { - keysToPurge.addAll(blockGroupList); - } + // Check if snapshot has been directory deep cleaned. Return if directory deep cleaning is not + // done. + if (!snapInfo.getDeepCleanedDeletedDir()) { + LOG.debug("Snapshot {} hasn't done deleted directory deep cleaning yet. Skipping the snapshot in this" + + " iteration.", snapInfo); + continue; } - - 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()); - } - - snapshotSeekMap.remove(currSnapInfo.getTableKey()); - } else { - // There are keys that still needs processing - // we can continue from it in the next iteration - if (lastKeyInCurrentRun != null) { - snapshotSeekMap.put(currSnapInfo.getTableKey(), - lastKeyInCurrentRun); - } + // Checking if snapshot has been key deep cleaned already. + if (snapInfo.getDeepClean()) { + LOG.debug("Snapshot {} has already done deleted key deep cleaning.", snapInfo); + continue; } - - if (!keysToPurge.isEmpty()) { - processKeyDeletes(keysToPurge, currOmSnapshot.getKeyManager(), - keysToModify, currSnapInfo.getTableKey(), - Optional.ofNullable(previousSnapshot).map(SnapshotInfo::getSnapshotId).orElse(null)); + try (ReferenceCounted omSnapshot = omSnapshotManager.getSnapshot(snapInfo.getVolumeName(), + snapInfo.getBucketName(), snapInfo.getName())) { + remainNum = processDeletedKeysForStore(snapInfo, omSnapshot.get().getKeyManager(), remainNum); } - } finally { - IOUtils.closeQuietly(rcPrevOmSnapshot, rcPrevToPrevOmSnapshot); + + } catch (IOException e) { + LOG.error("Error while running delete directories and files " + + "background task for snapshot: {}. Will retry at next run. on active object store", snapshotId, e); } } - } } - - updateDeepCleanedSnapshots(deepCleanedSnapshots); - updateSnapshotExclusiveSize(); - } - - private void updateSnapshotExclusiveSize() { - - if (completedExclusiveSizeSet.isEmpty()) { - return; - } - - Iterator completedSnapshotIterator = - completedExclusiveSizeSet.iterator(); - while (completedSnapshotIterator.hasNext()) { - ClientId clientId = ClientId.randomId(); - String dbKey = completedSnapshotIterator.next(); - SnapshotSize snapshotSize = SnapshotSize.newBuilder() - .setExclusiveSize(exclusiveSizeMap.getOrDefault(dbKey, 0L)) - .setExclusiveReplicatedSize( - exclusiveReplicatedSizeMap.getOrDefault(dbKey, 0L)) - .build(); - SetSnapshotPropertyRequest setSnapshotPropertyRequest = - SetSnapshotPropertyRequest.newBuilder() - .setSnapshotKey(dbKey) - .setSnapshotSize(snapshotSize) - .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) { - for (String deepCleanedSnapshot: deepCleanedSnapshots) { - ClientId clientId = ClientId.randomId(); - SetSnapshotPropertyRequest setSnapshotPropertyRequest = - SetSnapshotPropertyRequest.newBuilder() - .setSnapshotKey(deepCleanedSnapshot) - .setDeepCleanedDeletedKey(true) - .build(); - - OMRequest omRequest = OMRequest.newBuilder() - .setCmdType(Type.SetSnapshotProperty) - .setSetSnapshotPropertyRequest(setSnapshotPropertyRequest) - .setClientId(clientId.toString()) - .build(); - - submitRequest(omRequest, clientId); - } - } - - public void submitRequest(OMRequest omRequest, ClientId clientId) { - try { - OzoneManagerRatisUtils.submitRequest(getOzoneManager(), omRequest, clientId, getRunCount().get()); - } catch (ServiceException e) { - LOG.error("Snapshot deep cleaning request failed. " + - "Will retry at next run.", e); - } + // By design, no one cares about the results of this call back. + return EmptyTaskResult.newResult(); } } } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java index edc6c7a16296..85e3c8b9da85 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java @@ -37,9 +37,10 @@ import org.apache.hadoop.ozone.om.OzoneManager; import org.apache.hadoop.ozone.om.SnapshotChainManager; 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.SnapshotInfo; +import org.apache.hadoop.ozone.om.lock.IOzoneManagerLock; +import org.apache.hadoop.ozone.om.lock.MultiLocks; +import org.apache.hadoop.ozone.om.lock.OzoneManagerLock; import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils; import org.apache.hadoop.ozone.om.snapshot.ReferenceCounted; import org.apache.hadoop.ozone.om.snapshot.SnapshotUtils; @@ -54,8 +55,10 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Iterator; import java.util.List; +import java.util.Optional; import java.util.UUID; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -91,7 +94,7 @@ public class SnapshotDeletingService extends AbstractKeyDeletingService { private final int keyLimitPerTask; private final int snapshotDeletionPerTask; private final int ratisByteLimit; - private final long serviceTimeout; + private MultiLocks snapshotIdLocks; public SnapshotDeletingService(long interval, long serviceTimeout, OzoneManager ozoneManager) @@ -118,32 +121,8 @@ public SnapshotDeletingService(long interval, long serviceTimeout, this.keyLimitPerTask = conf.getInt( OZONE_SNAPSHOT_KEY_DELETING_LIMIT_PER_TASK, OZONE_SNAPSHOT_KEY_DELETING_LIMIT_PER_TASK_DEFAULT); - this.serviceTimeout = serviceTimeout; - } - - // Wait for a notification from KeyDeletingService if the key deletion is running. This is to ensure, merging of - // entries do not start while the AOS is still processing the deleted keys. - @VisibleForTesting - public void waitForKeyDeletingService() throws InterruptedException { - KeyDeletingService keyDeletingService = getOzoneManager().getKeyManager().getDeletingService(); - synchronized (keyDeletingService) { - while (keyDeletingService.isRunningOnAOS()) { - keyDeletingService.wait(serviceTimeout); - } - } - } - - // Wait for a notification from DirectoryDeletingService if the directory deletion is running. This is to ensure, - // merging of entries do not start while the AOS is still processing the deleted keys. - @VisibleForTesting - public void waitForDirDeletingService() throws InterruptedException { - DirectoryDeletingService directoryDeletingService = getOzoneManager().getKeyManager() - .getDirDeletingService(); - synchronized (directoryDeletingService) { - while (directoryDeletingService.isRunningOnAOS()) { - directoryDeletingService.wait(serviceTimeout); - } - } + IOzoneManagerLock lock = getOzoneManager().getMetadataManager().getLock(); + this.snapshotIdLocks = new MultiLocks<>(lock, OzoneManagerLock.Resource.SNAPSHOT_GC_LOCK, true); } private class SnapshotDeletingTask implements BackgroundTask { @@ -175,20 +154,28 @@ public BackgroundTaskResult call() throws InterruptedException { nextSnapshot.getSnapshotStatus() != SnapshotInfo.SnapshotStatus.SNAPSHOT_ACTIVE) { continue; } + SnapshotInfo nextToNextSnapshot = nextSnapshot == null ? null : SnapshotUtils.getNextSnapshot(ozoneManager, + chainManager, snapInfo); - // nextSnapshot = null means entries would be moved to AOS. - if (nextSnapshot == null) { - waitForKeyDeletingService(); - waitForDirDeletingService(); + // Wait for the next iteration if the next snapshot or next to next snapshot is still not deep cleaned + // since purge transaction will add entries and it could be processed by mistake. + if (isSnapshotNotDeepCleaned(nextSnapshot) || isSnapshotNotDeepCleaned(nextToNextSnapshot)) { + continue; } - try (ReferenceCounted snapshot = omSnapshotManager.getSnapshot( - snapInfo.getVolumeName(), snapInfo.getBucketName(), snapInfo.getName())) { + // Acquire write lock on current snapshot and next snapshot in chain. + if (!snapshotIdLocks.acquireLock(Arrays.asList(snapInfo.getSnapshotId(), + Optional.ofNullable(nextSnapshot).map(SnapshotInfo::getSnapshotId).orElse(null))) + .isLockAcquired()) { + continue; + } + try (ReferenceCounted snapshot = omSnapshotManager.getSnapshot(snapInfo.getVolumeName(), + snapInfo.getBucketName(), snapInfo.getName())) { KeyManager snapshotKeyManager = snapshot.get().getKeyManager(); int moveCount = 0; // Get all entries from deletedKeyTable. List>> deletedKeyEntries = snapshotKeyManager.getDeletedKeyEntries(snapInfo.getVolumeName(), snapInfo.getBucketName(), - null, remaining); + null, (kv) -> true, remaining); moveCount += deletedKeyEntries.size(); // Get all entries from deletedDirTable. List> deletedDirEntries = snapshotKeyManager.getDeletedDirEntries( @@ -196,7 +183,7 @@ public BackgroundTaskResult call() throws InterruptedException { moveCount += deletedDirEntries.size(); // Get all entries from snapshotRenamedTable. List> renameEntries = snapshotKeyManager.getRenamesKeyEntries( - snapInfo.getVolumeName(), snapInfo.getBucketName(), null, remaining - moveCount); + snapInfo.getVolumeName(), snapInfo.getBucketName(), null, (kv) -> true, remaining - moveCount); moveCount += renameEntries.size(); if (moveCount > 0) { List deletedKeys = new ArrayList<>(deletedKeyEntries.size()); @@ -227,6 +214,8 @@ public BackgroundTaskResult call() throws InterruptedException { } else { snapshotsToBePurged.add(snapInfo.getTableKey()); } + } finally { + snapshotIdLocks.releaseLock(); } successRunCount.incrementAndGet(); snapshotLimit--; @@ -314,61 +303,14 @@ private void submitRequest(OMRequest omRequest) { boolean shouldIgnoreSnapshot(SnapshotInfo snapInfo) throws IOException { SnapshotInfo.SnapshotStatus snapshotStatus = snapInfo.getSnapshotStatus(); return snapshotStatus != SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED || - !OmSnapshotManager.areSnapshotChangesFlushedToDB(getOzoneManager().getMetadataManager(), snapInfo); + !OmSnapshotManager.areSnapshotChangesFlushedToDB(getOzoneManager().getMetadataManager(), snapInfo) + || isSnapshotNotDeepCleaned(snapInfo); } - // TODO: Move this util class. - public static boolean isBlockLocationInfoSame(OmKeyInfo prevKeyInfo, - OmKeyInfo deletedKeyInfo) { - - if (prevKeyInfo == null && deletedKeyInfo == null) { - LOG.debug("Both prevKeyInfo and deletedKeyInfo are null."); - return true; - } - if (prevKeyInfo == null || deletedKeyInfo == null) { - LOG.debug("prevKeyInfo: '{}' or deletedKeyInfo: '{}' is null.", - prevKeyInfo, deletedKeyInfo); - return false; - } - // For hsync, Though the blockLocationInfo of a key may not be same - // at the time of snapshot and key deletion as blocks can be appended. - // If the objectId is same then the key is same. - if (prevKeyInfo.isHsync() && deletedKeyInfo.isHsync()) { - return true; - } - - if (prevKeyInfo.getKeyLocationVersions().size() != - deletedKeyInfo.getKeyLocationVersions().size()) { - return false; - } - - OmKeyLocationInfoGroup deletedOmKeyLocation = - deletedKeyInfo.getLatestVersionLocations(); - OmKeyLocationInfoGroup prevOmKeyLocation = - prevKeyInfo.getLatestVersionLocations(); - - if (deletedOmKeyLocation == null || prevOmKeyLocation == null) { - return false; - } - - List deletedLocationList = - deletedOmKeyLocation.getLocationList(); - List prevLocationList = - prevOmKeyLocation.getLocationList(); - - if (deletedLocationList.size() != prevLocationList.size()) { - return false; - } - - for (int idx = 0; idx < deletedLocationList.size(); idx++) { - OmKeyLocationInfo deletedLocationInfo = deletedLocationList.get(idx); - OmKeyLocationInfo prevLocationInfo = prevLocationList.get(idx); - if (!deletedLocationInfo.hasSameBlockAs(prevLocationInfo)) { - return false; - } - } - - return true; + @VisibleForTesting + boolean isSnapshotNotDeepCleaned(SnapshotInfo snapInfo) { + // if snapshot is null it means snapshot doesn't exist, so it means deep cleaned. + return snapInfo != null && !(snapInfo.getDeepCleanedDeletedDir() && snapInfo.getDeepClean()); } @Override diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDirectoryCleaningService.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDirectoryCleaningService.java deleted file mode 100644 index e7133e625896..000000000000 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDirectoryCleaningService.java +++ /dev/null @@ -1,485 +0,0 @@ -/* - * 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.service; - -import com.google.common.annotations.VisibleForTesting; -import com.google.protobuf.ServiceException; -import org.apache.commons.lang3.StringUtils; -import org.apache.hadoop.hdds.client.BlockID; -import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol; -import org.apache.hadoop.hdds.utils.BackgroundTask; -import org.apache.hadoop.hdds.utils.BackgroundTaskQueue; -import org.apache.hadoop.hdds.utils.BackgroundTaskResult; -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.common.BlockGroup; -import org.apache.hadoop.ozone.om.OMMetadataManager; -import org.apache.hadoop.ozone.om.OmMetadataManagerImpl; -import org.apache.hadoop.ozone.om.OmSnapshot; -import org.apache.hadoop.ozone.om.OmSnapshotManager; -import org.apache.hadoop.ozone.om.OzoneManager; -import org.apache.hadoop.ozone.om.SnapshotChainManager; -import org.apache.hadoop.ozone.om.helpers.OmBucketInfo; -import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo; -import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; -import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup; -import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; -import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils; -import org.apache.hadoop.ozone.om.request.file.OMFileRequest; -import org.apache.hadoop.ozone.om.snapshot.ReferenceCounted; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SetSnapshotPropertyRequest; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotSize; -import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type; -import org.apache.ratis.protocol.ClientId; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Stack; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.stream.Collectors; - -import static org.apache.hadoop.ozone.om.helpers.SnapshotInfo.SnapshotStatus.SNAPSHOT_ACTIVE; -import static org.apache.hadoop.ozone.om.request.file.OMFileRequest.getDirectoryInfo; -import static org.apache.hadoop.ozone.om.snapshot.SnapshotUtils.getOzonePathKeyForFso; -import static org.apache.hadoop.ozone.om.snapshot.SnapshotUtils.getPreviousSnapshot; - -/** - * Snapshot BG Service for deleted directory deep clean and exclusive size - * calculation for deleted directories. - */ -public class SnapshotDirectoryCleaningService - extends AbstractKeyDeletingService { - // Use only a single thread for DirDeletion. Multiple threads would read - // or write to same tables and can send deletion requests for same key - // multiple times. - private static final int SNAPSHOT_DIR_CORE_POOL_SIZE = 1; - - private final AtomicBoolean suspended; - private final Map exclusiveSizeMap; - private final Map exclusiveReplicatedSizeMap; - - public SnapshotDirectoryCleaningService(long interval, TimeUnit unit, - long serviceTimeout, - OzoneManager ozoneManager, - ScmBlockLocationProtocol scmClient) { - super(SnapshotDirectoryCleaningService.class.getSimpleName(), - interval, unit, SNAPSHOT_DIR_CORE_POOL_SIZE, serviceTimeout, - ozoneManager, scmClient); - this.suspended = new AtomicBoolean(false); - this.exclusiveSizeMap = new HashMap<>(); - this.exclusiveReplicatedSizeMap = new HashMap<>(); - } - - private boolean shouldRun() { - if (getOzoneManager() == null) { - // OzoneManager can be null for testing - return true; - } - return getOzoneManager().isLeaderReady() && !suspended.get(); - } - - /** - * Suspend the service. - */ - @VisibleForTesting - public void suspend() { - suspended.set(true); - } - - /** - * Resume the service if suspended. - */ - @VisibleForTesting - public void resume() { - suspended.set(false); - } - - @Override - public BackgroundTaskQueue getTasks() { - BackgroundTaskQueue queue = new BackgroundTaskQueue(); - queue.add(new SnapshotDirectoryCleaningService.SnapshotDirTask()); - return queue; - } - - private class SnapshotDirTask implements BackgroundTask { - - @Override - public BackgroundTaskResult call() { - if (!shouldRun()) { - return BackgroundTaskResult.EmptyTaskResult.newResult(); - } - LOG.debug("Running SnapshotDirectoryCleaningService"); - - getRunCount().incrementAndGet(); - OmSnapshotManager omSnapshotManager = - getOzoneManager().getOmSnapshotManager(); - Table snapshotInfoTable = - getOzoneManager().getMetadataManager().getSnapshotInfoTable(); - OmMetadataManagerImpl metadataManager = (OmMetadataManagerImpl) - getOzoneManager().getMetadataManager(); - SnapshotChainManager snapChainManager = metadataManager - .getSnapshotChainManager(); - - try (TableIterator> iterator = snapshotInfoTable.iterator()) { - - while (iterator.hasNext()) { - SnapshotInfo currSnapInfo = snapshotInfoTable.get(iterator.next().getKey()); - - // Expand deleted dirs only on active snapshot. Deleted Snapshots - // will be cleaned up by SnapshotDeletingService. - if (currSnapInfo == null || currSnapInfo.getSnapshotStatus() != SNAPSHOT_ACTIVE || - currSnapInfo.getDeepCleanedDeletedDir()) { - continue; - } - - ReferenceCounted rcPrevOmSnapshot = null; - ReferenceCounted rcPrevToPrevOmSnapshot = null; - try { - long volumeId = metadataManager - .getVolumeId(currSnapInfo.getVolumeName()); - // Get bucketInfo for the snapshot bucket to get bucket layout. - String dbBucketKey = metadataManager - .getBucketKey(currSnapInfo.getVolumeName(), - currSnapInfo.getBucketName()); - OmBucketInfo bucketInfo = metadataManager - .getBucketTable().get(dbBucketKey); - - if (bucketInfo == null) { - throw new IllegalStateException("Bucket " + "/" + - currSnapInfo.getVolumeName() + "/" + currSnapInfo - .getBucketName() + - " is not found. BucketInfo should not be " + - "null for snapshotted bucket. The OM is in " + - "unexpected state."); - } - - SnapshotInfo previousSnapshot = getPreviousSnapshot(getOzoneManager(), snapChainManager, currSnapInfo); - SnapshotInfo previousToPrevSnapshot = null; - - Table previousKeyTable = null; - Table prevRenamedTable = null; - - if (previousSnapshot != null) { - rcPrevOmSnapshot = omSnapshotManager.getActiveSnapshot( - previousSnapshot.getVolumeName(), - previousSnapshot.getBucketName(), - previousSnapshot.getName()); - OmSnapshot omPreviousSnapshot = rcPrevOmSnapshot.get(); - - previousKeyTable = omPreviousSnapshot.getMetadataManager() - .getKeyTable(bucketInfo.getBucketLayout()); - prevRenamedTable = omPreviousSnapshot - .getMetadataManager().getSnapshotRenamedTable(); - previousToPrevSnapshot = getPreviousSnapshot(getOzoneManager(), snapChainManager, previousSnapshot); - } - - Table previousToPrevKeyTable = null; - if (previousToPrevSnapshot != null) { - rcPrevToPrevOmSnapshot = omSnapshotManager.getActiveSnapshot( - previousToPrevSnapshot.getVolumeName(), - previousToPrevSnapshot.getBucketName(), - previousToPrevSnapshot.getName()); - OmSnapshot omPreviousToPrevSnapshot = rcPrevToPrevOmSnapshot.get(); - - previousToPrevKeyTable = omPreviousToPrevSnapshot - .getMetadataManager() - .getKeyTable(bucketInfo.getBucketLayout()); - } - - String dbBucketKeyForDir = getOzonePathKeyForFso(metadataManager, - currSnapInfo.getVolumeName(), currSnapInfo.getBucketName()); - try (ReferenceCounted - rcCurrOmSnapshot = omSnapshotManager.getActiveSnapshot( - currSnapInfo.getVolumeName(), - currSnapInfo.getBucketName(), - currSnapInfo.getName())) { - - OmSnapshot currOmSnapshot = rcCurrOmSnapshot.get(); - Table snapDeletedDirTable = - currOmSnapshot.getMetadataManager().getDeletedDirTable(); - - try (TableIterator> deletedDirIterator = snapDeletedDirTable - .iterator(dbBucketKeyForDir)) { - - while (deletedDirIterator.hasNext()) { - Table.KeyValue deletedDirInfo = - deletedDirIterator.next(); - - // For each deleted directory we do an in-memory DFS and - // do a deep clean and exclusive size calculation. - iterateDirectoryTree(deletedDirInfo, volumeId, bucketInfo, - previousSnapshot, previousToPrevSnapshot, - currOmSnapshot, previousKeyTable, prevRenamedTable, - previousToPrevKeyTable, dbBucketKeyForDir); - } - updateDeepCleanSnapshotDir(currSnapInfo.getTableKey()); - if (previousSnapshot != null) { - updateExclusiveSize(previousSnapshot.getTableKey()); - } - } - } - } finally { - IOUtils.closeQuietly(rcPrevOmSnapshot, rcPrevToPrevOmSnapshot); - } - } - } catch (IOException ex) { - LOG.error("Error while running directory deep clean on snapshots." + - " Will retry at next run.", ex); - } - return BackgroundTaskResult.EmptyTaskResult.newResult(); - } - } - - @SuppressWarnings("checkstyle:ParameterNumber") - private void iterateDirectoryTree( - Table.KeyValue deletedDirInfo, long volumeId, - OmBucketInfo bucketInfo, - SnapshotInfo previousSnapshot, - SnapshotInfo previousToPrevSnapshot, - OmSnapshot currOmSnapshot, - Table previousKeyTable, - Table prevRenamedTable, - Table previousToPrevKeyTable, - String dbBucketKeyForDir) throws IOException { - - Table snapDirTable = - currOmSnapshot.getMetadataManager().getDirectoryTable(); - Table snapRenamedTable = - currOmSnapshot.getMetadataManager().getSnapshotRenamedTable(); - - Stack stackNodes = new Stack<>(); - OmDirectoryInfo omDeletedDirectoryInfo = - getDirectoryInfo(deletedDirInfo.getValue()); - String dirPathDbKey = currOmSnapshot.getMetadataManager() - .getOzonePathKey(volumeId, bucketInfo.getObjectID(), - omDeletedDirectoryInfo); - // Stack Init - StackNode topLevelDir = new StackNode(); - topLevelDir.setDirKey(dirPathDbKey); - topLevelDir.setDirValue(omDeletedDirectoryInfo); - stackNodes.push(topLevelDir); - - try (TableIterator> - directoryIterator = snapDirTable.iterator(dbBucketKeyForDir)) { - - while (!stackNodes.isEmpty()) { - StackNode stackTop = stackNodes.peek(); - // First process all the files in the current directory - // and then do a DFS for directory. - if (StringUtils.isEmpty(stackTop.getSubDirSeek())) { - processFilesUnderDir(previousSnapshot, - previousToPrevSnapshot, - volumeId, - bucketInfo, - stackTop.getDirValue(), - currOmSnapshot.getMetadataManager(), - snapRenamedTable, - previousKeyTable, - prevRenamedTable, - previousToPrevKeyTable); - // Format : /volId/bucketId/parentId/ - String seekDirInDB = currOmSnapshot.getMetadataManager() - .getOzonePathKey(volumeId, bucketInfo.getObjectID(), - stackTop.getDirValue().getObjectID(), ""); - stackTop.setSubDirSeek(seekDirInDB); - } else { - // Adding \0 to seek the next greater element. - directoryIterator.seek(stackTop.getSubDirSeek() + "\0"); - if (directoryIterator.hasNext()) { - - Table.KeyValue deletedSubDirInfo = directoryIterator.next(); - String deletedSubDirKey = deletedSubDirInfo.getKey(); - String prefixCheck = currOmSnapshot.getMetadataManager() - .getOzoneDeletePathDirKey(stackTop.getSubDirSeek()); - // Exit if it is out of the sub dir prefix scope. - if (!deletedSubDirKey.startsWith(prefixCheck)) { - stackNodes.pop(); - } else { - stackTop.setSubDirSeek(deletedSubDirKey); - StackNode nextSubDir = new StackNode(); - nextSubDir.setDirKey(deletedSubDirInfo.getKey()); - nextSubDir.setDirValue(deletedSubDirInfo.getValue()); - stackNodes.push(nextSubDir); - } - } else { - stackNodes.pop(); - } - } - } - } - } - - private void updateExclusiveSize(String prevSnapshotKeyTable) { - ClientId clientId = ClientId.randomId(); - SnapshotSize snapshotSize = SnapshotSize.newBuilder() - .setExclusiveSize( - exclusiveSizeMap.getOrDefault(prevSnapshotKeyTable, 0L)) - .setExclusiveReplicatedSize( - exclusiveReplicatedSizeMap.getOrDefault( - prevSnapshotKeyTable, 0L)) - .build(); - exclusiveSizeMap.remove(prevSnapshotKeyTable); - exclusiveReplicatedSizeMap.remove(prevSnapshotKeyTable); - SetSnapshotPropertyRequest - setSnapshotPropertyRequest = - SetSnapshotPropertyRequest.newBuilder() - .setSnapshotKey(prevSnapshotKeyTable) - .setSnapshotSize(snapshotSize) - .build(); - - OMRequest omRequest = OMRequest.newBuilder() - .setCmdType(Type.SetSnapshotProperty) - .setSetSnapshotPropertyRequest(setSnapshotPropertyRequest) - .setClientId(clientId.toString()) - .build(); - - submitRequest(omRequest, clientId); - } - - @SuppressWarnings("checkstyle:ParameterNumber") - private void processFilesUnderDir( - SnapshotInfo previousSnapshot, - SnapshotInfo previousToPrevSnapshot, - long volumeId, - OmBucketInfo bucketInfo, - OmDirectoryInfo parentInfo, - OMMetadataManager metadataManager, - Table snapRenamedTable, - Table previousKeyTable, - Table prevRenamedTable, - Table previousToPrevKeyTable) - throws IOException { - String seekFileInDB = metadataManager.getOzonePathKey(volumeId, - bucketInfo.getObjectID(), - parentInfo.getObjectID(), ""); - List blocksForKeyDelete = new ArrayList<>(); - - Table fileTable = metadataManager.getFileTable(); - try (TableIterator> - iterator = fileTable.iterator(seekFileInDB)) { - - while (iterator.hasNext()) { - Table.KeyValue entry = iterator.next(); - OmKeyInfo fileInfo = entry.getValue(); - if (!OMFileRequest.isImmediateChild(fileInfo.getParentObjectID(), - parentInfo.getObjectID())) { - break; - } - - String ozoneDeletePathKey = metadataManager - .getOzoneDeletePathKey(fileInfo.getObjectID(), entry.getKey()); - if (isKeyReclaimable(previousKeyTable, snapRenamedTable, - fileInfo, bucketInfo, volumeId, null)) { - for (OmKeyLocationInfoGroup keyLocations : - fileInfo.getKeyLocationVersions()) { - List item = keyLocations.getLocationList().stream() - .map(b -> new BlockID(b.getContainerID(), b.getLocalID())) - .collect(Collectors.toList()); - BlockGroup keyBlocks = BlockGroup.newBuilder() - .setKeyName(ozoneDeletePathKey) - .addAllBlockIDs(item) - .build(); - blocksForKeyDelete.add(keyBlocks); - } - // TODO: Add Retry mechanism. - getScmClient().deleteKeyBlocks(blocksForKeyDelete); - } else if (previousSnapshot != null) { - calculateExclusiveSize(previousSnapshot, previousToPrevSnapshot, - fileInfo, bucketInfo, volumeId, snapRenamedTable, - previousKeyTable, prevRenamedTable, previousToPrevKeyTable, - exclusiveSizeMap, exclusiveReplicatedSizeMap); - } - } - } - } - - private void updateDeepCleanSnapshotDir(String snapshotKeyTable) { - ClientId clientId = ClientId.randomId(); - SetSnapshotPropertyRequest setSnapshotPropertyRequest = - SetSnapshotPropertyRequest.newBuilder() - .setSnapshotKey(snapshotKeyTable) - .setDeepCleanedDeletedDir(true) - .build(); - - OMRequest omRequest = OMRequest.newBuilder() - .setCmdType(Type.SetSnapshotProperty) - .setSetSnapshotPropertyRequest(setSnapshotPropertyRequest) - .setClientId(clientId.toString()) - .build(); - - submitRequest(omRequest, clientId); - } - - public void submitRequest(OMRequest omRequest, ClientId clientId) { - try { - OzoneManagerRatisUtils.submitRequest(getOzoneManager(), omRequest, clientId, getRunCount().get()); - } catch (ServiceException e) { - LOG.error("Snapshot deep cleaning request failed. " + - "Will retry at next run.", e); - } - } - - /** - * Stack node data for directory deep clean for snapshot. - */ - private static class StackNode { - private String dirKey; - private OmDirectoryInfo dirValue; - private String subDirSeek; - - public String getDirKey() { - return dirKey; - } - - public void setDirKey(String dirKey) { - this.dirKey = dirKey; - } - - public OmDirectoryInfo getDirValue() { - return dirValue; - } - - public void setDirValue(OmDirectoryInfo dirValue) { - this.dirValue = dirValue; - } - - public String getSubDirSeek() { - return subDirSeek; - } - - public void setSubDirSeek(String subDirSeek) { - this.subDirSeek = subDirSeek; - } - - @Override - public String toString() { - return "StackNode{" + - "dirKey='" + dirKey + '\'' + - ", dirObjectId=" + dirValue.getObjectID() + - ", subDirSeek='" + subDirSeek + '\'' + - '}'; - } - } -} diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java index 6393f12066c4..c6ee5709409c 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java @@ -46,7 +46,6 @@ import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; import org.apache.hadoop.ozone.om.helpers.WithObjectID; import org.apache.hadoop.ozone.om.helpers.WithParentObjectId; -import org.apache.hadoop.ozone.om.service.SnapshotDeletingService; import org.apache.hadoop.ozone.snapshot.CancelSnapshotDiffResponse; import org.apache.hadoop.ozone.snapshot.SnapshotDiffReportOzone; import org.apache.hadoop.ozone.snapshot.SnapshotDiffResponse; @@ -1417,7 +1416,7 @@ long generateDiffReport( private boolean isKeyModified(OmKeyInfo fromKey, OmKeyInfo toKey) { return !fromKey.isKeyInfoSame(toKey, false, false, false, false, true) - || !SnapshotDeletingService.isBlockLocationInfoSame( + || !SnapshotUtils.isBlockLocationInfoSame( fromKey, toKey); } @@ -1466,7 +1465,7 @@ private boolean isBlockLocationSame( "OmKeyInfo"); } - return SnapshotDeletingService.isBlockLocationInfoSame( + return SnapshotUtils.isBlockLocationInfoSame( (OmKeyInfo) fromObject, (OmKeyInfo) toObject); } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotUtils.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotUtils.java index 201a9fe0c9c9..c9c104db1613 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotUtils.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotUtils.java @@ -18,6 +18,7 @@ package org.apache.hadoop.ozone.om.snapshot; +import org.apache.hadoop.hdds.utils.BackgroundService; import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB; import org.apache.hadoop.ozone.om.OMMetadataManager; import org.apache.hadoop.ozone.om.OmMetadataManagerImpl; @@ -25,6 +26,8 @@ import org.apache.hadoop.ozone.om.SnapshotChainManager; import org.apache.hadoop.ozone.om.exceptions.OMException; 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.RepeatedOmKeyInfo; import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; import org.apache.hadoop.ozone.om.helpers.SnapshotInfo.SnapshotStatus; @@ -173,7 +176,7 @@ public static SnapshotInfo getNextSnapshot(OzoneManager ozoneManager, return getSnapshotInfo(ozoneManager, chainManager, nextPathSnapshot); } } catch (NoSuchElementException ex) { - LOG.error("The snapshot {} is not longer in snapshot chain, It " + + LOG.error("The snapshot {} is no longer in snapshot chain, It " + "maybe removed in the previous Snapshot purge request.", snapInfo.getTableKey()); } @@ -194,7 +197,7 @@ public static SnapshotInfo getPreviousSnapshot(OzoneManager ozoneManager, /** * Get the previous snapshot in the snapshot chain. */ - private static UUID getPreviousSnapshotId(SnapshotInfo snapInfo, SnapshotChainManager chainManager) + public static UUID getPreviousSnapshotId(SnapshotInfo snapInfo, SnapshotChainManager chainManager) throws IOException { // If the snapshot is deleted in the previous run, then the in-memory // SnapshotChainManager might throw NoSuchElementException as the snapshot @@ -320,6 +323,13 @@ private static boolean isSameAsLatestOmKeyInfo(List omKeyInfos, return false; } + public static SnapshotInfo getLatestGlobalSnapshotInfo(OzoneManager ozoneManager, + SnapshotChainManager snapshotChainManager) throws IOException { + Optional latestGlobalSnapshot = Optional.ofNullable(snapshotChainManager.getLatestGlobalSnapshotId()); + return latestGlobalSnapshot.isPresent() ? getSnapshotInfo(ozoneManager, snapshotChainManager, + latestGlobalSnapshot.get()) : null; + } + public static SnapshotInfo getLatestSnapshotInfo(String volumeName, String bucketName, OzoneManager ozoneManager, SnapshotChainManager snapshotChainManager) throws IOException { @@ -349,4 +359,56 @@ public static void validatePreviousSnapshotId(SnapshotInfo snapshotInfo, OMException.ResultCodes.INVALID_REQUEST); } } + + public static boolean isBlockLocationInfoSame(OmKeyInfo prevKeyInfo, + OmKeyInfo deletedKeyInfo) { + + if (prevKeyInfo == null && deletedKeyInfo == null) { + BackgroundService.LOG.debug("Both prevKeyInfo and deletedKeyInfo are null."); + return true; + } + if (prevKeyInfo == null || deletedKeyInfo == null) { + BackgroundService.LOG.debug("prevKeyInfo: '{}' or deletedKeyInfo: '{}' is null.", + prevKeyInfo, deletedKeyInfo); + return false; + } + // For hsync, Though the blockLocationInfo of a key may not be same + // at the time of snapshot and key deletion as blocks can be appended. + // If the objectId is same then the key is same. + if (prevKeyInfo.isHsync() && deletedKeyInfo.isHsync()) { + return true; + } + + if (prevKeyInfo.getKeyLocationVersions().size() != + deletedKeyInfo.getKeyLocationVersions().size()) { + return false; + } + + OmKeyLocationInfoGroup deletedOmKeyLocation = + deletedKeyInfo.getLatestVersionLocations(); + OmKeyLocationInfoGroup prevOmKeyLocation = + prevKeyInfo.getLatestVersionLocations(); + + if (deletedOmKeyLocation == null || prevOmKeyLocation == null) { + return false; + } + + List deletedLocationList = + deletedOmKeyLocation.getLocationList(); + List prevLocationList = + prevOmKeyLocation.getLocationList(); + + if (deletedLocationList.size() != prevLocationList.size()) { + return false; + } + + for (int idx = 0; idx < deletedLocationList.size(); idx++) { + OmKeyLocationInfo deletedLocationInfo = deletedLocationList.get(idx); + OmKeyLocationInfo prevLocationInfo = prevLocationList.get(idx); + if (!deletedLocationInfo.hasSameBlockAs(prevLocationInfo)) { + return false; + } + } + return true; + } } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/filter/ReclaimableDirFilter.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/filter/ReclaimableDirFilter.java new file mode 100644 index 000000000000..3410d73c8148 --- /dev/null +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/filter/ReclaimableDirFilter.java @@ -0,0 +1,116 @@ +/* + * 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.snapshot.filter; + +import org.apache.hadoop.hdds.utils.db.Table; +import org.apache.hadoop.ozone.om.OMMetadataManager; +import org.apache.hadoop.ozone.om.OmSnapshot; +import org.apache.hadoop.ozone.om.OmSnapshotManager; +import org.apache.hadoop.ozone.om.OzoneManager; +import org.apache.hadoop.ozone.om.SnapshotChainManager; +import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; +import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; +import org.apache.hadoop.ozone.om.lock.IOzoneManagerLock; +import org.apache.hadoop.ozone.om.snapshot.ReferenceCounted; + +import java.io.IOException; + +/** + * Filter to return deleted directories which are reclaimable based on their presence in previous snapshot in + * the snapshot chain. + */ +public class ReclaimableDirFilter extends ReclaimableFilter { + + private final OzoneManager ozoneManager; + + /** + * Filter to return deleted directories which are reclaimable based on their presence in previous snapshot in + * the snapshot chain. + * + * @param omSnapshotManager + * @param snapshotChainManager + * @param currentSnapshotInfo : If null the deleted keys in AOS needs to be processed, hence the latest snapshot + * in the snapshot chain corresponding to bucket key needs to be processed. + * @param metadataManager : MetadataManager corresponding to snapshot or AOS. + * @param lock : Lock for Active OM. + */ + public ReclaimableDirFilter(OzoneManager ozoneManager, + OmSnapshotManager omSnapshotManager, SnapshotChainManager snapshotChainManager, + SnapshotInfo currentSnapshotInfo, OMMetadataManager metadataManager, + IOzoneManagerLock lock) { + super(ozoneManager, omSnapshotManager, snapshotChainManager, currentSnapshotInfo, metadataManager, lock, 1); + this.ozoneManager = ozoneManager; + } + + @Override + protected String getVolumeName(Table.KeyValue keyValue) throws IOException { + return keyValue.getValue().getVolumeName(); + } + + @Override + protected String getBucketName(Table.KeyValue keyValue) throws IOException { + return keyValue.getValue().getBucketName(); + } + + @Override + protected Boolean isReclaimable(Table.KeyValue deletedDirInfo) throws IOException { + ReferenceCounted previousSnapshot = getPreviousOmSnapshot(0); + Table prevDirTable = previousSnapshot == null ? null : + previousSnapshot.get().getMetadataManager().getDirectoryTable(); + return isDirReclaimable(deletedDirInfo, prevDirTable, + getMetadataManager().getSnapshotRenamedTable()); + } + + private boolean isDirReclaimable(Table.KeyValue deletedDir, + Table previousDirTable, + Table renamedTable) throws IOException { + if (previousDirTable == null) { + return true; + } + + String deletedDirDbKey = deletedDir.getKey(); + OmKeyInfo deletedDirInfo = deletedDir.getValue(); + String dbRenameKey = ozoneManager.getMetadataManager().getRenameKey( + deletedDirInfo.getVolumeName(), deletedDirInfo.getBucketName(), + deletedDirInfo.getObjectID()); + + /* + snapshotRenamedTable: /volumeName/bucketName/objectID -> + /volumeId/bucketId/parentId/dirName + */ + String dbKeyBeforeRename = renamedTable.getIfExist(dbRenameKey); + String prevDbKey = null; + + if (dbKeyBeforeRename != null) { + prevDbKey = dbKeyBeforeRename; + } else { + // In OMKeyDeleteResponseWithFSO OzonePathKey is converted to + // OzoneDeletePathKey. Changing it back to check the previous DirTable. + prevDbKey = ozoneManager.getMetadataManager() + .getOzoneDeletePathDirKey(deletedDirDbKey); + } + + OmDirectoryInfo prevDirectoryInfo = previousDirTable.get(prevDbKey); + if (prevDirectoryInfo == null) { + return true; + } + return prevDirectoryInfo.getObjectID() != deletedDirInfo.getObjectID(); + } +} diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/filter/ReclaimableFilter.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/filter/ReclaimableFilter.java new file mode 100644 index 000000000000..200a35c47724 --- /dev/null +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/filter/ReclaimableFilter.java @@ -0,0 +1,222 @@ +/* + * 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.snapshot.filter; + +import com.google.common.collect.Lists; +import org.apache.hadoop.hdds.utils.db.Table; +import org.apache.hadoop.ozone.om.OMMetadataManager; +import org.apache.hadoop.ozone.om.OmSnapshot; +import org.apache.hadoop.ozone.om.OmSnapshotManager; +import org.apache.hadoop.ozone.om.OzoneManager; +import org.apache.hadoop.ozone.om.SnapshotChainManager; +import org.apache.hadoop.ozone.om.helpers.OmBucketInfo; +import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; +import org.apache.hadoop.ozone.om.lock.IOzoneManagerLock; +import org.apache.hadoop.ozone.om.lock.MultiLocks; +import org.apache.hadoop.ozone.om.lock.OzoneManagerLock; +import org.apache.hadoop.ozone.om.snapshot.ReferenceCounted; +import org.apache.hadoop.ozone.om.snapshot.SnapshotUtils; +import org.apache.hadoop.ozone.util.CheckedExceptionOperation; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.UUID; +import java.util.stream.Collectors; + +/** + * This class is responsible for opening last N snapshot given snapshot or AOS metadata manager by acquiring a lock. + */ +public abstract class ReclaimableFilter implements CheckedExceptionOperation, + Boolean, IOException>, Closeable { + + private final OzoneManager ozoneManager; + private final SnapshotInfo currentSnapshotInfo; + private final OmSnapshotManager omSnapshotManager; + private final SnapshotChainManager snapshotChainManager; + + private final List previousSnapshotInfos; + private final List> previousOmSnapshots; + private final MultiLocks snapshotIdLocks; + private Long volumeId; + private OmBucketInfo bucketInfo; + private final OMMetadataManager metadataManager; + private final int numberOfPreviousSnapshotsFromChain; + + /** + * Filter to return deleted keys/directories which are reclaimable based on their presence in previous snapshot in + * the snapshot chain. + * + * @param omSnapshotManager + * @param snapshotChainManager + * @param currentSnapshotInfo : If null the deleted keys in AOS needs to be processed, hence the latest snapshot + * in the snapshot chain corresponding to bucket key needs to be processed. + * @param metadataManager : MetadataManager corresponding to snapshot or AOS. + * @param lock : Lock for Active OM. + */ + public ReclaimableFilter(OzoneManager ozoneManager, OmSnapshotManager omSnapshotManager, + SnapshotChainManager snapshotChainManager, + SnapshotInfo currentSnapshotInfo, OMMetadataManager metadataManager, + IOzoneManagerLock lock, + int numberOfPreviousSnapshotsFromChain) { + this.ozoneManager = ozoneManager; + this.omSnapshotManager = omSnapshotManager; + this.currentSnapshotInfo = currentSnapshotInfo; + this.snapshotChainManager = snapshotChainManager; + this.snapshotIdLocks = new MultiLocks<>(lock, OzoneManagerLock.Resource.SNAPSHOT_GC_LOCK, false); + this.metadataManager = metadataManager; + this.numberOfPreviousSnapshotsFromChain = numberOfPreviousSnapshotsFromChain; + this.previousOmSnapshots = new ArrayList<>(numberOfPreviousSnapshotsFromChain); + this.previousSnapshotInfos = new ArrayList<>(numberOfPreviousSnapshotsFromChain); + } + + private List getLastNSnapshotInChain(String volume, String bucket) throws IOException { + if (currentSnapshotInfo != null && + (!currentSnapshotInfo.getVolumeName().equals(volume) || !currentSnapshotInfo.getBucketName().equals(bucket))) { + throw new IOException("Volume & Bucket name for snapshot : " + currentSnapshotInfo + " not matching for " + + "key in volume: " + volume + " bucket: " + bucket); + } + SnapshotInfo expectedPreviousSnapshotInfo = currentSnapshotInfo == null + ? SnapshotUtils.getLatestSnapshotInfo(volume, bucket, ozoneManager, snapshotChainManager) + : SnapshotUtils.getPreviousSnapshot(ozoneManager, snapshotChainManager, currentSnapshotInfo); + List snapshotInfos = Lists.newArrayList(expectedPreviousSnapshotInfo); + SnapshotInfo snapshotInfo = expectedPreviousSnapshotInfo; + while (snapshotInfos.size() < numberOfPreviousSnapshotsFromChain) { + snapshotInfo = snapshotInfo == null ? null + : SnapshotUtils.getPreviousSnapshot(ozoneManager, snapshotChainManager, snapshotInfo); + snapshotInfos.add(snapshotInfo); + // If changes made to the snapshot have not been flushed to disk, throw exception immediately, next run of + // garbage collection would process the snapshot. + if (!OmSnapshotManager.areSnapshotChangesFlushedToDB(ozoneManager.getMetadataManager(), snapshotInfo)) { + throw new IOException("Changes made to the snapshot " + snapshotInfo + " have not been flushed to the disk "); + } + } + + // Reversing list to get the correct order in chain. To ensure locking order is as per the chain ordering. + Collections.reverse(snapshotInfos); + return snapshotInfos; + } + + private boolean validateExistingLastNSnapshotsInChain(String volume, String bucket) throws IOException { + List expectedLastNSnapshotsInChain = getLastNSnapshotInChain(volume, bucket); + List expectedSnapshotIds = expectedLastNSnapshotsInChain.stream() + .map(snapshotInfo -> snapshotInfo == null ? null : snapshotInfo.getSnapshotId()) + .collect(Collectors.toList()); + List existingSnapshotIds = previousOmSnapshots.stream() + .map(omSnapshotReferenceCounted -> omSnapshotReferenceCounted == null ? null : + omSnapshotReferenceCounted.get().getSnapshotID()).collect(Collectors.toList()); + return expectedSnapshotIds.equals(existingSnapshotIds); + } + + // Initialize the last N snapshots in the chain by acquiring locks. Throw IOException if it fails. + private void initializePreviousSnapshotsFromChain(String volume, String bucket) throws IOException { + // If existing snapshotIds don't match then close all snapshots and reopen the previous N snapshots. + if (!validateExistingLastNSnapshotsInChain(volume, bucket)) { + close(); + try { + // Acquire lock only on last N-1 snapshot & current snapshot(AOS if it is null). + List expectedLastNSnapshotsInChain = getLastNSnapshotInChain(volume, bucket); + List expectedSnapshotIds = expectedLastNSnapshotsInChain.stream() + .map(snapshotInfo -> snapshotInfo == null ? null : snapshotInfo.getSnapshotId()) + .collect(Collectors.toList()); + List lockIds = new ArrayList<>(expectedSnapshotIds.subList(1, expectedSnapshotIds.size())); + lockIds.add(currentSnapshotInfo == null ? null : currentSnapshotInfo.getSnapshotId()); + + if (snapshotIdLocks.acquireLock(lockIds).isLockAcquired()) { + for (SnapshotInfo snapshotInfo : expectedLastNSnapshotsInChain) { + if (snapshotInfo != null) { + // For AOS fail operation if any of the previous snapshots are not active. currentSnapshotInfo for + // AOS will be null. + previousOmSnapshots.add(currentSnapshotInfo == null + ? omSnapshotManager.getActiveSnapshot(snapshotInfo.getVolumeName(), snapshotInfo.getBucketName(), + snapshotInfo.getName()) + : omSnapshotManager.getSnapshot(snapshotInfo.getVolumeName(), snapshotInfo.getBucketName(), + snapshotInfo.getName())); + previousSnapshotInfos.add(snapshotInfo); + } else { + previousOmSnapshots.add(null); + previousSnapshotInfos.add(null); + } + + // TODO: Getting volumeId and bucket from active OM. This would be wrong on volume & bucket renames + // support. + volumeId = ozoneManager.getMetadataManager().getVolumeId(volume); + String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(volume, bucket); + bucketInfo = ozoneManager.getMetadataManager().getBucketTable().get(dbBucketKey); + } + } else { + throw new IOException("Lock acquisition failed for last N snapshots : " + + expectedLastNSnapshotsInChain + " " + currentSnapshotInfo); + } + } catch (IOException e) { + this.close(); + throw e; + } + } + } + + @Override + public Boolean apply(Table.KeyValue keyValue) throws IOException { + String volume = getVolumeName(keyValue); + String bucket = getBucketName(keyValue); + initializePreviousSnapshotsFromChain(volume, bucket); + boolean isReclaimable = isReclaimable(keyValue); + // This is to ensure the reclamation ran on the same previous snapshot and no change occurred in the chain + // while processing the entry. + return isReclaimable && validateExistingLastNSnapshotsInChain(volume, bucket); + } + + protected abstract String getVolumeName(Table.KeyValue keyValue) throws IOException; + + protected abstract String getBucketName(Table.KeyValue keyValue) throws IOException; + + protected abstract Boolean isReclaimable(Table.KeyValue omKeyInfo) throws IOException; + + @Override + public void close() throws IOException { + this.snapshotIdLocks.releaseLock(); + for (ReferenceCounted previousOmSnapshot : previousOmSnapshots) { + previousOmSnapshot.close(); + } + previousOmSnapshots.clear(); + previousSnapshotInfos.clear(); + } + + public ReferenceCounted getPreviousOmSnapshot(int index) { + return previousOmSnapshots.get(index); + } + + public OMMetadataManager getMetadataManager() { + return metadataManager; + } + + public Long getVolumeId() { + return volumeId; + } + + public OmBucketInfo getBucketInfo() { + return bucketInfo; + } + + public SnapshotInfo getPreviousSnapshotInfo(int index) { + return previousSnapshotInfos.get(index); + } +} diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/filter/ReclaimableKeyFilter.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/filter/ReclaimableKeyFilter.java new file mode 100644 index 000000000000..00c7d30de882 --- /dev/null +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/filter/ReclaimableKeyFilter.java @@ -0,0 +1,277 @@ +/* + * 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.snapshot.filter; + +import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.utils.db.Table; +import org.apache.hadoop.ozone.om.OMMetadataManager; +import org.apache.hadoop.ozone.om.OmSnapshot; +import org.apache.hadoop.ozone.om.OmSnapshotManager; +import org.apache.hadoop.ozone.om.OzoneManager; +import org.apache.hadoop.ozone.om.SnapshotChainManager; +import org.apache.hadoop.ozone.om.helpers.OmBucketInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; +import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; +import org.apache.hadoop.ozone.om.lock.IOzoneManagerLock; +import org.apache.hadoop.ozone.om.snapshot.ReferenceCounted; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import static org.apache.hadoop.ozone.OzoneConsts.OBJECT_ID_RECLAIM_BLOCKS; +import static org.apache.hadoop.ozone.om.snapshot.SnapshotUtils.isBlockLocationInfoSame; + +/** + * Filter to return deleted keys which are reclaimable based on their presence in previous snapshot in + * the snapshot chain. + */ +public class ReclaimableKeyFilter extends ReclaimableFilter { + private final OzoneManager ozoneManager; + private final Map exclusiveSizeMap; + private final Map exclusiveReplicatedSizeMap; + + /** + * @param omSnapshotManager + * @param snapshotChainManager + * @param currentSnapshotInfo : If null the deleted keys in AOS needs to be processed, hence the latest snapshot + * in the snapshot chain corresponding to bucket key needs to be processed. + * @param metadataManager : MetadataManager corresponding to snapshot or AOS. + * @param lock : Lock for Active OM. + */ + public ReclaimableKeyFilter(OzoneManager ozoneManager, + OmSnapshotManager omSnapshotManager, SnapshotChainManager snapshotChainManager, + SnapshotInfo currentSnapshotInfo, OMMetadataManager metadataManager, + IOzoneManagerLock lock) { + super(ozoneManager, omSnapshotManager, snapshotChainManager, currentSnapshotInfo, metadataManager, lock, 2); + this.ozoneManager = ozoneManager; + this.exclusiveSizeMap = new HashMap<>(); + this.exclusiveReplicatedSizeMap = new HashMap<>(); + } + + @Override + protected String getVolumeName(Table.KeyValue keyValue) throws IOException { + return keyValue.getValue().getVolumeName(); + } + + @Override + protected String getBucketName(Table.KeyValue keyValue) throws IOException { + return keyValue.getValue().getBucketName(); + } + + @Override + protected Boolean isReclaimable(Table.KeyValue deletedKeyInfo) throws IOException { + ReferenceCounted previousSnapshot = getPreviousOmSnapshot(1); + ReferenceCounted previousToPreviousSnapshot = getPreviousOmSnapshot(0); + + Table previousKeyTable = null; + Table previousPrevKeyTable = null; + + Table renamedTable = getMetadataManager().getSnapshotRenamedTable(); + Table prevRenamedTable = null; + + SnapshotInfo previousSnapshotInfo = getPreviousSnapshotInfo(1); + SnapshotInfo prevPrevSnapshotInfo = getPreviousSnapshotInfo(0); + + if (previousSnapshot != null) { + previousKeyTable = previousSnapshot.get().getMetadataManager().getKeyTable(getBucketInfo().getBucketLayout()); + prevRenamedTable = previousSnapshot.get().getMetadataManager().getSnapshotRenamedTable(); + } + if (previousToPreviousSnapshot != null) { + previousPrevKeyTable = previousToPreviousSnapshot.get().getMetadataManager() + .getKeyTable(getBucketInfo().getBucketLayout()); + } + if (isKeyReclaimable(previousKeyTable, renamedTable, deletedKeyInfo.getValue(), + getBucketInfo(), getVolumeId(), + null)) { + return true; + } + calculateExclusiveSize(previousSnapshotInfo, prevPrevSnapshotInfo, deletedKeyInfo.getValue(), getBucketInfo(), + getVolumeId(), renamedTable, previousKeyTable, prevRenamedTable, previousPrevKeyTable, exclusiveSizeMap, + exclusiveReplicatedSizeMap); + return false; + } + + + public Map getExclusiveSizeMap() { + return exclusiveSizeMap; + } + + public Map getExclusiveReplicatedSizeMap() { + return exclusiveReplicatedSizeMap; + } + + private boolean isKeyReclaimable( + Table previousKeyTable, + Table renamedTable, + OmKeyInfo deletedKeyInfo, OmBucketInfo bucketInfo, + long volumeId, HddsProtos.KeyValue.Builder renamedKeyBuilder) + throws IOException { + + String dbKey; + // Handle case when the deleted snapshot is the first snapshot. + if (previousKeyTable == null) { + return true; + } + + // These are uncommitted blocks wrapped into a pseudo KeyInfo + if (deletedKeyInfo.getObjectID() == OBJECT_ID_RECLAIM_BLOCKS) { + return true; + } + + // Construct keyTable or fileTable DB key depending on the bucket type + if (bucketInfo.getBucketLayout().isFileSystemOptimized()) { + dbKey = ozoneManager.getMetadataManager().getOzonePathKey( + volumeId, + bucketInfo.getObjectID(), + deletedKeyInfo.getParentObjectID(), + deletedKeyInfo.getFileName()); + } else { + dbKey = ozoneManager.getMetadataManager().getOzoneKey( + deletedKeyInfo.getVolumeName(), + deletedKeyInfo.getBucketName(), + deletedKeyInfo.getKeyName()); + } + + /* + snapshotRenamedTable: + 1) /volumeName/bucketName/objectID -> + /volumeId/bucketId/parentId/fileName (FSO) + 2) /volumeName/bucketName/objectID -> + /volumeName/bucketName/keyName (non-FSO) + */ + String dbRenameKey = ozoneManager.getMetadataManager().getRenameKey( + deletedKeyInfo.getVolumeName(), deletedKeyInfo.getBucketName(), + deletedKeyInfo.getObjectID()); + + // Condition: key should not exist in snapshotRenamedTable + // of the current snapshot and keyTable of the previous snapshot. + // Check key exists in renamedTable of the Snapshot + String renamedKey = renamedTable.getIfExist(dbRenameKey); + + if (renamedKey != null && renamedKeyBuilder != null) { + renamedKeyBuilder.setKey(dbRenameKey).setValue(renamedKey); + } + // previousKeyTable is fileTable if the bucket is FSO, + // otherwise it is the keyTable. + OmKeyInfo prevKeyInfo = renamedKey != null ? previousKeyTable + .get(renamedKey) : previousKeyTable.get(dbKey); + + if (prevKeyInfo == null || + prevKeyInfo.getObjectID() != deletedKeyInfo.getObjectID()) { + return true; + } + + // For key overwrite the objectID will remain the same, In this + // case we need to check if OmKeyLocationInfo is also same. + return !isBlockLocationInfoSame(prevKeyInfo, deletedKeyInfo); + } + + /** + * 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") + public void calculateExclusiveSize( + SnapshotInfo previousSnapshot, + SnapshotInfo previousToPrevSnapshot, + OmKeyInfo keyInfo, + OmBucketInfo bucketInfo, long volumeId, + Table snapRenamedTable, + Table previousKeyTable, + Table prevRenamedTable, + Table previousToPrevKeyTable, + Map exclusiveSizes, + Map exclusiveReplicatedSizes) throws IOException { + String prevSnapKey = previousSnapshot.getTableKey(); + long exclusiveReplicatedSize = exclusiveReplicatedSizes.getOrDefault( + prevSnapKey, 0L) + keyInfo.getReplicatedSize(); + long exclusiveSize = exclusiveSizes.getOrDefault(prevSnapKey, 0L) + keyInfo.getDataSize(); + + // If there is no previous to previous snapshot, then + // the previous snapshot is the first snapshot. + if (previousToPrevSnapshot == null) { + exclusiveSizes.put(prevSnapKey, exclusiveSize); + exclusiveReplicatedSizes.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) { + exclusiveSizes.put(prevSnapKey, exclusiveSize); + exclusiveReplicatedSizes.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 = ozoneManager.getMetadataManager().getOzonePathKey( + volumeId, + bucketInfo.getObjectID(), + keyInfo.getParentObjectID(), + keyInfo.getFileName()); + } else { + dbKeyPrevSnap = ozoneManager.getMetadataManager().getOzoneKey( + keyInfo.getVolumeName(), + keyInfo.getBucketName(), + keyInfo.getKeyName()); + } + + String dbRenameKey = ozoneManager.getMetadataManager().getRenameKey( + keyInfo.getVolumeName(), + keyInfo.getBucketName(), + keyInfo.getObjectID()); + + String renamedKey = snapRenamedTable.getIfExist(dbRenameKey); + OmKeyInfo prevKeyInfo = renamedKey != null ? previousKeyTable.get(renamedKey) : previousKeyTable.get(dbKeyPrevSnap); + + if (prevKeyInfo == null || prevKeyInfo.getObjectID() != keyInfo.getObjectID()) { + return null; + } + + return isBlockLocationInfoSame(prevKeyInfo, keyInfo) ? prevKeyInfo : null; + } +} diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/filter/ReclaimableRenameEntryFilter.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/filter/ReclaimableRenameEntryFilter.java new file mode 100644 index 000000000000..48b08ed5bf42 --- /dev/null +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/filter/ReclaimableRenameEntryFilter.java @@ -0,0 +1,103 @@ +/* + * 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.snapshot.filter; + +import org.apache.hadoop.hdds.utils.db.Table; +import org.apache.hadoop.ozone.om.OMMetadataManager; +import org.apache.hadoop.ozone.om.OmSnapshot; +import org.apache.hadoop.ozone.om.OmSnapshotManager; +import org.apache.hadoop.ozone.om.OzoneManager; +import org.apache.hadoop.ozone.om.SnapshotChainManager; +import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo; +import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; +import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; +import org.apache.hadoop.ozone.om.lock.IOzoneManagerLock; +import org.apache.hadoop.ozone.om.snapshot.ReferenceCounted; + +import java.io.IOException; + +/** + * Filter to return rename table entries which are reclaimable based on the key presence in previous snapshot's + * keyTable/DirectoryTable in the snapshot chain. + */ +public class ReclaimableRenameEntryFilter extends ReclaimableFilter { + + /** + * + * + * @param omSnapshotManager + * @param snapshotChainManager + * @param currentSnapshotInfo : If null the deleted keys in AOS needs to be processed, hence the latest snapshot + * in the snapshot chain corresponding to bucket key needs to be processed. + * @param metadataManager : MetadataManager corresponding to snapshot or AOS. + * @param lock : Lock for Active OM. + */ + public ReclaimableRenameEntryFilter(OzoneManager ozoneManager, + OmSnapshotManager omSnapshotManager, SnapshotChainManager snapshotChainManager, + SnapshotInfo currentSnapshotInfo, OMMetadataManager metadataManager, + IOzoneManagerLock lock) { + super(ozoneManager, omSnapshotManager, snapshotChainManager, currentSnapshotInfo, metadataManager, lock, 1); + } + + @Override + protected Boolean isReclaimable(Table.KeyValue renameEntry) throws IOException { + ReferenceCounted previousSnapshot = getPreviousOmSnapshot(0); + Table previousKeyTable = null; + Table prevDirTable = null; + if (previousSnapshot != null) { + previousKeyTable = previousSnapshot.get().getMetadataManager().getKeyTable(getBucketInfo().getBucketLayout()); + prevDirTable = previousSnapshot.get().getMetadataManager().getDirectoryTable(); + } + return isRenameEntryReclaimable(renameEntry, prevDirTable, previousKeyTable); + } + + @Override + protected String getVolumeName(Table.KeyValue keyValue) throws IOException { + return getMetadataManager().splitRenameKey(keyValue.getKey())[0]; + } + + @Override + protected String getBucketName(Table.KeyValue keyValue) throws IOException { + return getMetadataManager().splitRenameKey(keyValue.getKey())[1]; + } + + private boolean isRenameEntryReclaimable(Table.KeyValue renameEntry, + Table previousDirTable, + Table prevKeyInfoTable) throws IOException { + + if (previousDirTable == null && prevKeyInfoTable == null) { + return true; + } + String prevDbKey = renameEntry.getValue(); + + + if (previousDirTable != null) { + OmDirectoryInfo prevDirectoryInfo = previousDirTable.getIfExist(prevDbKey); + if (prevDirectoryInfo != null) { + return false; + } + } + + if (prevKeyInfoTable != null) { + OmKeyInfo omKeyInfo = prevKeyInfoTable.getIfExist(prevDbKey); + return omKeyInfo == null; + } + return true; + } +} diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/filter/package-info.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/filter/package-info.java new file mode 100644 index 000000000000..700f7b9c6d05 --- /dev/null +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/filter/package-info.java @@ -0,0 +1,22 @@ +/* + * 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 containing filter to perform reclaimable check on snapshots. + */ +package org.apache.hadoop.ozone.om.snapshot.filter; diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/OMRequestTestUtils.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/OMRequestTestUtils.java index eff23a18e6e2..483f95c0e292 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/OMRequestTestUtils.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/OMRequestTestUtils.java @@ -289,6 +289,16 @@ public static void addKeyToTable(boolean openKeyTable, boolean addToCache, omMetadataManager); } + /** + * Add key entry to SnapshotRenamedTable. + */ + public static String addRenamedEntryToTable(long trxnLogIndex, String volumeName, String bucketName, String key, + OMMetadataManager omMetadataManager) throws Exception { + String renameKey = omMetadataManager.getRenameKey(volumeName, bucketName, trxnLogIndex); + omMetadataManager.getSnapshotRenamedTable().put(renameKey, key); + return renameKey; + } + /** * Add key entry to KeyTable. if openKeyTable flag is true, add's entries * to openKeyTable, else add's it to keyTable. diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyPurgeRequestAndResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyPurgeRequestAndResponse.java index c323fecd5015..928a14efde78 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyPurgeRequestAndResponse.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyPurgeRequestAndResponse.java @@ -23,6 +23,7 @@ import java.util.List; import java.util.UUID; +import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.hdds.utils.TransactionInfo; import org.apache.hadoop.ozone.om.OmSnapshot; import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; @@ -56,7 +57,7 @@ public class TestOMKeyPurgeRequestAndResponse extends TestOMKeyRequest { * Creates volume, bucket and key entries and adds to OM DB and then * deletes these keys to move them to deletedKeys table. */ - private List createAndDeleteKeys(Integer trxnIndex, String bucket) + private Pair, List> createAndDeleteKeysAndRenamedEntry(Integer trxnIndex, String bucket) throws Exception { if (bucket == null) { bucket = bucketName; @@ -66,11 +67,14 @@ private List createAndDeleteKeys(Integer trxnIndex, String bucket) omMetadataManager); List ozoneKeyNames = new ArrayList<>(numKeys); + List renamedEntries = new ArrayList<>(numKeys); for (int i = 1; i <= numKeys; i++) { String key = keyName + "-" + i; OMRequestTestUtils.addKeyToTable(false, false, volumeName, bucket, key, clientID, replicationConfig, trxnIndex++, omMetadataManager); + renamedEntries.add(OMRequestTestUtils.addRenamedEntryToTable(trxnIndex, volumeName, bucket, key, + omMetadataManager)); ozoneKeyNames.add(omMetadataManager.getOzoneKey( volumeName, bucket, key)); } @@ -82,14 +86,14 @@ private List createAndDeleteKeys(Integer trxnIndex, String bucket) deletedKeyNames.add(deletedKeyName); } - return deletedKeyNames; + return Pair.of(deletedKeyNames, renamedEntries); } /** * Create OMRequest which encapsulates DeleteKeyRequest. * @return OMRequest */ - private OMRequest createPurgeKeysRequest(List deletedKeys, + private OMRequest createPurgeKeysRequest(List deletedKeys, List renamedEntries, String snapshotDbKey) { DeletedKeys deletedKeysInBucket = DeletedKeys.newBuilder() .setVolumeName(volumeName) @@ -97,7 +101,7 @@ private OMRequest createPurgeKeysRequest(List deletedKeys, .addAllKeys(deletedKeys) .build(); PurgeKeysRequest.Builder purgeKeysRequest = PurgeKeysRequest.newBuilder() - .addDeletedKeys(deletedKeysInBucket); + .addDeletedKeys(deletedKeysInBucket).addAllRenamedKeys(renamedEntries); if (snapshotDbKey != null) { purgeKeysRequest.setSnapshotTableKey(snapshotDbKey); @@ -126,16 +130,20 @@ private OMRequest preExecute(OMRequest originalOmRequest) throws IOException { @Test public void testValidateAndUpdateCache() throws Exception { // Create and Delete keys. The keys should be moved to DeletedKeys table - List deletedKeyNames = createAndDeleteKeys(1, null); + Pair, List> deleteKeysAndRenamedEntry = createAndDeleteKeysAndRenamedEntry(1, null); // The keys should be present in the DeletedKeys table before purging - for (String deletedKey : deletedKeyNames) { + for (String deletedKey : deleteKeysAndRenamedEntry.getKey()) { assertTrue(omMetadataManager.getDeletedTable().isExist( deletedKey)); } + for (String renamedKey : deleteKeysAndRenamedEntry.getValue()) { + assertTrue(omMetadataManager.getSnapshotRenamedTable().isExist(renamedKey)); + } // Create PurgeKeysRequest to purge the deleted keys - OMRequest omRequest = createPurgeKeysRequest(deletedKeyNames, null); + OMRequest omRequest = createPurgeKeysRequest(deleteKeysAndRenamedEntry.getKey(), + deleteKeysAndRenamedEntry.getValue(), null); OMRequest preExecutedRequest = preExecute(omRequest); OMKeyPurgeRequest omKeyPurgeRequest = @@ -153,7 +161,8 @@ public void testValidateAndUpdateCache() throws Exception { omMetadataManager.getStore().initBatchOperation()) { OMKeyPurgeResponse omKeyPurgeResponse = new OMKeyPurgeResponse( - omResponse, deletedKeyNames, null, null); + omResponse, deleteKeysAndRenamedEntry.getKey(), deleteKeysAndRenamedEntry.getValue(), null, + null); omKeyPurgeResponse.addToDBBatch(omMetadataManager, batchOperation); // Do manual commit and see whether addToBatch is successful or not. @@ -161,37 +170,49 @@ public void testValidateAndUpdateCache() throws Exception { } // The keys should not exist in the DeletedKeys table - for (String deletedKey : deletedKeyNames) { + for (String deletedKey : deleteKeysAndRenamedEntry.getKey()) { assertFalse(omMetadataManager.getDeletedTable().isExist(deletedKey)); } + // Renamed entry should not exist + for (String renamedKey : deleteKeysAndRenamedEntry.getValue()) { + assertFalse(omMetadataManager.getSnapshotRenamedTable().isExist(renamedKey)); + } } @Test public void testKeyPurgeInSnapshot() throws Exception { // Create and Delete keys. The keys should be moved to DeletedKeys table - List deletedKeyNames = createAndDeleteKeys(1, null); + Pair, List> deleteKeysAndRenamedEntry = createAndDeleteKeysAndRenamedEntry(1, null); SnapshotInfo snapInfo = createSnapshot("snap1"); assertEquals(snapInfo.getLastTransactionInfo(), TransactionInfo.valueOf(TransactionInfo.getTermIndex(1L)).toByteString()); // The keys should be not present in the active Db's deletedTable - for (String deletedKey : deletedKeyNames) { + for (String deletedKey : deleteKeysAndRenamedEntry.getKey()) { assertFalse(omMetadataManager.getDeletedTable().isExist(deletedKey)); } + for (String renamedKey : deleteKeysAndRenamedEntry.getValue()) { + assertFalse(omMetadataManager.getSnapshotRenamedTable().isExist(renamedKey)); + } ReferenceCounted rcOmSnapshot = ozoneManager.getOmSnapshotManager() .getSnapshot(snapInfo.getVolumeName(), snapInfo.getBucketName(), snapInfo.getName()); OmSnapshot omSnapshot = rcOmSnapshot.get(); // The keys should be present in the snapshot's deletedTable - for (String deletedKey : deletedKeyNames) { + for (String deletedKey : deleteKeysAndRenamedEntry.getKey()) { assertTrue(omSnapshot.getMetadataManager() .getDeletedTable().isExist(deletedKey)); } + // The keys should be present in the snapshot's deletedTable + for (String renamedKey : deleteKeysAndRenamedEntry.getValue()) { + assertTrue(omSnapshot.getMetadataManager() + .getSnapshotRenamedTable().isExist(renamedKey)); + } // Create PurgeKeysRequest to purge the deleted keys - OMRequest omRequest = createPurgeKeysRequest(deletedKeyNames, - snapInfo.getTableKey()); + OMRequest omRequest = createPurgeKeysRequest(deleteKeysAndRenamedEntry.getKey(), + deleteKeysAndRenamedEntry.getValue(), snapInfo.getTableKey()); OMRequest preExecutedRequest = preExecute(omRequest); OMKeyPurgeRequest omKeyPurgeRequest = @@ -214,7 +235,8 @@ public void testKeyPurgeInSnapshot() throws Exception { try (BatchOperation batchOperation = omMetadataManager.getStore().initBatchOperation()) { - OMKeyPurgeResponse omKeyPurgeResponse = new OMKeyPurgeResponse(omResponse, deletedKeyNames, snapInfo, null); + OMKeyPurgeResponse omKeyPurgeResponse = new OMKeyPurgeResponse(omResponse, deleteKeysAndRenamedEntry.getKey(), + deleteKeysAndRenamedEntry.getValue(), snapInfo, null); omKeyPurgeResponse.addToDBBatch(omMetadataManager, batchOperation); // Do manual commit and see whether addToBatch is successful or not. @@ -223,11 +245,16 @@ public void testKeyPurgeInSnapshot() throws Exception { snapshotInfoOnDisk = omMetadataManager.getSnapshotInfoTable().getSkipCache(snapInfo.getTableKey()); assertEquals(snapshotInfoOnDisk, snapInfo); // The keys should not exist in the DeletedKeys table - for (String deletedKey : deletedKeyNames) { + for (String deletedKey : deleteKeysAndRenamedEntry.getKey()) { assertFalse(omSnapshot.getMetadataManager() .getDeletedTable().isExist(deletedKey)); } + for (String renamedEntry : deleteKeysAndRenamedEntry.getValue()) { + assertFalse(omSnapshot.getMetadataManager() + .getSnapshotRenamedTable().isExist(renamedEntry)); + } + omSnapshot = null; rcOmSnapshot.close(); } 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 ff6506da0347..9fdcece46661 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 @@ -57,6 +57,7 @@ import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; import org.apache.hadoop.ozone.om.request.OMRequestTestUtils; import org.apache.hadoop.ozone.om.snapshot.ReferenceCounted; +import org.apache.hadoop.ozone.util.CheckedExceptionOperation; import org.apache.ozone.test.OzoneTestBase; import org.apache.ratis.util.ExitUtils; import org.junit.jupiter.api.AfterAll; @@ -203,7 +204,7 @@ void checkIfDeleteServiceIsDeletingKeys() () -> getDeletedKeyCount() >= initialDeletedCount + keyCount, 100, 10000); assertThat(getRunCount()).isGreaterThan(initialRunCount); - assertThat(keyManager.getPendingDeletionKeys(Integer.MAX_VALUE).getKeyBlocksList()) + assertThat(keyManager.getPendingDeletionKeys((kv) -> true, Integer.MAX_VALUE).getKeyBlocksList()) .isEmpty(); } @@ -232,7 +233,7 @@ void checkDeletionForKeysWithMultipleVersions() throws Exception { 1000, 10000); assertThat(getRunCount()) .isGreaterThan(initialRunCount); - assertThat(keyManager.getPendingDeletionKeys(Integer.MAX_VALUE).getKeyBlocksList()) + assertThat(keyManager.getPendingDeletionKeys((kv) -> true, Integer.MAX_VALUE).getKeyBlocksList()) .isEmpty(); // The 1st version of the key has 1 block and the 2nd version has 2 @@ -274,7 +275,7 @@ void checkDeletedTableCleanUpForSnapshot() throws Exception { 1000, 10000); assertThat(getRunCount()) .isGreaterThan(initialRunCount); - assertThat(keyManager.getPendingDeletionKeys(Integer.MAX_VALUE).getKeyBlocksList()) + assertThat(keyManager.getPendingDeletionKeys(null, Integer.MAX_VALUE).getKeyBlocksList()) .isEmpty(); // deletedTable should have deleted key of the snapshot bucket @@ -381,7 +382,8 @@ public void testAOSKeyDeletingWithSnapshotCreateParallelExecution() Assertions.assertNotEquals(deletePathKey[0], group.getGroupID()); } return pendingKeysDeletion; - }).when(km).getPendingDeletionKeys(anyInt()); + }).when(km).getPendingDeletionKeys(Mockito.anyString(), Mockito.anyString(), Mockito.anyString(), + Mockito.any(CheckedExceptionOperation.class), anyInt()); service.runPeriodicalTaskNow(); service.runPeriodicalTaskNow(); assertTableRowCount(snapshotInfoTable, initialSnapshotCount + 2, metadataManager); @@ -882,7 +884,7 @@ private long getRunCount() { private int countKeysPendingDeletion() { try { - final int count = keyManager.getPendingDeletionKeys(Integer.MAX_VALUE) + final int count = keyManager.getPendingDeletionKeys(null, Integer.MAX_VALUE) .getKeyBlocksList().size(); LOG.debug("KeyManager keys pending deletion: {}", count); return count; @@ -893,7 +895,7 @@ private int countKeysPendingDeletion() { private long countBlocksPendingDeletion() { try { - return keyManager.getPendingDeletionKeys(Integer.MAX_VALUE) + return keyManager.getPendingDeletionKeys(null, Integer.MAX_VALUE) .getKeyBlocksList() .stream() .map(BlockGroup::getBlockIDList) diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestSnapshotDiffManager.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestSnapshotDiffManager.java index 0f2ab6150669..7c342a87b9d6 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestSnapshotDiffManager.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestSnapshotDiffManager.java @@ -799,7 +799,7 @@ public void testGenerateDiffReport() throws IOException { mockedSnapshotDeletingService = mockStatic( SnapshotDeletingService.class)) { mockedSnapshotDeletingService.when(() -> - SnapshotDeletingService.isBlockLocationInfoSame(any(OmKeyInfo.class), + SnapshotUtils.isBlockLocationInfoSame(any(OmKeyInfo.class), any(OmKeyInfo.class))) .thenAnswer(i -> { int keyVal = Integer.parseInt(((OmKeyInfo)i.getArgument(0))