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; }