diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml
index 5a4b9a22c887..1287dc6492e4 100644
--- a/hadoop-hdds/common/src/main/resources/ozone-default.xml
+++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml
@@ -4282,7 +4282,7 @@
- ozone.om.snapshot.diff.cleanup.service.run.internal
+ ozone.om.snapshot.diff.cleanup.service.run.interval
1m
OZONE, OM
@@ -4301,6 +4301,16 @@
+
+ ozone.om.snapshot.cache.cleanup.service.run.interval
+ 1m
+ OZONE, OM
+
+ Interval at which snapshot cache clean up will run.
+ Uses millisecond by default when no time unit is specified.
+
+
+
ozone.om.snapshot.load.native.lib
true
diff --git a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java
index 97d015fb2392..f3c322b56924 100644
--- a/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java
+++ b/hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java
@@ -301,7 +301,7 @@ private void createSstBackUpDir() {
}
@Override
- public void close() throws Exception {
+ public void close() {
if (!closed) {
synchronized (this) {
if (!closed) {
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java
index ff3a7beee509..f78f7785735a 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/OMConfigKeys.java
@@ -569,13 +569,20 @@ private OMConfigKeys() {
= TimeUnit.DAYS.toMillis(7);
public static final String OZONE_OM_SNAPSHOT_DIFF_CLEANUP_SERVICE_RUN_INTERVAL
- = "ozone.om.snapshot.diff.cleanup.service.run.internal";
+ = "ozone.om.snapshot.diff.cleanup.service.run.interval";
+ public static final String
+ OZONE_OM_SNAPSHOT_CACHE_CLEANUP_SERVICE_RUN_INTERVAL
+ = "ozone.om.snapshot.cache.cleanup.service.run.interval";
public static final long
OZONE_OM_SNAPSHOT_DIFF_CLEANUP_SERVICE_RUN_INTERVAL_DEFAULT
= TimeUnit.MINUTES.toMillis(1);
+ public static final long
+ OZONE_OM_SNAPSHOT_CACHE_CLEANUP_SERVICE_RUN_INTERVAL_DEFAULT
+ = TimeUnit.MINUTES.toMillis(1);
public static final String OZONE_OM_SNAPSHOT_DIFF_CLEANUP_SERVICE_TIMEOUT
= "ozone.om.snapshot.diff.cleanup.service.timeout";
+
public static final long
OZONE_OM_SNAPSHOT_DIFF_CLEANUP_SERVICE_TIMEOUT_DEFAULT
= TimeUnit.MINUTES.toMillis(5);
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 5839c61cf31f..f863c086028e 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
@@ -47,6 +47,7 @@
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
+import java.util.UUID;
import java.util.stream.Collectors;
/**
@@ -74,6 +75,7 @@ public class OmSnapshot implements IOmMetadataReader, Closeable {
private final String volumeName;
private final String bucketName;
private final String snapshotName;
+ private final UUID snapshotID;
// To access snapshot checkpoint DB metadata
private final OMMetadataManager omMetadataManager;
private final KeyManager keyManager;
@@ -83,7 +85,8 @@ public OmSnapshot(KeyManager keyManager,
OzoneManager ozoneManager,
String volumeName,
String bucketName,
- String snapshotName) {
+ String snapshotName,
+ UUID snapshotID) {
IAccessAuthorizer accessAuthorizer =
OzoneAuthorizerFactory.forSnapshot(ozoneManager,
keyManager, prefixManager);
@@ -93,6 +96,7 @@ public OmSnapshot(KeyManager keyManager,
this.snapshotName = snapshotName;
this.bucketName = bucketName;
this.volumeName = volumeName;
+ this.snapshotID = snapshotID;
this.keyManager = keyManager;
this.omMetadataManager = keyManager.getMetadataManager();
}
@@ -295,6 +299,10 @@ public String getName() {
return snapshotName;
}
+ public UUID getSnapshotID() {
+ return snapshotID;
+ }
+
@Override
public void close() throws IOException {
// Close DB
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 a3799b389c51..11edd28e9442 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
@@ -83,6 +83,8 @@
import static org.apache.hadoop.ozone.OzoneConsts.OM_SNAPSHOT_CHECKPOINT_DIR;
import static org.apache.hadoop.ozone.OzoneConsts.OM_SNAPSHOT_DIFF_DB_NAME;
import static org.apache.hadoop.ozone.OzoneConsts.OM_SNAPSHOT_INDICATOR;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_SNAPSHOT_CACHE_CLEANUP_SERVICE_RUN_INTERVAL;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_SNAPSHOT_CACHE_CLEANUP_SERVICE_RUN_INTERVAL_DEFAULT;
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_SNAPSHOT_CACHE_MAX_SIZE;
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_SNAPSHOT_CACHE_MAX_SIZE_DEFAULT;
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_SNAPSHOT_DB_MAX_OPEN_FILES;
@@ -272,7 +274,12 @@ public OmSnapshotManager(OzoneManager ozoneManager) {
};
// Init snapshot cache
- this.snapshotCache = new SnapshotCache(loader, softCacheSize, ozoneManager.getMetrics());
+ long cacheCleanupServiceInterval = ozoneManager.getConfiguration()
+ .getTimeDuration(OZONE_OM_SNAPSHOT_CACHE_CLEANUP_SERVICE_RUN_INTERVAL,
+ OZONE_OM_SNAPSHOT_CACHE_CLEANUP_SERVICE_RUN_INTERVAL_DEFAULT,
+ TimeUnit.MILLISECONDS);
+ this.snapshotCache = new SnapshotCache(loader, softCacheSize, ozoneManager.getMetrics(),
+ cacheCleanupServiceInterval);
this.snapshotDiffManager = new SnapshotDiffManager(snapshotDiffDb, differ,
ozoneManager, snapDiffJobCf, snapDiffReportCf,
@@ -382,7 +389,8 @@ public OmSnapshot load(@Nonnull UUID snapshotId) throws IOException {
return new OmSnapshot(km, pm, ozoneManager,
snapshotInfo.getVolumeName(),
snapshotInfo.getBucketName(),
- snapshotInfo.getName());
+ snapshotInfo.getName(),
+ snapshotInfo.getSnapshotId());
} catch (Exception e) {
// Close RocksDB if there is any failure.
if (!snapshotMetadataManager.getStore().isClosed()) {
@@ -426,7 +434,7 @@ public void invalidateCache() {
*
* @param key SnapshotId.
*/
- public void invalidateCacheEntry(UUID key) throws IOException {
+ public void invalidateCacheEntry(UUID key) {
if (snapshotCache != null) {
snapshotCache.invalidate(key);
}
@@ -949,7 +957,9 @@ public void close() {
snapshotDiffManager.close();
}
- invalidateCache();
+ if (snapshotCache != null) {
+ snapshotCache.close();
+ }
if (snapshotDiffCleanupService != null) {
snapshotDiffCleanupService.shutdown();
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
index 0a9d47fc861c..97e19eb969d8 100644
--- 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
@@ -51,10 +51,10 @@ public class ReferenceCounted
/**
* Parent instance whose callback will be triggered upon this RC closure.
*/
- private final Object parentWithCallback;
+ private final ReferenceCountedCallback parentWithCallback;
public ReferenceCounted(T obj, boolean disableCounter,
- Object parentWithCallback) {
+ ReferenceCountedCallback parentWithCallback) {
// A param to allow disabling ref counting to reduce active DB
// access penalties due to AtomicLong operations.
this.obj = obj;
@@ -126,7 +126,9 @@ public long decrementRefCount() {
Preconditions.checkState(newValTotal >= 0L,
"Total reference count underflow");
}
-
+ if (refCount.get() == 0) {
+ this.parentWithCallback.callback(this);
+ }
return refCount.get();
}
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
index f14837462b0a..035fc80d3468 100644
--- 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
@@ -20,23 +20,24 @@
import com.google.common.annotations.VisibleForTesting;
import com.google.common.cache.CacheLoader;
import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.hdds.utils.Scheduler;
import org.apache.hadoop.ozone.om.OmSnapshot;
import org.apache.hadoop.ozone.om.exceptions.OMException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
-import java.util.Iterator;
-import java.util.Map;
+import java.util.Set;
import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.FILE_NOT_FOUND;
/**
* Thread-safe custom unbounded LRU cache to manage open snapshot DB instances.
*/
-public class SnapshotCache {
+public class SnapshotCache implements ReferenceCountedCallback, AutoCloseable {
static final Logger LOG = LoggerFactory.getLogger(SnapshotCache.class);
@@ -51,14 +52,28 @@ public class SnapshotCache {
// Soft-limit of the total number of snapshot DB instances allowed to be
// opened on the OM.
private final int cacheSizeLimit;
+ private final Set pendingEvictionQueue;
+ private final Scheduler scheduler;
+ private static final String SNAPSHOT_CACHE_CLEANUP_SERVICE =
+ "SnapshotCacheCleanupService";
private final OMMetrics omMetrics;
- public SnapshotCache(CacheLoader cacheLoader, int cacheSizeLimit, OMMetrics omMetrics) {
+ public SnapshotCache(CacheLoader cacheLoader, int cacheSizeLimit, OMMetrics omMetrics,
+ long cleanupInterval) {
this.dbMap = new ConcurrentHashMap<>();
this.cacheLoader = cacheLoader;
this.cacheSizeLimit = cacheSizeLimit;
this.omMetrics = omMetrics;
+ this.pendingEvictionQueue = ConcurrentHashMap.newKeySet();
+ if (cleanupInterval > 0) {
+ this.scheduler = new Scheduler(SNAPSHOT_CACHE_CLEANUP_SERVICE,
+ true, 1);
+ this.scheduler.scheduleWithFixedDelay(this::cleanup, cleanupInterval,
+ cleanupInterval, TimeUnit.MILLISECONDS);
+ } else {
+ this.scheduler = null;
+ }
}
@VisibleForTesting
@@ -77,7 +92,7 @@ public int size() {
* Immediately invalidate an entry.
* @param key SnapshotId
*/
- public void invalidate(UUID key) throws IOException {
+ public void invalidate(UUID key) {
dbMap.compute(key, (k, v) -> {
if (v == null) {
LOG.warn("SnapshotId: '{}' does not exist in snapshot cache.", k);
@@ -97,19 +112,16 @@ public void invalidate(UUID key) throws IOException {
* Immediately invalidate all entries and close their DB instances in cache.
*/
public void invalidateAll() {
- Iterator>> it = dbMap.entrySet().iterator();
-
- while (it.hasNext()) {
- Map.Entry> entry = it.next();
- 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();
- omMetrics.decNumSnapshotCacheSize();
+ for (UUID key : dbMap.keySet()) {
+ invalidate(key);
+ }
+ }
+
+ @Override
+ public void close() {
+ invalidateAll();
+ if (this.scheduler != null) {
+ this.scheduler.close();
}
}
@@ -170,13 +182,6 @@ public ReferenceCounted get(UUID key) throws IOException {
throw new OMException("SnapshotId: '" + key + "' not found, or the snapshot is no longer active.",
OMException.ResultCodes.FILE_NOT_FOUND);
}
-
- // 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;
}
@@ -185,58 +190,60 @@ public ReferenceCounted get(UUID key) throws IOException {
* @param key SnapshotId
*/
public void release(UUID key) {
- dbMap.compute(key, (k, v) -> {
- if (v == null) {
- throw new IllegalArgumentException("SnapshotId '" + key + "' does not exist in cache.");
- } else {
- v.decrementRefCount();
- }
- return v;
- });
-
- // The cache size might have already exceeded the soft limit
- // Thus triggering cleanup() to check and evict if applicable
- cleanup();
+ ReferenceCounted val = dbMap.get(key);
+ if (val == null) {
+ throw new IllegalArgumentException("Key '" + key + "' does not " +
+ "exist in cache.");
+ }
+ val.decrementRefCount();
}
+
/**
- * Wrapper for cleanupInternal() that is synchronized to prevent multiple
- * threads from interleaving into the cleanup method.
+ * If cache size exceeds soft limit, attempt to clean up and close the
+ instances that has zero reference count.
*/
- private synchronized void cleanup() {
+ @VisibleForTesting
+ void cleanup() {
if (dbMap.size() > cacheSizeLimit) {
- cleanupInternal();
+ for (UUID evictionKey : pendingEvictionQueue) {
+ dbMap.compute(evictionKey, (k, v) -> {
+ pendingEvictionQueue.remove(k);
+ if (v == null) {
+ throw new IllegalStateException("SnapshotId '" + k + "' does not exist in cache. The RocksDB " +
+ "instance of the Snapshot may not be closed properly.");
+ }
+
+ if (v.getTotalRefCount() > 0) {
+ LOG.debug("SnapshotId {} is still being referenced ({}), skipping its clean up.", k, v.getTotalRefCount());
+ return v;
+ } else {
+ LOG.debug("Closing SnapshotId {}. It is not being referenced anymore.", k);
+ // Close the instance, which also closes its DB handle.
+ try {
+ v.get().close();
+ } catch (IOException ex) {
+ throw new IllegalStateException("Error while closing snapshot DB.", ex);
+ }
+ omMetrics.decNumSnapshotCacheSize();
+ return null;
+ }
+ });
+ }
}
}
/**
- * 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.
+ * Callback method used to enqueue or dequeue ReferenceCounted from
+ * pendingEvictionList.
+ * @param referenceCounted ReferenceCounted object
*/
- private void cleanupInternal() {
- for (Map.Entry> entry : dbMap.entrySet()) {
- dbMap.compute(entry.getKey(), (k, v) -> {
- if (v == null) {
- throw new IllegalStateException("SnapshotId '" + k + "' does not exist in cache. The RocksDB " +
- "instance of the Snapshot may not be closed properly.");
- }
-
- if (v.getTotalRefCount() > 0) {
- LOG.debug("SnapshotId {} is still being referenced ({}), skipping its clean up.", k, v.getTotalRefCount());
- return v;
- } else {
- LOG.debug("Closing SnapshotId {}. It is not being referenced anymore.", k);
- // Close the instance, which also closes its DB handle.
- try {
- v.get().close();
- } catch (IOException ex) {
- throw new IllegalStateException("Error while closing snapshot DB.", ex);
- }
- omMetrics.decNumSnapshotCacheSize();
- return null;
- }
- });
+ @Override
+ public void callback(ReferenceCounted referenceCounted) {
+ if (referenceCounted.getTotalRefCount() == 0L) {
+ // Reference count reaches zero, add to pendingEvictionList
+ pendingEvictionQueue.add(((OmSnapshot) referenceCounted.get())
+ .getSnapshotID());
}
}
}
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
index 2a1e2ec99fca..8546690fa0ae 100644
--- 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
@@ -33,6 +33,7 @@
import java.io.IOException;
import java.util.UUID;
+import java.util.concurrent.TimeoutException;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
@@ -65,6 +66,7 @@ static void beforeAll() throws Exception {
// Mock the snapshotTable return value for the lookup inside release()
final UUID snapshotID = (UUID) invocation.getArguments()[0];
when(omSnapshot.getSnapshotTableKey()).thenReturn(snapshotID.toString());
+ when(omSnapshot.getSnapshotID()).thenReturn(snapshotID);
return omSnapshot;
}
@@ -78,7 +80,7 @@ static void beforeAll() throws Exception {
void setUp() {
// Reset cache for each test case
omMetrics = OMMetrics.create();
- snapshotCache = new SnapshotCache(cacheLoader, CACHE_SIZE_LIMIT, omMetrics);
+ snapshotCache = new SnapshotCache(cacheLoader, CACHE_SIZE_LIMIT, omMetrics, 50);
}
@AfterEach
@@ -205,7 +207,7 @@ private void assertEntryExistence(UUID key, boolean shouldExist) {
@Test
@DisplayName("Basic cache eviction")
- void testEviction1() throws IOException {
+ void testEviction1() throws IOException, InterruptedException, TimeoutException {
final UUID dbKey1 = UUID.randomUUID();
snapshotCache.get(dbKey1);
@@ -233,8 +235,9 @@ void testEviction1() throws IOException {
final UUID dbKey4 = UUID.randomUUID();
snapshotCache.get(dbKey4);
- // dbKey1, dbKey2 and dbKey3 would have been evicted by the end of the last get() because
- // those were release()d.
+ // dbKey1, dbKey2 and dbKey3 would have been evicted by the end of the last scheduled cleanup() because
+ // those were released.
+ GenericTestUtils.waitFor(() -> snapshotCache.size() == 1, 50, 3000);
assertEquals(1, snapshotCache.size());
assertEquals(1, omMetrics.getNumSnapshotCacheSize());
assertEntryExistence(dbKey1, false);
@@ -242,7 +245,7 @@ void testEviction1() throws IOException {
@Test
@DisplayName("Cache eviction while exceeding soft limit")
- void testEviction2() throws IOException {
+ void testEviction2() throws IOException, InterruptedException, TimeoutException {
final UUID dbKey1 = UUID.randomUUID();
snapshotCache.get(dbKey1);
@@ -269,6 +272,7 @@ void testEviction2() throws IOException {
// Releasing dbKey2 at this point should immediately trigger its eviction
// because the cache size exceeded the soft limit
snapshotCache.release(dbKey2);
+ GenericTestUtils.waitFor(() -> snapshotCache.size() == 3, 50, 3000);
assertEquals(3, snapshotCache.size());
assertEquals(3, omMetrics.getNumSnapshotCacheSize());
assertEntryExistence(dbKey2, false);
@@ -277,7 +281,7 @@ void testEviction2() throws IOException {
@Test
@DisplayName("Cache eviction with try-with-resources")
- void testEviction3WithClose() throws IOException {
+ void testEviction3WithClose() throws IOException, InterruptedException, TimeoutException {
final UUID dbKey1 = UUID.randomUUID();
try (ReferenceCounted rcOmSnapshot = snapshotCache.get(dbKey1)) {
@@ -321,6 +325,7 @@ void testEviction3WithClose() throws IOException {
final UUID dbKey4 = UUID.randomUUID();
try (ReferenceCounted rcOmSnapshot = snapshotCache.get(dbKey4)) {
+ GenericTestUtils.waitFor(() -> snapshotCache.size() == 1, 50, 3000);
assertEquals(1L, rcOmSnapshot.getTotalRefCount());
assertEquals(1, snapshotCache.size());
assertEquals(1, omMetrics.getNumSnapshotCacheSize());
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 a9e67b00cc9e..6b2aee8680f2 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
@@ -364,7 +364,7 @@ public void init() throws RocksDBException, IOException, ExecutionException {
omSnapshotManager = mock(OmSnapshotManager.class);
when(ozoneManager.getOmSnapshotManager()).thenReturn(omSnapshotManager);
when(omSnapshotManager.isSnapshotStatus(any(), any())).thenReturn(true);
- SnapshotCache snapshotCache = new SnapshotCache(mockCacheLoader(), 10, omMetrics);
+ SnapshotCache snapshotCache = new SnapshotCache(mockCacheLoader(), 10, omMetrics, 0);
when(omSnapshotManager.getActiveSnapshot(anyString(), anyString(), anyString()))
.thenAnswer(invocationOnMock -> {
@@ -400,6 +400,7 @@ private OmSnapshot getMockedOmSnapshot(UUID snapshotId) {
when(omSnapshot.getName()).thenReturn(snapshotId.toString());
when(omSnapshot.getMetadataManager()).thenReturn(omMetadataManager);
when(omMetadataManager.getStore()).thenReturn(dbStore);
+ when(omSnapshot.getSnapshotID()).thenReturn(snapshotId);
return omSnapshot;
}