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 b399d6bb9ceb..da080be68cac 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 @@ -131,6 +131,7 @@ import org.apache.hadoop.hdds.utils.BackgroundService; import org.apache.hadoop.hdds.utils.db.StringCodec; 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.hdds.utils.db.cache.CacheKey; import org.apache.hadoop.hdds.utils.db.cache.CacheValue; @@ -160,6 +161,7 @@ import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils; import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus; import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo; +import org.apache.hadoop.ozone.om.helpers.WithParentObjectId; import org.apache.hadoop.ozone.om.request.OMClientRequest; import org.apache.hadoop.ozone.om.request.file.OMFileRequest; import org.apache.hadoop.ozone.om.request.util.OMMultipartUploadUtils; @@ -734,7 +736,7 @@ public ListKeysResult listKeys(String volumeName, String bucketName, @Override public PendingKeysDeletion getPendingDeletionKeys( - final CheckedFunction, Boolean, IOException> filter, final int count) + final CheckedFunction, Boolean, IOException> filter, final int count) throws IOException { return getPendingDeletionKeys(null, null, null, filter, count); } @@ -742,13 +744,13 @@ public PendingKeysDeletion getPendingDeletionKeys( @Override public PendingKeysDeletion getPendingDeletionKeys( String volume, String bucket, String startKey, - CheckedFunction, Boolean, IOException> filter, + CheckedFunction, Boolean, IOException> filter, int count) throws IOException { List keyBlocksList = Lists.newArrayList(); Map keysToModify = new HashMap<>(); // Bucket prefix would be empty if volume is empty i.e. either null or "". Optional bucketPrefix = getBucketPrefix(volume, bucket, false); - try (TableIterator> + 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 @@ -760,7 +762,7 @@ public PendingKeysDeletion getPendingDeletionKeys( int currentCount = 0; while (delKeyIter.hasNext() && currentCount < count) { RepeatedOmKeyInfo notReclaimableKeyInfo = new RepeatedOmKeyInfo(); - Table.KeyValue kv = delKeyIter.next(); + KeyValue kv = delKeyIter.next(); if (kv != null) { List blockGroupList = Lists.newArrayList(); // Multiple keys with the same path can be queued in one DB entry @@ -795,12 +797,12 @@ public PendingKeysDeletion getPendingDeletionKeys( return new PendingKeysDeletion(keyBlocksList, keysToModify); } - private List> getTableEntries(String startKey, - TableIterator> tableIterator, + private List> getTableEntries(String startKey, + TableIterator> tableIterator, Function valueFunction, - CheckedFunction, Boolean, IOException> filter, + CheckedFunction, Boolean, IOException> filter, int size) throws IOException { - List> entries = new ArrayList<>(); + List> entries = new ArrayList<>(); /* Seek to the start key if it's 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. */ @@ -811,7 +813,7 @@ private List> getTableEntries(String startKey, } int currentCount = 0; while (tableIterator.hasNext() && currentCount < size) { - Table.KeyValue kv = tableIterator.next(); + KeyValue kv = tableIterator.next(); if (kv != null && filter.apply(kv)) { entries.add(Table.newKeyValue(kv.getKey(), valueFunction.apply(kv.getValue()))); currentCount++; @@ -833,11 +835,11 @@ private Optional getBucketPrefix(String volumeName, String bucketName, b } @Override - public List> getRenamesKeyEntries( + public List> getRenamesKeyEntries( String volume, String bucket, String startKey, - CheckedFunction, Boolean, IOException> filter, int size) throws IOException { + CheckedFunction, Boolean, IOException> filter, int size) throws IOException { Optional bucketPrefix = getBucketPrefix(volume, bucket, false); - try (TableIterator> + try (TableIterator> renamedKeyIter = metadataManager.getSnapshotRenamedTable().iterator(bucketPrefix.orElse(""))) { return getTableEntries(startKey, renamedKeyIter, Function.identity(), filter, size); } @@ -882,12 +884,12 @@ private CheckedFunction getPreviousSnapshotOzone } @Override - public List>> getDeletedKeyEntries( + public List>> getDeletedKeyEntries( String volume, String bucket, String startKey, - CheckedFunction, Boolean, IOException> filter, + CheckedFunction, Boolean, IOException> filter, int size) throws IOException { Optional bucketPrefix = getBucketPrefix(volume, bucket, false); - try (TableIterator> + try (TableIterator> delKeyIter = metadataManager.getDeletedTable().iterator(bucketPrefix.orElse(""))) { return getTableEntries(startKey, delKeyIter, RepeatedOmKeyInfo::cloneOmKeyInfoList, filter, size); } @@ -1537,10 +1539,10 @@ private OmKeyInfo createFakeDirIfShould(String volume, String bucket, } } - try (TableIterator> + try (TableIterator> keyTblItr = keyTable.iterator(targetKey)) { while (keyTblItr.hasNext()) { - Table.KeyValue keyValue = keyTblItr.next(); + KeyValue keyValue = keyTblItr.next(); if (keyValue != null) { String key = keyValue.getKey(); // HDDS-7871: RocksIterator#seek() may position at the key @@ -1851,7 +1853,7 @@ public List listStatus(OmKeyArgs args, boolean recursive, String keyArgs = OzoneFSUtils.addTrailingSlashIfNeeded( metadataManager.getOzoneKey(volumeName, bucketName, keyName)); - TableIterator> iterator; + TableIterator> iterator; Table keyTable; metadataManager.getLock().acquireReadLock(BUCKET_LOCK, volumeName, bucketName); @@ -1908,12 +1910,12 @@ public List listStatus(OmKeyArgs args, boolean recursive, return fileStatusList; } - private TableIterator> + private TableIterator> getIteratorForKeyInTableCache( boolean recursive, String startKey, String volumeName, String bucketName, TreeMap cacheKeyMap, String keyArgs, Table keyTable) throws IOException { - TableIterator> iterator; + TableIterator> iterator; Iterator, CacheValue>> cacheIter = keyTable.cacheIterator(); String startCacheKey = metadataManager.getOzoneKey(volumeName, bucketName, startKey); @@ -1931,12 +1933,12 @@ private void findKeyInDbWithIterator(boolean recursive, String startKey, TreeMap cacheKeyMap, String keyArgs, Table keyTable, TableIterator> iterator) + ? extends KeyValue> iterator) throws IOException { // Then, find key in DB String seekKeyInDb = metadataManager.getOzoneKey(volumeName, bucketName, startKey); - Table.KeyValue entry = iterator.seek(seekKeyInDb); + KeyValue entry = iterator.seek(seekKeyInDb); int countEntries = 0; if (iterator.hasNext()) { if (entry.getKey().equals(keyArgs)) { @@ -2187,7 +2189,7 @@ private void slimLocationVersion(OmKeyInfo... keyInfos) { } @Override - public TableIterator> getDeletedDirEntries( + public TableIterator> getDeletedDirEntries( String volume, String bucket) throws IOException { Optional bucketPrefix = getBucketPrefix(volume, bucket, true); return metadataManager.getDeletedDirTable().iterator(bucketPrefix.orElse("")); @@ -2196,101 +2198,52 @@ private void slimLocationVersion(OmKeyInfo... keyInfos) { @Override public DeleteKeysResult getPendingDeletionSubDirs(long volumeId, long bucketId, OmKeyInfo parentInfo, long remainingBufLimit) throws IOException { - String seekDirInDB = metadataManager.getOzonePathKey(volumeId, bucketId, - parentInfo.getObjectID(), ""); - long countEntries = 0; - - Table dirTable = metadataManager.getDirectoryTable(); - try (TableIterator> - iterator = dirTable.iterator(seekDirInDB)) { - return gatherSubDirsWithIterator(parentInfo, countEntries, iterator, remainingBufLimit); - } - + return gatherSubPathsWithIterator(volumeId, bucketId, parentInfo, metadataManager.getDirectoryTable(), + omDirectoryInfo -> OMFileRequest.getKeyInfoWithFullPath(parentInfo, omDirectoryInfo), remainingBufLimit); } - private DeleteKeysResult gatherSubDirsWithIterator(OmKeyInfo parentInfo, - long countEntries, - TableIterator> iterator, long remainingBufLimit) - throws IOException { - List directories = new ArrayList<>(); - long consumedSize = 0; - boolean processedSubDirs = false; - - while (iterator.hasNext() && remainingBufLimit > 0) { - Table.KeyValue entry = iterator.next(); - OmDirectoryInfo dirInfo = entry.getValue(); - long objectSerializedSize = entry.getRawSize(); - if (!OMFileRequest.isImmediateChild(dirInfo.getParentObjectID(), - parentInfo.getObjectID())) { - processedSubDirs = true; - break; - } - if (!metadataManager.getDirectoryTable().isExist(entry.getKey())) { - continue; - } - if (remainingBufLimit - objectSerializedSize < 0) { - break; - } - String dirName = OMFileRequest.getAbsolutePath(parentInfo.getKeyName(), - dirInfo.getName()); - OmKeyInfo omKeyInfo = OMFileRequest.getOmKeyInfo( - parentInfo.getVolumeName(), parentInfo.getBucketName(), dirInfo, - dirName); - directories.add(omKeyInfo); - countEntries++; - remainingBufLimit -= objectSerializedSize; - consumedSize += objectSerializedSize; - } - - processedSubDirs = processedSubDirs || (!iterator.hasNext()); - - return new DeleteKeysResult(directories, consumedSize, processedSubDirs); - } - - @Override - public DeleteKeysResult getPendingDeletionSubFiles(long volumeId, - long bucketId, OmKeyInfo parentInfo, long remainingBufLimit) - throws IOException { - List files = new ArrayList<>(); + private DeleteKeysResult gatherSubPathsWithIterator( + long volumeId, long bucketId, OmKeyInfo parentInfo, + Table table, Function deleteKeyTransformer, + long remainingBufLimit) throws IOException { + List keyInfos = new ArrayList<>(); String seekFileInDB = metadataManager.getOzonePathKey(volumeId, bucketId, parentInfo.getObjectID(), ""); long consumedSize = 0; - boolean processedSubFiles = false; - - Table fileTable = metadataManager.getFileTable(); - try (TableIterator> - iterator = fileTable.iterator(seekFileInDB)) { - + boolean processedSubPaths = false; + try (TableIterator> iterator = table.iterator(seekFileInDB)) { while (iterator.hasNext() && remainingBufLimit > 0) { - Table.KeyValue entry = iterator.next(); - OmKeyInfo fileInfo = entry.getValue(); + KeyValue entry = iterator.next(); + T withParentObjectId = entry.getValue(); long objectSerializedSize = entry.getRawSize(); - if (!OMFileRequest.isImmediateChild(fileInfo.getParentObjectID(), + if (!OMFileRequest.isImmediateChild(withParentObjectId.getParentObjectID(), parentInfo.getObjectID())) { - processedSubFiles = true; + processedSubPaths = true; break; } - if (!metadataManager.getFileTable().isExist(entry.getKey())) { + if (!table.isExist(entry.getKey())) { continue; } if (remainingBufLimit - objectSerializedSize < 0) { break; } - fileInfo.setFileName(fileInfo.getKeyName()); - String fullKeyPath = OMFileRequest.getAbsolutePath( - parentInfo.getKeyName(), fileInfo.getKeyName()); - fileInfo.setKeyName(fullKeyPath); - - files.add(fileInfo); + OmKeyInfo keyInfo = deleteKeyTransformer.apply(withParentObjectId); + keyInfos.add(keyInfo); remainingBufLimit -= objectSerializedSize; consumedSize += objectSerializedSize; } - processedSubFiles = processedSubFiles || (!iterator.hasNext()); + processedSubPaths = processedSubPaths || (!iterator.hasNext()); + return new DeleteKeysResult(keyInfos, consumedSize, processedSubPaths); } + } - return new DeleteKeysResult(files, consumedSize, processedSubFiles); + @Override + public DeleteKeysResult getPendingDeletionSubFiles(long volumeId, + long bucketId, OmKeyInfo parentInfo, long remainingBufLimit) + throws IOException { + return gatherSubPathsWithIterator(volumeId, bucketId, parentInfo, metadataManager.getFileTable(), + keyInfo -> OMFileRequest.getKeyInfoWithFullPath(parentInfo, keyInfo), + remainingBufLimit); } public boolean isBucketFSOptimized(String volName, String buckName) 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 f8058bd7a897..75ec1d5b7277 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 @@ -720,6 +720,22 @@ public static OzoneFileStatus getOMKeyInfoIfExists( return null; } + public static OmKeyInfo getKeyInfoWithFullPath(OmKeyInfo parentInfo, OmDirectoryInfo directoryInfo) { + String dirName = OMFileRequest.getAbsolutePath(parentInfo.getKeyName(), + directoryInfo.getName()); + return OMFileRequest.getOmKeyInfo( + parentInfo.getVolumeName(), parentInfo.getBucketName(), directoryInfo, + dirName); + } + + public static OmKeyInfo getKeyInfoWithFullPath(OmKeyInfo parentInfo, OmKeyInfo omKeyInfo) { + omKeyInfo.setFileName(omKeyInfo.getKeyName()); + String fullKeyPath = OMFileRequest.getAbsolutePath( + parentInfo.getKeyName(), omKeyInfo.getKeyName()); + omKeyInfo.setKeyName(fullKeyPath); + return omKeyInfo; + } + /** * Prepare OmKeyInfo from OmDirectoryInfo. *