diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestSnapshotDeletingService.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestSnapshotDeletingService.java index 57501d11aead..fc9c2960ffbb 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestSnapshotDeletingService.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestSnapshotDeletingService.java @@ -32,6 +32,7 @@ import org.apache.hadoop.ozone.client.BucketArgs; import org.apache.hadoop.ozone.client.OzoneBucket; import org.apache.hadoop.ozone.client.OzoneClient; +import org.apache.hadoop.ozone.om.IOmMetadataReader; import org.apache.hadoop.ozone.om.OMConfigKeys; import org.apache.hadoop.ozone.om.OMMetadataManager; import org.apache.hadoop.ozone.om.OmMetadataManagerImpl; @@ -43,6 +44,8 @@ import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo; import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; import org.apache.hadoop.ozone.om.service.SnapshotDeletingService; +import org.apache.hadoop.ozone.om.snapshot.ReferenceCounted; +import org.apache.hadoop.ozone.om.snapshot.SnapshotCache; import org.apache.ozone.test.GenericTestUtils; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; @@ -125,7 +128,7 @@ public void teardown() { @Test public void testSnapshotSplitAndMove() throws Exception { - SnapshotDeletingService snapshotDeletingService = (SnapshotDeletingService) + SnapshotDeletingService snapshotDeletingService = om.getKeyManager().getSnapshotDeletingService(); Table snapshotInfoTable = om.getMetadataManager().getSnapshotInfoTable(); @@ -138,7 +141,7 @@ public void testSnapshotSplitAndMove() throws Exception { OmSnapshot bucket1snap3 = (OmSnapshot) om.getOmSnapshotManager() .checkForSnapshot(VOLUME_NAME, BUCKET_NAME_ONE, - getSnapshotPrefix("bucket1snap3"), true); + getSnapshotPrefix("bucket1snap3"), true).get(); // Check bucket1key1 added to next non deleted snapshot db. List> omKeyInfos = @@ -359,7 +362,7 @@ public void testSnapshotWithFSO() throws Exception { verifySnapshotChain(deletedSnap, "/vol1/bucket2/snap3"); OmSnapshot snap3 = (OmSnapshot) om.getOmSnapshotManager() .checkForSnapshot(VOLUME_NAME, BUCKET_NAME_TWO, - getSnapshotPrefix("snap3"), true); + getSnapshotPrefix("snap3"), true).get(); Table snapDeletedDirTable = snap3.getMetadataManager().getDeletedDirTable(); @@ -386,9 +389,10 @@ public void testSnapshotWithFSO() throws Exception { assertTableRowCount(renamedTable, 4); assertTableRowCount(deletedDirTable, 3); - OmSnapshot snap1 = (OmSnapshot) om.getOmSnapshotManager() - .checkForSnapshot(VOLUME_NAME, BUCKET_NAME_TWO, - getSnapshotPrefix("snap1"), true); + ReferenceCounted rcSnap1 = + om.getOmSnapshotManager().checkForSnapshot( + VOLUME_NAME, BUCKET_NAME_TWO, getSnapshotPrefix("snap1"), true); + OmSnapshot snap1 = (OmSnapshot) rcSnap1.get(); Table snap1KeyTable = snap1.getMetadataManager().getFileTable(); try (TableIterator tableLockMap = new HashMap<>(); - @Override - public ReentrantReadWriteLock getTableLock(String tableName) { - return tableLockMap.get(tableName); - } + private OzoneManager ozoneManager; // Epoch is used to generate the objectIDs. The most significant 2 bits of // objectIDs is set to this epoch. For clusters before HDDS-4315 there is @@ -301,7 +300,16 @@ public ReentrantReadWriteLock getTableLock(String tableName) { new HashMap<>(); private SnapshotChainManager snapshotChainManager; - public OmMetadataManagerImpl(OzoneConfiguration conf) throws IOException { + /** + * OmMetadataManagerImpl constructor. + * @param conf OzoneConfiguration + * @param ozoneManager Points to parent OzoneManager. + * Can be null if not used (in some tests). + * @throws IOException + */ + public OmMetadataManagerImpl(OzoneConfiguration conf, + OzoneManager ozoneManager) throws IOException { + this.ozoneManager = ozoneManager; this.lock = new OzoneManagerLock(conf); // TODO: This is a temporary check. Once fully implemented, all OM state // change should go through Ratis - be it standalone (for non-HA) or @@ -390,6 +398,15 @@ private OmMetadataManagerImpl(OzoneConfiguration conf, File dir, String name) } } + @Override + public ReentrantReadWriteLock getTableLock(String tableName) { + return tableLockMap.get(tableName); + } + + public OzoneManager getOzoneManager() { + return ozoneManager; + } + @Override public Table getUserTable() { return userTable; @@ -1524,96 +1541,102 @@ public PendingKeysDeletion getPendingDeletionKeys(final int keyCount, List blockGroupList = Lists.newArrayList(); // Get volume name and bucket name String[] keySplit = kv.getKey().split(OM_KEY_PREFIX); - // Get the latest snapshot in snapshot path. - OmSnapshot latestSnapshot = getLatestActiveSnapshot(keySplit[1], - keySplit[2], omSnapshotManager); String bucketKey = getBucketKey(keySplit[1], keySplit[2]); OmBucketInfo bucketInfo = getBucketTable().get(bucketKey); - // 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 (latestSnapshot != null) { - Table prevKeyTable = - latestSnapshot.getMetadataManager().getKeyTable( - bucketInfo.getBucketLayout()); - - Table prevDeletedTable = - latestSnapshot.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.getKeyName()); - } else if (prevKeyTableDBKey == null) { - prevKeyTableDBKey = getOzoneKey(info.getVolumeName(), - info.getBucketName(), - info.getKeyName()); - } + // Get the latest snapshot in snapshot path. + try (ReferenceCounted + rcLatestSnapshot = getLatestActiveSnapshot( + keySplit[1], keySplit[2], omSnapshotManager)) { + + // 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 = + ((OmSnapshot) rcLatestSnapshot.get()) + .getMetadataManager() + .getKeyTable(bucketInfo.getBucketLayout()); + + Table prevDeletedTable = + ((OmSnapshot) 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.getKeyName()); + } 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); + 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; } - 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); + // 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++; } - currentCount++; - } - List notReclaimableKeyInfoList = - notReclaimableKeyInfo.getOmKeyInfoList(); + List notReclaimableKeyInfoList = + notReclaimableKeyInfo.getOmKeyInfoList(); - // 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 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); + if (notReclaimableKeyInfoList.size() != + infoList.getOmKeyInfoList().size()) { + keyBlocksList.addAll(blockGroupList); + } } } } @@ -1632,12 +1655,12 @@ private boolean versionExistsInPreviousSnapshot(OmKeyInfo omKeyInfo, /** * Get the latest OmSnapshot for a snapshot path. */ - public OmSnapshot getLatestActiveSnapshot(String volumeName, - String bucketName, - OmSnapshotManager snapshotManager) + public ReferenceCounted< + IOmMetadataReader, SnapshotCache> getLatestActiveSnapshot( + String volumeName, String bucketName, + OmSnapshotManager snapshotManager) throws IOException { - String snapshotPath = volumeName + OM_KEY_PREFIX + bucketName; Optional latestPathSnapshot = Optional.ofNullable( snapshotChainManager.getLatestPathSnapshotId(snapshotPath)); @@ -1667,16 +1690,17 @@ public OmSnapshot getLatestActiveSnapshot(String volumeName, } } - Optional omSnapshot = snapshotInfo.isPresent() ? - Optional.ofNullable( - (OmSnapshot) snapshotManager.checkForSnapshot(volumeName, - bucketName, - getSnapshotPrefix(snapshotInfo.get().getName()), - true) - ) : - Optional.empty(); - - return omSnapshot.orElse(null); + Optional> rcOmSnapshot = + snapshotInfo.isPresent() ? + Optional.ofNullable( + snapshotManager.checkForSnapshot(volumeName, + bucketName, + getSnapshotPrefix(snapshotInfo.get().getName()), + true) + ) : + Optional.empty(); + + return rcOmSnapshot.orElse(null); } @Override diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmSnapshot.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmSnapshot.java index 8d026326bb48..54b526735d1d 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmSnapshot.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmSnapshot.java @@ -28,6 +28,7 @@ import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup; import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus; +import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; import org.apache.hadoop.ozone.security.acl.OzoneObj; import org.apache.hadoop.ozone.security.acl.OzoneObjInfo; import org.apache.hadoop.util.Time; @@ -44,14 +45,14 @@ /** * Metadata Reading class for OM Snapshots. - * + *

* This abstraction manages all the metadata key/acl reading from a * rocksDb instance, for OM snapshots. It's basically identical to * the ozoneManager OmMetadataReader with two exceptions: - * + *

* 1. Its keymanager and prefix manager contain an OmMetadataManager * that reads from a snapshot. - * + *

* 2. It normalizes/denormalizes each request as it comes in to * remove/replace the ".snapshot/snapshotName" prefix. */ @@ -257,6 +258,7 @@ public String getName() { @Override public void close() throws IOException { + // Close DB omMetadataManager.getStore().close(); } @@ -281,4 +283,11 @@ public OMMetadataManager getMetadataManager() { public KeyManager getKeyManager() { return keyManager; } + + /** + * @return DB snapshot table key for this OmSnapshot instance. + */ + public String getSnapshotTableKey() { + return SnapshotInfo.getTableKey(volumeName, bucketName, snapshotName); + } } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmSnapshotManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmSnapshotManager.java index 355eb7f832d6..0c52dba7b98d 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmSnapshotManager.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmSnapshotManager.java @@ -20,11 +20,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; -import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; -import com.google.common.cache.LoadingCache; -import com.google.common.cache.RemovalListener; - import java.io.File; import java.io.IOException; import java.nio.file.Files; @@ -37,10 +33,10 @@ import java.util.List; import java.util.Objects; import java.util.Set; -import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +import com.google.common.cache.RemovalListener; import org.apache.hadoop.hdds.StringUtils; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.server.ServerUtils; @@ -65,6 +61,8 @@ import org.apache.hadoop.ozone.om.service.SnapshotDiffCleanupService; import org.apache.hadoop.ozone.om.snapshot.SnapshotDiffObject; import org.apache.hadoop.ozone.om.helpers.SnapshotDiffJob; +import org.apache.hadoop.ozone.om.snapshot.ReferenceCounted; +import org.apache.hadoop.ozone.om.snapshot.SnapshotCache; import org.apache.hadoop.ozone.om.snapshot.SnapshotDiffManager; import org.apache.hadoop.ozone.om.snapshot.SnapshotUtils; import org.apache.hadoop.ozone.snapshot.CancelSnapshotDiffResponse; @@ -116,7 +114,8 @@ public final class OmSnapshotManager implements AutoCloseable { private final OzoneManager ozoneManager; private final SnapshotDiffManager snapshotDiffManager; - private final LoadingCache snapshotCache; + // Per-OM instance of snapshot cache map + private final SnapshotCache snapshotCache; private final ManagedRocksDB snapshotDiffDb; public static final String DELIMITER = "-"; @@ -233,13 +232,15 @@ public OmSnapshotManager(OzoneManager ozoneManager) { .getStore() .getRocksDBCheckpointDiffer(); - // snapshot cache size, soft-limit + // Soft-limit of lru cache size this.softCacheSize = ozoneManager.getConfiguration().getInt( OZONE_OM_SNAPSHOT_CACHE_MAX_SIZE, OZONE_OM_SNAPSHOT_CACHE_MAX_SIZE_DEFAULT); CacheLoader loader = createCacheLoader(); + // TODO: [SNAPSHOT] Remove this if not going to make SnapshotCache impl + // pluggable. RemovalListener removalListener = notification -> { try { final String snapshotTableKey = notification.getKey(); @@ -262,15 +263,8 @@ public OmSnapshotManager(OzoneManager ozoneManager) { } }; - // init LRU cache - this.snapshotCache = CacheBuilder.newBuilder() - // Indicating OmSnapshot instances are softly referenced from the cache. - // If no thread is holding a strong reference to an OmSnapshot instance - // (e.g. SnapDiff), the instance could be garbage collected by JVM at - // its discretion. - .softValues() - .removalListener(removalListener) - .build(loader); + // Init snapshot cache + this.snapshotCache = new SnapshotCache(this, loader, softCacheSize); this.snapshotDiffManager = new SnapshotDiffManager(snapshotDiffDb, differ, ozoneManager, snapshotCache, snapDiffJobCf, snapDiffReportCf, @@ -310,8 +304,8 @@ private CacheLoader createCacheLoader() { @Override public OmSnapshot load(@Nonnull String snapshotTableKey) throws IOException { - // see if the snapshot exists - SnapshotInfo snapshotInfo = getSnapshotInfo(snapshotTableKey); + // Check if the snapshot exists + final SnapshotInfo snapshotInfo = getSnapshotInfo(snapshotTableKey); // Block snapshot from loading when it is no longer active e.g. DELETED, // unless this is called from SnapshotDeletingService. @@ -336,7 +330,7 @@ public OmSnapshot load(@Nonnull String snapshotTableKey) snapshotInfo.getCheckpointDirName(), isSnapshotInCache, maxOpenSstFilesInSnapshotDb); } catch (IOException e) { - LOG.error("Failed to retrieve snapshot: {}", snapshotTableKey); + LOG.error("Failed to retrieve snapshot: {}", snapshotTableKey, e); throw e; } @@ -379,7 +373,7 @@ private static CodecRegistry createCodecRegistryForSnapDiff() { * Get snapshot instance LRU cache. * @return LoadingCache */ - public LoadingCache getSnapshotCache() { + public SnapshotCache getSnapshotCache() { return snapshotCache; } @@ -632,17 +626,17 @@ private static void deleteKeysFromDelKeyTableInSnapshotScope( } // Get OmSnapshot if the keyName has ".snapshot" key indicator - public IOmMetadataReader checkForSnapshot(String volumeName, - String bucketName, - String keyName, - boolean skipActiveCheck) - throws IOException { + public ReferenceCounted checkForSnapshot( + String volumeName, + String bucketName, + String keyName, + boolean skipActiveCheck) throws IOException { if (keyName == null || !ozoneManager.isFilesystemSnapshotEnabled()) { return ozoneManager.getOmMetadataReader(); } // see if key is for a snapshot - String[] keyParts = keyName.split("/"); + String[] keyParts = keyName.split(OM_KEY_PREFIX); if (isSnapshotKey(keyParts)) { String snapshotName = keyParts[1]; if (snapshotName == null || snapshotName.isEmpty()) { @@ -664,16 +658,24 @@ public IOmMetadataReader checkForSnapshot(String volumeName, } // retrieve the snapshot from the cache - try { - return snapshotCache.get(snapshotTableKey); - } catch (ExecutionException e) { - throw new IOException(e.getCause()); - } + return snapshotCache.get(snapshotTableKey, skipActiveCheck); } else { return ozoneManager.getOmMetadataReader(); } } + /** + * Returns true if the snapshot is in given status. + * @param key DB snapshot table key + * @param status SnapshotStatus + * @return true if the snapshot is in given status, false otherwise + */ + public boolean isSnapshotStatus(String key, + SnapshotInfo.SnapshotStatus status) + throws IOException { + return getSnapshotInfo(key).getSnapshotStatus().equals(status); + } + public SnapshotInfo getSnapshotInfo(String key) throws IOException { return SnapshotUtils.getSnapshotInfo(ozoneManager, key); } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java index c6588a4e3e76..30655fdfafd3 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java @@ -94,6 +94,8 @@ import org.apache.hadoop.ozone.om.s3.S3SecretStoreProvider; import org.apache.hadoop.ozone.om.service.OMRangerBGSyncService; import org.apache.hadoop.ozone.om.snapshot.OmSnapshotUtils; +import org.apache.hadoop.ozone.om.snapshot.ReferenceCounted; +import org.apache.hadoop.ozone.om.snapshot.SnapshotCache; import org.apache.hadoop.ozone.om.upgrade.OMLayoutFeature; import org.apache.hadoop.ozone.snapshot.CancelSnapshotDiffResponse; import org.apache.hadoop.ozone.snapshot.SnapshotDiffResponse; @@ -470,6 +472,9 @@ private enum State { // This metadata reader points to the active filesystem private OmMetadataReader omMetadataReader; + // Wrap active DB metadata reader in ReferenceCounted once to avoid + // instance creation every single time. + private ReferenceCounted rcOmMetadataReader; private OmSnapshotManager omSnapshotManager; @SuppressWarnings("methodlength") @@ -779,7 +784,7 @@ private void logVersionMismatch(OzoneConfiguration conf, ScmInfo scmInfo) { private void instantiateServices(boolean withNewSnapshot) throws IOException { OmMetadataManagerImpl metadataManagerImpl = - new OmMetadataManagerImpl(configuration); + new OmMetadataManagerImpl(configuration, this); this.metadataManager = metadataManagerImpl; LOG.info("S3 Multi-Tenancy is {}", isS3MultiTenancyEnabled ? "enabled" : "disabled"); @@ -823,6 +828,9 @@ private void instantiateServices(boolean withNewSnapshot) throws IOException { perfMetrics); omMetadataReader = new OmMetadataReader(keyManager, prefixManager, this, LOG, AUDIT, metrics); + // Active DB's OmMetadataReader instance does not need to be reference + // counted, but it still needs to be wrapped to be consistent. + rcOmMetadataReader = new ReferenceCounted<>(omMetadataReader, true, null); // TODO: [SNAPSHOT] Revisit this in HDDS-8529. omSnapshotManager = new OmSnapshotManager(this); @@ -2560,8 +2568,9 @@ public boolean getAllowListAllVolumes() { return allowListAllVolumes; } - public OmMetadataReader getOmMetadataReader() { - return omMetadataReader; + public ReferenceCounted< + IOmMetadataReader, SnapshotCache> getOmMetadataReader() { + return rcOmMetadataReader; } /** @@ -2791,29 +2800,40 @@ public OmBucketInfo getBucketInfo(String volume, String bucket) } /** - * Lookup a key. - * - * @param args - attributes of the key. - * @return OmKeyInfo - the info about the requested key. + * {@inheritDoc} */ @Override public OmKeyInfo lookupKey(OmKeyArgs args) throws IOException { - return getReader(args).lookupKey(args); + try (ReferenceCounted + rcReader = getReader(args)) { + return rcReader.get().lookupKey(args); + } } + /** + * {@inheritDoc} + */ @Override public KeyInfoWithVolumeContext getKeyInfo(final OmKeyArgs args, boolean assumeS3Context) throws IOException { - return getReader(args).getKeyInfo(args, assumeS3Context); + try (ReferenceCounted rcReader = + getReader(args)) { + return rcReader.get().getKeyInfo(args, assumeS3Context); + } } + /** + * {@inheritDoc} + */ @Override public List listKeys(String volumeName, String bucketName, String startKey, String keyPrefix, int maxKeys) throws IOException { - - return getReader(volumeName, bucketName, keyPrefix).listKeys( - volumeName, bucketName, startKey, keyPrefix, maxKeys); + try (ReferenceCounted rcReader = + getReader(volumeName, bucketName, keyPrefix)) { + return rcReader.get().listKeys( + volumeName, bucketName, startKey, keyPrefix, maxKeys); + } } @Override @@ -3495,16 +3515,31 @@ public OmMultipartUploadList listMultipartUploads(String volumeName, } + /** + * {@inheritDoc} + */ @Override public OzoneFileStatus getFileStatus(OmKeyArgs args) throws IOException { - return getReader(args).getFileStatus(args); + try (ReferenceCounted rcReader = + getReader(args)) { + return rcReader.get().getFileStatus(args); + } } + /** + * {@inheritDoc} + */ @Override public OmKeyInfo lookupFile(OmKeyArgs args) throws IOException { - return getReader(args).lookupFile(args); + try (ReferenceCounted rcReader = + getReader(args)) { + return rcReader.get().lookupFile(args); + } } + /** + * {@inheritDoc} + */ @Override public List listStatus(OmKeyArgs args, boolean recursive, String startKey, long numEntries) @@ -3515,20 +3550,22 @@ public List listStatus(OmKeyArgs args, boolean recursive, public List listStatus(OmKeyArgs args, boolean recursive, String startKey, long numEntries, boolean allowPartialPrefixes) throws IOException { - - return getReader(args).listStatus(args, recursive, - startKey, numEntries, allowPartialPrefixes); + try (ReferenceCounted rcReader = + getReader(args)) { + return rcReader.get().listStatus( + args, recursive, startKey, numEntries, allowPartialPrefixes); + } } /** - * Returns list of ACLs for given Ozone object. - * - * @param obj Ozone object. - * @throws IOException if there is error. + * {@inheritDoc} */ @Override public List getAcl(OzoneObj obj) throws IOException { - return getReader(obj).getAcl(obj); + try (ReferenceCounted rcReader = + getReader(obj)) { + return rcReader.get().getAcl(obj); + } } /** @@ -4509,22 +4546,52 @@ public static HddsProtos.OzoneManagerDetailsProto getOmDetailsProto( return omDetailsProto; } - private IOmMetadataReader getReader(OmKeyArgs keyArgs) throws IOException { + /** + * Get a referenced counted OmMetadataReader instance. + * Caller is responsible of closing the return value. + * Using try-with-resources is recommended. + * @param keyArgs OmKeyArgs + * @return ReferenceCounted + */ + private ReferenceCounted< + IOmMetadataReader, SnapshotCache> getReader(OmKeyArgs keyArgs) + throws IOException { return omSnapshotManager.checkForSnapshot( keyArgs.getVolumeName(), keyArgs.getBucketName(), keyArgs.getKeyName(), false); } - private IOmMetadataReader getReader(String volumeName, String bucketName, - String key) throws IOException { - return omSnapshotManager.checkForSnapshot(volumeName, bucketName, key, - false); + /** + * Get a referenced counted OmMetadataReader instance. + * Caller is responsible of closing the return value. + * Using try-with-resources is recommended. + * @param volumeName volume name + * @param bucketName bucket name + * @param key key path + * @return ReferenceCounted + */ + private ReferenceCounted< + IOmMetadataReader, SnapshotCache> getReader( + String volumeName, String bucketName, String key) throws IOException { + return omSnapshotManager.checkForSnapshot( + volumeName, bucketName, key, false); } - private IOmMetadataReader getReader(OzoneObj ozoneObj) throws IOException { + /** + * Get a referenced counted OmMetadataReader instance. + * Caller is responsible of closing the return value. + * Using try-with-resources is recommended. + * @param ozoneObj OzoneObj + * @return ReferenceCounted + */ + private ReferenceCounted< + IOmMetadataReader, SnapshotCache> getReader(OzoneObj ozoneObj) + throws IOException { return omSnapshotManager.checkForSnapshot( - ozoneObj.getVolumeName(), ozoneObj.getBucketName(), - ozoneObj.getKeyName(), false); + ozoneObj.getVolumeName(), + ozoneObj.getBucketName(), + ozoneObj.getKeyName(), + false); } public SnapshotDiffResponse snapshotDiff(String volume, diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerStateMachine.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerStateMachine.java index d3bf26b772c5..62c387f33211 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerStateMachine.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerStateMachine.java @@ -568,7 +568,7 @@ private OMResponse runCommand(OMRequest request, long trxLogIndex) { trxLogIndex).getOMResponse(); } catch (Throwable e) { // For any Runtime exceptions, terminate OM. - String errorMessage = "Request " + request + "failed with exception"; + String errorMessage = "Request " + request + " failed with exception"; ExitUtils.terminate(1, errorMessage, e, LOG); } return null; diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/OMClientRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/OMClientRequest.java index 2ca3b8debf7e..e769717cf202 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/OMClientRequest.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/OMClientRequest.java @@ -28,6 +28,8 @@ import org.apache.hadoop.ozone.audit.AuditEventStatus; import org.apache.hadoop.ozone.audit.AuditLogger; import org.apache.hadoop.ozone.audit.AuditMessage; +import org.apache.hadoop.ozone.om.IOmMetadataReader; +import org.apache.hadoop.ozone.om.OmMetadataReader; import org.apache.hadoop.ozone.om.OzoneAclUtils; import org.apache.hadoop.ozone.om.OzoneManager; import org.apache.hadoop.ozone.om.OzonePrefixPathImpl; @@ -36,6 +38,8 @@ import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper; import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils; import org.apache.hadoop.ozone.om.response.OMClientResponse; +import org.apache.hadoop.ozone.om.snapshot.ReferenceCounted; +import org.apache.hadoop.ozone.om.snapshot.SnapshotCache; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.LayoutVersion; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest; @@ -271,8 +275,13 @@ protected void checkACLsWithFSO(OzoneManager ozoneManager, String volumeName, contextBuilder.setOwnerName(bucketOwner); } - ozoneManager.getOmMetadataReader().checkAcls(obj, - contextBuilder.build(), true); + try (ReferenceCounted rcMetadataReader = + ozoneManager.getOmMetadataReader()) { + OmMetadataReader omMetadataReader = + (OmMetadataReader) rcMetadataReader.get(); + + omMetadataReader.checkAcls(obj, contextBuilder.build(), true); + } } } @@ -332,10 +341,13 @@ public void checkAcls(OzoneManager ozoneManager, String bucketOwner) throws IOException { - OzoneAclUtils.checkAllAcls(ozoneManager.getOmMetadataReader(), - resType, storeType, aclType, - vol, bucket, key, volOwner, bucketOwner, createUGI(), - getRemoteAddress(), getHostName()); + try (ReferenceCounted rcMetadataReader = + ozoneManager.getOmMetadataReader()) { + OzoneAclUtils.checkAllAcls((OmMetadataReader) rcMetadataReader.get(), + resType, storeType, aclType, + vol, bucket, key, volOwner, bucketOwner, createUGI(), + getRemoteAddress(), getHostName()); + } } /** diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketSetPropertyRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketSetPropertyRequest.java index 7b7998e0139f..f1e8d202a1f3 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketSetPropertyRequest.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketSetPropertyRequest.java @@ -26,6 +26,8 @@ import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.audit.AuditLogger; import org.apache.hadoop.ozone.audit.OMAction; +import org.apache.hadoop.ozone.om.IOmMetadataReader; +import org.apache.hadoop.ozone.om.OmMetadataReader; import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs; import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper; import org.apache.hadoop.ozone.om.request.util.OmResponseUtil; @@ -33,6 +35,8 @@ import org.apache.hadoop.ozone.om.request.validation.RequestProcessingPhase; import org.apache.hadoop.ozone.om.request.validation.ValidationCondition; import org.apache.hadoop.ozone.om.request.validation.ValidationContext; +import org.apache.hadoop.ozone.om.snapshot.ReferenceCounted; +import org.apache.hadoop.ozone.om.snapshot.SnapshotCache; import org.apache.hadoop.ozone.om.upgrade.OMLayoutFeature; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type; import org.apache.hadoop.security.UserGroupInformation; @@ -270,7 +274,13 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, private void checkAclPermission( OzoneManager ozoneManager, String volumeName, String bucketName) throws IOException { - if (ozoneManager.getOmMetadataReader().isNativeAuthorizerEnabled()) { + final boolean nativeAuthorizerEnabled; + try (ReferenceCounted rcMetadataReader = + ozoneManager.getOmMetadataReader()) { + OmMetadataReader mdReader = (OmMetadataReader) rcMetadataReader.get(); + nativeAuthorizerEnabled = mdReader.isNativeAuthorizerEnabled(); + } + if (nativeAuthorizerEnabled) { UserGroupInformation ugi = createUGI(); String bucketOwner = ozoneManager.getBucketOwner(volumeName, bucketName, IAccessAuthorizer.ACLType.READ, OzoneObj.ResourceType.BUCKET); diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMDirectoriesPurgeRequestWithFSO.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMDirectoriesPurgeRequestWithFSO.java index a63ae3265c13..411ed889d388 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMDirectoriesPurgeRequestWithFSO.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMDirectoriesPurgeRequestWithFSO.java @@ -25,8 +25,6 @@ import java.util.Set; import org.apache.commons.lang3.tuple.Pair; 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.helpers.BucketLayout; import org.apache.hadoop.ozone.om.helpers.OmBucketInfo; @@ -42,7 +40,6 @@ import java.util.List; -import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix; import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK; /** @@ -59,27 +56,21 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, long trxnLogIndex, OzoneManagerDoubleBufferHelper omDoubleBufferHelper) { OzoneManagerProtocolProtos.PurgeDirectoriesRequest purgeDirsRequest = getOmRequest().getPurgeDirectoriesRequest(); - OmSnapshotManager omSnapshotManager = ozoneManager.getOmSnapshotManager(); String fromSnapshot = purgeDirsRequest.hasSnapshotTableKey() ? purgeDirsRequest.getSnapshotTableKey() : null; List purgeRequests = purgeDirsRequest.getDeletedPathList(); - OmSnapshot omFromSnapshot = null; + SnapshotInfo fromSnapshotInfo = null; Set> lockSet = new HashSet<>(); Map, OmBucketInfo> volBucketInfoMap = new HashMap<>(); OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager(); try { if (fromSnapshot != null) { - SnapshotInfo snapshotInfo = - ozoneManager.getMetadataManager().getSnapshotInfoTable() - .get(fromSnapshot); - // TODO: [SNAPSHOT] Revisit in HDDS-8529. - omFromSnapshot = (OmSnapshot) omSnapshotManager - .checkForSnapshot(snapshotInfo.getVolumeName(), - snapshotInfo.getBucketName(), - getSnapshotPrefix(snapshotInfo.getName()), true); + fromSnapshotInfo = ozoneManager.getMetadataManager() + .getSnapshotInfoTable() + .get(fromSnapshot); } for (OzoneManagerProtocolProtos.PurgePathRequest path : purgeRequests) { @@ -148,7 +139,7 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, getOmRequest()); OMClientResponse omClientResponse = new OMDirectoriesPurgeResponseWithFSO( omResponse.build(), purgeRequests, ozoneManager.isRatisEnabled(), - getBucketLayout(), volBucketInfoMap, omFromSnapshot); + getBucketLayout(), volBucketInfoMap, fromSnapshotInfo); addResponseToDoubleBuffer(trxnLogIndex, omClientResponse, omDoubleBufferHelper); 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 3d317e148e8f..2344dd14c81d 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,8 +21,6 @@ import java.io.IOException; import java.util.ArrayList; -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.SnapshotInfo; import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper; @@ -40,8 +38,6 @@ import java.util.List; -import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix; - /** * Handles purging of keys from OM DB. */ @@ -62,7 +58,6 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, .getDeletedKeysList(); List keysToUpdateList = purgeKeysRequest .getKeysToUpdateList(); - OmSnapshotManager omSnapshotManager = ozoneManager.getOmSnapshotManager(); String fromSnapshot = purgeKeysRequest.hasSnapshotTableKey() ? purgeKeysRequest.getSnapshotTableKey() : null; List keysToBePurgedList = new ArrayList<>(); @@ -71,7 +66,6 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, getOmRequest()); OMClientResponse omClientResponse = null; - for (DeletedKeys bucketWithDeleteKeys : bucketDeletedKeysList) { for (String deletedKey : bucketWithDeleteKeys.getKeysList()) { keysToBePurgedList.add(deletedKey); @@ -79,20 +73,13 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, } try { - OmSnapshot omFromSnapshot = null; + SnapshotInfo fromSnapshotInfo = null; if (fromSnapshot != null) { - SnapshotInfo snapshotInfo = - ozoneManager.getMetadataManager().getSnapshotInfoTable() - .get(fromSnapshot); - // TODO: [SNAPSHOT] Revisit in HDDS-8529. - omFromSnapshot = (OmSnapshot) omSnapshotManager - .checkForSnapshot(snapshotInfo.getVolumeName(), - snapshotInfo.getBucketName(), - getSnapshotPrefix(snapshotInfo.getName()), true); + fromSnapshotInfo = ozoneManager.getMetadataManager() + .getSnapshotInfoTable().get(fromSnapshot); } - omClientResponse = new OMKeyPurgeResponse(omResponse.build(), - keysToBePurgedList, omFromSnapshot, keysToUpdateList); + keysToBePurgedList, fromSnapshotInfo, keysToUpdateList); } catch (IOException ex) { omClientResponse = new OMKeyPurgeResponse( createErrorOMResponse(omResponse, ex)); diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRequest.java index a3845fb2b1f3..e8b0f8cc24a3 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRequest.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRequest.java @@ -39,7 +39,9 @@ import org.apache.hadoop.ozone.OmUtils; import org.apache.hadoop.ozone.OzoneAcl; import org.apache.hadoop.ozone.OzoneConsts; +import org.apache.hadoop.ozone.om.IOmMetadataReader; import org.apache.hadoop.ozone.om.OMMetrics; +import org.apache.hadoop.ozone.om.OmMetadataReader; import org.apache.hadoop.ozone.om.PrefixManager; import org.apache.hadoop.ozone.om.ResolvedBucket; import org.apache.hadoop.ozone.om.helpers.BucketEncryptionKeyInfo; @@ -57,6 +59,8 @@ import org.apache.hadoop.ozone.om.lock.OzoneLockStrategy; import org.apache.hadoop.ozone.om.request.OMClientRequestUtils; import org.apache.hadoop.ozone.om.request.file.OMFileRequest; +import org.apache.hadoop.ozone.om.snapshot.ReferenceCounted; +import org.apache.hadoop.ozone.om.snapshot.SnapshotCache; import org.apache.hadoop.ozone.protocolPB.OMPBHelper; import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer; import org.apache.hadoop.ozone.security.acl.OzoneObj; @@ -381,11 +385,18 @@ protected void checkKeyAcls(OzoneManager ozoneManager, String volume, protected void checkKeyAclsInOpenKeyTable(OzoneManager ozoneManager, String volume, String bucket, String key, IAccessAuthorizer.ACLType aclType, long clientId) throws IOException { + final boolean nativeAuthorizerEnabled; + try (ReferenceCounted rcMetadataReader = + ozoneManager.getOmMetadataReader()) { + OmMetadataReader mdReader = (OmMetadataReader) rcMetadataReader.get(); + nativeAuthorizerEnabled = mdReader.isNativeAuthorizerEnabled(); + } + String keyNameForAclCheck = key; // Native authorizer requires client id as part of key name to check // write ACL on key. Add client id to key name if ozone native // authorizer is configured. - if (ozoneManager.getOmMetadataReader().isNativeAuthorizerEnabled()) { + if (nativeAuthorizerEnabled) { keyNameForAclCheck = key + "/" + clientId; } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/security/OMSetSecretRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/security/OMSetSecretRequest.java index 2039dc09b406..129cef809372 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/security/OMSetSecretRequest.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/security/OMSetSecretRequest.java @@ -164,8 +164,7 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, if (exception == null) { LOG.debug("Success: SetSecret for accessKey '{}'", accessId); } else { - LOG.error("Failed to SetSecret for accessKey '{}': {}", - accessId, exception); + LOG.error("Failed to SetSecret for accessKey '{}'", accessId, exception); } return omClientResponse; } 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 9eee231a6993..7dacebea9ecc 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 @@ -21,7 +21,6 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos; 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; @@ -43,7 +42,6 @@ import java.io.IOException; import java.util.List; -import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix; import static org.apache.hadoop.ozone.om.upgrade.OMLayoutFeature.FILESYSTEM_SNAPSHOT; /** @@ -81,11 +79,6 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, OzoneManagerProtocolProtos.OMResponse.Builder omResponse = OmResponseUtil.getOMResponseBuilder(getOmRequest()); try { - OmSnapshot omFromSnapshot = (OmSnapshot) omSnapshotManager - .checkForSnapshot(fromSnapshot.getVolumeName(), - fromSnapshot.getBucketName(), - getSnapshotPrefix(fromSnapshot.getName()), true); - nextSnapshot = SnapshotUtils.getNextActiveSnapshot(fromSnapshot, snapshotChainManager, omSnapshotManager); @@ -99,17 +92,8 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager, List movedDirs = moveDeletedKeysRequest.getDeletedDirsToMoveList(); - OmSnapshot omNextSnapshot = null; - - if (nextSnapshot != null) { - omNextSnapshot = (OmSnapshot) omSnapshotManager - .checkForSnapshot(nextSnapshot.getVolumeName(), - nextSnapshot.getBucketName(), - getSnapshotPrefix(nextSnapshot.getName()), true); - } - omClientResponse = new OMSnapshotMoveDeletedKeysResponse( - omResponse.build(), omFromSnapshot, omNextSnapshot, + omResponse.build(), fromSnapshot, nextSnapshot, nextDBKeysList, reclaimKeysList, renamedKeysList, movedDirs); } catch (IOException ex) { 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 a82ccc9d6e31..758133b41113 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 @@ -23,14 +23,20 @@ import org.apache.hadoop.hdds.utils.db.BatchOperation; import org.apache.hadoop.hdds.utils.db.DBStore; import org.apache.hadoop.ozone.OmUtils; +import org.apache.hadoop.ozone.om.IOmMetadataReader; 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.helpers.BucketLayout; 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.request.key.OMDirectoriesPurgeRequestWithFSO; import org.apache.hadoop.ozone.om.response.CleanupTableInfo; +import org.apache.hadoop.ozone.om.snapshot.ReferenceCounted; +import org.apache.hadoop.ozone.om.snapshot.SnapshotCache; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse; import org.slf4j.Logger; @@ -44,6 +50,7 @@ import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DELETED_TABLE; import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DIRECTORY_TABLE; import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.FILE_TABLE; +import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix; /** * Response for {@link OMDirectoriesPurgeRequestWithFSO} request. @@ -57,30 +64,43 @@ public class OMDirectoriesPurgeResponseWithFSO extends OmKeyResponse { private List paths; private boolean isRatisEnabled; private Map, OmBucketInfo> volBucketInfoMap; - private OmSnapshot fromSnapshot; - + private SnapshotInfo fromSnapshotInfo; public OMDirectoriesPurgeResponseWithFSO(@Nonnull OMResponse omResponse, @Nonnull List paths, boolean isRatisEnabled, @Nonnull BucketLayout bucketLayout, Map, OmBucketInfo> volBucketInfoMap, - OmSnapshot fromSnapshot) { + SnapshotInfo fromSnapshotInfo) { super(omResponse, bucketLayout); this.paths = paths; this.isRatisEnabled = isRatisEnabled; this.volBucketInfoMap = volBucketInfoMap; - this.fromSnapshot = fromSnapshot; + this.fromSnapshotInfo = fromSnapshotInfo; } @Override public void addToDBBatch(OMMetadataManager metadataManager, BatchOperation batchOp) throws IOException { - if (fromSnapshot != null) { - DBStore fromSnapshotStore = fromSnapshot.getMetadataManager().getStore(); - // Init Batch Operation for snapshot db. - try (BatchOperation writeBatch = fromSnapshotStore.initBatchOperation()) { - processPaths(fromSnapshot.getMetadataManager(), writeBatch); - fromSnapshotStore.commitBatchOperation(writeBatch); + if (fromSnapshotInfo != null) { + OmSnapshotManager omSnapshotManager = + ((OmMetadataManagerImpl) metadataManager) + .getOzoneManager().getOmSnapshotManager(); + + try (ReferenceCounted + rcFromSnapshotInfo = omSnapshotManager.checkForSnapshot( + fromSnapshotInfo.getVolumeName(), + fromSnapshotInfo.getBucketName(), + getSnapshotPrefix(fromSnapshotInfo.getName()), + true)) { + OmSnapshot fromSnapshot = (OmSnapshot) rcFromSnapshotInfo.get(); + DBStore fromSnapshotStore = fromSnapshot.getMetadataManager() + .getStore(); + // Init Batch Operation for snapshot db. + try (BatchOperation writeBatch = + fromSnapshotStore.initBatchOperation()) { + processPaths(fromSnapshot.getMetadataManager(), writeBatch); + fromSnapshotStore.commitBatchOperation(writeBatch); + } } } else { processPaths(metadataManager, batchOp); 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 6c89e16d6a5b..719ece21c151 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 @@ -19,11 +19,17 @@ package org.apache.hadoop.ozone.om.response.key; import org.apache.hadoop.hdds.utils.db.DBStore; +import org.apache.hadoop.ozone.om.IOmMetadataReader; 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.helpers.RepeatedOmKeyInfo; +import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; import org.apache.hadoop.ozone.om.response.CleanupTableInfo; import org.apache.hadoop.ozone.om.request.key.OMKeyPurgeRequest; +import org.apache.hadoop.ozone.om.snapshot.ReferenceCounted; +import org.apache.hadoop.ozone.om.snapshot.SnapshotCache; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveKeyInfos; @@ -35,6 +41,7 @@ import javax.annotation.Nonnull; import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DELETED_TABLE; +import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix; import static org.apache.hadoop.ozone.om.response.snapshot.OMSnapshotMoveDeletedKeysResponse.createRepeatedOmKeyInfo; /** @@ -43,11 +50,12 @@ @CleanupTableInfo(cleanupTables = {DELETED_TABLE}) public class OMKeyPurgeResponse extends OmKeyResponse { private List purgeKeyList; - private OmSnapshot fromSnapshot; + private SnapshotInfo fromSnapshot; private List keysToUpdateList; public OMKeyPurgeResponse(@Nonnull OMResponse omResponse, - @Nonnull List keyList, OmSnapshot fromSnapshot, + @Nonnull List keyList, + SnapshotInfo fromSnapshot, List keysToUpdate) { super(omResponse); this.purgeKeyList = keyList; @@ -69,12 +77,27 @@ public void addToDBBatch(OMMetadataManager omMetadataManager, BatchOperation batchOperation) throws IOException { if (fromSnapshot != null) { - DBStore fromSnapshotStore = fromSnapshot.getMetadataManager().getStore(); - // Init Batch Operation for snapshot db. - try (BatchOperation writeBatch = fromSnapshotStore.initBatchOperation()) { - processKeys(writeBatch, fromSnapshot.getMetadataManager()); - processKeysToUpdate(writeBatch, fromSnapshot.getMetadataManager()); - fromSnapshotStore.commitBatchOperation(writeBatch); + OmSnapshotManager omSnapshotManager = + ((OmMetadataManagerImpl) omMetadataManager) + .getOzoneManager().getOmSnapshotManager(); + + try (ReferenceCounted rcOmFromSnapshot = + omSnapshotManager.checkForSnapshot( + fromSnapshot.getVolumeName(), + fromSnapshot.getBucketName(), + getSnapshotPrefix(fromSnapshot.getName()), + true)) { + + OmSnapshot fromOmSnapshot = (OmSnapshot) rcOmFromSnapshot.get(); + DBStore fromSnapshotStore = + fromOmSnapshot.getMetadataManager().getStore(); + // Init Batch Operation for snapshot db. + try (BatchOperation writeBatch = + fromSnapshotStore.initBatchOperation()) { + processKeys(writeBatch, fromOmSnapshot.getMetadataManager()); + processKeysToUpdate(writeBatch, fromOmSnapshot.getMetadataManager()); + fromSnapshotStore.commitBatchOperation(writeBatch); + } } } else { processKeys(batchOperation, omMetadataManager); diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/snapshot/OMSnapshotMoveDeletedKeysResponse.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/snapshot/OMSnapshotMoveDeletedKeysResponse.java index 3846dfbf5e07..f4142400d7cf 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/snapshot/OMSnapshotMoveDeletedKeysResponse.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/snapshot/OMSnapshotMoveDeletedKeysResponse.java @@ -21,12 +21,18 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.utils.db.BatchOperation; import org.apache.hadoop.hdds.utils.db.RDBStore; +import org.apache.hadoop.ozone.om.IOmMetadataReader; 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.helpers.OmKeyInfo; import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo; +import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; import org.apache.hadoop.ozone.om.response.CleanupTableInfo; import org.apache.hadoop.ozone.om.response.OMClientResponse; +import org.apache.hadoop.ozone.om.snapshot.ReferenceCounted; +import org.apache.hadoop.ozone.om.snapshot.SnapshotCache; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveKeyInfos; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse; @@ -36,6 +42,7 @@ import java.util.List; import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.SNAPSHOT_INFO_TABLE; +import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix; /** * Response for OMSnapshotMoveDeletedKeysRequest. @@ -43,22 +50,23 @@ @CleanupTableInfo(cleanupTables = {SNAPSHOT_INFO_TABLE}) public class OMSnapshotMoveDeletedKeysResponse extends OMClientResponse { - private OmSnapshot fromSnapshot; - private OmSnapshot nextSnapshot; + private SnapshotInfo fromSnapshot; + private SnapshotInfo nextSnapshot; private List nextDBKeysList; private List reclaimKeysList; private List renamedKeysList; private List movedDirs; public OMSnapshotMoveDeletedKeysResponse(OMResponse omResponse, - @Nonnull OmSnapshot omFromSnapshot, OmSnapshot omNextSnapshot, - List nextDBKeysList, - List reclaimKeysList, - List renamedKeysList, - List movedDirs) { + @Nonnull SnapshotInfo fromSnapshot, + SnapshotInfo nextSnapshot, + List nextDBKeysList, + List reclaimKeysList, + List renamedKeysList, + List movedDirs) { super(omResponse); - this.fromSnapshot = omFromSnapshot; - this.nextSnapshot = omNextSnapshot; + this.fromSnapshot = fromSnapshot; + this.nextSnapshot = nextSnapshot; this.nextDBKeysList = nextDBKeysList; this.reclaimKeysList = reclaimKeysList; this.renamedKeysList = renamedKeysList; @@ -78,42 +86,74 @@ public OMSnapshotMoveDeletedKeysResponse(@Nonnull OMResponse omResponse) { protected void addToDBBatch(OMMetadataManager omMetadataManager, BatchOperation batchOperation) throws IOException { - if (nextSnapshot != null) { - RDBStore nextSnapshotStore = - (RDBStore) nextSnapshot.getMetadataManager().getStore(); - // Init Batch Operation for snapshot db. - try (BatchOperation writeBatch = nextSnapshotStore.initBatchOperation()) { - processKeys(writeBatch, nextSnapshot.getMetadataManager()); - processDirs(writeBatch, nextSnapshot.getMetadataManager()); - nextSnapshotStore.commitBatchOperation(writeBatch); - nextSnapshotStore.getDb().flushWal(true); - nextSnapshotStore.getDb().flush(); + // Note: a trick to get + // To do this properly, refactor OzoneManagerDoubleBuffer#addToBatch and + // add OmSnapshotManager as a parameter. + OmSnapshotManager omSnapshotManager = + ((OmMetadataManagerImpl) omMetadataManager) + .getOzoneManager().getOmSnapshotManager(); + + try (ReferenceCounted rcOmFromSnapshot = + omSnapshotManager.checkForSnapshot( + fromSnapshot.getVolumeName(), + fromSnapshot.getBucketName(), + getSnapshotPrefix(fromSnapshot.getName()), + true)) { + + OmSnapshot fromOmSnapshot = (OmSnapshot) rcOmFromSnapshot.get(); + + if (nextSnapshot != null) { + try (ReferenceCounted + rcOmNextSnapshot = omSnapshotManager.checkForSnapshot( + nextSnapshot.getVolumeName(), + nextSnapshot.getBucketName(), + getSnapshotPrefix(nextSnapshot.getName()), + true)) { + + OmSnapshot nextOmSnapshot = (OmSnapshot) rcOmNextSnapshot.get(); + RDBStore nextSnapshotStore = + (RDBStore) nextOmSnapshot.getMetadataManager().getStore(); + // Init Batch Operation for snapshot db. + try (BatchOperation writeBatch = + nextSnapshotStore.initBatchOperation()) { + processKeys(writeBatch, nextOmSnapshot.getMetadataManager()); + processDirs(writeBatch, nextOmSnapshot.getMetadataManager(), + fromOmSnapshot); + nextSnapshotStore.commitBatchOperation(writeBatch); + nextSnapshotStore.getDb().flushWal(true); + nextSnapshotStore.getDb().flush(); + } + } + } else { + // Handle the case where there is no next Snapshot. + processKeys(batchOperation, omMetadataManager); + processDirs(batchOperation, omMetadataManager, fromOmSnapshot); } - } else { - // Handle the case where there is no next Snapshot. - processKeys(batchOperation, omMetadataManager); - processDirs(batchOperation, omMetadataManager); - } - // Update From Snapshot Deleted Table. - RDBStore fromSnapshotStore = - (RDBStore) fromSnapshot.getMetadataManager().getStore(); - try (BatchOperation fromSnapshotBatchOp = - fromSnapshotStore.initBatchOperation()) { - processReclaimKeys(fromSnapshotBatchOp, - fromSnapshot.getMetadataManager()); - deleteDirsFromSnapshot(fromSnapshotBatchOp); - fromSnapshotStore.commitBatchOperation(fromSnapshotBatchOp); - fromSnapshotStore.getDb().flushWal(true); - fromSnapshotStore.getDb().flush(); + // Update From Snapshot Deleted Table. + RDBStore fromSnapshotStore = + (RDBStore) fromOmSnapshot.getMetadataManager().getStore(); + try (BatchOperation fromSnapshotBatchOp = + fromSnapshotStore.initBatchOperation()) { + processReclaimKeys(fromSnapshotBatchOp, + fromOmSnapshot.getMetadataManager()); + deleteDirsFromSnapshot(fromSnapshotBatchOp, fromOmSnapshot); + fromSnapshotStore.commitBatchOperation(fromSnapshotBatchOp); + fromSnapshotStore.getDb().flushWal(true); + fromSnapshotStore.getDb().flush(); + } } + } - private void deleteDirsFromSnapshot(BatchOperation batchOp) + private void deleteDirsFromSnapshot(BatchOperation batchOp, + OmSnapshot fromOmSnapshot) throws IOException { for (String movedDirsKey : movedDirs) { // Delete dirs from current snapshot that are moved to next snapshot. - fromSnapshot.getMetadataManager().getDeletedDirTable() + fromOmSnapshot + .getMetadataManager() + .getDeletedDirTable() .deleteWithBatch(batchOp, movedDirsKey); } } @@ -139,10 +179,12 @@ private void processReclaimKeys(BatchOperation batchOp, } private void processDirs(BatchOperation batchOp, - OMMetadataManager omMetadataManager) + OMMetadataManager omMetadataManager, + OmSnapshot fromOmSnapshot) throws IOException { for (String movedDirsKey : movedDirs) { - OmKeyInfo keyInfo = fromSnapshot.getMetadataManager().getDeletedDirTable() + OmKeyInfo keyInfo = fromOmSnapshot.getMetadataManager() + .getDeletedDirTable() .get(movedDirsKey); if (keyInfo == null) { continue; 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 5a39afe502ec..59b116bca5d2 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 @@ -25,12 +25,15 @@ 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.IOmMetadataReader; 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.helpers.OmKeyInfo; +import org.apache.hadoop.ozone.om.snapshot.ReferenceCounted; +import org.apache.hadoop.ozone.om.snapshot.SnapshotCache; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.PurgePathRequest; import org.apache.hadoop.util.Time; import org.apache.ratis.protocol.ClientId; @@ -206,32 +209,38 @@ private boolean previousSnapshotHasDir( OmMetadataManagerImpl metadataManager = (OmMetadataManagerImpl) getOzoneManager().getMetadataManager(); - OmSnapshot latestSnapshot = metadataManager.getLatestActiveSnapshot( - deletedDirInfo.getVolumeName(), deletedDirInfo.getBucketName(), - omSnapshotManager); - - if (latestSnapshot != null) { - String dbRenameKey = metadataManager - .getRenameKey(deletedDirInfo.getVolumeName(), - deletedDirInfo.getBucketName(), deletedDirInfo.getObjectID()); - Table prevDirTable = - latestSnapshot.getMetadataManager().getDirectoryTable(); - Table prevDeletedDirTable = - latestSnapshot.getMetadataManager().getDeletedDirTable(); - OmKeyInfo prevDeletedDirInfo = prevDeletedDirTable.get(key); - if (prevDeletedDirInfo != null) { - return true; + try (ReferenceCounted rcLatestSnapshot = + metadataManager.getLatestActiveSnapshot( + deletedDirInfo.getVolumeName(), + deletedDirInfo.getBucketName(), + omSnapshotManager)) { + + if (rcLatestSnapshot != null) { + String dbRenameKey = metadataManager + .getRenameKey(deletedDirInfo.getVolumeName(), + deletedDirInfo.getBucketName(), deletedDirInfo.getObjectID()); + Table prevDirTable = + ((OmSnapshot) rcLatestSnapshot.get()) + .getMetadataManager().getDirectoryTable(); + Table prevDeletedDirTable = + ((OmSnapshot) rcLatestSnapshot.get()) + .getMetadataManager().getDeletedDirTable(); + OmKeyInfo prevDeletedDirInfo = prevDeletedDirTable.get(key); + if (prevDeletedDirInfo != null) { + return true; + } + 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); + return prevDirInfo != null && + prevDirInfo.getObjectID() == deletedDirInfo.getObjectID(); } - 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); - return prevDirInfo != null && - prevDirInfo.getObjectID() == deletedDirInfo.getObjectID(); } + return false; } } 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 92ff8861433e..5a1f8336fc0c 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 @@ -31,6 +31,7 @@ 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.IOmMetadataReader; import org.apache.hadoop.ozone.om.KeyManager; import org.apache.hadoop.ozone.om.OmMetadataManagerImpl; import org.apache.hadoop.ozone.om.OmSnapshot; @@ -38,6 +39,8 @@ import org.apache.hadoop.ozone.om.OzoneManager; import org.apache.hadoop.ozone.om.helpers.OMRatisHelper; import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer; +import org.apache.hadoop.ozone.om.snapshot.ReferenceCounted; +import org.apache.hadoop.ozone.om.snapshot.SnapshotCache; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotPurgeRequest; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type; @@ -235,107 +238,119 @@ private void processSnapshotDeepClean(int delCount) continue; } - OmSnapshot currOmSnapshot = (OmSnapshot) omSnapshotManager - .checkForSnapshot(currSnapInfo.getVolumeName(), + try (ReferenceCounted + rcCurrOmSnapshot = omSnapshotManager.checkForSnapshot( + currSnapInfo.getVolumeName(), currSnapInfo.getBucketName(), getSnapshotPrefix(currSnapInfo.getName()), - true); - - 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."); - } + true)) { + OmSnapshot currOmSnapshot = (OmSnapshot) 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 = getPreviousActiveSnapshot( - currSnapInfo, snapChainManager, omSnapshotManager); - Table previousKeyTable = null; - OmSnapshot omPreviousSnapshot = null; - - // Split RepeatedOmKeyInfo and update current snapshot deletedKeyTable - // and next snapshot deletedKeyTable. - if (previousSnapshot != null) { - omPreviousSnapshot = (OmSnapshot) omSnapshotManager - .checkForSnapshot(previousSnapshot.getVolumeName(), - previousSnapshot.getBucketName(), - getSnapshotPrefix(previousSnapshot.getName()), true); - - previousKeyTable = omPreviousSnapshot - .getMetadataManager().getKeyTable(bucketInfo.getBucketLayout()); - } + String snapshotBucketKey = dbBucketKey + OzoneConsts.OM_KEY_PREFIX; + SnapshotInfo previousSnapshot = getPreviousActiveSnapshot( + currSnapInfo, snapChainManager, omSnapshotManager); + Table previousKeyTable = null; + ReferenceCounted + rcPrevOmSnapshot = null; + OmSnapshot omPreviousSnapshot = null; + + // Split RepeatedOmKeyInfo and update current snapshot + // deletedKeyTable and next snapshot deletedKeyTable. + if (previousSnapshot != null) { + rcPrevOmSnapshot = omSnapshotManager.checkForSnapshot( + previousSnapshot.getVolumeName(), + previousSnapshot.getBucketName(), + getSnapshotPrefix(previousSnapshot.getName()), true); + omPreviousSnapshot = (OmSnapshot) rcPrevOmSnapshot.get(); + + previousKeyTable = omPreviousSnapshot.getMetadataManager() + .getKeyTable(bucketInfo.getBucketLayout()); + } - try (TableIterator> deletedIterator = snapDeletedTable - .iterator()) { + try (TableIterator> deletedIterator = snapDeletedTable + .iterator()) { - deletedIterator.seek(snapshotBucketKey); - while (deletedIterator.hasNext() && delCount < keyLimitPerTask) { - Table.KeyValue - deletedKeyValue = deletedIterator.next(); - String deletedKey = deletedKeyValue.getKey(); + deletedIterator.seek(snapshotBucketKey); + while (deletedIterator.hasNext() && delCount < keyLimitPerTask) { + Table.KeyValue + deletedKeyValue = deletedIterator.next(); + String deletedKey = deletedKeyValue.getKey(); - // Exit if it is out of the bucket scope. - if (!deletedKey.startsWith(snapshotBucketKey)) { - break; - } + // 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 (isKeyReclaimable(previousKeyTable, snapRenamedTable, - keyInfo, bucketInfo, volumeId, null)) { - List blocksForKeyDelete = currOmSnapshot - .getMetadataManager() - .getBlocksForKeyDelete(deletedKey); - if (blocksForKeyDelete != null) { - blockGroupList.addAll(blocksForKeyDelete); + RepeatedOmKeyInfo repeatedOmKeyInfo = + deletedKeyValue.getValue(); + + List blockGroupList = new ArrayList<>(); + RepeatedOmKeyInfo newRepeatedOmKeyInfo = + new RepeatedOmKeyInfo(); + for (OmKeyInfo keyInfo : repeatedOmKeyInfo.getOmKeyInfoList()) { + 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); } - delCount++; - } else { - newRepeatedOmKeyInfo.addOmKeyInfo(keyInfo); } - } - if (newRepeatedOmKeyInfo.getOmKeyInfoList().size() > 0 && - newRepeatedOmKeyInfo.getOmKeyInfoList().size() != - repeatedOmKeyInfo.getOmKeyInfoList().size()) { - keysToModify.put(deletedKey, newRepeatedOmKeyInfo); - } + 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); + if (newRepeatedOmKeyInfo.getOmKeyInfoList().size() != + repeatedOmKeyInfo.getOmKeyInfoList().size()) { + keysToPurge.addAll(blockGroupList); + } } - } - if (delCount < keyLimitPerTask) { - // Deep clean is completed, we can update the SnapInfo. - deepCleanedSnapshots.add(currSnapInfo.getTableKey()); - } + if (delCount < keyLimitPerTask) { + // Deep clean is completed, we can update the SnapInfo. + deepCleanedSnapshots.add(currSnapInfo.getTableKey()); + } - if (!keysToPurge.isEmpty()) { - processKeyDeletes(keysToPurge, currOmSnapshot.getKeyManager(), - keysToModify, currSnapInfo.getTableKey()); + if (!keysToPurge.isEmpty()) { + processKeyDeletes(keysToPurge, currOmSnapshot.getKeyManager(), + keysToModify, currSnapInfo.getTableKey()); + } + } finally { + if (previousSnapshot != null) { + rcPrevOmSnapshot.close(); + } } - } + } } updateDeepCleanedSnapshots(deepCleanedSnapshots); 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 7c542e5c963f..60c4b9422d39 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 @@ -33,6 +33,7 @@ import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.common.BlockGroup; import org.apache.hadoop.ozone.lock.BootstrapStateHandler; +import org.apache.hadoop.ozone.om.IOmMetadataReader; import org.apache.hadoop.ozone.om.OmMetadataManagerImpl; import org.apache.hadoop.ozone.om.OmSnapshot; import org.apache.hadoop.ozone.om.OmSnapshotManager; @@ -47,6 +48,8 @@ import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo; import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer; +import org.apache.hadoop.ozone.om.snapshot.ReferenceCounted; +import org.apache.hadoop.ozone.om.snapshot.SnapshotCache; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.PurgePathRequest; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest; @@ -122,13 +125,18 @@ private class SnapshotDeletingTask implements BackgroundTask { @SuppressWarnings("checkstyle:MethodLength") @Override - public BackgroundTaskResult call() throws Exception { + public BackgroundTaskResult call() throws InterruptedException { if (!shouldRun()) { return BackgroundTaskResult.EmptyTaskResult.newResult(); } getRunCount().incrementAndGet(); + ReferenceCounted rcOmSnapshot = + null; + ReferenceCounted rcOmPreviousSnapshot = + null; + Table snapshotInfoTable = ozoneManager.getMetadataManager().getSnapshotInfoTable(); List purgeSnapshotKeys = new ArrayList<>(); @@ -148,10 +156,14 @@ public BackgroundTaskResult call() throws Exception { continue; } - OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager - .checkForSnapshot(snapInfo.getVolumeName(), - snapInfo.getBucketName(), - getSnapshotPrefix(snapInfo.getName()), true); + // Note: Can refactor this to use try-with-resources. + // Handling RC decrements manually for now to minimize conflicts. + rcOmSnapshot = omSnapshotManager.checkForSnapshot( + snapInfo.getVolumeName(), + snapInfo.getBucketName(), + getSnapshotPrefix(snapInfo.getName()), + true); + OmSnapshot omSnapshot = (OmSnapshot) rcOmSnapshot.get(); Table snapshotDeletedTable = omSnapshot.getMetadataManager().getDeletedTable(); @@ -170,6 +182,9 @@ public BackgroundTaskResult call() throws Exception { .getBucketTable().get(dbBucketKey); if (bucketInfo == null) { + // Decrement ref count + rcOmSnapshot.close(); + rcOmSnapshot = null; throw new IllegalStateException("Bucket " + "/" + snapInfo.getVolumeName() + "/" + snapInfo.getBucketName() + " is not found. BucketInfo should not be null for snapshotted" + @@ -184,6 +199,9 @@ public BackgroundTaskResult call() throws Exception { if (isSnapshotReclaimable(snapshotDeletedTable, snapshotDeletedDirTable, snapshotBucketKey, dbBucketKeyForDir)) { purgeSnapshotKeys.add(snapInfo.getTableKey()); + // Decrement ref count + rcOmSnapshot.close(); + rcOmSnapshot = null; continue; } @@ -197,10 +215,12 @@ public BackgroundTaskResult call() throws Exception { // Split RepeatedOmKeyInfo and update current snapshot deletedKeyTable // and next snapshot deletedKeyTable. if (previousSnapshot != null) { - omPreviousSnapshot = (OmSnapshot) omSnapshotManager - .checkForSnapshot(previousSnapshot.getVolumeName(), - previousSnapshot.getBucketName(), - getSnapshotPrefix(previousSnapshot.getName()), true); + rcOmPreviousSnapshot = omSnapshotManager.checkForSnapshot( + previousSnapshot.getVolumeName(), + previousSnapshot.getBucketName(), + getSnapshotPrefix(previousSnapshot.getName()), + true); + omPreviousSnapshot = (OmSnapshot) rcOmPreviousSnapshot.get(); previousKeyTable = omPreviousSnapshot .getMetadataManager().getKeyTable(bucketInfo.getBucketLayout()); @@ -295,9 +315,23 @@ public BackgroundTaskResult call() throws Exception { // Submit Move request to OM. submitSnapshotMoveDeletedKeys(snapInfo, toReclaimList, toNextDBList, renamedList, dirsToMove); + + // Properly decrement ref count for rcOmPreviousSnapshot + if (rcOmPreviousSnapshot != null) { + rcOmPreviousSnapshot.close(); + rcOmPreviousSnapshot = null; + } } } catch (IOException e) { LOG.error("Error while running Snapshot Deleting Service", e); + } finally { + // Decrement ref counts + if (rcOmPreviousSnapshot != null) { + rcOmPreviousSnapshot.close(); + } + if (rcOmSnapshot != null) { + rcOmSnapshot.close(); + } } submitSnapshotPurgeRequest(purgeSnapshotKeys); diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/ReferenceCounted.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/ReferenceCounted.java new file mode 100644 index 000000000000..e064812de8ef --- /dev/null +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/ReferenceCounted.java @@ -0,0 +1,173 @@ +/* + * 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 com.google.common.base.Preconditions; + +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicLong; + +/** + * Add reference counter to an object instance. + */ +public class ReferenceCounted + implements AutoCloseable { + + /** + * Object that is being reference counted. e.g. OmSnapshot + */ + private final T obj; + + /** + * A map of thread IDs holding the reference of the object and its count. + */ + private final ConcurrentHashMap threadMap; + + /** + * Sum of reference counts from all threads. + */ + private final AtomicLong refCount; + + /** + * Object lock to synchronize refCount and threadMap operations. + */ + private final Object refCountLock = new Object(); + + /** + * Parent instance whose callback will be triggered upon this RC closure. + */ + private final U parentWithCallback; + + public ReferenceCounted(T obj, boolean disableCounter, + U parentWithCallback) { + // A param to allow disabling ref counting to reduce active DB + // access penalties due to AtomicLong operations. + this.obj = obj; + if (disableCounter) { + this.threadMap = null; + this.refCount = null; + } else { + this.threadMap = new ConcurrentHashMap<>(); + this.refCount = new AtomicLong(0L); + } + this.parentWithCallback = parentWithCallback; + } + + /** + * @return Object being referenced counted. + */ + public T get() { + return obj; + } + + public long incrementRefCount() { + if (refCount == null || threadMap == null) { + return -1L; + } + + long tid = Thread.currentThread().getId(); + + threadMap.putIfAbsent(tid, 0L); + + synchronized (refCountLock) { + threadMap.computeIfPresent(tid, (k, v) -> { + long newVal = v + 1; + Preconditions.checkState(newVal > 0L, + "Thread reference count overflown"); + return newVal; + }); + + long newValTotal = refCount.incrementAndGet(); + Preconditions.checkState(newValTotal > 0L, + "Total reference count overflown"); + + if (refCount.get() == 1L) { + // ref count increased to one (from zero), remove from + // pendingEvictionList if added + parentWithCallback.callback(this); + } + } + + return refCount.get(); + } + + public long decrementRefCount() { + if (refCount == null || threadMap == null) { + return -1L; + } + + long tid = Thread.currentThread().getId(); + + Preconditions.checkState(threadMap.containsKey(tid), + "Current thread have not holden reference before"); + + Preconditions.checkState(threadMap.get(tid) > 0L, "This thread " + tid + + " already have a reference count of zero."); + + synchronized (refCountLock) { + threadMap.computeIfPresent(tid, (k, v) -> { + long newValue = v - 1L; + Preconditions.checkState(newValue >= 0L, + "Thread reference count underflow"); + // Remove entry by returning null here if thread ref count reaches zero + return newValue != 0L ? newValue : null; + }); + + long newValTotal = refCount.decrementAndGet(); + Preconditions.checkState(newValTotal >= 0L, + "Total reference count underflow"); + + if (refCount.get() == 0L) { + // ref count decreased to zero, add to pendingEvictionList + parentWithCallback.callback(this); + } + } + + return refCount.get(); + } + + /** + * @return The total number of times the object has been held reference to. + */ + public long getTotalRefCount() { + if (refCount == null) { + return -1L; + } + + return refCount.get(); + } + + /** + * @return Number of times current thread has held reference to the object. + */ + public long getCurrentThreadRefCount() { + if (refCount == null || threadMap == null) { + return -1L; + } + + long tid = Thread.currentThread().getId(); + return threadMap.getOrDefault(tid, 0L); + } + + @Override + public void close() { + // Decrease ref count by 1 when close() is called on this object + // so it is eligible to be used with try-with-resources. + decrementRefCount(); + } +} diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/ReferenceCountedCallback.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/ReferenceCountedCallback.java new file mode 100644 index 000000000000..d63f5783b808 --- /dev/null +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/ReferenceCountedCallback.java @@ -0,0 +1,25 @@ +/* + * 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; + +/** + * Callback interface for ReferenceCounted. + */ +public interface ReferenceCountedCallback { + void callback(ReferenceCounted referenceCounted); +} diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotCache.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotCache.java new file mode 100644 index 000000000000..78e43af7501b --- /dev/null +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotCache.java @@ -0,0 +1,397 @@ +/* + * 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 com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.cache.CacheLoader; +import org.apache.hadoop.ozone.om.IOmMetadataReader; +import org.apache.hadoop.ozone.om.OmSnapshot; +import org.apache.hadoop.ozone.om.OmSnapshotManager; +import org.apache.hadoop.ozone.om.exceptions.OMException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Collections; +import java.util.Iterator; +import java.util.LinkedHashSet; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.FILE_NOT_FOUND; +import static org.apache.hadoop.ozone.om.helpers.SnapshotInfo.SnapshotStatus.SNAPSHOT_ACTIVE; + +/** + * Thread-safe custom unbounded LRU cache to manage open snapshot DB instances. + */ +public class SnapshotCache implements ReferenceCountedCallback { + + static final Logger LOG = LoggerFactory.getLogger(SnapshotCache.class); + + // Snapshot cache internal hash map. + // Key: DB snapshot table key + // Value: OmSnapshot instance, each holds a DB instance handle inside + // TODO: [SNAPSHOT] Consider wrapping SoftReference<> around IOmMetadataReader + private final ConcurrentHashMap> dbMap; + + // Linked hash set that holds OmSnapshot instances whose reference count + // has reached zero. Those entries are eligible to be evicted and closed. + // Sorted in last used order. + // Least-recently-used entry located at the beginning. + private final Set< + ReferenceCounted> pendingEvictionList; + private final OmSnapshotManager omSnapshotManager; + private final CacheLoader cacheLoader; + // Soft-limit of the total number of snapshot DB instances allowed to be + // opened on the OM. + private final int cacheSizeLimit; + + public SnapshotCache( + OmSnapshotManager omSnapshotManager, + CacheLoader cacheLoader, + int cacheSizeLimit) { + this.dbMap = new ConcurrentHashMap<>(); + this.pendingEvictionList = + Collections.synchronizedSet(new LinkedHashSet<>()); + this.omSnapshotManager = omSnapshotManager; + this.cacheLoader = cacheLoader; + this.cacheSizeLimit = cacheSizeLimit; + } + + @VisibleForTesting + ConcurrentHashMap> getDbMap() { + return dbMap; + } + + @VisibleForTesting + Set> getPendingEvictionList() { + return pendingEvictionList; + } + + /** + * @return number of DB instances currently held in cache. + */ + public int size() { + return dbMap.size(); + } + + /** + * Immediately invalidate an entry. + * @param key DB snapshot table key + */ + public void invalidate(String key) throws IOException { + dbMap.computeIfPresent(key, (k, v) -> { + pendingEvictionList.remove(v); + try { + ((OmSnapshot) v.get()).close(); + } catch (IOException e) { + throw new IllegalStateException("Failed to close snapshot: " + key, e); + } + // Remove the entry from map by returning null + return null; + }); + } + + /** + * Immediately invalidate all entries and close their DB instances in cache. + */ + public void invalidateAll() { + Iterator< + Map.Entry>> + it = dbMap.entrySet().iterator(); + + while (it.hasNext()) { + Map.Entry> + entry = it.next(); + pendingEvictionList.remove(entry.getValue()); + OmSnapshot omSnapshot = (OmSnapshot) entry.getValue().get(); + try { + // TODO: If wrapped with SoftReference<>, omSnapshot could be null? + omSnapshot.close(); + } catch (IOException e) { + throw new IllegalStateException("Failed to close snapshot", e); + } + it.remove(); + } + } + + /** + * State the reason the current thread is getting the OmSnapshot instance. + * Unused for now. + */ + public enum Reason { + FS_API_READ, + SNAPDIFF_READ, + DEEP_CLEAN_WRITE, + GARBAGE_COLLECTION_WRITE + } + + public ReferenceCounted get(String key) + throws IOException { + return get(key, false); + } + + /** + * Get or load OmSnapshot. Shall be close()d after use. + * TODO: [SNAPSHOT] Can add reason enum to param list later. + * @param key snapshot table key + * @return an OmSnapshot instance, or null on error + */ + public ReferenceCounted get(String key, + boolean skipActiveCheck) throws IOException { + // Atomic operation to initialize the OmSnapshot instance (once) if the key + // does not exist, and increment the reference count on the instance. + ReferenceCounted rcOmSnapshot = + dbMap.compute(key, (k, v) -> { + LOG.info("Loading snapshot. Table key: {}", k); + if (v == null) { + try { + v = new ReferenceCounted<>(cacheLoader.load(k), false, this); + } catch (OMException omEx) { + // Return null if the snapshot is no longer active + if (!omEx.getResult().equals(FILE_NOT_FOUND)) { + throw new IllegalStateException(omEx); + } + } catch (IOException ioEx) { + // Failed to load snapshot DB + throw new IllegalStateException(ioEx); + } catch (Exception ex) { + // Unexpected and unknown exception thrown from CacheLoader#load + throw new IllegalStateException(ex); + } + } + if (v != null) { + // When RC OmSnapshot is successfully loaded + v.incrementRefCount(); + } + return v; + }); + + if (rcOmSnapshot == null) { + // The only exception that would fall through the loader logic above + // is OMException with FILE_NOT_FOUND. + throw new OMException("Snapshot table key '" + key + "' not found, " + + "or the snapshot is no longer active", + OMException.ResultCodes.FILE_NOT_FOUND); + } + + // If the snapshot is already loaded in cache, the check inside the loader + // above is ignored. But we would still want to reject all get()s except + // when called from SDT (and some) if the snapshot is not active anymore. + if (!skipActiveCheck && + !omSnapshotManager.isSnapshotStatus(key, SNAPSHOT_ACTIVE)) { + // Ref count was incremented. Need to decrement on exception here. + rcOmSnapshot.decrementRefCount(); + throw new OMException("Unable to load snapshot. " + + "Snapshot with table key '" + key + "' is no longer active", + FILE_NOT_FOUND); + } + + synchronized (pendingEvictionList) { + // Remove instance from clean up list when it exists. + pendingEvictionList.remove(rcOmSnapshot); + } + + // Check if any entries can be cleaned up. + // At this point, cache size might temporarily exceed cacheSizeLimit + // even if there are entries that can be evicted, which is fine since it + // is a soft limit. + cleanup(); + + return rcOmSnapshot; + } + + /** + * Release the reference count on the OmSnapshot instance. + * @param key snapshot table key + */ + public void release(String key) { + dbMap.compute(key, (k, v) -> { + if (v == null) { + throw new IllegalArgumentException( + "Key '" + key + "' does not exist in cache"); + } + + if (v.decrementRefCount() == 0L) { + synchronized (pendingEvictionList) { + // v is eligible to be evicted and closed + pendingEvictionList.add(v); + } + } + + return v; + }); + + // The cache size might have already exceed the soft limit + // Thus triggering cleanup() to check and evict if applicable + cleanup(); + } + + /** + * Alternatively, can release with OmSnapshot instance directly. + * @param omSnapshot OmSnapshot + */ + public void release(OmSnapshot omSnapshot) { + final String snapshotTableKey = omSnapshot.getSnapshotTableKey(); + release(snapshotTableKey); + } + + /** + * Callback method used to enqueue or dequeue ReferenceCounted from + * pendingEvictionList. + * @param referenceCounted ReferenceCounted object + */ + @Override + public void callback(ReferenceCounted referenceCounted) { + synchronized (pendingEvictionList) { + if (referenceCounted.getTotalRefCount() == 0L) { + // Reference count reaches zero, add to pendingEvictionList + Preconditions.checkState( + !pendingEvictionList.contains(referenceCounted), + "SnapshotCache is inconsistent. Entry should not be in the " + + "pendingEvictionList when ref count just reached zero."); + pendingEvictionList.add(referenceCounted); + } else if (referenceCounted.getTotalRefCount() == 1L) { + pendingEvictionList.remove(referenceCounted); + } + } + } + + /** + * Wrapper for cleanupInternal() that is synchronized to prevent multiple + * threads from interleaving into the cleanup method. + */ + private synchronized void cleanup() { + synchronized (pendingEvictionList) { + cleanupInternal(); + } + } + + /** + * If cache size exceeds soft limit, attempt to clean up and close the + * instances that has zero reference count. + * TODO: [SNAPSHOT] Add new ozone debug CLI command to trigger this directly. + */ + private void cleanupInternal() { + long numEntriesToEvict = (long) dbMap.size() - cacheSizeLimit; + while (numEntriesToEvict > 0L && pendingEvictionList.size() > 0) { + // Get the first instance in the clean up list + ReferenceCounted rcOmSnapshot = + pendingEvictionList.iterator().next(); + OmSnapshot omSnapshot = (OmSnapshot) rcOmSnapshot.get(); + LOG.debug("Evicting OmSnapshot instance {} with table key {}", + rcOmSnapshot, omSnapshot.getSnapshotTableKey()); + // Sanity check + Preconditions.checkState(rcOmSnapshot.getTotalRefCount() == 0L, + "Illegal state: OmSnapshot reference count non-zero (" + + rcOmSnapshot.getTotalRefCount() + ") but shows up in the " + + "clean up list"); + + final String key = omSnapshot.getSnapshotTableKey(); + final ReferenceCounted result = + dbMap.remove(key); + // Sanity check + Preconditions.checkState(rcOmSnapshot == result, + "Cache map entry removal failure. The cache is in an inconsistent " + + "state. Expected OmSnapshot instance: " + rcOmSnapshot + + ", actual: " + result); + + pendingEvictionList.remove(result); + + // Close the instance, which also closes its DB handle. + try { + ((OmSnapshot) rcOmSnapshot.get()).close(); + } catch (IOException ex) { + throw new IllegalStateException("Error while closing snapshot DB", ex); + } + + --numEntriesToEvict; + } + + // Print warning message if actual cache size is exceeding the soft limit + // even after the cleanup procedure above. + if ((long) dbMap.size() > cacheSizeLimit) { + LOG.warn("Current snapshot cache size ({}) is exceeding configured " + + "soft-limit ({}) after possible evictions.", + dbMap.size(), cacheSizeLimit); + + Preconditions.checkState(pendingEvictionList.size() == 0); + } + } + + /** + * Check cache consistency. + * @return true if the cache internal structure is consistent to the best of + * its knowledge, false if found to be inconsistent and details logged. + */ + @VisibleForTesting + public boolean isConsistent() { + // Uses dbMap as the source of truth for this check, whether dbMap entries + // are in OM DB's snapshotInfoTable is out of the scope of this check. + + LOG.info("dbMap has {} entries", dbMap.size()); + LOG.info("pendingEvictionList has {} entries", + pendingEvictionList.size()); + + // pendingEvictionList must be empty if cache size exceeds limit + if (dbMap.size() > cacheSizeLimit) { + if (pendingEvictionList.size() != 0) { + // cleanup() is not functioning correctly + LOG.error("pendingEvictionList is not empty even when cache size" + + "exceeds limit"); + } + } + + dbMap.forEach((k, v) -> { + if (v.getTotalRefCount() == 0L) { + long threadRefCount = v.getCurrentThreadRefCount(); + if (threadRefCount != 0L) { + LOG.error("snapshotTableKey='{}' instance has inconsistent " + + "ref count. Total ref count is 0 but thread " + + "ref count is {}", k, threadRefCount); + } + // Zero ref count values in dbMap must be in pendingEvictionList + if (!pendingEvictionList.contains(v)) { + LOG.error("snapshotTableKey='{}' instance has zero ref count but " + + "not in pendingEvictionList", k); + } + } + }); + + pendingEvictionList.forEach(v -> { + // Objects in pendingEvictionList should still be in dbMap + if (!dbMap.contains(v)) { + LOG.error("Instance '{}' is in pendingEvictionList but not in " + + "dbMap", v); + } + // Instances in pendingEvictionList must have ref count equals 0 + if (v.getTotalRefCount() != 0L) { + LOG.error("Instance '{}' is in pendingEvictionList but ref count " + + "is not zero", v); + } + }); + + return true; + } + +} 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 244c64674400..0378e9edc0a5 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 @@ -19,7 +19,6 @@ package org.apache.hadoop.ozone.om.snapshot; import com.google.common.annotations.VisibleForTesting; -import com.google.common.cache.LoadingCache; import com.google.common.collect.Maps; import com.google.common.util.concurrent.ThreadFactoryBuilder; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; @@ -38,6 +37,7 @@ import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry; import org.apache.hadoop.ozone.OFSPath; import org.apache.hadoop.ozone.OzoneConsts; +import org.apache.hadoop.ozone.om.IOmMetadataReader; import org.apache.hadoop.ozone.om.OMConfigKeys; import org.apache.hadoop.ozone.om.OMMetadataManager; import org.apache.hadoop.ozone.om.OmMetadataManagerImpl; @@ -89,7 +89,6 @@ import java.util.UUID; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.Callable; -import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.SynchronousQueue; @@ -152,7 +151,7 @@ public class SnapshotDiffManager implements AutoCloseable { private final ManagedRocksDB db; private final RocksDBCheckpointDiffer differ; private final OzoneManager ozoneManager; - private final LoadingCache snapshotCache; + private final SnapshotCache snapshotCache; private final CodecRegistry codecRegistry; private final ManagedColumnFamilyOptions familyOptions; // TODO: [SNAPSHOT] Use different wait time based of job status. @@ -198,7 +197,7 @@ public class SnapshotDiffManager implements AutoCloseable { public SnapshotDiffManager(ManagedRocksDB db, RocksDBCheckpointDiffer differ, OzoneManager ozoneManager, - LoadingCache snapshotCache, + SnapshotCache snapshotCache, ColumnFamilyHandle snapDiffJobCfh, ColumnFamilyHandle snapDiffReportCfh, ManagedColumnFamilyOptions familyOptions, @@ -818,8 +817,8 @@ void generateSnapshotDiffReport(final String jobKey, final String fromSnapshotName, final String toSnapshotName, final boolean forceFullDiff) { - LOG.info("Started snap diff report generation for volume: {} " + - "bucket: {}, fromSnapshot: {} and toSnapshot: {}", + LOG.info("Started snap diff report generation for volume: '{}', " + + "bucket: '{}', fromSnapshot: '{}', toSnapshot: '{}'", volumeName, bucketName, fromSnapshotName, toSnapshotName); ColumnFamilyHandle fromSnapshotColumnFamily = null; @@ -832,6 +831,9 @@ void generateSnapshotDiffReport(final String jobKey, // job by RocksDBCheckpointDiffer#pruneOlderSnapshotsWithCompactionHistory. Path path = Paths.get(sstBackupDirForSnapDiffJobs + "/" + jobId); + ReferenceCounted rcFromSnapshot = null; + ReferenceCounted rcToSnapshot = null; + try { if (!areDiffJobAndSnapshotsActive(volumeName, bucketName, fromSnapshotName, toSnapshotName)) { @@ -841,8 +843,11 @@ void generateSnapshotDiffReport(final String jobKey, String fsKey = getTableKey(volumeName, bucketName, fromSnapshotName); String tsKey = getTableKey(volumeName, bucketName, toSnapshotName); - OmSnapshot fromSnapshot = snapshotCache.get(fsKey); - OmSnapshot toSnapshot = snapshotCache.get(tsKey); + rcFromSnapshot = snapshotCache.get(fsKey); + rcToSnapshot = snapshotCache.get(tsKey); + + OmSnapshot fromSnapshot = (OmSnapshot) rcFromSnapshot.get(); + OmSnapshot toSnapshot = (OmSnapshot) rcToSnapshot.get(); SnapshotInfo fsInfo = getSnapshotInfo(ozoneManager, volumeName, bucketName, fromSnapshotName); SnapshotInfo tsInfo = getSnapshotInfo(ozoneManager, @@ -991,7 +996,7 @@ void generateSnapshotDiffReport(final String jobKey, } methodCall.call(); } - } catch (ExecutionException | IOException | RocksDBException exception) { + } catch (IOException | RocksDBException exception) { updateJobStatusToFailed(jobKey, exception.getMessage()); LOG.error("Caught checked exception during diff report generation for " + "volume: {} bucket: {}, fromSnapshot: {} and toSnapshot: {}", @@ -1014,6 +1019,13 @@ void generateSnapshotDiffReport(final String jobKey, dropAndCloseColumnFamilyHandle(objectIDsColumnFamily); // Delete SST files backup directory. deleteDir(path); + // Decrement ref counts + if (rcFromSnapshot != null) { + rcFromSnapshot.close(); + } + if (rcToSnapshot != null) { + rcToSnapshot.close(); + } } } diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOMMultiTenantManagerImpl.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOMMultiTenantManagerImpl.java index 9029c5b8395c..073f1d334fcc 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOMMultiTenantManagerImpl.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOMMultiTenantManagerImpl.java @@ -66,7 +66,7 @@ public void setUp() throws IOException { conf.set(OZONE_OM_DB_DIRS, folder.newFolder().getAbsolutePath()); conf.set(OZONE_OM_TENANT_DEV_SKIP_RANGER, "true"); - omMetadataManager = new OmMetadataManagerImpl(conf); + omMetadataManager = new OmMetadataManagerImpl(conf, ozoneManager); createTenantInDB(TENANT_ID); assignUserToTenantInDB(TENANT_ID, "seed-accessId1", "seed-user1", false, diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOMTenantCreateRequest.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOMTenantCreateRequest.java index fb95fba6ed85..ff7407e3dbad 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOMTenantCreateRequest.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOMTenantCreateRequest.java @@ -69,7 +69,8 @@ public void setup() throws Exception { OzoneConfiguration ozoneConfiguration = new OzoneConfiguration(); ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS, folder.newFolder().getAbsolutePath()); - omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration); + omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration, + ozoneManager); when(ozoneManager.getMetrics()).thenReturn(omMetrics); when(ozoneManager.getMetadataManager()).thenReturn(omMetadataManager); when(ozoneManager.getMaxUserVolumeCount()).thenReturn(10L); diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOmMetadataManager.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOmMetadataManager.java index 8f51143d5e95..148dc94b2208 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOmMetadataManager.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOmMetadataManager.java @@ -82,7 +82,7 @@ public void setup() throws Exception { ozoneConfiguration = new OzoneConfiguration(); ozoneConfiguration.set(OZONE_OM_DB_DIRS, folder.getAbsolutePath()); - omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration); + omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration, null); } @Test diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOmSnapshotManager.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOmSnapshotManager.java index d5887126de55..e15a540487ca 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOmSnapshotManager.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOmSnapshotManager.java @@ -160,7 +160,7 @@ public void testCloseOnEviction() throws IOException { OmSnapshotManager omSnapshotManager = om.getOmSnapshotManager(); OmSnapshot firstSnapshot = (OmSnapshot) omSnapshotManager .checkForSnapshot(first.getVolumeName(), - first.getBucketName(), getSnapshotPrefix(first.getName()), false); + first.getBucketName(), getSnapshotPrefix(first.getName()), false).get(); DBStore firstSnapshotStore = mock(DBStore.class); HddsWhiteboxTestUtils.setInternalState( firstSnapshot.getMetadataManager(), "store", firstSnapshotStore); diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestSnapshotChain.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestSnapshotChain.java index 6613ffeb7edd..53527a3f37fc 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestSnapshotChain.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestSnapshotChain.java @@ -58,7 +58,7 @@ public class TestSnapshotChain { public void setup() throws Exception { OzoneConfiguration conf = new OzoneConfiguration(); conf.set(OZONE_OM_DB_DIRS, folder.toString()); - omMetadataManager = new OmMetadataManagerImpl(conf); + omMetadataManager = new OmMetadataManagerImpl(conf, null); snapshotIdToSnapshotInfoMap = new HashMap<>(); chainManager = new SnapshotChainManager(omMetadataManager); } diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestSnapshotInfo.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestSnapshotInfo.java index d3651b06d2d1..a1abd9a4be6a 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestSnapshotInfo.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestSnapshotInfo.java @@ -51,7 +51,7 @@ public void setup() throws Exception { OzoneConfiguration conf = new OzoneConfiguration(); conf.set(OZONE_OM_DB_DIRS, folder.getRoot().getAbsolutePath()); - omMetadataManager = new OmMetadataManagerImpl(conf); + omMetadataManager = new OmMetadataManagerImpl(conf, null); } private SnapshotInfo createSnapshotInfo() { diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerDoubleBuffer.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerDoubleBuffer.java index 939f5a210e25..22279dc23eb4 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerDoubleBuffer.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerDoubleBuffer.java @@ -96,7 +96,7 @@ public void setup() throws IOException { ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS, tempDir.getAbsolutePath()); OMMetadataManager omMetadataManager = - new OmMetadataManagerImpl(ozoneConfiguration); + new OmMetadataManagerImpl(ozoneConfiguration, null); OzoneManager ozoneManager = mock(OzoneManager.class); when(ozoneManager.getMetrics()).thenReturn(omMetrics); when(ozoneManager.getMetadataManager()).thenReturn(omMetadataManager); diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerDoubleBufferWithDummyResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerDoubleBufferWithDummyResponse.java index 1048a1851a3a..3970a97380a2 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerDoubleBufferWithDummyResponse.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerDoubleBufferWithDummyResponse.java @@ -74,7 +74,7 @@ public void setup() throws IOException { configuration.set(OZONE_METADATA_DIRS, folder.newFolder().getAbsolutePath()); omMetadataManager = - new OmMetadataManagerImpl(configuration); + new OmMetadataManagerImpl(configuration, null); OzoneManagerRatisSnapshot ozoneManagerRatisSnapshot = index -> { lastAppliedIndex = index.get(index.size() - 1); }; diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerDoubleBufferWithOMResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerDoubleBufferWithOMResponse.java index a182e0c94823..0aac1e5653f8 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerDoubleBufferWithOMResponse.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerDoubleBufferWithOMResponse.java @@ -99,7 +99,8 @@ public void setup() throws IOException { OzoneConfiguration ozoneConfiguration = new OzoneConfiguration(); ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS, folder.newFolder().getAbsolutePath()); - omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration); + omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration, + ozoneManager); when(ozoneManager.getMetrics()).thenReturn(omMetrics); when(ozoneManager.getMetadataManager()).thenReturn(omMetadataManager); when(ozoneManager.getMaxUserVolumeCount()).thenReturn(10L); diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerRatisServer.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerRatisServer.java index 2aa74c4c1de0..c04e8fdcdf25 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerRatisServer.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerRatisServer.java @@ -122,7 +122,8 @@ public void init() throws Exception { OzoneConfiguration ozoneConfiguration = new OzoneConfiguration(); ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS, folder.newFolder().getAbsolutePath()); - omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration); + omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration, + ozoneManager); when(ozoneManager.getMetadataManager()).thenReturn(omMetadataManager); initialTermIndex = TermIndex.valueOf(0, 0); RatisSnapshotInfo omRatisSnapshotInfo = new RatisSnapshotInfo(); diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerStateMachine.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerStateMachine.java index db63fadf301a..0cfa9dae9ef5 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerStateMachine.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerStateMachine.java @@ -74,7 +74,8 @@ public void setup() throws Exception { conf.set(OMConfigKeys.OZONE_OM_DB_DIRS, tempDir.newFolder().getAbsolutePath().toString()); - OMMetadataManager omMetadataManager = new OmMetadataManagerImpl(conf); + OMMetadataManager omMetadataManager = new OmMetadataManagerImpl(conf, + ozoneManager); when(ozoneManager.getMetadataManager()).thenReturn(omMetadataManager); diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/TestOMClientRequestWithUserInfo.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/TestOMClientRequestWithUserInfo.java index df53dc972d49..ab289ab1337e 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/TestOMClientRequestWithUserInfo.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/TestOMClientRequestWithUserInfo.java @@ -70,7 +70,8 @@ public void setup() throws Exception { OzoneConfiguration ozoneConfiguration = new OzoneConfiguration(); ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS, folder.newFolder().getAbsolutePath()); - omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration); + omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration, + ozoneManager); when(ozoneManager.getMetrics()).thenReturn(omMetrics); when(ozoneManager.getMetadataManager()).thenReturn(omMetadataManager); inetAddress = InetAddress.getByName("127.0.0.1"); diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/bucket/TestBucketRequest.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/bucket/TestBucketRequest.java index cd036c373e34..a825b82a319d 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/bucket/TestBucketRequest.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/bucket/TestBucketRequest.java @@ -74,7 +74,8 @@ public void setup() throws Exception { ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS, folder.newFolder().getAbsolutePath()); when(ozoneManager.getConfiguration()).thenReturn(ozoneConfiguration); - omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration); + omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration, + ozoneManager); when(ozoneManager.getMetrics()).thenReturn(omMetrics); when(ozoneManager.getMetadataManager()).thenReturn(omMetadataManager); when(ozoneManager.isRatisEnabled()).thenReturn(true); diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/file/TestOMDirectoryCreateRequest.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/file/TestOMDirectoryCreateRequest.java index 9caff1bed723..f3debcc484c7 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/file/TestOMDirectoryCreateRequest.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/file/TestOMDirectoryCreateRequest.java @@ -85,7 +85,8 @@ public void setup() throws Exception { OzoneConfiguration ozoneConfiguration = new OzoneConfiguration(); ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS, folder.newFolder().getAbsolutePath()); - omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration); + omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration, + ozoneManager); when(ozoneManager.getMetrics()).thenReturn(omMetrics); when(ozoneManager.getMetadataManager()).thenReturn(omMetadataManager); auditLogger = Mockito.mock(AuditLogger.class); diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/file/TestOMDirectoryCreateRequestWithFSO.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/file/TestOMDirectoryCreateRequestWithFSO.java index 3dfa79790d59..0a8b4b44b593 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/file/TestOMDirectoryCreateRequestWithFSO.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/file/TestOMDirectoryCreateRequestWithFSO.java @@ -90,7 +90,8 @@ public void setup() throws Exception { ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS, folder.newFolder().getAbsolutePath()); OMRequestTestUtils.configureFSOptimizedPaths(ozoneConfiguration, true); - omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration); + omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration, + ozoneManager); when(ozoneManager.getMetrics()).thenReturn(omMetrics); when(ozoneManager.getMetadataManager()).thenReturn(omMetadataManager); auditLogger = Mockito.mock(AuditLogger.class); 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 3385046e0c1d..1a10a3f71020 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,12 +23,15 @@ import java.util.List; import java.util.UUID; +import org.apache.hadoop.ozone.om.IOmMetadataReader; import org.apache.hadoop.ozone.om.OmSnapshot; import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; import org.apache.hadoop.ozone.om.request.OMRequestTestUtils; import org.apache.hadoop.ozone.om.request.snapshot.OMSnapshotCreateRequest; import org.apache.hadoop.ozone.om.request.snapshot.TestOMSnapshotCreateRequest; import org.apache.hadoop.ozone.om.response.snapshot.OMSnapshotCreateResponse; +import org.apache.hadoop.ozone.om.snapshot.ReferenceCounted; +import org.apache.hadoop.ozone.om.snapshot.SnapshotCache; import org.junit.Assert; import org.junit.Test; @@ -212,9 +215,19 @@ public void testKeyPurgeInSnapshot() throws Exception { deletedKey)); } - OmSnapshot omSnapshot = (OmSnapshot) ozoneManager.getOmSnapshotManager() - .checkForSnapshot(volumeName, bucketName, - getSnapshotPrefix("snap1"), true); + SnapshotInfo fromSnapshotInfo = new SnapshotInfo.Builder() + .setVolumeName(volumeName) + .setBucketName(bucketName) + .setName("snap1") + .build(); + + ReferenceCounted rcOmSnapshot = + ozoneManager.getOmSnapshotManager().checkForSnapshot( + fromSnapshotInfo.getVolumeName(), + fromSnapshotInfo.getBucketName(), + getSnapshotPrefix(fromSnapshotInfo.getName()), + true); + OmSnapshot omSnapshot = (OmSnapshot) rcOmSnapshot.get(); // The keys should be present in the snapshot's deletedTable for (String deletedKey : deletedKeyNames) { @@ -240,10 +253,10 @@ public void testKeyPurgeInSnapshot() throws Exception { .build(); try (BatchOperation batchOperation = - omMetadataManager.getStore().initBatchOperation()) { + omMetadataManager.getStore().initBatchOperation()) { OMKeyPurgeResponse omKeyPurgeResponse = new OMKeyPurgeResponse( - omResponse, deletedKeyNames, omSnapshot, null); + omResponse, deletedKeyNames, fromSnapshotInfo, null); omKeyPurgeResponse.addToDBBatch(omMetadataManager, batchOperation); // Do manual commit and see whether addToBatch is successful or not. @@ -255,5 +268,8 @@ public void testKeyPurgeInSnapshot() throws Exception { Assert.assertFalse(omSnapshot.getMetadataManager() .getDeletedTable().isExist(deletedKey)); } + + omSnapshot = null; + rcOmSnapshot.close(); } } diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyRequest.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyRequest.java index d76872e3b972..ff943bee8d90 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyRequest.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyRequest.java @@ -27,7 +27,9 @@ import org.apache.hadoop.hdds.client.ReplicationConfig; import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList; import org.apache.hadoop.ozone.OzoneConfigKeys; +import org.apache.hadoop.ozone.om.IOmMetadataReader; import org.apache.hadoop.ozone.om.OMPerformanceMetrics; +import org.apache.hadoop.ozone.om.OmMetadataReader; import org.apache.hadoop.ozone.om.OmSnapshotManager; import org.apache.hadoop.ozone.om.OzoneManagerPrepareState; import org.apache.hadoop.ozone.om.ResolvedBucket; @@ -38,6 +40,8 @@ import org.apache.hadoop.ozone.om.helpers.OmKeyInfo; import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper; import org.apache.hadoop.ozone.om.request.OMClientRequest; +import org.apache.hadoop.ozone.om.snapshot.ReferenceCounted; +import org.apache.hadoop.ozone.om.snapshot.SnapshotCache; import org.apache.hadoop.ozone.om.upgrade.OMLayoutVersionManager; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyArgs; import org.apache.hadoop.security.UserGroupInformation; @@ -64,7 +68,6 @@ import org.apache.hadoop.ozone.om.OMMetadataManager; import org.apache.hadoop.ozone.om.OMMetrics; import org.apache.hadoop.ozone.om.OmMetadataManagerImpl; -import org.apache.hadoop.ozone.om.OmMetadataReader; import org.apache.hadoop.ozone.om.OzoneManager; import org.apache.hadoop.ozone.om.ScmClient; import org.apache.hadoop.hdds.security.token.OzoneBlockTokenSecretManager; @@ -129,7 +132,8 @@ public void setup() throws Exception { folder.newFolder().getAbsolutePath()); ozoneConfiguration.set(OzoneConfigKeys.OZONE_METADATA_DIRS, folder.newFolder().getAbsolutePath()); - omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration); + omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration, + ozoneManager); when(ozoneManager.getMetrics()).thenReturn(omMetrics); when(ozoneManager.getMetadataManager()).thenReturn(omMetadataManager); when(ozoneManager.getConfiguration()).thenReturn(ozoneConfiguration); @@ -165,8 +169,13 @@ public void setup() throws Exception { when(scmClient.getBlockClient()).thenReturn(scmBlockLocationProtocol); when(ozoneManager.getKeyManager()).thenReturn(keyManager); - OmMetadataReader omMetadataReader = Mockito.mock(OmMetadataReader.class); - when(ozoneManager.getOmMetadataReader()).thenReturn(omMetadataReader); + ReferenceCounted rcOmMetadataReader = + mock(ReferenceCounted.class); + when(ozoneManager.getOmMetadataReader()).thenReturn(rcOmMetadataReader); + // Init OmMetadataReader to let the test pass + OmMetadataReader omMetadataReader = mock(OmMetadataReader.class); + when(omMetadataReader.isNativeAuthorizerEnabled()).thenReturn(true); + when(rcOmMetadataReader.get()).thenReturn(omMetadataReader); prepareState = new OzoneManagerPrepareState(ozoneConfiguration); when(ozoneManager.getPrepareState()).thenReturn(prepareState); diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/s3/multipart/TestS3MultipartRequest.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/s3/multipart/TestS3MultipartRequest.java index 591c3bfa76af..d36ad74d5da0 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/s3/multipart/TestS3MultipartRequest.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/s3/multipart/TestS3MultipartRequest.java @@ -79,7 +79,8 @@ public void setup() throws Exception { OzoneConfiguration ozoneConfiguration = new OzoneConfiguration(); ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS, folder.newFolder().getAbsolutePath()); - omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration); + omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration, + ozoneManager); when(ozoneManager.getMetrics()).thenReturn(omMetrics); when(ozoneManager.getMetadataManager()).thenReturn(omMetadataManager); auditLogger = Mockito.mock(AuditLogger.class); diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/s3/security/TestS3GetSecretRequest.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/s3/security/TestS3GetSecretRequest.java index fcfd6fab5081..195ad71658d0 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/s3/security/TestS3GetSecretRequest.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/s3/security/TestS3GetSecretRequest.java @@ -131,7 +131,8 @@ public void setUp() throws Exception { folder.newFolder().getAbsolutePath()); // No need to conf.set(OzoneConfigKeys.OZONE_ADMINISTRATORS, ...) here // as we did the trick earlier with mockito. - OmMetadataManagerImpl omMetadataManager = new OmMetadataManagerImpl(conf); + OmMetadataManagerImpl omMetadataManager = new OmMetadataManagerImpl(conf, + ozoneManager); when(ozoneManager.getMetrics()).thenReturn(omMetrics); when(ozoneManager.getMetadataManager()).thenReturn(omMetadataManager); when(ozoneManager.isRatisEnabled()).thenReturn(true); diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/s3/tenant/TestSetRangerServiceVersionRequest.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/s3/tenant/TestSetRangerServiceVersionRequest.java index b30a607c1699..65ce001f5b27 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/s3/tenant/TestSetRangerServiceVersionRequest.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/s3/tenant/TestSetRangerServiceVersionRequest.java @@ -62,7 +62,7 @@ public void setUp() throws Exception { conf.set(OMConfigKeys.OZONE_OM_DB_DIRS, folder.newFolder().getAbsolutePath()); Mockito.when(ozoneManager.getMetadataManager()) - .thenReturn(new OmMetadataManagerImpl(conf)); + .thenReturn(new OmMetadataManagerImpl(conf, ozoneManager)); } @After diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/security/TestOMDelegationTokenRequest.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/security/TestOMDelegationTokenRequest.java index 08c95f6e8f2e..7b695f73737f 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/security/TestOMDelegationTokenRequest.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/security/TestOMDelegationTokenRequest.java @@ -60,7 +60,8 @@ public void setup() throws Exception { conf = new OzoneConfiguration(); ((OzoneConfiguration) conf) .set(OZONE_OM_DB_DIRS, folder.newFolder().getAbsolutePath()); - omMetadataManager = new OmMetadataManagerImpl((OzoneConfiguration) conf); + omMetadataManager = new OmMetadataManagerImpl((OzoneConfiguration) conf, + ozoneManager); when(ozoneManager.getMetadataManager()).thenReturn(omMetadataManager); } diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/snapshot/TestOMSnapshotCreateRequest.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/snapshot/TestOMSnapshotCreateRequest.java index 864d1003c4ef..b5b2c362a983 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/snapshot/TestOMSnapshotCreateRequest.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/snapshot/TestOMSnapshotCreateRequest.java @@ -96,7 +96,8 @@ public void setup() throws Exception { OzoneConfiguration ozoneConfiguration = new OzoneConfiguration(); ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS, anotherTempDir.getAbsolutePath()); - omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration); + omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration, + ozoneManager); when(ozoneManager.getMetrics()).thenReturn(omMetrics); when(ozoneManager.getMetadataManager()).thenReturn(omMetadataManager); when(ozoneManager.isRatisEnabled()).thenReturn(true); diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/snapshot/TestOMSnapshotDeleteRequest.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/snapshot/TestOMSnapshotDeleteRequest.java index 035bf5b860f1..0c53e3fab613 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/snapshot/TestOMSnapshotDeleteRequest.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/snapshot/TestOMSnapshotDeleteRequest.java @@ -19,7 +19,6 @@ package org.apache.hadoop.ozone.om.request.snapshot; -import com.google.common.cache.LoadingCache; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.utils.db.cache.CacheKey; import org.apache.hadoop.hdds.utils.db.cache.CacheValue; @@ -35,6 +34,7 @@ import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper; import org.apache.hadoop.ozone.om.request.OMRequestTestUtils; import org.apache.hadoop.ozone.om.response.OMClientResponse; +import org.apache.hadoop.ozone.om.snapshot.SnapshotCache; import org.apache.hadoop.ozone.om.upgrade.OMLayoutVersionManager; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse; @@ -95,7 +95,8 @@ public void setup() throws Exception { OzoneConfiguration ozoneConfiguration = new OzoneConfiguration(); ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS, folder.getAbsolutePath()); - omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration); + omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration, + ozoneManager); when(ozoneManager.getMetrics()).thenReturn(omMetrics); when(ozoneManager.getMetadataManager()).thenReturn(omMetadataManager); when(ozoneManager.isRatisEnabled()).thenReturn(true); @@ -113,7 +114,7 @@ public void setup() throws Exception { OmSnapshotManager omSnapshotManager = mock(OmSnapshotManager.class); when(omSnapshotManager.getSnapshotCache()) - .thenReturn(mock(LoadingCache.class)); + .thenReturn(mock(SnapshotCache.class)); when(ozoneManager.getOmSnapshotManager()).thenReturn(omSnapshotManager); volumeName = UUID.randomUUID().toString(); diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/snapshot/TestOMSnapshotPurgeRequestAndResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/snapshot/TestOMSnapshotPurgeRequestAndResponse.java index 8102735a4aa2..bb2b933f4c05 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/snapshot/TestOMSnapshotPurgeRequestAndResponse.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/snapshot/TestOMSnapshotPurgeRequestAndResponse.java @@ -24,11 +24,11 @@ import org.apache.hadoop.hdds.utils.db.RDBStore; import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.audit.AuditLogger; +import org.apache.hadoop.ozone.om.IOmMetadataReader; import org.apache.hadoop.ozone.om.OMConfigKeys; import org.apache.hadoop.ozone.om.OMMetadataManager; import org.apache.hadoop.ozone.om.OMMetrics; import org.apache.hadoop.ozone.om.OmMetadataManagerImpl; -import org.apache.hadoop.ozone.om.OmMetadataReader; import org.apache.hadoop.ozone.om.OmSnapshotManager; import org.apache.hadoop.ozone.om.OzoneManager; import org.apache.hadoop.ozone.om.SnapshotChainManager; @@ -37,6 +37,8 @@ import org.apache.hadoop.ozone.om.request.OMRequestTestUtils; import org.apache.hadoop.ozone.om.response.snapshot.OMSnapshotCreateResponse; import org.apache.hadoop.ozone.om.response.snapshot.OMSnapshotPurgeResponse; +import org.apache.hadoop.ozone.om.snapshot.ReferenceCounted; +import org.apache.hadoop.ozone.om.snapshot.SnapshotCache; import org.apache.hadoop.ozone.om.upgrade.OMLayoutVersionManager; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotPurgeRequest; @@ -104,7 +106,8 @@ public void setup() throws Exception { testDir.getAbsolutePath()); ozoneConfiguration.set(OzoneConfigKeys.OZONE_METADATA_DIRS, testDir.getAbsolutePath()); - omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration); + omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration, + ozoneManager); when(ozoneManager.getMetrics()).thenReturn(omMetrics); when(ozoneManager.getMetadataManager()).thenReturn(omMetadataManager); when(ozoneManager.getConfiguration()).thenReturn(ozoneConfiguration); @@ -112,8 +115,9 @@ public void setup() throws Exception { .thenReturn(true); when(ozoneManager.isFilesystemSnapshotEnabled()).thenReturn(true); - OmMetadataReader omMetadataReader = Mockito.mock(OmMetadataReader.class); - when(ozoneManager.getOmMetadataReader()).thenReturn(omMetadataReader); + ReferenceCounted rcOmMetadataReader = + Mockito.mock(ReferenceCounted.class); + when(ozoneManager.getOmMetadataReader()).thenReturn(rcOmMetadataReader); omSnapshotManager = new OmSnapshotManager(ozoneManager); when(ozoneManager.getOmSnapshotManager()).thenReturn(omSnapshotManager); volumeName = UUID.randomUUID().toString(); diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/volume/TestOMVolumeRequest.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/volume/TestOMVolumeRequest.java index bdec623d17b8..8f6f16f0e7e4 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/volume/TestOMVolumeRequest.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/volume/TestOMVolumeRequest.java @@ -74,7 +74,8 @@ public void setup() throws Exception { OzoneConfiguration ozoneConfiguration = new OzoneConfiguration(); ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS, folder.newFolder().getAbsolutePath()); - omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration); + omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration, + ozoneManager); when(ozoneManager.getMetrics()).thenReturn(omMetrics); when(ozoneManager.getMetadataManager()).thenReturn(omMetadataManager); when(ozoneManager.getMaxUserVolumeCount()).thenReturn(10L); diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/TestCleanupTableInfo.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/TestCleanupTableInfo.java index 5d61da399fa4..f49837254497 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/TestCleanupTableInfo.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/TestCleanupTableInfo.java @@ -283,7 +283,7 @@ private OMMetadataManager createOMMetadataManagerSpy() throws IOException { Assert.assertTrue(newFolder.mkdirs()); } ServerUtils.setOzoneMetaDirPath(conf, newFolder.toString()); - return spy(new OmMetadataManagerImpl(conf)); + return spy(new OmMetadataManagerImpl(conf, null)); } private OMFileCreateRequest anOMFileCreateRequest() { diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/bucket/TestOMBucketCreateResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/bucket/TestOMBucketCreateResponse.java index 3e71cbaa0a9a..354b6e02eb2e 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/bucket/TestOMBucketCreateResponse.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/bucket/TestOMBucketCreateResponse.java @@ -58,7 +58,7 @@ public void setup() throws Exception { OzoneConfiguration ozoneConfiguration = new OzoneConfiguration(); ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS, folder.newFolder().getAbsolutePath()); - omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration); + omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration, null); batchOperation = omMetadataManager.getStore().initBatchOperation(); } diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/bucket/TestOMBucketDeleteResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/bucket/TestOMBucketDeleteResponse.java index 838f2b289a02..22e1a57e7bfe 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/bucket/TestOMBucketDeleteResponse.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/bucket/TestOMBucketDeleteResponse.java @@ -60,7 +60,7 @@ public void setup() throws Exception { OzoneConfiguration ozoneConfiguration = new OzoneConfiguration(); ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS, folder.newFolder().getAbsolutePath()); - omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration); + omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration, null); batchOperation = omMetadataManager.getStore().initBatchOperation(); } diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/bucket/TestOMBucketSetPropertyResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/bucket/TestOMBucketSetPropertyResponse.java index d0f4e21e9c09..a8d63ce215b8 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/bucket/TestOMBucketSetPropertyResponse.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/bucket/TestOMBucketSetPropertyResponse.java @@ -58,7 +58,7 @@ public void setup() throws Exception { OzoneConfiguration ozoneConfiguration = new OzoneConfiguration(); ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS, folder.newFolder().getAbsolutePath()); - omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration); + omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration, null); batchOperation = omMetadataManager.getStore().initBatchOperation(); } diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/file/TestOMDirectoryCreateResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/file/TestOMDirectoryCreateResponse.java index e9198220e543..7f5500fe40ad 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/file/TestOMDirectoryCreateResponse.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/file/TestOMDirectoryCreateResponse.java @@ -61,7 +61,7 @@ public void setup() throws Exception { OzoneConfiguration ozoneConfiguration = new OzoneConfiguration(); ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS, folder.newFolder().getAbsolutePath()); - omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration); + omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration, null); batchOperation = omMetadataManager.getStore().initBatchOperation(); } diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/file/TestOMDirectoryCreateResponseWithFSO.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/file/TestOMDirectoryCreateResponseWithFSO.java index 61d5abb2de45..f44bc09952d0 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/file/TestOMDirectoryCreateResponseWithFSO.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/file/TestOMDirectoryCreateResponseWithFSO.java @@ -62,7 +62,7 @@ public void setup() throws Exception { OzoneConfiguration ozoneConfiguration = new OzoneConfiguration(); ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS, folder.newFolder().getAbsolutePath()); - omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration); + omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration, null); batchOperation = omMetadataManager.getStore().initBatchOperation(); } diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyResponse.java index 72a7107d929b..5c4692a68958 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyResponse.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyResponse.java @@ -72,7 +72,7 @@ public void setup() throws Exception { OzoneConfiguration ozoneConfiguration = getOzoneConfiguration(); ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS, folder.newFolder().getAbsolutePath()); - omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration); + omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration, null); batchOperation = omMetadataManager.getStore().initBatchOperation(); volumeName = UUID.randomUUID().toString(); diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/s3/multipart/TestS3MultipartResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/s3/multipart/TestS3MultipartResponse.java index af96f565902f..598e2d8a7ed5 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/s3/multipart/TestS3MultipartResponse.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/s3/multipart/TestS3MultipartResponse.java @@ -77,7 +77,7 @@ public void setup() throws Exception { OzoneConfiguration ozoneConfiguration = new OzoneConfiguration(); ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS, folder.newFolder().getAbsolutePath()); - omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration); + omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration, null); batchOperation = omMetadataManager.getStore().initBatchOperation(); } diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/security/TestOMDelegationTokenResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/security/TestOMDelegationTokenResponse.java index b35c33542f4a..be83237afc66 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/security/TestOMDelegationTokenResponse.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/security/TestOMDelegationTokenResponse.java @@ -46,7 +46,8 @@ public void setup() throws IOException { conf = new OzoneConfiguration(); ((OzoneConfiguration) conf).set(OMConfigKeys.OZONE_OM_DB_DIRS, folder.newFolder().getAbsolutePath()); - omMetadataManager = new OmMetadataManagerImpl((OzoneConfiguration) conf); + omMetadataManager = new OmMetadataManagerImpl((OzoneConfiguration) conf, + null); batchOperation = omMetadataManager.getStore().initBatchOperation(); } diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/snapshot/TestOMSnapshotCreateResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/snapshot/TestOMSnapshotCreateResponse.java index b4071b6a92fa..b8b05708c020 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/snapshot/TestOMSnapshotCreateResponse.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/snapshot/TestOMSnapshotCreateResponse.java @@ -73,7 +73,7 @@ public void setup() throws Exception { String fsPath = folder.newFolder().getAbsolutePath(); ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS, fsPath); - omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration); + omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration, null); batchOperation = omMetadataManager.getStore().initBatchOperation(); } diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/snapshot/TestOMSnapshotDeleteResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/snapshot/TestOMSnapshotDeleteResponse.java index dfbbf6b22320..164b882fdb94 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/snapshot/TestOMSnapshotDeleteResponse.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/snapshot/TestOMSnapshotDeleteResponse.java @@ -66,7 +66,7 @@ public void setup() throws Exception { String fsPath = folder.newFolder().getAbsolutePath(); ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS, fsPath); - omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration); + omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration, null); batchOperation = omMetadataManager.getStore().initBatchOperation(); } diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeCreateResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeCreateResponse.java index 9cfd21db2295..c0c706a95f24 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeCreateResponse.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeCreateResponse.java @@ -60,7 +60,7 @@ public void setup() throws Exception { OzoneConfiguration ozoneConfiguration = new OzoneConfiguration(); ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS, folder.newFolder().getAbsolutePath()); - omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration); + omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration, null); batchOperation = omMetadataManager.getStore().initBatchOperation(); } diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeDeleteResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeDeleteResponse.java index f956b0d9b976..fabad94370ca 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeDeleteResponse.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeDeleteResponse.java @@ -59,7 +59,7 @@ public void setup() throws Exception { OzoneConfiguration ozoneConfiguration = new OzoneConfiguration(); ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS, folder.newFolder().getAbsolutePath()); - omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration); + omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration, null); batchOperation = omMetadataManager.getStore().initBatchOperation(); } diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeSetOwnerResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeSetOwnerResponse.java index 96129e45bcb2..b7c6b0fb3ad8 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeSetOwnerResponse.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeSetOwnerResponse.java @@ -60,7 +60,7 @@ public void setup() throws Exception { OzoneConfiguration ozoneConfiguration = new OzoneConfiguration(); ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS, folder.newFolder().getAbsolutePath()); - omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration); + omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration, null); batchOperation = omMetadataManager.getStore().initBatchOperation(); } diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeSetQuotaResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeSetQuotaResponse.java index dfbaeac3726a..3a09743b6224 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeSetQuotaResponse.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/volume/TestOMVolumeSetQuotaResponse.java @@ -59,7 +59,7 @@ public void setup() throws Exception { OzoneConfiguration ozoneConfiguration = new OzoneConfiguration(); ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS, folder.newFolder().getAbsolutePath()); - omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration); + omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration, null); batchOperation = omMetadataManager.getStore().initBatchOperation(); } 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 b49988571a24..5b1c1325d669 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 @@ -32,6 +32,7 @@ import org.apache.hadoop.hdds.utils.db.Table; import org.apache.hadoop.hdds.utils.db.TableIterator; +import org.apache.hadoop.ozone.om.IOmMetadataReader; import org.apache.hadoop.ozone.om.KeyManager; import org.apache.hadoop.ozone.om.OMMetadataManager; import org.apache.hadoop.ozone.om.OmSnapshot; @@ -46,6 +47,8 @@ import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo; 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.om.snapshot.SnapshotCache; import org.apache.ratis.util.ExitUtils; import org.junit.BeforeClass; import org.slf4j.Logger; @@ -537,21 +540,25 @@ public void testSnapshotDeepClean() throws Exception { keyDeletingService.resume(); - OmSnapshot snap3 = (OmSnapshot) om.getOmSnapshotManager() - .checkForSnapshot(volumeName, bucketName, - getSnapshotPrefix("snap3"), true); - Table snap3deletedTable = - snap3.getMetadataManager().getDeletedTable(); + try (ReferenceCounted rcOmSnapshot = + om.getOmSnapshotManager().checkForSnapshot( + volumeName, bucketName, getSnapshotPrefix("snap3"), true)) { + OmSnapshot snap3 = (OmSnapshot) rcOmSnapshot.get(); - // 5 keys can be deep cleaned as it was stuck previously - assertTableRowCount(snap3deletedTable, 10, metadataManager); - checkSnapDeepCleanStatus(snapshotInfoTable, false); + Table snap3deletedTable = + snap3.getMetadataManager().getDeletedTable(); - writeClient.deleteSnapshot(volumeName, bucketName, "snap2"); - assertTableRowCount(snapshotInfoTable, 2, metadataManager); + // 5 keys can be deep cleaned as it was stuck previously + assertTableRowCount(snap3deletedTable, 10, metadataManager); + checkSnapDeepCleanStatus(snapshotInfoTable, false); + + writeClient.deleteSnapshot(volumeName, bucketName, "snap2"); + assertTableRowCount(snapshotInfoTable, 2, metadataManager); + + assertTableRowCount(snap3deletedTable, 0, metadataManager); + assertTableRowCount(deletedTable, 0, metadataManager); + } - assertTableRowCount(snap3deletedTable, 0, metadataManager); - assertTableRowCount(deletedTable, 0, metadataManager); } private void checkSnapDeepCleanStatus(Table diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestSnapshotCache.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestSnapshotCache.java new file mode 100644 index 000000000000..fa3ac8609656 --- /dev/null +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestSnapshotCache.java @@ -0,0 +1,394 @@ +/* + * 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 com.google.common.cache.CacheLoader; +import org.apache.hadoop.ozone.om.IOmMetadataReader; +import org.apache.hadoop.ozone.om.OmSnapshot; +import org.apache.hadoop.ozone.om.OmSnapshotManager; +import org.apache.ozone.test.GenericTestUtils; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.DisplayName; +import org.junit.jupiter.api.MethodOrderer; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestMethodOrder; +import org.mockito.stubbing.Answer; +import org.slf4j.event.Level; + +import java.io.IOException; + +import static org.apache.hadoop.ozone.om.helpers.SnapshotInfo.SnapshotStatus.SNAPSHOT_ACTIVE; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +/** + * Tests SnapshotCache. + */ +@TestMethodOrder(MethodOrderer.DisplayName.class) +class TestSnapshotCache { + + private static final int CACHE_SIZE_LIMIT = 3; + private static OmSnapshotManager omSnapshotManager; + private static CacheLoader cacheLoader; + private SnapshotCache snapshotCache; + + @BeforeAll + static void beforeAll() throws Exception { + omSnapshotManager = mock(OmSnapshotManager.class); + when(omSnapshotManager.isSnapshotStatus(any(), eq(SNAPSHOT_ACTIVE))) + .thenReturn(true); + cacheLoader = mock(CacheLoader.class); + // Create a difference mock OmSnapshot instance each time load() is called + when(cacheLoader.load(any())).thenAnswer( + (Answer) invocation -> { + final OmSnapshot omSnapshot = mock(OmSnapshot.class); + // Mock the snapshotTable return value for the lookup inside release() + final String dbKey = (String) invocation.getArguments()[0]; + when(omSnapshot.getSnapshotTableKey()).thenReturn(dbKey); + + return omSnapshot; + } + ); + + // Set SnapshotCache log level. Set to DEBUG for verbose output + GenericTestUtils.setLogLevel(SnapshotCache.LOG, Level.DEBUG); + } + + @BeforeEach + void setUp() { + // Reset cache for each test case + snapshotCache = new SnapshotCache( + omSnapshotManager, cacheLoader, CACHE_SIZE_LIMIT); + } + + @AfterEach + void tearDown() { + // Not strictly needed. Added for symmetry + snapshotCache = null; + } + + @Test + @DisplayName("01. get()") + void testGet() throws IOException { + final String dbKey1 = "dbKey1"; + ReferenceCounted omSnapshot = + snapshotCache.get(dbKey1); + assertNotNull(omSnapshot); + assertNotNull(omSnapshot.get()); + assertTrue(omSnapshot.get() instanceof OmSnapshot); + assertEquals(1, snapshotCache.size()); + assertTrue(snapshotCache.isConsistent()); + } + + @Test + @DisplayName("02. get() same entry twice yields one cache entry only") + void testGetTwice() throws IOException { + final String dbKey1 = "dbKey1"; + ReferenceCounted omSnapshot1 = + snapshotCache.get(dbKey1); + assertNotNull(omSnapshot1); + assertEquals(1, snapshotCache.size()); + assertTrue(snapshotCache.isConsistent()); + + ReferenceCounted omSnapshot1again = + snapshotCache.get(dbKey1); + // Should be the same instance + assertEquals(omSnapshot1, omSnapshot1again); + assertEquals(omSnapshot1.get(), omSnapshot1again.get()); + assertEquals(1, snapshotCache.size()); + assertTrue(snapshotCache.isConsistent()); + } + + @Test + @DisplayName("03. release(String)") + void testReleaseByDbKey() throws IOException { + final String dbKey1 = "dbKey1"; + ReferenceCounted omSnapshot1 = + snapshotCache.get(dbKey1); + assertNotNull(omSnapshot1); + assertNotNull(omSnapshot1.get()); + assertEquals(1, snapshotCache.size()); + assertEquals(0, snapshotCache.getPendingEvictionList().size()); + assertTrue(snapshotCache.isConsistent()); + + snapshotCache.release(dbKey1); + // Entry will not be immediately evicted + assertEquals(1, snapshotCache.size()); + // Entry is queued for eviction as its ref count reaches zero + assertEquals(1, snapshotCache.getPendingEvictionList().size()); + assertTrue(snapshotCache.isConsistent()); + } + + @Test + @DisplayName("04. release(OmSnapshot)") + void testReleaseByOmSnapshotInstance() throws IOException { + final String dbKey1 = "dbKey1"; + ReferenceCounted omSnapshot1 = + snapshotCache.get(dbKey1); + assertNotNull(omSnapshot1); + assertEquals(1, snapshotCache.size()); + assertEquals(0, snapshotCache.getPendingEvictionList().size()); + assertTrue(snapshotCache.isConsistent()); + + snapshotCache.release((OmSnapshot) omSnapshot1.get()); + // Entry will not be immediately evicted + assertEquals(1, snapshotCache.size()); + // Entry is queued for eviction as its ref count reaches zero + assertEquals(1, snapshotCache.getPendingEvictionList().size()); + assertTrue(snapshotCache.isConsistent()); + } + + @Test + @DisplayName("05. invalidate()") + void testInvalidate() throws IOException { + final String dbKey1 = "dbKey1"; + ReferenceCounted omSnapshot = + snapshotCache.get(dbKey1); + assertNotNull(omSnapshot); + assertEquals(1, snapshotCache.size()); + assertTrue(snapshotCache.isConsistent()); + + snapshotCache.release(dbKey1); + // Entry will not be immediately evicted + assertEquals(1, snapshotCache.size()); + assertTrue(snapshotCache.isConsistent()); + + snapshotCache.invalidate(dbKey1); + assertEquals(0, snapshotCache.size()); + assertTrue(snapshotCache.isConsistent()); + } + + @Test + @DisplayName("06. invalidateAll()") + void testInvalidateAll() throws IOException { + final String dbKey1 = "dbKey1"; + ReferenceCounted omSnapshot1 = + snapshotCache.get(dbKey1); + assertNotNull(omSnapshot1); + assertEquals(1, snapshotCache.size()); + assertTrue(snapshotCache.isConsistent()); + + final String dbKey2 = "dbKey2"; + ReferenceCounted omSnapshot2 = + snapshotCache.get(dbKey2); + assertNotNull(omSnapshot2); + assertEquals(2, snapshotCache.size()); + // Should be difference omSnapshot instances + assertNotEquals(omSnapshot1, omSnapshot2); + assertTrue(snapshotCache.isConsistent()); + + final String dbKey3 = "dbKey3"; + ReferenceCounted omSnapshot3 = + snapshotCache.get(dbKey3); + assertNotNull(omSnapshot3); + assertEquals(3, snapshotCache.size()); + assertEquals(0, snapshotCache.getPendingEvictionList().size()); + assertTrue(snapshotCache.isConsistent()); + + snapshotCache.release(dbKey1); + // Entry will not be immediately evicted + assertEquals(3, snapshotCache.size()); + assertEquals(1, snapshotCache.getPendingEvictionList().size()); + assertTrue(snapshotCache.isConsistent()); + + snapshotCache.invalidate(dbKey1); + assertEquals(2, snapshotCache.size()); + assertEquals(0, snapshotCache.getPendingEvictionList().size()); + assertTrue(snapshotCache.isConsistent()); + + snapshotCache.invalidateAll(); + assertEquals(0, snapshotCache.size()); + assertEquals(0, snapshotCache.getPendingEvictionList().size()); + assertTrue(snapshotCache.isConsistent()); + } + + private void assertEntryExistence(String key, boolean shouldExist) { + if (shouldExist) { + snapshotCache.getDbMap().computeIfAbsent(key, k -> { + fail(k + " should not have been evicted"); + return null; + }); + } else { + snapshotCache.getDbMap().computeIfPresent(key, (k, v) -> { + fail(k + " should have been evicted"); + return null; + }); + } + } + + @Test + @DisplayName("07. Basic cache eviction") + void testEviction1() throws IOException { + + final String dbKey1 = "dbKey1"; + snapshotCache.get(dbKey1); + assertEquals(1, snapshotCache.size()); + assertTrue(snapshotCache.isConsistent()); + snapshotCache.release(dbKey1); + assertEquals(1, snapshotCache.size()); + assertEquals(1, snapshotCache.getPendingEvictionList().size()); + assertTrue(snapshotCache.isConsistent()); + + final String dbKey2 = "dbKey2"; + snapshotCache.get(dbKey2); + assertEquals(2, snapshotCache.size()); + assertTrue(snapshotCache.isConsistent()); + snapshotCache.release(dbKey2); + assertEquals(2, snapshotCache.size()); + assertEquals(2, snapshotCache.getPendingEvictionList().size()); + assertTrue(snapshotCache.isConsistent()); + + final String dbKey3 = "dbKey3"; + snapshotCache.get(dbKey3); + assertEquals(3, snapshotCache.size()); + assertTrue(snapshotCache.isConsistent()); + snapshotCache.release(dbKey3); + assertEquals(3, snapshotCache.size()); + assertEquals(3, snapshotCache.getPendingEvictionList().size()); + assertTrue(snapshotCache.isConsistent()); + + final String dbKey4 = "dbKey4"; + snapshotCache.get(dbKey4); + // dbKey1 would have been evicted by the end of the last get() because + // it was release()d. + assertEquals(3, snapshotCache.size()); + assertTrue(snapshotCache.isConsistent()); + assertEntryExistence(dbKey1, false); + assertEquals(2, snapshotCache.getPendingEvictionList().size()); + assertTrue(snapshotCache.isConsistent()); + } + + @Test + @DisplayName("08. Cache eviction while exceeding soft limit") + void testEviction2() throws IOException { + + final String dbKey1 = "dbKey1"; + snapshotCache.get(dbKey1); + assertEquals(1, snapshotCache.size()); + assertTrue(snapshotCache.isConsistent()); + + final String dbKey2 = "dbKey2"; + snapshotCache.get(dbKey2); + assertEquals(2, snapshotCache.size()); + assertTrue(snapshotCache.isConsistent()); + + final String dbKey3 = "dbKey3"; + snapshotCache.get(dbKey3); + assertEquals(3, snapshotCache.size()); + assertTrue(snapshotCache.isConsistent()); + + final String dbKey4 = "dbKey4"; + snapshotCache.get(dbKey4); + // dbKey1 would not have been evicted because it is not release()d + assertEquals(4, snapshotCache.size()); + assertEntryExistence(dbKey1, true); + assertEquals(0, snapshotCache.getPendingEvictionList().size()); + assertTrue(snapshotCache.isConsistent()); + + // Releasing dbKey2 at this point should immediately trigger its eviction + // because the cache size exceeded the soft limit + snapshotCache.release(dbKey2); + assertEquals(3, snapshotCache.size()); + assertEntryExistence(dbKey2, false); + assertEntryExistence(dbKey1, true); + assertEquals(0, snapshotCache.getPendingEvictionList().size()); + assertTrue(snapshotCache.isConsistent()); + } + + @Test + @DisplayName("09. Cache eviction with try-with-resources") + void testEviction3WithClose() throws IOException { + + final String dbKey1 = "dbKey1"; + try (ReferenceCounted rcOmSnapshot = + snapshotCache.get(dbKey1)) { + assertEquals(1L, rcOmSnapshot.getTotalRefCount()); + assertEquals(1, snapshotCache.size()); + assertEquals(0, snapshotCache.getPendingEvictionList().size()); + assertTrue(snapshotCache.isConsistent()); + } + // ref count should have been decreased because it would be close()d + // upon exiting try-with-resources. + assertEquals(0L, snapshotCache.getDbMap().get(dbKey1).getTotalRefCount()); + assertEquals(1, snapshotCache.size()); + assertEquals(1, snapshotCache.getPendingEvictionList().size()); + assertTrue(snapshotCache.isConsistent()); + + final String dbKey2 = "dbKey2"; + try (ReferenceCounted rcOmSnapshot = + snapshotCache.get(dbKey2)) { + assertEquals(1L, rcOmSnapshot.getTotalRefCount()); + assertEquals(2, snapshotCache.size()); + assertEquals(1, snapshotCache.getPendingEvictionList().size()); + assertTrue(snapshotCache.isConsistent()); + // Get dbKey2 entry a second time + try (ReferenceCounted rcOmSnapshot2 = + snapshotCache.get(dbKey2)) { + assertEquals(2L, rcOmSnapshot.getTotalRefCount()); + assertEquals(2L, rcOmSnapshot2.getTotalRefCount()); + assertEquals(2, snapshotCache.size()); + assertEquals(1, snapshotCache.getPendingEvictionList().size()); + assertTrue(snapshotCache.isConsistent()); + } + assertEquals(1L, rcOmSnapshot.getTotalRefCount()); + assertTrue(snapshotCache.isConsistent()); + } + assertEquals(0L, snapshotCache.getDbMap().get(dbKey2).getTotalRefCount()); + assertEquals(2, snapshotCache.size()); + assertEquals(2, snapshotCache.getPendingEvictionList().size()); + assertTrue(snapshotCache.isConsistent()); + + final String dbKey3 = "dbKey3"; + try (ReferenceCounted rcOmSnapshot = + snapshotCache.get(dbKey3)) { + assertEquals(1L, rcOmSnapshot.getTotalRefCount()); + assertEquals(3, snapshotCache.size()); + assertEquals(2, snapshotCache.getPendingEvictionList().size()); + assertTrue(snapshotCache.isConsistent()); + } + assertEquals(0L, snapshotCache.getDbMap().get(dbKey3).getTotalRefCount()); + assertEquals(3, snapshotCache.size()); + assertEquals(3, snapshotCache.getPendingEvictionList().size()); + assertTrue(snapshotCache.isConsistent()); + + final String dbKey4 = "dbKey4"; + try (ReferenceCounted rcOmSnapshot = + snapshotCache.get(dbKey4)) { + assertEquals(1L, rcOmSnapshot.getTotalRefCount()); + assertEquals(3, snapshotCache.size()); + // An entry has been evicted at this point + assertEquals(2, snapshotCache.getPendingEvictionList().size()); + assertTrue(snapshotCache.isConsistent()); + } + assertEquals(0L, snapshotCache.getDbMap().get(dbKey4).getTotalRefCount()); + // Reached cache size limit + assertEquals(3, snapshotCache.size()); + assertEquals(3, snapshotCache.getPendingEvictionList().size()); + assertTrue(snapshotCache.isConsistent()); + } + +} 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 08dfc2c755cf..8a3f124d281e 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 @@ -17,9 +17,7 @@ */ package org.apache.hadoop.ozone.om.snapshot; -import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; -import com.google.common.cache.LoadingCache; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Sets; @@ -42,9 +40,11 @@ import org.apache.hadoop.hdfs.DFSUtilClient; import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport; import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry; +import org.apache.hadoop.ozone.om.IOmMetadataReader; 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.helpers.BucketLayout; import org.apache.hadoop.ozone.om.helpers.OmBucketInfo; @@ -83,6 +83,7 @@ import org.junit.jupiter.params.provider.CsvSource; import org.junit.jupiter.params.provider.MethodSource; import org.junit.jupiter.params.provider.ValueSource; +import org.mockito.Matchers; import org.mockito.Mock; import org.mockito.MockedConstruction; import org.mockito.MockedStatic; @@ -208,6 +209,9 @@ public class TestSnapshotDiffManager { private OzoneManager ozoneManager; @Mock private OzoneConfiguration configuration; + + private SnapshotCache snapshotCache; + @Mock private Table snapshotInfoTable; @Mock @@ -219,11 +223,12 @@ public class TestSnapshotDiffManager { @Mock private RDBStore dbStore; - private LoadingCache snapshotCache; - @Mock private RocksIterator jobTableIterator; + @Mock + private OmSnapshotManager omSnapshotManager; + private static CodecRegistry codecRegistry; private final BiFunction @@ -371,7 +376,10 @@ public OmSnapshot load(@NotNull String key) { } }; - snapshotCache = CacheBuilder.newBuilder().maximumSize(10).build(loader); + omSnapshotManager = Mockito.mock(OmSnapshotManager.class); + Mockito.when(omSnapshotManager.isSnapshotStatus( + Matchers.any(), Matchers.any())).thenReturn(true); + snapshotCache = new SnapshotCache(omSnapshotManager, loader, 10); snapshotDiffManager = new SnapshotDiffManager(db, differ, ozoneManager, snapshotCache, snapDiffJobTable, snapDiffReportTable, @@ -422,16 +430,26 @@ public void testGetDeltaFilesWithDag(int numberOfFiles) eq(diffDir)) ).thenReturn(Lists.newArrayList(randomStrings)); + ReferenceCounted rcFromSnapshot = + snapshotCache.get(snap1.toString()); + ReferenceCounted rcToSnapshot = + snapshotCache.get(snap2.toString()); + OmSnapshot fromSnapshot = (OmSnapshot) rcFromSnapshot.get(); + OmSnapshot toSnapshot = (OmSnapshot) rcToSnapshot.get(); + SnapshotInfo fromSnapshotInfo = getMockedSnapshotInfo(snap1); SnapshotInfo toSnapshotInfo = getMockedSnapshotInfo(snap2); when(jobTableIterator.isValid()).thenReturn(false); Set deltaFiles = snapshotDiffManager.getDeltaFiles( - snapshotCache.get(snap1.toString()), - snapshotCache.get(snap2.toString()), + fromSnapshot, + toSnapshot, Arrays.asList("cf1", "cf2"), fromSnapshotInfo, toSnapshotInfo, false, Collections.emptyMap(), diffDir); assertEquals(randomStrings, deltaFiles); + + rcFromSnapshot.close(); + rcToSnapshot.close(); } @ParameterizedTest @@ -478,12 +496,19 @@ public void testGetDeltaFilesWithFullDiff(int numberOfFiles, .thenReturn(Collections.emptyList()); } + ReferenceCounted rcFromSnapshot = + snapshotCache.get(snap1.toString()); + ReferenceCounted rcToSnapshot = + snapshotCache.get(snap2.toString()); + OmSnapshot fromSnapshot = (OmSnapshot) rcFromSnapshot.get(); + OmSnapshot toSnapshot = (OmSnapshot) rcToSnapshot.get(); + SnapshotInfo fromSnapshotInfo = getMockedSnapshotInfo(snap1); SnapshotInfo toSnapshotInfo = getMockedSnapshotInfo(snap1); when(jobTableIterator.isValid()).thenReturn(false); Set deltaFiles = snapshotDiffManager.getDeltaFiles( - snapshotCache.get(snap1.toString()), - snapshotCache.get(snap2.toString()), + fromSnapshot, + toSnapshot, Arrays.asList("cf1", "cf2"), fromSnapshotInfo, toSnapshotInfo, @@ -536,12 +561,19 @@ public void testGetDeltaFilesWithDifferThrowException(int numberOfFiles) any(DifferSnapshotInfo.class), anyString()); + ReferenceCounted rcFromSnapshot = + snapshotCache.get(snap1.toString()); + ReferenceCounted rcToSnapshot = + snapshotCache.get(snap2.toString()); + OmSnapshot fromSnapshot = (OmSnapshot) rcFromSnapshot.get(); + OmSnapshot toSnapshot = (OmSnapshot) rcToSnapshot.get(); + SnapshotInfo fromSnapshotInfo = getMockedSnapshotInfo(snap1); SnapshotInfo toSnapshotInfo = getMockedSnapshotInfo(snap1); when(jobTableIterator.isValid()).thenReturn(false); Set deltaFiles = snapshotDiffManager.getDeltaFiles( - snapshotCache.get(snap1.toString()), - snapshotCache.get(snap2.toString()), + fromSnapshot, + toSnapshot, Arrays.asList("cf1", "cf2"), fromSnapshotInfo, toSnapshotInfo, @@ -550,6 +582,9 @@ public void testGetDeltaFilesWithDifferThrowException(int numberOfFiles) Files.createTempDirectory("snapdiff_dir").toAbsolutePath() .toString()); assertEquals(deltaStrings, deltaFiles); + + rcFromSnapshot.close(); + rcToSnapshot.close(); } } diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/security/TestOzoneDelegationTokenSecretManager.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/security/TestOzoneDelegationTokenSecretManager.java index 0325f6531661..94022a329078 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/security/TestOzoneDelegationTokenSecretManager.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/security/TestOzoneDelegationTokenSecretManager.java @@ -106,7 +106,7 @@ public void setUp() throws Exception { s3Secrets.put("abc", new S3SecretValue("abc", "djakjahkd")); om = Mockito.mock(OzoneManager.class); - OMMetadataManager metadataManager = new OmMetadataManagerImpl(conf); + OMMetadataManager metadataManager = new OmMetadataManagerImpl(conf, om); Mockito.when(om.getMetadataManager()).thenReturn(metadataManager); s3SecretManager = new S3SecretLockedManager( new S3SecretManagerImpl(new S3SecretStoreMap(s3Secrets), diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/OMMetadataManagerTestUtils.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/OMMetadataManagerTestUtils.java index 3f00eb2adc0c..cadfc5ca9b59 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/OMMetadataManagerTestUtils.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/OMMetadataManagerTestUtils.java @@ -79,7 +79,7 @@ public static OMMetadataManager initializeNewOmMetadataManager( omConfiguration.set(OZONE_OM_DB_DIRS, omDbDir.getAbsolutePath()); OMMetadataManager omMetadataManager = new OmMetadataManagerImpl( - omConfiguration); + omConfiguration, null); String volumeKey = omMetadataManager.getVolumeKey("sampleVol"); OmVolumeArgs args = @@ -113,7 +113,7 @@ public static OMMetadataManager initializeEmptyOmMetadataManager( OzoneConfiguration omConfiguration = new OzoneConfiguration(); omConfiguration.set(OZONE_OM_DB_DIRS, omDbDir.getAbsolutePath()); - return new OmMetadataManagerImpl(omConfiguration); + return new OmMetadataManagerImpl(omConfiguration, null); } /** diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithFSO.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithFSO.java index aa792ee00b3d..4830b0e6e058 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithFSO.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithFSO.java @@ -835,7 +835,7 @@ private static OMMetadataManager initializeNewOmMetadataManager( omConfiguration.set(OZONE_OM_DB_DIRS, omDbDir.getAbsolutePath()); OMMetadataManager omMetadataManager = new OmMetadataManagerImpl( - omConfiguration); + omConfiguration, null); String volumeKey = omMetadataManager.getVolumeKey(VOL); OmVolumeArgs args = diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithLegacy.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithLegacy.java index 7412a7cacebc..01517d296cf6 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithLegacy.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithLegacy.java @@ -873,7 +873,7 @@ private static OMMetadataManager initializeNewOmMetadataManager( omConfiguration.set(OMConfigKeys .OZONE_OM_ENABLE_FILESYSTEM_PATHS, "true"); OMMetadataManager omMetadataManager = new OmMetadataManagerImpl( - omConfiguration); + omConfiguration, null); String volumeKey = omMetadataManager.getVolumeKey(VOL); OmVolumeArgs args = diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/recovery/TestReconOmMetadataManagerImpl.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/recovery/TestReconOmMetadataManagerImpl.java index 2f398b4c4607..4b14c740ed8f 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/recovery/TestReconOmMetadataManagerImpl.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/recovery/TestReconOmMetadataManagerImpl.java @@ -155,7 +155,7 @@ private OMMetadataManager getOMMetadataManager() throws IOException { omConfiguration.set(OZONE_OM_DB_DIRS, omDbDir.getAbsolutePath()); OMMetadataManager omMetadataManager = new OmMetadataManagerImpl( - omConfiguration); + omConfiguration, null); //Create a volume + bucket + 2 keys. String volumeKey = omMetadataManager.getVolumeKey("sampleVol"); diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTask.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTask.java index 59c22aecd636..7b78319b8449 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTask.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTask.java @@ -436,7 +436,7 @@ private static void initializeNewOmMetadataManager( omConfiguration.set(OMConfigKeys .OZONE_OM_ENABLE_FILESYSTEM_PATHS, "true"); omMetadataManager = new OmMetadataManagerImpl( - omConfiguration); + omConfiguration, null); String volumeKey = omMetadataManager.getVolumeKey(VOL); OmVolumeArgs args = diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithLegacy.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithLegacy.java index 65f1170c9460..17230beb4cba 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithLegacy.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithLegacy.java @@ -704,7 +704,7 @@ private static void initializeNewOmMetadataManager( omConfiguration.set(OMConfigKeys .OZONE_OM_ENABLE_FILESYSTEM_PATHS, "true"); omMetadataManager = new OmMetadataManagerImpl( - omConfiguration); + omConfiguration, null); String volumeKey = omMetadataManager.getVolumeKey(VOL); OmVolumeArgs args = diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestOMDBUpdatesHandler.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestOMDBUpdatesHandler.java index 5469d85c7bd9..e84e6621155f 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestOMDBUpdatesHandler.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestOMDBUpdatesHandler.java @@ -83,10 +83,11 @@ private OzoneConfiguration createNewTestPath() throws IOException { @Before public void setUp() throws Exception { OzoneConfiguration configuration = createNewTestPath(); - omMetadataManager = new OmMetadataManagerImpl(configuration); + omMetadataManager = new OmMetadataManagerImpl(configuration, null); OzoneConfiguration reconConfiguration = createNewTestPath(); - reconOmMetadataManager = new OmMetadataManagerImpl(reconConfiguration); + reconOmMetadataManager = new OmMetadataManagerImpl(reconConfiguration, + null); } @Test diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/PrefixParser.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/PrefixParser.java index 6e5ce0149ad0..e877597d172e 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/PrefixParser.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/debug/PrefixParser.java @@ -130,7 +130,7 @@ public void parse(String vol, String buck, String db, conf.set(OMConfigKeys.OZONE_OM_DB_DIRS, db); OmMetadataManagerImpl metadataManager = - new OmMetadataManagerImpl(conf); + new OmMetadataManagerImpl(conf, null); metadataManager.start(conf); org.apache.hadoop.fs.Path effectivePath = diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/fsck/ContainerMapper.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/fsck/ContainerMapper.java index f0693b80109c..a7a74c2e3729 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/fsck/ContainerMapper.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/fsck/ContainerMapper.java @@ -78,7 +78,7 @@ public static void main(String[] args) throws IOException { throw new IOException(OZONE_OM_DB_DIRS + "should be set "); } OmMetadataManagerImpl metadataManager = - new OmMetadataManagerImpl(configuration); + new OmMetadataManagerImpl(configuration, null); try { Table keyTable = metadataManager.getKeyTable(getBucketLayout());