From aef7ecace31a4ff3986bb287fec42f3fcfb30bad Mon Sep 17 00:00:00 2001 From: Sachin Kale Date: Fri, 4 Oct 2024 10:50:53 +0530 Subject: [PATCH] Add more unit tests for RemoteStoreUtils and RemoteFsTimestampAwareTranslog (#16151) Signed-off-by: Sachin Kale --- .../index/remote/RemoteStoreUtils.java | 1 + .../RemoteFsTimestampAwareTranslog.java | 15 +- .../index/remote/RemoteStoreUtilsTests.java | 97 +++++++ .../RemoteFsTimestampAwareTranslogTests.java | 249 +++++++++++++++++- 4 files changed, 353 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java b/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java index edcfe17d31fb8..32a1ca0e5d5ab 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java @@ -477,6 +477,7 @@ private static Set getPinnedTimestampLockedFiles( for (Long pinnedTimestamp : pinnedTimestampSet) { String cachedFile = metadataFilePinnedTimestampMap.get(pinnedTimestamp); if (cachedFile != null) { + assert metadataFiles.contains(cachedFile) : "Metadata files should contain [" + cachedFile + "]"; implicitLockedFiles.add(cachedFile); } else { newPinnedTimestamps.add(pinnedTimestamp); diff --git a/server/src/main/java/org/opensearch/index/translog/RemoteFsTimestampAwareTranslog.java b/server/src/main/java/org/opensearch/index/translog/RemoteFsTimestampAwareTranslog.java index ede422ea3c4f7..3ccacde22bbfc 100644 --- a/server/src/main/java/org/opensearch/index/translog/RemoteFsTimestampAwareTranslog.java +++ b/server/src/main/java/org/opensearch/index/translog/RemoteFsTimestampAwareTranslog.java @@ -233,6 +233,7 @@ public void onResponse(List blobMetadata) { // Update cache to keep only those metadata files that are not getting deleted oldFormatMetadataFileGenerationMap.keySet().retainAll(metadataFilesNotToBeDeleted); + oldFormatMetadataFilePrimaryTermMap.keySet().retainAll(metadataFilesNotToBeDeleted); // Delete stale primary terms deleteStaleRemotePrimaryTerms(metadataFilesNotToBeDeleted); } else { @@ -408,9 +409,9 @@ protected Tuple getMinMaxTranslogGenerationFromMetadataFile( } } - private void deleteStaleRemotePrimaryTerms(List metadataFiles) { + private void deleteStaleRemotePrimaryTerms(List metadataFilesNotToBeDeleted) { deleteStaleRemotePrimaryTerms( - metadataFiles, + metadataFilesNotToBeDeleted, translogTransferManager, oldFormatMetadataFilePrimaryTermMap, minPrimaryTermInRemote, @@ -425,7 +426,7 @@ private void deleteStaleRemotePrimaryTerms(List metadataFiles) { * This will also delete all stale translog metadata files from remote except the latest basis the metadata file comparator. */ protected static void deleteStaleRemotePrimaryTerms( - List metadataFiles, + List metadataFilesNotToBeDeleted, TranslogTransferManager translogTransferManager, Map> oldFormatMetadataFilePrimaryTermMap, AtomicLong minPrimaryTermInRemoteAtomicLong, @@ -434,15 +435,15 @@ protected static void deleteStaleRemotePrimaryTerms( // The deletion of older translog files in remote store is on best-effort basis, there is a possibility that there // are older files that are no longer needed and should be cleaned up. In here, we delete all files that are part // of older primary term. - if (metadataFiles.isEmpty()) { + if (metadataFilesNotToBeDeleted.isEmpty()) { logger.trace("No metadata is uploaded yet, returning from deleteStaleRemotePrimaryTerms"); return; } - Optional minPrimaryTermFromMetadataFiles = metadataFiles.stream().map(file -> { + Optional minPrimaryTermFromMetadataFiles = metadataFilesNotToBeDeleted.stream().map(file -> { try { return getMinMaxPrimaryTermFromMetadataFile(file, translogTransferManager, oldFormatMetadataFilePrimaryTermMap).v1(); } catch (IOException e) { - return Long.MAX_VALUE; + return Long.MIN_VALUE; } }).min(Long::compareTo); // First we delete all stale primary terms folders from remote store @@ -459,7 +460,7 @@ protected static void deleteStaleRemotePrimaryTerms( } } - private static Long getMinPrimaryTermInRemote( + protected static Long getMinPrimaryTermInRemote( AtomicLong minPrimaryTermInRemote, TranslogTransferManager translogTransferManager, Logger logger diff --git a/server/src/test/java/org/opensearch/index/remote/RemoteStoreUtilsTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteStoreUtilsTests.java index be30de97ee830..d134a2bad7b74 100644 --- a/server/src/test/java/org/opensearch/index/remote/RemoteStoreUtilsTests.java +++ b/server/src/test/java/org/opensearch/index/remote/RemoteStoreUtilsTests.java @@ -1083,4 +1083,101 @@ public void testIsPinnedTimestampStateStaleFeatureEnabled() { setupRemotePinnedTimestampFeature(true); assertTrue(RemoteStoreUtils.isPinnedTimestampStateStale()); } + + public void testGetPinnedTimestampLockedFilesWithCache() { + setupRemotePinnedTimestampFeature(true); + + Map metadataFilePinnedTimestampCache = new HashMap<>(); + + // Pinned timestamps 800, 900, 1000, 2000 + // Metadata with timestamp 990, 995, 1000, 1001 + // Metadata timestamp 1000 <= Pinned Timestamp 1000 + // Metadata timestamp 1001 <= Pinned Timestamp 2000 + Tuple, Set> metadataAndLocks = testGetPinnedTimestampLockedFilesWithPinnedTimestamps( + List.of(990L, 995L, 1000L, 1001L), + Set.of(800L, 900L, 1000L, 2000L), + metadataFilePinnedTimestampCache + ); + Map metadataFiles = metadataAndLocks.v1(); + Set implicitLockedFiles = metadataAndLocks.v2(); + + assertEquals(2, implicitLockedFiles.size()); + assertTrue(implicitLockedFiles.contains(metadataFiles.get(1000L))); + assertTrue(implicitLockedFiles.contains(metadataFiles.get(1001L))); + // Now we cache all the matches except the last one. + assertEquals(1, metadataFilePinnedTimestampCache.size()); + assertEquals(metadataFiles.get(1000L), metadataFilePinnedTimestampCache.get(1000L)); + + metadataAndLocks = testGetPinnedTimestampLockedFilesWithPinnedTimestamps( + List.of(990L, 995L, 1000L, 1001L, 2000L, 2200L), + Set.of(800L, 900L, 1000L, 2000L, 3000L), + metadataFilePinnedTimestampCache + ); + metadataFiles = metadataAndLocks.v1(); + implicitLockedFiles = metadataAndLocks.v2(); + assertEquals(3, implicitLockedFiles.size()); + assertTrue(implicitLockedFiles.contains(metadataFiles.get(1000L))); + assertTrue(implicitLockedFiles.contains(metadataFiles.get(2000L))); + assertTrue(implicitLockedFiles.contains(metadataFiles.get(2200L))); + assertEquals(2, metadataFilePinnedTimestampCache.size()); + assertEquals(metadataFiles.get(1000L), metadataFilePinnedTimestampCache.get(1000L)); + assertEquals(metadataFiles.get(2000L), metadataFilePinnedTimestampCache.get(2000L)); + + metadataAndLocks = testGetPinnedTimestampLockedFilesWithPinnedTimestamps( + List.of(990L, 995L, 1000L, 1001L, 2000L, 2200L, 2500L), + Set.of(2000L, 3000L), + metadataFilePinnedTimestampCache + ); + metadataFiles = metadataAndLocks.v1(); + implicitLockedFiles = metadataAndLocks.v2(); + assertEquals(2, implicitLockedFiles.size()); + assertTrue(implicitLockedFiles.contains(metadataFiles.get(2000L))); + assertTrue(implicitLockedFiles.contains(metadataFiles.get(2500L))); + assertEquals(1, metadataFilePinnedTimestampCache.size()); + assertEquals(metadataFiles.get(2000L), metadataFilePinnedTimestampCache.get(2000L)); + + metadataAndLocks = testGetPinnedTimestampLockedFilesWithPinnedTimestamps( + List.of(2000L, 2200L, 2500L, 3001L, 4200L, 4600L, 5010L), + Set.of(3000L, 4000L, 5000L, 6000L), + metadataFilePinnedTimestampCache + ); + metadataFiles = metadataAndLocks.v1(); + implicitLockedFiles = metadataAndLocks.v2(); + assertEquals(4, implicitLockedFiles.size()); + assertTrue(implicitLockedFiles.contains(metadataFiles.get(2500L))); + assertTrue(implicitLockedFiles.contains(metadataFiles.get(3001L))); + assertTrue(implicitLockedFiles.contains(metadataFiles.get(4600L))); + assertTrue(implicitLockedFiles.contains(metadataFiles.get(5010L))); + assertEquals(3, metadataFilePinnedTimestampCache.size()); + assertEquals(metadataFiles.get(2500L), metadataFilePinnedTimestampCache.get(3000L)); + assertEquals(metadataFiles.get(3001L), metadataFilePinnedTimestampCache.get(4000L)); + assertEquals(metadataFiles.get(4600L), metadataFilePinnedTimestampCache.get(5000L)); + + metadataAndLocks = testGetPinnedTimestampLockedFilesWithPinnedTimestamps( + List.of(), + Set.of(3000L, 4000L, 5000L, 6000L), + metadataFilePinnedTimestampCache + ); + implicitLockedFiles = metadataAndLocks.v2(); + assertEquals(0, implicitLockedFiles.size()); + assertEquals(3, metadataFilePinnedTimestampCache.size()); + + assertThrows( + AssertionError.class, + () -> testGetPinnedTimestampLockedFilesWithPinnedTimestamps( + List.of(2000L, 2200L, 3001L, 4200L, 4600L, 5010L), + Set.of(3000L, 4000L, 5000L, 6000L), + metadataFilePinnedTimestampCache + ) + ); + + metadataAndLocks = testGetPinnedTimestampLockedFilesWithPinnedTimestamps( + List.of(2000L, 2200L, 2500L, 3001L, 4200L, 4600L, 5010L), + Set.of(), + metadataFilePinnedTimestampCache + ); + implicitLockedFiles = metadataAndLocks.v2(); + assertEquals(0, implicitLockedFiles.size()); + assertEquals(0, metadataFilePinnedTimestampCache.size()); + } } diff --git a/server/src/test/java/org/opensearch/index/translog/RemoteFsTimestampAwareTranslogTests.java b/server/src/test/java/org/opensearch/index/translog/RemoteFsTimestampAwareTranslogTests.java index e6871414cf5e0..73db3314f4d1e 100644 --- a/server/src/test/java/org/opensearch/index/translog/RemoteFsTimestampAwareTranslogTests.java +++ b/server/src/test/java/org/opensearch/index/translog/RemoteFsTimestampAwareTranslogTests.java @@ -721,6 +721,42 @@ public void testGetGenerationsToBeDeleted() throws IOException { assertTrue(generations.isEmpty()); } + public void testGetGenerationsToBeDeletedWithGenerationInRemote() throws IOException { + List metadataFilesNotToBeDeleted = List.of( + // 1 to 4 + "metadata__9223372036854775806__9223372036854775803__9223370311919910398__31__9223372036854775806__1__1", + // 26 to 30 + "metadata__9223372036854775806__9223372036854775777__9223370311919910398__31__9223372036854775781__1__1", + // 42 to 100 + "metadata__9223372036854775806__9223372036854775707__9223370311919910403__31__9223372036854775765__1__1" + ); + List metadataFilesToBeDeleted = List.of( + // 4 to 7 + "metadata__9223372036854775806__9223372036854775800__9223370311919910398__31__9223372036854775803__1__1", + // 17 to 37 + "metadata__9223372036854775806__9223372036854775770__9223370311919910398__31__9223372036854775790__1__1", + // 27 to 42 + "metadata__9223372036854775806__9223372036854775765__9223370311919910403__31__9223372036854775780__1__1" + ); + Set generations = ((RemoteFsTimestampAwareTranslog) translog).getGenerationsToBeDeleted( + metadataFilesNotToBeDeleted, + metadataFilesToBeDeleted, + 35 + ); + Set md1Generations = LongStream.rangeClosed(5, 7).boxed().collect(Collectors.toSet()); + Set md2Generations = LongStream.rangeClosed(17, 25).boxed().collect(Collectors.toSet()); + Set md3Generations = LongStream.rangeClosed(31, 34).boxed().collect(Collectors.toSet()); + + assertTrue(generations.containsAll(md1Generations)); + assertTrue(generations.containsAll(md2Generations)); + assertTrue(generations.containsAll(md3Generations)); + + generations.removeAll(md1Generations); + generations.removeAll(md2Generations); + generations.removeAll(md3Generations); + assertTrue(generations.isEmpty()); + } + public void testGetMetadataFilesToBeDeletedNoExclusion() { updatePinnedTimstampTask.run(); @@ -982,6 +1018,16 @@ public void testGetMinMaxTranslogGenerationFromMetadataFile() throws IOException translogTransferManager ) ); + // Calling it again to check if the details are getting fetched from the cache. + // Number of calls to readMetadata will be able to verify this. + assertEquals( + new Tuple<>(701L, 1008L), + translog.getMinMaxTranslogGenerationFromMetadataFile( + "metadata__9223372036438563903__9223372036854774799__9223370311919910393__31__1", + translogTransferManager + ) + ); + TranslogTransferMetadata md2 = mock(TranslogTransferMetadata.class); when(md2.getMinTranslogGeneration()).thenReturn(4L); when(md2.getGeneration()).thenReturn(7L); @@ -994,9 +1040,81 @@ public void testGetMinMaxTranslogGenerationFromMetadataFile() throws IOException translogTransferManager ) ); + // Calling it again to check if the details are getting fetched from the cache. + // Number of calls to readMetadata will be able to verify this. + assertEquals( + new Tuple<>(4L, 7L), + translog.getMinMaxTranslogGenerationFromMetadataFile( + "metadata__9223372036438563903__9223372036854775800__9223370311919910398__31__1", + translogTransferManager + ) + ); + + verify(translogTransferManager, times(1)).readMetadata( + "metadata__9223372036438563903__9223372036854774799__9223370311919910393__31__1" + ); + verify(translogTransferManager, times(1)).readMetadata( + "metadata__9223372036438563903__9223372036854775800__9223370311919910398__31__1" + ); + } + + public void testGetMinMaxPrimaryTermFromMetadataFile() throws IOException { + TranslogTransferManager translogTransferManager = mock(TranslogTransferManager.class); + + RemoteFsTimestampAwareTranslog translog = (RemoteFsTimestampAwareTranslog) this.translog; + + // Fetch generations directly from the filename + assertEquals( + new Tuple<>(1L, 1008L), + RemoteFsTimestampAwareTranslog.getMinMaxPrimaryTermFromMetadataFile( + "metadata__9223372036854774799__9223372036854774799__9223370311919910393__31__9223372036854775106__1__1", + translogTransferManager, + new HashMap<>() + ) + ); + assertEquals( + new Tuple<>(4L, 7L), + RemoteFsTimestampAwareTranslog.getMinMaxPrimaryTermFromMetadataFile( + "metadata__9223372036854775800__9223372036854775800__9223370311919910398__31__9223372036854775803__4__1", + translogTransferManager, + new HashMap<>() + ) + ); + assertEquals( + new Tuple<>(10L, 10L), + RemoteFsTimestampAwareTranslog.getMinMaxPrimaryTermFromMetadataFile( + "metadata__9223372036854775797__9223372036854775800__9223370311919910398__31__9223372036854775803__10__1", + translogTransferManager, + new HashMap<>() + ) + ); + + // For older md filenames, it needs to read the content + TranslogTransferMetadata md1 = mock(TranslogTransferMetadata.class); + when(md1.getGenerationToPrimaryTermMapper()).thenReturn(Map.of("12", "1", "23", "1", "34", "2")); + when(translogTransferManager.readMetadata("metadata__9223372036854775805__9223372036854774799__9223370311919910393__31__1")) + .thenReturn(md1); + assertEquals( + new Tuple<>(1L, 2L), + RemoteFsTimestampAwareTranslog.getMinMaxPrimaryTermFromMetadataFile( + "metadata__9223372036854775805__9223372036854774799__9223370311919910393__31__1", + translogTransferManager, + new HashMap<>() + ) + ); + assertEquals( + new Tuple<>(4L, 7L), + RemoteFsTimestampAwareTranslog.getMinMaxPrimaryTermFromMetadataFile( + "metadata__9223372036438563903__9223372036854775800__9223370311919910398__31__1", + translogTransferManager, + Map.of("metadata__9223372036438563903__9223372036854775800__9223370311919910398__31__1", new Tuple<>(4L, 7L)) + ) + ); - verify(translogTransferManager).readMetadata("metadata__9223372036438563903__9223372036854774799__9223370311919910393__31__1"); - verify(translogTransferManager).readMetadata("metadata__9223372036438563903__9223372036854775800__9223370311919910398__31__1"); + verify(translogTransferManager).readMetadata("metadata__9223372036854775805__9223372036854774799__9223370311919910393__31__1"); + verify(translogTransferManager, times(0)).readMetadata( + "metadata__9223372036438563903__9223372036854775800__9223370311919910398__31__1" + ); } public void testDeleteStaleRemotePrimaryTerms() throws IOException { @@ -1035,6 +1153,96 @@ public void testDeleteStaleRemotePrimaryTerms() throws IOException { verify(translogTransferManager, times(1)).deletePrimaryTermsAsync(anyLong()); } + public void testDeleteStaleRemotePrimaryTermsOldFormat() throws IOException { + TranslogTransferManager translogTransferManager = mock(TranslogTransferManager.class); + + List metadataFiles = List.of( + // PT 4 to 9 + "metadata__9223372036854775798__9223372036854774799__9223370311919910393__node1__1", + // PT 2 to 7 + "metadata__9223372036854775800__9223372036854774799__9223370311919910393__node1__1", + // PT 2 to 6 + "metadata__9223372036854775801__9223372036854774799__9223370311919910393__node1__1" + ); + + Logger staticLogger = LogManager.getLogger(RemoteFsTimestampAwareTranslogTests.class); + when(translogTransferManager.listPrimaryTermsInRemote()).thenReturn(Set.of(1L, 2L, 3L, 4L)); + + TranslogTransferMetadata md1 = mock(TranslogTransferMetadata.class); + when(md1.getGenerationToPrimaryTermMapper()).thenReturn(Map.of("12", "4", "23", "5", "34", "5")); + when(translogTransferManager.readMetadata("metadata__9223372036854775798__9223372036854774799__9223370311919910393__node1__1")) + .thenReturn(md1); + TranslogTransferMetadata md2 = mock(TranslogTransferMetadata.class); + when(md2.getGenerationToPrimaryTermMapper()).thenReturn(Map.of("12", "2", "23", "2", "34", "3")); + when(translogTransferManager.readMetadata("metadata__9223372036854775800__9223372036854774799__9223370311919910393__node1__1")) + .thenReturn(md2); + TranslogTransferMetadata md3 = mock(TranslogTransferMetadata.class); + when(md3.getGenerationToPrimaryTermMapper()).thenReturn(Map.of("12", "2", "23", "2", "34", "2")); + when(translogTransferManager.readMetadata("metadata__9223372036854775801__9223372036854774799__9223370311919910393__node1__1")) + .thenReturn(md3); + + AtomicLong minPrimaryTermInRemote = new AtomicLong(Long.MAX_VALUE); + RemoteFsTimestampAwareTranslog.deleteStaleRemotePrimaryTerms( + metadataFiles, + translogTransferManager, + new HashMap<>(), + minPrimaryTermInRemote, + staticLogger + ); + verify(translogTransferManager).deletePrimaryTermsAsync(2L); + assertEquals(2, minPrimaryTermInRemote.get()); + + RemoteFsTimestampAwareTranslog.deleteStaleRemotePrimaryTerms( + metadataFiles, + translogTransferManager, + new HashMap<>(), + minPrimaryTermInRemote, + staticLogger + ); + // This means there are no new invocations of deletePrimaryTermAsync + verify(translogTransferManager, times(1)).deletePrimaryTermsAsync(anyLong()); + } + + public void testDeleteStaleRemotePrimaryTermsOldFormatException() throws IOException { + TranslogTransferManager translogTransferManager = mock(TranslogTransferManager.class); + + List metadataFiles = List.of( + // PT 4 to 9 + "metadata__9223372036854775798__9223372036854774799__9223370311919910393__node1__1", + // PT 2 to 7 + "metadata__9223372036854775800__9223372036854774799__9223370311919910393__node1__1", + // PT 2 to 6 + "metadata__9223372036854775801__9223372036854774799__9223370311919910393__node1__1" + ); + + Logger staticLogger = LogManager.getLogger(RemoteFsTimestampAwareTranslogTests.class); + when(translogTransferManager.listPrimaryTermsInRemote()).thenReturn(Set.of(1L, 2L, 3L, 4L)); + + TranslogTransferMetadata md1 = mock(TranslogTransferMetadata.class); + when(md1.getGenerationToPrimaryTermMapper()).thenReturn(Map.of("12", "4", "23", "5", "34", "5")); + when(translogTransferManager.readMetadata("metadata__9223372036854775798__9223372036854774799__9223370311919910393__node1__1")) + .thenReturn(md1); + TranslogTransferMetadata md2 = mock(TranslogTransferMetadata.class); + when(md2.getGenerationToPrimaryTermMapper()).thenReturn(Map.of("12", "2", "23", "2", "34", "3")); + when(translogTransferManager.readMetadata("metadata__9223372036854775800__9223372036854774799__9223370311919910393__node1__1")) + .thenReturn(md2); + TranslogTransferMetadata md3 = mock(TranslogTransferMetadata.class); + when(md3.getGenerationToPrimaryTermMapper()).thenReturn(Map.of("12", "2", "23", "2", "34", "2")); + // Exception while reading this file + when(translogTransferManager.readMetadata("metadata__9223372036854775801__9223372036854774799__9223370311919910393__node1__1")) + .thenThrow(new IOException()); + + AtomicLong minPrimaryTermInRemote = new AtomicLong(4); + RemoteFsTimestampAwareTranslog.deleteStaleRemotePrimaryTerms( + metadataFiles, + translogTransferManager, + new HashMap<>(), + minPrimaryTermInRemote, + staticLogger + ); + verify(translogTransferManager, times(0)).deletePrimaryTermsAsync(anyLong()); + } + public void testDeleteStaleRemotePrimaryTermsNoPrimaryTermInRemote() throws IOException { TranslogTransferManager translogTransferManager = mock(TranslogTransferManager.class); @@ -1087,4 +1295,41 @@ public void testDeleteStaleRemotePrimaryTermsPrimaryTermInRemoteIsBigger() throw assertEquals(2, minPrimaryTermInRemote.get()); } + public void testGetMinPrimaryTermInRemoteAlreadyFetched() { + Long expectedMinPrimaryTerm = 12L; + assertEquals(expectedMinPrimaryTerm, RemoteFsTimestampAwareTranslog.getMinPrimaryTermInRemote(new AtomicLong(12), null, null)); + } + + public void testGetMinPrimaryTermInRemoteNotFetchedEmpty() throws IOException { + TranslogTransferManager translogTransferManager = mock(TranslogTransferManager.class); + when(translogTransferManager.listPrimaryTermsInRemote()).thenReturn(Set.of()); + Long expectedMinPrimaryTerm = Long.MAX_VALUE; + assertEquals( + expectedMinPrimaryTerm, + RemoteFsTimestampAwareTranslog.getMinPrimaryTermInRemote(new AtomicLong(Long.MAX_VALUE), translogTransferManager, null) + ); + verify(translogTransferManager).listPrimaryTermsInRemote(); + } + + public void testGetMinPrimaryTermInRemoteNotFetchedException() throws IOException { + TranslogTransferManager translogTransferManager = mock(TranslogTransferManager.class); + when(translogTransferManager.listPrimaryTermsInRemote()).thenThrow(new IOException()); + Long expectedMinPrimaryTerm = Long.MAX_VALUE; + assertEquals( + expectedMinPrimaryTerm, + RemoteFsTimestampAwareTranslog.getMinPrimaryTermInRemote(new AtomicLong(Long.MAX_VALUE), translogTransferManager, logger) + ); + verify(translogTransferManager).listPrimaryTermsInRemote(); + } + + public void testGetMinPrimaryTermInRemoteNotFetched() throws IOException { + TranslogTransferManager translogTransferManager = mock(TranslogTransferManager.class); + when(translogTransferManager.listPrimaryTermsInRemote()).thenReturn(Set.of(12L, 23L, 34L, 4L, 89L)); + Long expectedMinPrimaryTerm = 4L; + assertEquals( + expectedMinPrimaryTerm, + RemoteFsTimestampAwareTranslog.getMinPrimaryTermInRemote(new AtomicLong(Long.MAX_VALUE), translogTransferManager, logger) + ); + verify(translogTransferManager).listPrimaryTermsInRemote(); + } }