From 5a396a80f4777d840b13312fa03f613de01be177 Mon Sep 17 00:00:00 2001 From: Bryan Burkholder Date: Mon, 12 Aug 2024 14:29:15 -0700 Subject: [PATCH 1/7] Initial chunk store implementation --- .../com/slack/astra/blobfs/ChunkStore.java | 100 +++++++++++++++++ .../slack/astra/chunk/ReadOnlyChunkImpl.java | 36 +++--- .../com/slack/astra/chunk/ReadWriteChunk.java | 2 + .../chunk/SerialS3ChunkDownloaderImpl.java | 1 + .../chunkManager/CachingChunkManager.java | 16 +-- .../java/com/slack/astra/server/Astra.java | 7 +- .../slack/astra/blobfs/ChunkStoreTest.java | 106 ++++++++++++++++++ .../astra/chunk/ReadOnlyChunkImplTest.java | 13 ++- .../chunkManager/CachingChunkManagerTest.java | 5 +- 9 files changed, 251 insertions(+), 35 deletions(-) create mode 100644 astra/src/main/java/com/slack/astra/blobfs/ChunkStore.java create mode 100644 astra/src/test/java/com/slack/astra/blobfs/ChunkStoreTest.java diff --git a/astra/src/main/java/com/slack/astra/blobfs/ChunkStore.java b/astra/src/main/java/com/slack/astra/blobfs/ChunkStore.java new file mode 100644 index 0000000000..806fbfdcd7 --- /dev/null +++ b/astra/src/main/java/com/slack/astra/blobfs/ChunkStore.java @@ -0,0 +1,100 @@ +package com.slack.astra.blobfs; + +import static software.amazon.awssdk.services.s3.model.ListObjectsV2Request.builder; + +import java.nio.file.Path; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicBoolean; +import software.amazon.awssdk.services.s3.S3AsyncClient; +import software.amazon.awssdk.services.s3.model.Delete; +import software.amazon.awssdk.services.s3.model.DeleteObjectsRequest; +import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; +import software.amazon.awssdk.services.s3.model.ObjectIdentifier; +import software.amazon.awssdk.services.s3.paginators.ListObjectsV2Publisher; +import software.amazon.awssdk.transfer.s3.S3TransferManager; +import software.amazon.awssdk.transfer.s3.model.DownloadDirectoryRequest; +import software.amazon.awssdk.transfer.s3.model.UploadDirectoryRequest; + +public class ChunkStore { + + private final String bucketName; + private final S3AsyncClient s3AsyncClient; + private final S3TransferManager transferManager; + + public ChunkStore(S3AsyncClient s3AsyncClient, String bucketName) { + this.bucketName = bucketName; + this.s3AsyncClient = s3AsyncClient; + this.transferManager = S3TransferManager.builder().s3Client(s3AsyncClient).build(); + } + + public void upload(String chunkId, Path directoryToUpload) { + try { + transferManager + .uploadDirectory( + UploadDirectoryRequest.builder() + .source(directoryToUpload) + .s3Prefix(chunkId) + .bucket(bucketName) + .build()) + .completionFuture() + .get(); + } catch (ExecutionException | InterruptedException e) { + throw new RuntimeException(e); + } + } + + // todo - make destination optional + public Path download(String chunkId, Path destination) { + try { + transferManager + .downloadDirectory( + DownloadDirectoryRequest.builder() + .bucket(bucketName) + .destination(destination) + .listObjectsV2RequestTransformer(l -> l.prefix(chunkId)) + .build()) + .completionFuture() + .get(); + return destination; + } catch (ExecutionException | InterruptedException e) { + throw new RuntimeException(e); + } + } + + public boolean delete(String chunkId) { + ListObjectsV2Request listRequest = builder().bucket(bucketName).prefix(chunkId).build(); + ListObjectsV2Publisher asyncPaginatedListResponse = + s3AsyncClient.listObjectsV2Paginator(listRequest); + + AtomicBoolean deleted = new AtomicBoolean(false); + try { + asyncPaginatedListResponse + .subscribe( + listResponse -> { + List objects = + listResponse.contents().stream() + .map(s3Object -> ObjectIdentifier.builder().key(s3Object.key()).build()) + .toList(); + if (objects.isEmpty()) { + return; + } + DeleteObjectsRequest deleteRequest = + DeleteObjectsRequest.builder() + .bucket(bucketName) + .delete(Delete.builder().objects(objects).build()) + .build(); + try { + s3AsyncClient.deleteObjects(deleteRequest).get(); + deleted.set(true); + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException(e); + } + }) + .get(); + return deleted.get(); + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException(e); + } + } +} diff --git a/astra/src/main/java/com/slack/astra/chunk/ReadOnlyChunkImpl.java b/astra/src/main/java/com/slack/astra/chunk/ReadOnlyChunkImpl.java index 5646254eca..8dc1b5bd3c 100644 --- a/astra/src/main/java/com/slack/astra/chunk/ReadOnlyChunkImpl.java +++ b/astra/src/main/java/com/slack/astra/chunk/ReadOnlyChunkImpl.java @@ -4,7 +4,7 @@ import static com.slack.astra.server.AstraConfig.DEFAULT_ZK_TIMEOUT_SECS; import com.google.common.annotations.VisibleForTesting; -import com.slack.astra.blobfs.BlobFs; +import com.slack.astra.blobfs.ChunkStore; import com.slack.astra.logstore.search.LogIndexSearcher; import com.slack.astra.logstore.search.LogIndexSearcherImpl; import com.slack.astra.logstore.search.SearchQuery; @@ -64,7 +64,6 @@ public class ReadOnlyChunkImpl implements Chunk { private Metadata.CacheNodeAssignment.CacheNodeAssignmentState lastKnownAssignmentState; private final String dataDirectoryPrefix; - private final String s3Bucket; protected final SearchContext searchContext; protected final String slotId; private final CacheSlotMetadataStore cacheSlotMetadataStore; @@ -73,7 +72,7 @@ public class ReadOnlyChunkImpl implements Chunk { private final SearchMetadataStore searchMetadataStore; private CacheNodeAssignmentStore cacheNodeAssignmentStore; private final MeterRegistry meterRegistry; - private final BlobFs blobFs; + private final ChunkStore chunkStore; public static final String CHUNK_ASSIGNMENT_TIMER = "chunk_assignment_timer"; public static final String CHUNK_EVICTION_TIMER = "chunk_eviction_timer"; @@ -91,7 +90,7 @@ public class ReadOnlyChunkImpl implements Chunk { public ReadOnlyChunkImpl( AsyncCuratorFramework curatorFramework, MeterRegistry meterRegistry, - BlobFs blobFs, + ChunkStore chunkStore, SearchContext searchContext, String s3Bucket, String dataDirectoryPrefix, @@ -107,7 +106,7 @@ public ReadOnlyChunkImpl( this( curatorFramework, meterRegistry, - blobFs, + chunkStore, searchContext, s3Bucket, dataDirectoryPrefix, @@ -125,7 +124,7 @@ public ReadOnlyChunkImpl( public ReadOnlyChunkImpl( AsyncCuratorFramework curatorFramework, MeterRegistry meterRegistry, - BlobFs blobFs, + ChunkStore chunkStore, SearchContext searchContext, String s3Bucket, String dataDirectoryPrefix, @@ -136,8 +135,7 @@ public ReadOnlyChunkImpl( SearchMetadataStore searchMetadataStore) throws Exception { this.meterRegistry = meterRegistry; - this.blobFs = blobFs; - this.s3Bucket = s3Bucket; + this.chunkStore = chunkStore; this.dataDirectoryPrefix = dataDirectoryPrefix; this.searchContext = searchContext; this.slotId = UUID.randomUUID().toString(); @@ -235,12 +233,12 @@ public void downloadChunkData() { } } } - // init SerialS3DownloaderImpl w/ bucket, snapshotId, blob, data directory - SerialS3ChunkDownloaderImpl chunkDownloader = - new SerialS3ChunkDownloaderImpl( - s3Bucket, snapshotMetadata.snapshotId, blobFs, dataDirectory); - if (chunkDownloader.download()) { - throw new IOException("No files found on blob storage, released slot for re-assignment"); + + chunkStore.download(snapshotMetadata.snapshotId, dataDirectory); + try (Stream fileList = Files.list(dataDirectory)) { + if (fileList.findAny().isEmpty()) { + throw new IOException("No files found on blob storage, released slot for re-assignment"); + } } Path schemaPath = Path.of(dataDirectory.toString(), ReadWriteChunk.SCHEMA_FILE_NAME); @@ -379,11 +377,11 @@ private void handleChunkAssignment(CacheSlotMetadata cacheSlotMetadata) { } SnapshotMetadata snapshotMetadata = getSnapshotMetadata(cacheSlotMetadata.replicaId); - SerialS3ChunkDownloaderImpl chunkDownloader = - new SerialS3ChunkDownloaderImpl( - s3Bucket, snapshotMetadata.snapshotId, blobFs, dataDirectory); - if (chunkDownloader.download()) { - throw new IOException("No files found on blob storage, released slot for re-assignment"); + chunkStore.download(snapshotMetadata.snapshotId, dataDirectory); + try (Stream fileList = Files.list(dataDirectory)) { + if (fileList.findAny().isEmpty()) { + throw new IOException("No files found on blob storage, released slot for re-assignment"); + } } Path schemaPath = Path.of(dataDirectory.toString(), ReadWriteChunk.SCHEMA_FILE_NAME); diff --git a/astra/src/main/java/com/slack/astra/chunk/ReadWriteChunk.java b/astra/src/main/java/com/slack/astra/chunk/ReadWriteChunk.java index fe10992218..49fe7bc5c1 100644 --- a/astra/src/main/java/com/slack/astra/chunk/ReadWriteChunk.java +++ b/astra/src/main/java/com/slack/astra/chunk/ReadWriteChunk.java @@ -254,7 +254,9 @@ public boolean snapshotToS3(String bucket, String prefix, BlobFs blobFs) { } this.fileUploadAttempts.increment(filesToUpload.size()); Timer.Sample snapshotTimer = Timer.start(meterRegistry); + final int success = copyToS3(dirPath, filesToUpload, bucket, prefix, blobFs); + snapshotTimer.stop(meterRegistry.timer(SNAPSHOT_TIMER)); this.fileUploadFailures.increment(filesToUpload.size() - success); chunkInfo.setSnapshotPath(createURI(bucket, prefix, "").toString()); diff --git a/astra/src/main/java/com/slack/astra/chunk/SerialS3ChunkDownloaderImpl.java b/astra/src/main/java/com/slack/astra/chunk/SerialS3ChunkDownloaderImpl.java index 1c50493329..c88f0488a3 100644 --- a/astra/src/main/java/com/slack/astra/chunk/SerialS3ChunkDownloaderImpl.java +++ b/astra/src/main/java/com/slack/astra/chunk/SerialS3ChunkDownloaderImpl.java @@ -9,6 +9,7 @@ * SerialS3ChunkDownloaderImpl downloads all the chunk related data from S3 to a local store one * file at a time. */ +@Deprecated public class SerialS3ChunkDownloaderImpl implements ChunkDownloader { private final String s3Bucket; private final String snapshotId; diff --git a/astra/src/main/java/com/slack/astra/chunkManager/CachingChunkManager.java b/astra/src/main/java/com/slack/astra/chunkManager/CachingChunkManager.java index dd20758331..e454bc2825 100644 --- a/astra/src/main/java/com/slack/astra/chunkManager/CachingChunkManager.java +++ b/astra/src/main/java/com/slack/astra/chunkManager/CachingChunkManager.java @@ -2,7 +2,7 @@ import static com.slack.astra.clusterManager.CacheNodeAssignmentService.snapshotMetadataBySnapshotId; -import com.slack.astra.blobfs.BlobFs; +import com.slack.astra.blobfs.ChunkStore; import com.slack.astra.chunk.Chunk; import com.slack.astra.chunk.ReadOnlyChunkImpl; import com.slack.astra.chunk.SearchContext; @@ -39,7 +39,7 @@ public class CachingChunkManager extends ChunkManagerBase { private final MeterRegistry meterRegistry; private final AsyncCuratorFramework curatorFramework; - private final BlobFs blobFs; + private final ChunkStore chunkStore; private final SearchContext searchContext; private final String s3Bucket; private final String dataDirectoryPrefix; @@ -61,7 +61,7 @@ public class CachingChunkManager extends ChunkManagerBase { public CachingChunkManager( MeterRegistry registry, AsyncCuratorFramework curatorFramework, - BlobFs blobFs, + ChunkStore chunkStore, SearchContext searchContext, String s3Bucket, String dataDirectoryPrefix, @@ -70,7 +70,7 @@ public CachingChunkManager( long capacityBytes) { this.meterRegistry = registry; this.curatorFramework = curatorFramework; - this.blobFs = blobFs; + this.chunkStore = chunkStore; this.searchContext = searchContext; this.s3Bucket = s3Bucket; this.dataDirectoryPrefix = dataDirectoryPrefix; @@ -103,7 +103,7 @@ protected void startUp() throws Exception { new ReadOnlyChunkImpl<>( curatorFramework, meterRegistry, - blobFs, + chunkStore, searchContext, s3Bucket, dataDirectoryPrefix, @@ -153,12 +153,12 @@ public static CachingChunkManager fromConfig( AsyncCuratorFramework curatorFramework, AstraConfigs.S3Config s3Config, AstraConfigs.CacheConfig cacheConfig, - BlobFs blobFs) + ChunkStore chunkStore) throws Exception { return new CachingChunkManager<>( meterRegistry, curatorFramework, - blobFs, + chunkStore, SearchContext.fromConfig(cacheConfig.getServerConfig()), s3Config.getS3Bucket(), cacheConfig.getDataDirectory(), @@ -211,7 +211,7 @@ private void onAssignmentHandler(CacheNodeAssignment assignment) { new ReadOnlyChunkImpl<>( curatorFramework, meterRegistry, - blobFs, + chunkStore, searchContext, s3Bucket, dataDirectoryPrefix, diff --git a/astra/src/main/java/com/slack/astra/server/Astra.java b/astra/src/main/java/com/slack/astra/server/Astra.java index 0cb2370f60..d8f69a5a41 100644 --- a/astra/src/main/java/com/slack/astra/server/Astra.java +++ b/astra/src/main/java/com/slack/astra/server/Astra.java @@ -4,6 +4,7 @@ import com.google.common.util.concurrent.Service; import com.google.common.util.concurrent.ServiceManager; import com.slack.astra.blobfs.BlobFs; +import com.slack.astra.blobfs.ChunkStore; import com.slack.astra.blobfs.s3.S3CrtBlobFs; import com.slack.astra.bulkIngestApi.BulkIngestApi; import com.slack.astra.bulkIngestApi.BulkIngestKafkaProducer; @@ -142,9 +143,10 @@ public void start() throws Exception { // Initialize blobfs. Only S3 is supported currently. S3CrtBlobFs s3BlobFs = new S3CrtBlobFs(s3Client); + ChunkStore chunkStore = new ChunkStore(s3Client, astraConfig.getS3Config().getS3Bucket()); Set services = - getServices(curatorFramework, astraConfig, s3BlobFs, prometheusMeterRegistry); + getServices(curatorFramework, astraConfig, s3BlobFs, chunkStore, prometheusMeterRegistry); serviceManager = new ServiceManager(services); serviceManager.addListener(getServiceManagerListener(), MoreExecutors.directExecutor()); @@ -155,6 +157,7 @@ private static Set getServices( AsyncCuratorFramework curatorFramework, AstraConfigs.AstraConfig astraConfig, BlobFs blobFs, + ChunkStore chunkStore, PrometheusMeterRegistry meterRegistry) throws Exception { Set services = new HashSet<>(); @@ -238,7 +241,7 @@ private static Set getServices( curatorFramework, astraConfig.getS3Config(), astraConfig.getCacheConfig(), - blobFs); + chunkStore); services.add(chunkManager); HpaMetricMetadataStore hpaMetricMetadataStore = diff --git a/astra/src/test/java/com/slack/astra/blobfs/ChunkStoreTest.java b/astra/src/test/java/com/slack/astra/blobfs/ChunkStoreTest.java new file mode 100644 index 0000000000..d3125fed9e --- /dev/null +++ b/astra/src/test/java/com/slack/astra/blobfs/ChunkStoreTest.java @@ -0,0 +1,106 @@ +package com.slack.astra.blobfs; + +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +import com.adobe.testing.s3mock.junit5.S3MockExtension; +import com.slack.astra.blobfs.s3.S3TestUtils; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Objects; +import java.util.UUID; +import java.util.concurrent.ExecutionException; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import software.amazon.awssdk.services.s3.S3AsyncClient; +import software.amazon.awssdk.services.s3.model.ListObjectsRequest; + +class ChunkStoreTest { + private static final String TEST_BUCKET = "chunkStoreTest"; + + @RegisterExtension + public static final S3MockExtension S3_MOCK_EXTENSION = + S3MockExtension.builder() + .silent() + .withInitialBuckets(TEST_BUCKET) + .withSecureConnection(false) + .build(); + + private final S3AsyncClient s3Client = + S3TestUtils.createS3CrtClient(S3_MOCK_EXTENSION.getServiceEndpoint()); + + @Test + void testUploadDownload() throws IOException { + ChunkStore chunkStore = new ChunkStore(s3Client, TEST_BUCKET); + + Path directoryUpload = Files.createTempDirectory(""); + Path foo = Files.createTempFile(directoryUpload, "", ""); + try (FileWriter fileWriter = new FileWriter(foo.toFile())) { + fileWriter.write("Example test"); + } + String chunkId = UUID.randomUUID().toString(); + chunkStore.upload(chunkId, directoryUpload); + + // what goes up, must come down + Path directoryDownloaded = Files.createTempDirectory(""); + chunkStore.download(chunkId, directoryDownloaded); + + File[] filesDownloaded = directoryDownloaded.toFile().listFiles(); + assertThat(Objects.requireNonNull(filesDownloaded).length).isEqualTo(1); + + // contents of the file we uploaded should match + assertThat(Files.readAllBytes(filesDownloaded[0].toPath())).isEqualTo(Files.readAllBytes(foo)); + } + + @Test + void testDownloadDoesNotExist() throws IOException { + ChunkStore chunkStore = new ChunkStore(s3Client, TEST_BUCKET); + Path directoryDownloaded = Files.createTempDirectory(""); + chunkStore.download(UUID.randomUUID().toString(), directoryDownloaded); + } + + @Test + void testDeleteMultipleFiles() throws IOException, ExecutionException, InterruptedException { + ChunkStore chunkStore = new ChunkStore(s3Client, TEST_BUCKET); + + Path directoryUpload = Files.createTempDirectory(""); + Path foo = Files.createTempFile(directoryUpload, "", ""); + try (FileWriter fileWriter = new FileWriter(foo.toFile())) { + fileWriter.write("Example test 1"); + } + Path bar = Files.createTempFile(directoryUpload, "", ""); + try (FileWriter fileWriter = new FileWriter(bar.toFile())) { + fileWriter.write("Example test 2"); + } + String chunkId = UUID.randomUUID().toString(); + chunkStore.upload(chunkId, directoryUpload); + assertThat( + s3Client + .listObjects( + ListObjectsRequest.builder().bucket(TEST_BUCKET).prefix(chunkId).build()) + .get() + .contents() + .size()) + .isEqualTo(2); + + boolean deleted = chunkStore.delete(chunkId); + assertThat(deleted).isTrue(); + assertThat( + s3Client + .listObjects( + ListObjectsRequest.builder().bucket(TEST_BUCKET).prefix(chunkId).build()) + .get() + .contents() + .size()) + .isEqualTo(0); + } + + @Test + void testDeleteDoesNotExist() { + ChunkStore chunkStore = new ChunkStore(s3Client, TEST_BUCKET); + boolean deleted = chunkStore.delete(UUID.randomUUID().toString()); + assertThat(deleted).isFalse(); + } +} diff --git a/astra/src/test/java/com/slack/astra/chunk/ReadOnlyChunkImplTest.java b/astra/src/test/java/com/slack/astra/chunk/ReadOnlyChunkImplTest.java index cf579a446c..4dc0d2dccb 100644 --- a/astra/src/test/java/com/slack/astra/chunk/ReadOnlyChunkImplTest.java +++ b/astra/src/test/java/com/slack/astra/chunk/ReadOnlyChunkImplTest.java @@ -17,6 +17,7 @@ import brave.Tracing; import com.adobe.testing.s3mock.junit5.S3MockExtension; +import com.slack.astra.blobfs.ChunkStore; import com.slack.astra.blobfs.LocalBlobFs; import com.slack.astra.blobfs.s3.S3CrtBlobFs; import com.slack.astra.blobfs.s3.S3TestUtils; @@ -73,6 +74,7 @@ public class ReadOnlyChunkImplTest { private TestingServer testingServer; private MeterRegistry meterRegistry; private S3CrtBlobFs s3CrtBlobFs; + private ChunkStore chunkStore; @RegisterExtension public static final S3MockExtension S3_MOCK_EXTENSION = @@ -91,6 +93,7 @@ public void startup() throws Exception { S3AsyncClient s3AsyncClient = S3TestUtils.createS3CrtClient(S3_MOCK_EXTENSION.getServiceEndpoint()); s3CrtBlobFs = new S3CrtBlobFs(s3AsyncClient); + chunkStore = new ChunkStore(s3AsyncClient, TEST_S3_BUCKET); } @AfterEach @@ -132,7 +135,7 @@ public void shouldHandleChunkLivecycle() throws Exception { new ReadOnlyChunkImpl<>( curatorFramework, meterRegistry, - s3CrtBlobFs, + chunkStore, searchContext, AstraConfig.getS3Config().getS3Bucket(), AstraConfig.getCacheConfig().getDataDirectory(), @@ -264,7 +267,7 @@ public void shouldHandleMissingS3Assets() throws Exception { new ReadOnlyChunkImpl<>( curatorFramework, meterRegistry, - s3CrtBlobFs, + chunkStore, SearchContext.fromConfig(AstraConfig.getCacheConfig().getServerConfig()), AstraConfig.getS3Config().getS3Bucket(), AstraConfig.getCacheConfig().getDataDirectory(), @@ -330,7 +333,7 @@ public void shouldHandleMissingZkData() throws Exception { new ReadOnlyChunkImpl<>( curatorFramework, meterRegistry, - s3CrtBlobFs, + chunkStore, SearchContext.fromConfig(AstraConfig.getCacheConfig().getServerConfig()), AstraConfig.getS3Config().getS3Bucket(), AstraConfig.getCacheConfig().getDataDirectory(), @@ -397,7 +400,7 @@ public void closeShouldCleanupLiveChunkCorrectly() throws Exception { new ReadOnlyChunkImpl<>( curatorFramework, meterRegistry, - s3CrtBlobFs, + chunkStore, SearchContext.fromConfig(AstraConfig.getCacheConfig().getServerConfig()), AstraConfig.getS3Config().getS3Bucket(), AstraConfig.getCacheConfig().getDataDirectory(), @@ -528,7 +531,7 @@ public void shouldHandleDynamicChunkSizeLifecycle() throws Exception { new ReadOnlyChunkImpl<>( curatorFramework, meterRegistry, - s3CrtBlobFs, + chunkStore, searchContext, AstraConfig.getS3Config().getS3Bucket(), AstraConfig.getCacheConfig().getDataDirectory(), diff --git a/astra/src/test/java/com/slack/astra/chunkManager/CachingChunkManagerTest.java b/astra/src/test/java/com/slack/astra/chunkManager/CachingChunkManagerTest.java index 84333b2510..d7c52e8d56 100644 --- a/astra/src/test/java/com/slack/astra/chunkManager/CachingChunkManagerTest.java +++ b/astra/src/test/java/com/slack/astra/chunkManager/CachingChunkManagerTest.java @@ -16,6 +16,7 @@ import static org.awaitility.Awaitility.await; import com.adobe.testing.s3mock.junit5.S3MockExtension; +import com.slack.astra.blobfs.ChunkStore; import com.slack.astra.blobfs.LocalBlobFs; import com.slack.astra.blobfs.s3.S3CrtBlobFs; import com.slack.astra.blobfs.s3.S3TestUtils; @@ -64,6 +65,7 @@ public class CachingChunkManagerTest { private TestingServer testingServer; private MeterRegistry meterRegistry; private S3CrtBlobFs s3CrtBlobFs; + private ChunkStore chunkStore; @RegisterExtension public static final S3MockExtension S3_MOCK_EXTENSION = @@ -86,6 +88,7 @@ public void startup() throws Exception { S3AsyncClient s3AsyncClient = S3TestUtils.createS3CrtClient(S3_MOCK_EXTENSION.getServiceEndpoint()); s3CrtBlobFs = new S3CrtBlobFs(s3AsyncClient); + chunkStore = new ChunkStore(s3AsyncClient, TEST_S3_BUCKET); } @AfterEach @@ -146,7 +149,7 @@ private CachingChunkManager initChunkManager() throws TimeoutExcepti new CachingChunkManager<>( meterRegistry, curatorFramework, - s3CrtBlobFs, + chunkStore, SearchContext.fromConfig(AstraConfig.getCacheConfig().getServerConfig()), AstraConfig.getS3Config().getS3Bucket(), AstraConfig.getCacheConfig().getDataDirectory(), From d1efb1851352769e8e08af7d18453b387e9d23fc Mon Sep 17 00:00:00 2001 From: Bryan Burkholder Date: Tue, 13 Aug 2024 09:07:19 -0700 Subject: [PATCH 2/7] Remove unnecessary object storage code, part 1 --- .../java/com/slack/astra/blobfs/BlobFs.java | 139 +---- .../com/slack/astra/blobfs/BlobFsConfig.java | 5 - .../{logstore => blobfs}/BlobFsUtils.java | 18 +- .../com/slack/astra/blobfs/LocalBlobFs.java | 157 ----- .../astra/blobfs/{s3 => }/S3CrtBlobFs.java | 222 +------- .../com/slack/astra/blobfs/s3/S3BlobFs.java | 539 ------------------ .../slack/astra/chunk/ChunkDownloader.java | 6 - .../com/slack/astra/chunk/ReadWriteChunk.java | 4 +- .../chunk/SerialS3ChunkDownloaderImpl.java | 31 - .../java/com/slack/astra/server/Astra.java | 2 +- .../slack/astra/blobfs/LocalBlobFsTest.java | 245 -------- .../slack/astra/blobfs/s3/S3BlobFsTest.java | 353 ------------ .../astra/blobfs/s3/S3CrtBlobFsTest.java | 91 +-- .../slack/astra/blobfs/s3/S3TestUtils.java | 24 - .../astra/chunk/IndexingChunkImplTest.java | 5 +- .../astra/chunk/ReadOnlyChunkImplTest.java | 11 +- .../astra/chunk/RecoveryChunkImplTest.java | 2 +- .../chunkManager/CachingChunkManagerTest.java | 12 +- .../IndexingChunkManagerTest.java | 2 +- .../RecoveryChunkManagerTest.java | 2 +- ...MessageCountBasedRolloverStrategyTest.java | 2 +- .../SnapshotDeletionServiceTest.java | 4 +- .../logstore/LuceneIndexStoreImplTest.java | 67 +-- .../astra/recovery/RecoveryServiceTest.java | 4 +- .../slack/astra/testlib/ChunkManagerUtil.java | 2 +- 25 files changed, 59 insertions(+), 1890 deletions(-) delete mode 100644 astra/src/main/java/com/slack/astra/blobfs/BlobFsConfig.java rename astra/src/main/java/com/slack/astra/{logstore => blobfs}/BlobFsUtils.java (81%) delete mode 100644 astra/src/main/java/com/slack/astra/blobfs/LocalBlobFs.java rename astra/src/main/java/com/slack/astra/blobfs/{s3 => }/S3CrtBlobFs.java (70%) delete mode 100644 astra/src/main/java/com/slack/astra/blobfs/s3/S3BlobFs.java delete mode 100644 astra/src/main/java/com/slack/astra/chunk/ChunkDownloader.java delete mode 100644 astra/src/main/java/com/slack/astra/chunk/SerialS3ChunkDownloaderImpl.java delete mode 100644 astra/src/test/java/com/slack/astra/blobfs/LocalBlobFsTest.java delete mode 100644 astra/src/test/java/com/slack/astra/blobfs/s3/S3BlobFsTest.java diff --git a/astra/src/main/java/com/slack/astra/blobfs/BlobFs.java b/astra/src/main/java/com/slack/astra/blobfs/BlobFs.java index 218a4ad314..d5e711d7ba 100644 --- a/astra/src/main/java/com/slack/astra/blobfs/BlobFs.java +++ b/astra/src/main/java/com/slack/astra/blobfs/BlobFs.java @@ -3,14 +3,8 @@ import java.io.Closeable; import java.io.File; import java.io.IOException; -import java.io.InputStream; import java.io.Serializable; import java.net.URI; -import java.net.URISyntaxException; -import java.nio.file.Path; -import java.nio.file.Paths; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * BlobFs is a restricted FS API that exposes functionality that is required for a store to use @@ -25,23 +19,8 @@ *

NOTE: This code is a fork of PinotFS from Apache Pinot. In future, we will import this code as * an external lib. */ +@Deprecated public abstract class BlobFs implements Closeable, Serializable { - private static final Logger LOGGER = LoggerFactory.getLogger(BlobFs.class); - - /** - * Initializes the configurations specific to that filesystem. For instance, any security related - * parameters can be initialized here and will not be logged. - */ - public abstract void init(BlobFsConfig config); - - /** - * Creates a new directory. If parent directories are not created, it will create them. If the - * directory exists, it will return true without doing anything. - * - * @return true if mkdir is successful - * @throws IOException on IO failure - */ - public abstract boolean mkdir(URI uri) throws IOException; /** * Deletes the file at the location provided. If the segmentUri is a directory, it will delete the @@ -53,76 +32,9 @@ public abstract class BlobFs implements Closeable, Serializable { * @return true if delete is successful else false * @throws IOException on IO failure, e.g Uri is not present or not valid */ + @Deprecated public abstract boolean delete(URI segmentUri, boolean forceDelete) throws IOException; - /** - * Moves the file or directory from the src to dst. Does not keep the original file. If the dst - * has parent directories that haven't been created, this method will create all the necessary - * parent directories. Note: In blobfs we recommend the full paths of both src and dst be - * specified. For example, if a file /a/b/c is moved to a file /x/y/z, in the case of overwrite, - * the directory /a/b still exists, but will not contain the file 'c'. Instead, /x/y/z will - * contain the contents of 'c'. If src is a directory /a/b which contains two files /a/b/c and - * /a/b/d, and the dst is /x/y, the result would be that the directory /a/b under /a gets removed - * and dst directory contains two files which is /x/y/c and /x/y/d. If src is a directory /a/b - * needs to be moved under another directory /x/y, please specify the dst to /x/y/b. - * - * @param srcUri URI of the original file - * @param dstUri URI of the final file location - * @param overwrite true if we want to overwrite the dstURI, false otherwise - * @return true if move is successful - * @throws IOException on IO failure - */ - public boolean move(URI srcUri, URI dstUri, boolean overwrite) throws IOException { - if (!exists(srcUri)) { - LOGGER.warn("Source {} does not exist", srcUri); - return false; - } - if (exists(dstUri)) { - if (overwrite) { - delete(dstUri, true); - } else { - // dst file exists, returning - LOGGER.warn( - "Cannot move {} to {}. Destination exists and overwrite flag set to false.", - srcUri, - dstUri); - return false; - } - } else { - // ensures the parent path of dst exists. - try { - Path parentPath = Paths.get(dstUri.getPath()).getParent(); - URI parentUri = new URI(dstUri.getScheme(), dstUri.getHost(), parentPath.toString(), null); - mkdir(parentUri); - } catch (URISyntaxException e) { - throw new IOException(e); - } - } - return doMove(srcUri, dstUri); - } - - /** Does the actual behavior of move in each FS. */ - public abstract boolean doMove(URI srcUri, URI dstUri) throws IOException; - - /** - * Copies the file or directory from the src to dst. The original file is retained. If the dst has - * parent directories that haven't been created, this method will create all the necessary parent - * directories. If dst already exists, this will overwrite the existing file/directory in the - * path. - * - *

Note: In Pinot we recommend the full paths of both src and dst be specified. For example, if - * a file /a/b/c is copied to a file /x/y/z, the directory /a/b still exists containing the file - * 'c'. The dst file /x/y/z will contain the contents of 'c'. If a directory /a/b is copied to - * another directory /x/y, the directory /x/y will contain the content of /a/b. If a directory - * /a/b is copied under the directory /x/y, the dst needs to be specify as /x/y/b. - * - * @param srcUri URI of the original file - * @param dstUri URI of the final file location - * @return true if copy is successful - * @throws IOException on IO failure - */ - public abstract boolean copy(URI srcUri, URI dstUri) throws IOException; - /** * Checks whether the file or directory at the provided location exists. * @@ -130,17 +42,9 @@ public boolean move(URI srcUri, URI dstUri, boolean overwrite) throws IOExceptio * @return true if path exists * @throws IOException on IO failure */ + @Deprecated public abstract boolean exists(URI fileUri) throws IOException; - /** - * Returns the length of the file at the provided location. - * - * @param fileUri location of file - * @return the number of bytes - * @throws IOException on IO failure, e.g if it's a directory. - */ - public abstract long length(URI fileUri) throws IOException; - /** * Lists all the files and directories at the location provided. Lists recursively if {@code * recursive} is set to true. Throws IOException if this abstract pathname is not valid, or if an @@ -151,6 +55,7 @@ public boolean move(URI srcUri, URI dstUri, boolean overwrite) throws IOExceptio * @return an array of strings that contains file paths * @throws IOException on IO failure. See specific implementation */ + @Deprecated public abstract String[] listFiles(URI fileUri, boolean recursive) throws IOException; /** @@ -161,6 +66,7 @@ public boolean move(URI srcUri, URI dstUri, boolean overwrite) throws IOExceptio * @throws Exception if srcUri is not valid or not present, or timeout when downloading file to * local */ + @Deprecated public abstract void copyToLocalFile(URI srcUri, File dstFile) throws Exception; /** @@ -172,6 +78,7 @@ public boolean move(URI srcUri, URI dstUri, boolean overwrite) throws IOExceptio * @throws Exception if fileUri is not valid or not present, or timeout when uploading file from * local */ + @Deprecated public abstract void copyFromLocalFile(File srcFile, URI dstUri) throws Exception; /** @@ -181,40 +88,9 @@ public boolean move(URI srcUri, URI dstUri, boolean overwrite) throws IOExceptio * @return true if uri is a directory, false otherwise. * @throws IOException on IO failure, e.g uri is not valid or not present */ + @Deprecated public abstract boolean isDirectory(URI uri) throws IOException; - /** - * Returns the age of the file - * - * @param uri location of file or directory - * @return A long value representing the time the file was last modified, measured in milliseconds - * since epoch (00:00:00 GMT, January 1, 1970) or 0L if the file does not exist or if an I/O - * error occurs - * @throws IOException if uri is not valid or not present - */ - public abstract long lastModified(URI uri) throws IOException; - - /** - * Updates the last modified time of an existing file or directory to be current time. If the file - * system object does not exist, creates an empty file. - * - * @param uri location of file or directory - * @throws IOException if the parent directory doesn't exist - */ - public abstract boolean touch(URI uri) throws IOException; - - /** - * Opens a file in the underlying filesystem and returns an InputStream to read it. Note that the - * caller can invoke close on this inputstream. Some implementations can choose to copy the - * original file to local temp file and return the inputstream. In this case, the implementation - * it should delete the temp file when close is invoked. - * - * @param uri location of the file to open - * @return a new InputStream - * @throws IOException on any IO error - missing file, not a file etc - */ - public abstract InputStream open(URI uri) throws IOException; - /** * For certain filesystems, we may need to close the filesystem and do relevant operations to * prevent leaks. By default, this method does nothing. @@ -222,5 +98,6 @@ public boolean move(URI srcUri, URI dstUri, boolean overwrite) throws IOExceptio * @throws IOException on IO failure */ @Override + @Deprecated public void close() throws IOException {} } diff --git a/astra/src/main/java/com/slack/astra/blobfs/BlobFsConfig.java b/astra/src/main/java/com/slack/astra/blobfs/BlobFsConfig.java deleted file mode 100644 index 886b807a1b..0000000000 --- a/astra/src/main/java/com/slack/astra/blobfs/BlobFsConfig.java +++ /dev/null @@ -1,5 +0,0 @@ -package com.slack.astra.blobfs; - -public interface BlobFsConfig { - String getProperty(String propertyName); -} diff --git a/astra/src/main/java/com/slack/astra/logstore/BlobFsUtils.java b/astra/src/main/java/com/slack/astra/blobfs/BlobFsUtils.java similarity index 81% rename from astra/src/main/java/com/slack/astra/logstore/BlobFsUtils.java rename to astra/src/main/java/com/slack/astra/blobfs/BlobFsUtils.java index f99ccc9e46..c46c858f33 100644 --- a/astra/src/main/java/com/slack/astra/logstore/BlobFsUtils.java +++ b/astra/src/main/java/com/slack/astra/blobfs/BlobFsUtils.java @@ -1,17 +1,16 @@ -package com.slack.astra.logstore; +package com.slack.astra.blobfs; -import com.slack.astra.blobfs.BlobFs; import java.io.File; import java.io.IOException; import java.net.URI; import java.nio.file.Path; -import java.nio.file.Paths; import java.util.Arrays; import java.util.Collection; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** This class contains static methods that help with blobfs operations. */ +@Deprecated public class BlobFsUtils { private static final Logger LOG = LoggerFactory.getLogger(BlobFsUtils.class); @@ -20,6 +19,7 @@ public class BlobFsUtils { public static final String DELIMITER = "/"; public static final String FILE_FORMAT = "%s://%s/%s"; + @Deprecated public static int copyToS3( Path sourceDirPath, Collection files, String bucket, String prefix, BlobFs blobFs) throws Exception { @@ -35,6 +35,7 @@ public static int copyToS3( return success; } + @Deprecated public static URI createURI(String bucket, String prefix, String fileName) { return (prefix != null && !prefix.isEmpty()) ? URI.create(String.format(FILE_FORMAT, SCHEME, bucket + DELIMITER + prefix, fileName)) @@ -43,6 +44,7 @@ public static URI createURI(String bucket, String prefix, String fileName) { // TODO: Can we copy files without list files and a prefix only? // TODO: Take a complete URI as this is the format stored in snapshot data + @Deprecated public static String[] copyFromS3( String bucket, String prefix, BlobFs s3BlobFs, Path localDirPath) throws Exception { LOG.debug("Copying files from bucket={} prefix={} using directory", bucket, prefix); @@ -54,14 +56,4 @@ public static String[] copyFromS3( .distinct() .toArray(String[]::new); } - - public static void copyToLocalPath( - Path sourceDirPath, Collection files, Path destDirPath, BlobFs blobFs) - throws IOException { - for (String file : files) { - blobFs.copy( - Paths.get(sourceDirPath.toAbsolutePath().toString(), file).toUri(), - Paths.get(destDirPath.toAbsolutePath().toString(), file).toUri()); - } - } } diff --git a/astra/src/main/java/com/slack/astra/blobfs/LocalBlobFs.java b/astra/src/main/java/com/slack/astra/blobfs/LocalBlobFs.java deleted file mode 100644 index c5ef2e750f..0000000000 --- a/astra/src/main/java/com/slack/astra/blobfs/LocalBlobFs.java +++ /dev/null @@ -1,157 +0,0 @@ -package com.slack.astra.blobfs; - -import java.io.BufferedInputStream; -import java.io.File; -import java.io.FileInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.UnsupportedEncodingException; -import java.net.URI; -import java.net.URLDecoder; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.Arrays; -import java.util.stream.Stream; -import org.apache.commons.io.FileUtils; - -/** - * Implementation of BlobFs for a local filesystem. Methods in this class may throw a - * SecurityException at runtime if access to the file is denied. - */ -public class LocalBlobFs extends BlobFs { - - @Override - public void init(BlobFsConfig configuration) {} - - @Override - public boolean mkdir(URI uri) throws IOException { - FileUtils.forceMkdir(toFile(uri)); - return true; - } - - @Override - public boolean delete(URI segmentUri, boolean forceDelete) throws IOException { - File file = toFile(segmentUri); - if (file.isDirectory()) { - // Returns false if directory isn't empty - if (listFiles(segmentUri, false).length > 0 && !forceDelete) { - return false; - } - // Throws an IOException if it is unable to delete - FileUtils.deleteDirectory(file); - } else { - // Returns false if delete fails - return FileUtils.deleteQuietly(file); - } - return true; - } - - @Override - public boolean doMove(URI srcUri, URI dstUri) throws IOException { - File srcFile = toFile(srcUri); - File dstFile = toFile(dstUri); - if (srcFile.isDirectory()) { - FileUtils.moveDirectory(srcFile, dstFile); - } else { - FileUtils.moveFile(srcFile, dstFile); - } - return true; - } - - @Override - public boolean copy(URI srcUri, URI dstUri) throws IOException { - copy(toFile(srcUri), toFile(dstUri)); - return true; - } - - @Override - public boolean exists(URI fileUri) { - return toFile(fileUri).exists(); - } - - @Override - public long length(URI fileUri) { - File file = toFile(fileUri); - if (file.isDirectory()) { - throw new IllegalArgumentException("File is directory"); - } - return FileUtils.sizeOf(file); - } - - @Override - public String[] listFiles(URI fileUri, boolean recursive) throws IOException { - File file = toFile(fileUri); - if (!recursive) { - return Arrays.stream(file.list()) - .map(s -> new File(file, s)) - .map(File::getAbsolutePath) - .toArray(String[]::new); - } else { - try (Stream files = Files.walk(Paths.get(fileUri))) { - return files - .filter(s -> !s.equals(file.toPath())) - .map(Path::toString) - .toArray(String[]::new); - } - } - } - - @Override - public void copyToLocalFile(URI srcUri, File dstFile) throws Exception { - copy(toFile(srcUri), dstFile); - } - - @Override - public void copyFromLocalFile(File srcFile, URI dstUri) throws Exception { - copy(srcFile, toFile(dstUri)); - } - - @Override - public boolean isDirectory(URI uri) { - return toFile(uri).isDirectory(); - } - - @Override - public long lastModified(URI uri) { - return toFile(uri).lastModified(); - } - - @Override - public boolean touch(URI uri) throws IOException { - File file = toFile(uri); - if (!file.exists()) { - return file.createNewFile(); - } - return file.setLastModified(System.currentTimeMillis()); - } - - @Override - public InputStream open(URI uri) throws IOException { - return new BufferedInputStream(new FileInputStream(toFile(uri))); - } - - private static File toFile(URI uri) { - // NOTE: Do not use new File(uri) because scheme might not exist and it does not decode '+' to ' - // ' - // Do not use uri.getPath() because it does not decode '+' to ' ' - try { - return new File(URLDecoder.decode(uri.getRawPath(), "UTF-8")); - } catch (UnsupportedEncodingException e) { - throw new RuntimeException(e); - } - } - - private static void copy(File srcFile, File dstFile) throws IOException { - if (dstFile.exists()) { - FileUtils.deleteQuietly(dstFile); - } - if (srcFile.isDirectory()) { - // Throws Exception on failure - FileUtils.copyDirectory(srcFile, dstFile); - } else { - // Will create parent directories, throws Exception on failure - FileUtils.copyFile(srcFile, dstFile); - } - } -} diff --git a/astra/src/main/java/com/slack/astra/blobfs/s3/S3CrtBlobFs.java b/astra/src/main/java/com/slack/astra/blobfs/S3CrtBlobFs.java similarity index 70% rename from astra/src/main/java/com/slack/astra/blobfs/s3/S3CrtBlobFs.java rename to astra/src/main/java/com/slack/astra/blobfs/S3CrtBlobFs.java index 33a94679ff..5922758329 100644 --- a/astra/src/main/java/com/slack/astra/blobfs/s3/S3CrtBlobFs.java +++ b/astra/src/main/java/com/slack/astra/blobfs/S3CrtBlobFs.java @@ -1,24 +1,14 @@ -package com.slack.astra.blobfs.s3; +package com.slack.astra.blobfs; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; -import com.slack.astra.blobfs.BlobFs; -import com.slack.astra.blobfs.BlobFsConfig; import com.slack.astra.proto.config.AstraConfigs; import java.io.File; import java.io.IOException; -import java.io.InputStream; -import java.io.UnsupportedEncodingException; import java.net.URI; import java.net.URISyntaxException; -import java.net.URLEncoder; -import java.nio.charset.StandardCharsets; -import java.nio.file.Path; -import java.nio.file.Paths; import java.time.Duration; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.concurrent.ExecutionException; import org.apache.commons.io.FileUtils; import org.slf4j.Logger; @@ -27,8 +17,6 @@ import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider; -import software.amazon.awssdk.core.async.AsyncRequestBody; -import software.amazon.awssdk.core.async.AsyncResponseTransformer; import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.services.s3.S3AsyncClient; import software.amazon.awssdk.services.s3.S3CrtAsyncClientBuilder; @@ -36,19 +24,14 @@ import software.amazon.awssdk.services.s3.crt.S3CrtHttpConfiguration; import software.amazon.awssdk.services.s3.crt.S3CrtProxyConfiguration; import software.amazon.awssdk.services.s3.crt.S3CrtRetryConfiguration; -import software.amazon.awssdk.services.s3.model.CopyObjectRequest; -import software.amazon.awssdk.services.s3.model.CopyObjectResponse; import software.amazon.awssdk.services.s3.model.DeleteObjectRequest; import software.amazon.awssdk.services.s3.model.DeleteObjectResponse; import software.amazon.awssdk.services.s3.model.GetObjectRequest; import software.amazon.awssdk.services.s3.model.HeadObjectRequest; -import software.amazon.awssdk.services.s3.model.HeadObjectResponse; import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; import software.amazon.awssdk.services.s3.model.ListObjectsV2Response; -import software.amazon.awssdk.services.s3.model.MetadataDirective; import software.amazon.awssdk.services.s3.model.NoSuchKeyException; import software.amazon.awssdk.services.s3.model.PutObjectRequest; -import software.amazon.awssdk.services.s3.model.PutObjectResponse; import software.amazon.awssdk.services.s3.model.S3Exception; import software.amazon.awssdk.services.s3.model.S3Object; import software.amazon.awssdk.transfer.s3.S3TransferManager; @@ -68,6 +51,7 @@ * assumptions this was based on no longer apply. Additionally, several retrofits have been made to * support new API approaches which has left this overly complex. */ +@Deprecated public class S3CrtBlobFs extends BlobFs { public static final String S3_SCHEME = "s3://"; private static final Logger LOG = LoggerFactory.getLogger(S3CrtBlobFs.class); @@ -149,30 +133,6 @@ public static S3AsyncClient initS3Client(AstraConfigs.S3Config config) { } } - @Override - public void init(BlobFsConfig config) { - // Not sure if this interface works for a library. So on ice for now. - throw new UnsupportedOperationException( - "This class doesn't support initialization via blobfsconfig."); - } - - private HeadObjectResponse getS3ObjectMetadata(URI uri) throws IOException { - URI base = getBase(uri); - String path = sanitizePath(base.relativize(uri).getPath()); - HeadObjectRequest headObjectRequest = - HeadObjectRequest.builder().bucket(uri.getHost()).key(path).build(); - - try { - return s3AsyncClient.headObject(headObjectRequest).get(); - } catch (InterruptedException | ExecutionException e) { - if (e instanceof ExecutionException && e.getCause() instanceof NoSuchKeyException) { - throw NoSuchKeyException.builder().cause(e.getCause()).build(); - } else { - throw new IOException(e); - } - } - } - private boolean isPathTerminatedByDelimiter(URI uri) { return uri.getPath().endsWith(DELIMITER); } @@ -186,17 +146,6 @@ private String normalizeToDirectoryPrefix(URI uri) throws IOException { return sanitizePath(strippedUri.getPath() + DELIMITER); } - private URI normalizeToDirectoryUri(URI uri) throws IOException { - if (isPathTerminatedByDelimiter(uri)) { - return uri; - } - try { - return new URI(uri.getScheme(), uri.getHost(), sanitizePath(uri.getPath() + DELIMITER), null); - } catch (URISyntaxException e) { - throw new IOException(e); - } - } - private String sanitizePath(String path) { path = path.replaceAll(DELIMITER + "+", DELIMITER); if (path.startsWith(DELIMITER) && !path.equals(DELIMITER)) { @@ -263,55 +212,6 @@ private boolean isEmptyDirectory(URI uri) throws IOException { return isEmpty; } - private boolean copyFile(URI srcUri, URI dstUri) throws IOException { - try { - String encodedUrl = null; - try { - encodedUrl = - URLEncoder.encode( - srcUri.getHost() + srcUri.getPath(), StandardCharsets.UTF_8.toString()); - } catch (UnsupportedEncodingException e) { - throw new RuntimeException(e); - } - - String dstPath = sanitizePath(dstUri.getPath()); - CopyObjectRequest copyReq = - CopyObjectRequest.builder() - .copySource(encodedUrl) - .destinationBucket(dstUri.getHost()) - .destinationKey(dstPath) - .build(); - - CopyObjectResponse copyObjectResponse = s3AsyncClient.copyObject(copyReq).get(); - return copyObjectResponse.sdkHttpResponse().isSuccessful(); - } catch (S3Exception | ExecutionException | InterruptedException e) { - throw new IOException(e); - } - } - - @Override - public boolean mkdir(URI uri) throws IOException { - LOG.debug("mkdir {}", uri); - try { - Preconditions.checkNotNull(uri, "uri is null"); - String path = normalizeToDirectoryPrefix(uri); - // Bucket root directory already exists and cannot be created - if (path.equals(DELIMITER)) { - return true; - } - - PutObjectRequest putObjectRequest = - PutObjectRequest.builder().bucket(uri.getHost()).key(path).build(); - - PutObjectResponse putObjectResponse = - s3AsyncClient.putObject(putObjectRequest, AsyncRequestBody.fromBytes(new byte[0])).get(); - - return putObjectResponse.sdkHttpResponse().isSuccessful(); - } catch (Throwable t) { - throw new IOException(t); - } - } - @Override public boolean delete(URI segmentUri, boolean forceDelete) throws IOException { LOG.debug("Deleting uri {} force {}", segmentUri, forceDelete); @@ -369,44 +269,6 @@ public boolean delete(URI segmentUri, boolean forceDelete) throws IOException { } } - @Override - public boolean doMove(URI srcUri, URI dstUri) throws IOException { - if (copy(srcUri, dstUri)) { - return delete(srcUri, true); - } - return false; - } - - @Override - public boolean copy(URI srcUri, URI dstUri) throws IOException { - LOG.debug("Copying uri {} to uri {}", srcUri, dstUri); - Preconditions.checkState(exists(srcUri), "Source URI '%s' does not exist", srcUri); - if (srcUri.equals(dstUri)) { - return true; - } - if (!isDirectory(srcUri)) { - delete(dstUri, true); - return copyFile(srcUri, dstUri); - } - dstUri = normalizeToDirectoryUri(dstUri); - Path srcPath = Paths.get(srcUri.getPath()); - try { - boolean copySucceeded = true; - for (String filePath : listFiles(srcUri, true)) { - URI srcFileURI = URI.create(filePath); - String directoryEntryPrefix = srcFileURI.getPath(); - URI src = new URI(srcUri.getScheme(), srcUri.getHost(), directoryEntryPrefix, null); - String relativeSrcPath = srcPath.relativize(Paths.get(directoryEntryPrefix)).toString(); - String dstPath = dstUri.resolve(relativeSrcPath).getPath(); - URI dst = new URI(dstUri.getScheme(), dstUri.getHost(), dstPath, null); - copySucceeded &= copyFile(src, dst); - } - return copySucceeded; - } catch (URISyntaxException e) { - throw new IOException(e); - } - } - @Override public boolean exists(URI fileUri) throws IOException { try { @@ -422,21 +284,6 @@ public boolean exists(URI fileUri) throws IOException { } } - @Override - public long length(URI fileUri) throws IOException { - try { - Preconditions.checkState(!isPathTerminatedByDelimiter(fileUri), "URI is a directory"); - HeadObjectResponse s3ObjectMetadata = getS3ObjectMetadata(fileUri); - Preconditions.checkState((s3ObjectMetadata != null), "File '%s' does not exist", fileUri); - if (s3ObjectMetadata.contentLength() == null) { - return 0; - } - return s3ObjectMetadata.contentLength(); - } catch (Throwable t) { - throw new IOException(t); - } - } - @Override public String[] listFiles(URI fileUri, boolean recursive) throws IOException { try { @@ -605,71 +452,6 @@ public boolean isDirectory(URI uri) throws IOException { } } - @Override - public long lastModified(URI uri) throws IOException { - return getS3ObjectMetadata(uri).lastModified().toEpochMilli(); - } - - @Override - public boolean touch(URI uri) throws IOException { - try { - HeadObjectResponse s3ObjectMetadata = getS3ObjectMetadata(uri); - String encodedUrl = null; - try { - encodedUrl = - URLEncoder.encode(uri.getHost() + uri.getPath(), StandardCharsets.UTF_8.toString()); - } catch (UnsupportedEncodingException e) { - throw new RuntimeException(e); - } - - String path = sanitizePath(uri.getPath()); - Map mp = new HashMap<>(); - mp.put("lastModified", String.valueOf(System.currentTimeMillis())); - CopyObjectRequest request = - CopyObjectRequest.builder() - .copySource(encodedUrl) - .destinationBucket(uri.getHost()) - .destinationKey(path) - .metadata(mp) - .metadataDirective(MetadataDirective.REPLACE) - .build(); - - s3AsyncClient.copyObject(request).get(); - long newUpdateTime = getS3ObjectMetadata(uri).lastModified().toEpochMilli(); - return newUpdateTime > s3ObjectMetadata.lastModified().toEpochMilli(); - } catch (NoSuchKeyException e) { - String path = sanitizePath(uri.getPath()); - try { - s3AsyncClient - .putObject( - PutObjectRequest.builder().bucket(uri.getHost()).key(path).build(), - AsyncRequestBody.fromBytes(new byte[0])) - .get(); - } catch (InterruptedException | ExecutionException ex) { - throw new IOException(ex); - } - return true; - } catch (S3Exception | ExecutionException | InterruptedException e) { - throw new IOException(e); - } - } - - @Override - public InputStream open(URI uri) throws IOException { - try { - String path = sanitizePath(uri.getPath()); - GetObjectRequest getObjectRequest = - GetObjectRequest.builder().bucket(uri.getHost()).key(path).build(); - return s3AsyncClient - .getObject(getObjectRequest, AsyncResponseTransformer.toBlockingInputStream()) - .get(); - } catch (S3Exception e) { - throw e; - } catch (ExecutionException | InterruptedException e) { - throw new IOException(e); - } - } - @Override public void close() throws IOException { super.close(); diff --git a/astra/src/main/java/com/slack/astra/blobfs/s3/S3BlobFs.java b/astra/src/main/java/com/slack/astra/blobfs/s3/S3BlobFs.java deleted file mode 100644 index 28a6bfc95d..0000000000 --- a/astra/src/main/java/com/slack/astra/blobfs/s3/S3BlobFs.java +++ /dev/null @@ -1,539 +0,0 @@ -package com.slack.astra.blobfs.s3; - -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; -import com.slack.astra.blobfs.BlobFs; -import com.slack.astra.blobfs.BlobFsConfig; -import com.slack.astra.proto.config.AstraConfigs; -import java.io.File; -import java.io.IOException; -import java.io.InputStream; -import java.io.UnsupportedEncodingException; -import java.net.URI; -import java.net.URISyntaxException; -import java.net.URLEncoder; -import java.nio.charset.StandardCharsets; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import org.apache.commons.io.FileUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; -import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; -import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; -import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider; -import software.amazon.awssdk.core.SdkSystemSetting; -import software.amazon.awssdk.core.sync.RequestBody; -import software.amazon.awssdk.core.sync.ResponseTransformer; -import software.amazon.awssdk.regions.Region; -import software.amazon.awssdk.services.s3.S3Client; -import software.amazon.awssdk.services.s3.S3ClientBuilder; -import software.amazon.awssdk.services.s3.model.CopyObjectRequest; -import software.amazon.awssdk.services.s3.model.CopyObjectResponse; -import software.amazon.awssdk.services.s3.model.DeleteObjectRequest; -import software.amazon.awssdk.services.s3.model.DeleteObjectResponse; -import software.amazon.awssdk.services.s3.model.GetObjectRequest; -import software.amazon.awssdk.services.s3.model.HeadObjectRequest; -import software.amazon.awssdk.services.s3.model.HeadObjectResponse; -import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; -import software.amazon.awssdk.services.s3.model.ListObjectsV2Response; -import software.amazon.awssdk.services.s3.model.MetadataDirective; -import software.amazon.awssdk.services.s3.model.NoSuchKeyException; -import software.amazon.awssdk.services.s3.model.PutObjectRequest; -import software.amazon.awssdk.services.s3.model.PutObjectResponse; -import software.amazon.awssdk.services.s3.model.S3Exception; -import software.amazon.awssdk.services.s3.model.S3Object; - -/** - * @see S3CrtBlobFs - */ -@Deprecated -public class S3BlobFs extends BlobFs { - public static final String S3_SCHEME = "s3://"; - private static final Logger LOG = LoggerFactory.getLogger(S3BlobFs.class); - private static final String DELIMITER = "/"; - private static final int LIST_MAX_KEYS = 2500; - private S3Client s3Client; - - public S3BlobFs(S3Client s3Client) { - this.s3Client = s3Client; - } - - static boolean isNullOrEmpty(String target) { - return target == null || "".equals(target); - } - - public static S3Client initS3Client(AstraConfigs.S3Config config) { - Preconditions.checkArgument(!isNullOrEmpty(config.getS3Region())); - String region = config.getS3Region(); - - AwsCredentialsProvider awsCredentialsProvider; - try { - - if (!isNullOrEmpty(config.getS3AccessKey()) && !isNullOrEmpty(config.getS3SecretKey())) { - String accessKey = config.getS3AccessKey(); - String secretKey = config.getS3SecretKey(); - AwsBasicCredentials awsBasicCredentials = AwsBasicCredentials.create(accessKey, secretKey); - awsCredentialsProvider = StaticCredentialsProvider.create(awsBasicCredentials); - } else { - awsCredentialsProvider = DefaultCredentialsProvider.create(); - } - - // TODO: Remove hard coded HTTP IMPL property setting by only having 1 http client on the - // classpath. - System.setProperty( - SdkSystemSetting.SYNC_HTTP_SERVICE_IMPL.property(), - "software.amazon.awssdk.http.apache.ApacheSdkHttpService"); - S3ClientBuilder s3ClientBuilder = - S3Client.builder().region(Region.of(region)).credentialsProvider(awsCredentialsProvider); - if (!isNullOrEmpty(config.getS3EndPoint())) { - String endpoint = config.getS3EndPoint(); - try { - s3ClientBuilder.endpointOverride(new URI(endpoint)); - } catch (URISyntaxException e) { - throw new RuntimeException(e); - } - } - return s3ClientBuilder.build(); - } catch (S3Exception e) { - throw new RuntimeException("Could not initialize S3blobFs", e); - } - } - - @Override - public void init(BlobFsConfig config) { - // Not sure if this interface works for a library. So on ice for now. - throw new UnsupportedOperationException( - "This class doesn't support initialization via blobfsconfig."); - } - - private HeadObjectResponse getS3ObjectMetadata(URI uri) throws IOException { - URI base = getBase(uri); - String path = sanitizePath(base.relativize(uri).getPath()); - HeadObjectRequest headObjectRequest = - HeadObjectRequest.builder().bucket(uri.getHost()).key(path).build(); - - return s3Client.headObject(headObjectRequest); - } - - private boolean isPathTerminatedByDelimiter(URI uri) { - return uri.getPath().endsWith(DELIMITER); - } - - private String normalizeToDirectoryPrefix(URI uri) throws IOException { - Preconditions.checkNotNull(uri, "uri is null"); - URI strippedUri = getBase(uri).relativize(uri); - if (isPathTerminatedByDelimiter(strippedUri)) { - return sanitizePath(strippedUri.getPath()); - } - return sanitizePath(strippedUri.getPath() + DELIMITER); - } - - private URI normalizeToDirectoryUri(URI uri) throws IOException { - if (isPathTerminatedByDelimiter(uri)) { - return uri; - } - try { - return new URI(uri.getScheme(), uri.getHost(), sanitizePath(uri.getPath() + DELIMITER), null); - } catch (URISyntaxException e) { - throw new IOException(e); - } - } - - private String sanitizePath(String path) { - path = path.replaceAll(DELIMITER + "+", DELIMITER); - if (path.startsWith(DELIMITER) && !path.equals(DELIMITER)) { - path = path.substring(1); - } - return path; - } - - private URI getBase(URI uri) throws IOException { - try { - return new URI(uri.getScheme(), uri.getHost(), null, null); - } catch (URISyntaxException e) { - throw new IOException(e); - } - } - - private boolean existsFile(URI uri) throws IOException { - try { - URI base = getBase(uri); - String path = sanitizePath(base.relativize(uri).getPath()); - HeadObjectRequest headObjectRequest = - HeadObjectRequest.builder().bucket(uri.getHost()).key(path).build(); - - s3Client.headObject(headObjectRequest); - return true; - } catch (NoSuchKeyException e) { - return false; - } catch (S3Exception e) { - throw new IOException(e); - } - } - - private boolean isEmptyDirectory(URI uri) throws IOException { - if (!isDirectory(uri)) { - return false; - } - String prefix = normalizeToDirectoryPrefix(uri); - boolean isEmpty = true; - ListObjectsV2Response listObjectsV2Response; - ListObjectsV2Request.Builder listObjectsV2RequestBuilder = - ListObjectsV2Request.builder().bucket(uri.getHost()); - - if (!prefix.equals(DELIMITER)) { - listObjectsV2RequestBuilder = listObjectsV2RequestBuilder.prefix(prefix); - } - - ListObjectsV2Request listObjectsV2Request = listObjectsV2RequestBuilder.build(); - listObjectsV2Response = s3Client.listObjectsV2(listObjectsV2Request); - - for (S3Object s3Object : listObjectsV2Response.contents()) { - if (s3Object.key().equals(prefix)) { - continue; - } else { - isEmpty = false; - break; - } - } - return isEmpty; - } - - private boolean copyFile(URI srcUri, URI dstUri) throws IOException { - try { - String encodedUrl = null; - try { - encodedUrl = - URLEncoder.encode( - srcUri.getHost() + srcUri.getPath(), StandardCharsets.UTF_8.toString()); - } catch (UnsupportedEncodingException e) { - throw new RuntimeException(e); - } - - String dstPath = sanitizePath(dstUri.getPath()); - CopyObjectRequest copyReq = - CopyObjectRequest.builder() - .copySource(encodedUrl) - .destinationBucket(dstUri.getHost()) - .destinationKey(dstPath) - .build(); - - CopyObjectResponse copyObjectResponse = s3Client.copyObject(copyReq); - return copyObjectResponse.sdkHttpResponse().isSuccessful(); - } catch (S3Exception e) { - throw new IOException(e); - } - } - - @Override - public boolean mkdir(URI uri) throws IOException { - LOG.debug("mkdir {}", uri); - try { - Preconditions.checkNotNull(uri, "uri is null"); - String path = normalizeToDirectoryPrefix(uri); - // Bucket root directory already exists and cannot be created - if (path.equals(DELIMITER)) { - return true; - } - - PutObjectRequest putObjectRequest = - PutObjectRequest.builder().bucket(uri.getHost()).key(path).build(); - - PutObjectResponse putObjectResponse = - s3Client.putObject(putObjectRequest, RequestBody.fromBytes(new byte[0])); - - return putObjectResponse.sdkHttpResponse().isSuccessful(); - } catch (Throwable t) { - throw new IOException(t); - } - } - - @Override - public boolean delete(URI segmentUri, boolean forceDelete) throws IOException { - LOG.debug("Deleting uri {} force {}", segmentUri, forceDelete); - try { - if (isDirectory(segmentUri)) { - if (!forceDelete) { - Preconditions.checkState( - isEmptyDirectory(segmentUri), - "ForceDelete flag is not set and directory '%s' is not empty", - segmentUri); - } - String prefix = normalizeToDirectoryPrefix(segmentUri); - ListObjectsV2Response listObjectsV2Response; - ListObjectsV2Request.Builder listObjectsV2RequestBuilder = - ListObjectsV2Request.builder().bucket(segmentUri.getHost()); - - if (prefix.equals(DELIMITER)) { - ListObjectsV2Request listObjectsV2Request = listObjectsV2RequestBuilder.build(); - listObjectsV2Response = s3Client.listObjectsV2(listObjectsV2Request); - } else { - ListObjectsV2Request listObjectsV2Request = - listObjectsV2RequestBuilder.prefix(prefix).build(); - listObjectsV2Response = s3Client.listObjectsV2(listObjectsV2Request); - } - boolean deleteSucceeded = true; - for (S3Object s3Object : listObjectsV2Response.contents()) { - DeleteObjectRequest deleteObjectRequest = - DeleteObjectRequest.builder() - .bucket(segmentUri.getHost()) - .key(s3Object.key()) - .build(); - - DeleteObjectResponse deleteObjectResponse = s3Client.deleteObject(deleteObjectRequest); - - deleteSucceeded &= deleteObjectResponse.sdkHttpResponse().isSuccessful(); - } - return deleteSucceeded; - } else { - String prefix = sanitizePath(segmentUri.getPath()); - DeleteObjectRequest deleteObjectRequest = - DeleteObjectRequest.builder().bucket(segmentUri.getHost()).key(prefix).build(); - - DeleteObjectResponse deleteObjectResponse = s3Client.deleteObject(deleteObjectRequest); - - return deleteObjectResponse.sdkHttpResponse().isSuccessful(); - } - } catch (NoSuchKeyException e) { - return false; - } catch (S3Exception e) { - throw e; - } catch (Exception e) { - throw new IOException(e); - } - } - - @Override - public boolean doMove(URI srcUri, URI dstUri) throws IOException { - if (copy(srcUri, dstUri)) { - return delete(srcUri, true); - } - return false; - } - - @Override - public boolean copy(URI srcUri, URI dstUri) throws IOException { - LOG.debug("Copying uri {} to uri {}", srcUri, dstUri); - Preconditions.checkState(exists(srcUri), "Source URI '%s' does not exist", srcUri); - if (srcUri.equals(dstUri)) { - return true; - } - if (!isDirectory(srcUri)) { - delete(dstUri, true); - return copyFile(srcUri, dstUri); - } - dstUri = normalizeToDirectoryUri(dstUri); - Path srcPath = Paths.get(srcUri.getPath()); - try { - boolean copySucceeded = true; - for (String filePath : listFiles(srcUri, true)) { - URI srcFileURI = URI.create(filePath); - String directoryEntryPrefix = srcFileURI.getPath(); - URI src = new URI(srcUri.getScheme(), srcUri.getHost(), directoryEntryPrefix, null); - String relativeSrcPath = srcPath.relativize(Paths.get(directoryEntryPrefix)).toString(); - String dstPath = dstUri.resolve(relativeSrcPath).getPath(); - URI dst = new URI(dstUri.getScheme(), dstUri.getHost(), dstPath, null); - copySucceeded &= copyFile(src, dst); - } - return copySucceeded; - } catch (URISyntaxException e) { - throw new IOException(e); - } - } - - @Override - public boolean exists(URI fileUri) throws IOException { - try { - if (isDirectory(fileUri)) { - return true; - } - if (isPathTerminatedByDelimiter(fileUri)) { - return false; - } - return existsFile(fileUri); - } catch (NoSuchKeyException e) { - return false; - } - } - - @Override - public long length(URI fileUri) throws IOException { - try { - Preconditions.checkState(!isPathTerminatedByDelimiter(fileUri), "URI is a directory"); - HeadObjectResponse s3ObjectMetadata = getS3ObjectMetadata(fileUri); - Preconditions.checkState((s3ObjectMetadata != null), "File '%s' does not exist", fileUri); - if (s3ObjectMetadata.contentLength() == null) { - return 0; - } - return s3ObjectMetadata.contentLength(); - } catch (Throwable t) { - throw new IOException(t); - } - } - - @Override - public String[] listFiles(URI fileUri, boolean recursive) throws IOException { - try { - ImmutableList.Builder builder = ImmutableList.builder(); - String continuationToken = null; - boolean isDone = false; - String prefix = normalizeToDirectoryPrefix(fileUri); - int fileCount = 0; - while (!isDone) { - ListObjectsV2Request.Builder listObjectsV2RequestBuilder = - ListObjectsV2Request.builder().bucket(fileUri.getHost()); - if (!prefix.equals(DELIMITER)) { - listObjectsV2RequestBuilder = listObjectsV2RequestBuilder.prefix(prefix); - } - if (!recursive) { - listObjectsV2RequestBuilder = listObjectsV2RequestBuilder.delimiter(DELIMITER); - } - if (continuationToken != null) { - listObjectsV2RequestBuilder.continuationToken(continuationToken); - } - ListObjectsV2Request listObjectsV2Request = listObjectsV2RequestBuilder.build(); - LOG.debug("Trying to send ListObjectsV2Request {}", listObjectsV2Request); - ListObjectsV2Response listObjectsV2Response = s3Client.listObjectsV2(listObjectsV2Request); - LOG.debug("Getting ListObjectsV2Response: {}", listObjectsV2Response); - List filesReturned = listObjectsV2Response.contents(); - fileCount += filesReturned.size(); - filesReturned.stream() - .forEach( - object -> { - // Only add files and not directories - if (!object.key().equals(fileUri.getPath()) - && !object.key().endsWith(DELIMITER)) { - String fileKey = object.key(); - if (fileKey.startsWith(DELIMITER)) { - fileKey = fileKey.substring(1); - } - builder.add(S3_SCHEME + fileUri.getHost() + DELIMITER + fileKey); - } - }); - if (fileCount == LIST_MAX_KEYS) { - // check if we reached the max keys returned, if so abort and throw an error message - LOG.error( - "Too many files ({}) returned from S3 when attempting to list object prefixes", - LIST_MAX_KEYS); - throw new IllegalStateException( - String.format( - "Max keys (%s) reached when attempting to list S3 objects", LIST_MAX_KEYS)); - } - isDone = !listObjectsV2Response.isTruncated(); - continuationToken = listObjectsV2Response.nextContinuationToken(); - } - String[] listedFiles = builder.build().toArray(new String[0]); - LOG.debug( - "Listed {} files from URI: {}, is recursive: {}", listedFiles.length, fileUri, recursive); - return listedFiles; - } catch (Throwable t) { - throw new IOException(t); - } - } - - @Override - public void copyToLocalFile(URI srcUri, File dstFile) throws Exception { - LOG.debug("Copy {} to local {}", srcUri, dstFile.getAbsolutePath()); - URI base = getBase(srcUri); - FileUtils.forceMkdir(dstFile.getParentFile()); - String prefix = sanitizePath(base.relativize(srcUri).getPath()); - GetObjectRequest getObjectRequest = - GetObjectRequest.builder().bucket(srcUri.getHost()).key(prefix).build(); - - s3Client.getObject(getObjectRequest, ResponseTransformer.toFile(dstFile)); - } - - @Override - public void copyFromLocalFile(File srcFile, URI dstUri) throws Exception { - LOG.debug("Copy {} from local to {}", srcFile.getAbsolutePath(), dstUri); - URI base = getBase(dstUri); - String prefix = sanitizePath(base.relativize(dstUri).getPath()); - PutObjectRequest putObjectRequest = - PutObjectRequest.builder().bucket(dstUri.getHost()).key(prefix).build(); - - s3Client.putObject(putObjectRequest, srcFile.toPath()); - } - - @Override - public boolean isDirectory(URI uri) throws IOException { - try { - String prefix = normalizeToDirectoryPrefix(uri); - if (prefix.equals(DELIMITER)) { - return true; - } - - ListObjectsV2Request listObjectsV2Request = - ListObjectsV2Request.builder().bucket(uri.getHost()).prefix(prefix).maxKeys(2).build(); - ListObjectsV2Response listObjectsV2Response = s3Client.listObjectsV2(listObjectsV2Request); - return listObjectsV2Response.hasContents(); - } catch (NoSuchKeyException e) { - LOG.error("Could not get directory entry for {}", uri); - return false; - } - } - - @Override - public long lastModified(URI uri) throws IOException { - return getS3ObjectMetadata(uri).lastModified().toEpochMilli(); - } - - @Override - public boolean touch(URI uri) throws IOException { - try { - HeadObjectResponse s3ObjectMetadata = getS3ObjectMetadata(uri); - String encodedUrl = null; - try { - encodedUrl = - URLEncoder.encode(uri.getHost() + uri.getPath(), StandardCharsets.UTF_8.toString()); - } catch (UnsupportedEncodingException e) { - throw new RuntimeException(e); - } - - String path = sanitizePath(uri.getPath()); - Map mp = new HashMap<>(); - mp.put("lastModified", String.valueOf(System.currentTimeMillis())); - CopyObjectRequest request = - CopyObjectRequest.builder() - .copySource(encodedUrl) - .destinationBucket(uri.getHost()) - .destinationKey(path) - .metadata(mp) - .metadataDirective(MetadataDirective.REPLACE) - .build(); - - s3Client.copyObject(request); - long newUpdateTime = getS3ObjectMetadata(uri).lastModified().toEpochMilli(); - return newUpdateTime > s3ObjectMetadata.lastModified().toEpochMilli(); - } catch (NoSuchKeyException e) { - String path = sanitizePath(uri.getPath()); - s3Client.putObject( - PutObjectRequest.builder().bucket(uri.getHost()).key(path).build(), - RequestBody.fromBytes(new byte[0])); - return true; - } catch (S3Exception e) { - throw new IOException(e); - } - } - - @Override - public InputStream open(URI uri) throws IOException { - try { - String path = sanitizePath(uri.getPath()); - GetObjectRequest getObjectRequest = - GetObjectRequest.builder().bucket(uri.getHost()).key(path).build(); - - return s3Client.getObjectAsBytes(getObjectRequest).asInputStream(); - } catch (S3Exception e) { - throw e; - } - } - - @Override - public void close() throws IOException { - super.close(); - } -} diff --git a/astra/src/main/java/com/slack/astra/chunk/ChunkDownloader.java b/astra/src/main/java/com/slack/astra/chunk/ChunkDownloader.java deleted file mode 100644 index ff9ed73ba9..0000000000 --- a/astra/src/main/java/com/slack/astra/chunk/ChunkDownloader.java +++ /dev/null @@ -1,6 +0,0 @@ -package com.slack.astra.chunk; - -/** A ChunkDownloader is used to download chunk data from a remote store a local store. */ -public interface ChunkDownloader { - boolean download() throws Exception; -} diff --git a/astra/src/main/java/com/slack/astra/chunk/ReadWriteChunk.java b/astra/src/main/java/com/slack/astra/chunk/ReadWriteChunk.java index 49fe7bc5c1..7d6ac38080 100644 --- a/astra/src/main/java/com/slack/astra/chunk/ReadWriteChunk.java +++ b/astra/src/main/java/com/slack/astra/chunk/ReadWriteChunk.java @@ -1,8 +1,8 @@ package com.slack.astra.chunk; +import static com.slack.astra.blobfs.BlobFsUtils.copyToS3; +import static com.slack.astra.blobfs.BlobFsUtils.createURI; import static com.slack.astra.chunk.ChunkInfo.toSnapshotMetadata; -import static com.slack.astra.logstore.BlobFsUtils.copyToS3; -import static com.slack.astra.logstore.BlobFsUtils.createURI; import static com.slack.astra.writer.SpanFormatter.isValidTimestamp; import com.google.common.annotations.VisibleForTesting; diff --git a/astra/src/main/java/com/slack/astra/chunk/SerialS3ChunkDownloaderImpl.java b/astra/src/main/java/com/slack/astra/chunk/SerialS3ChunkDownloaderImpl.java deleted file mode 100644 index c88f0488a3..0000000000 --- a/astra/src/main/java/com/slack/astra/chunk/SerialS3ChunkDownloaderImpl.java +++ /dev/null @@ -1,31 +0,0 @@ -package com.slack.astra.chunk; - -import static com.slack.astra.logstore.BlobFsUtils.copyFromS3; - -import com.slack.astra.blobfs.BlobFs; -import java.nio.file.Path; - -/** - * SerialS3ChunkDownloaderImpl downloads all the chunk related data from S3 to a local store one - * file at a time. - */ -@Deprecated -public class SerialS3ChunkDownloaderImpl implements ChunkDownloader { - private final String s3Bucket; - private final String snapshotId; - private BlobFs blobFs; - private Path dataDirectory; - - public SerialS3ChunkDownloaderImpl( - String s3Bucket, String snapshotId, BlobFs blobFs, Path localDataDirectory) { - this.s3Bucket = s3Bucket; - this.snapshotId = snapshotId; - this.blobFs = blobFs; - this.dataDirectory = localDataDirectory; - } - - @Override - public boolean download() throws Exception { - return copyFromS3(s3Bucket, snapshotId, blobFs, dataDirectory).length == 0; - } -} diff --git a/astra/src/main/java/com/slack/astra/server/Astra.java b/astra/src/main/java/com/slack/astra/server/Astra.java index d8f69a5a41..efcd37a4a1 100644 --- a/astra/src/main/java/com/slack/astra/server/Astra.java +++ b/astra/src/main/java/com/slack/astra/server/Astra.java @@ -5,7 +5,7 @@ import com.google.common.util.concurrent.ServiceManager; import com.slack.astra.blobfs.BlobFs; import com.slack.astra.blobfs.ChunkStore; -import com.slack.astra.blobfs.s3.S3CrtBlobFs; +import com.slack.astra.blobfs.S3CrtBlobFs; import com.slack.astra.bulkIngestApi.BulkIngestApi; import com.slack.astra.bulkIngestApi.BulkIngestKafkaProducer; import com.slack.astra.bulkIngestApi.DatasetRateLimitingService; diff --git a/astra/src/test/java/com/slack/astra/blobfs/LocalBlobFsTest.java b/astra/src/test/java/com/slack/astra/blobfs/LocalBlobFsTest.java deleted file mode 100644 index c7bb733f48..0000000000 --- a/astra/src/test/java/com/slack/astra/blobfs/LocalBlobFsTest.java +++ /dev/null @@ -1,245 +0,0 @@ -package com.slack.astra.blobfs; - -import static org.assertj.core.api.Fail.fail; -import static org.junit.jupiter.api.Assertions.assertArrayEquals; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertTrue; - -import java.io.File; -import java.io.IOException; -import java.net.URI; -import org.apache.commons.io.FileUtils; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -public class LocalBlobFsTest { - private File testFile; - private File absoluteTmpDirPath; - private File newTmpDir; - private File nonExistentTmpFolder; - - @BeforeEach - public void setUp() { - absoluteTmpDirPath = - new File( - System.getProperty("java.io.tmpdir"), LocalBlobFsTest.class.getSimpleName() + "first"); - FileUtils.deleteQuietly(absoluteTmpDirPath); - assertTrue( - absoluteTmpDirPath.mkdir(), "Could not make directory " + absoluteTmpDirPath.getPath()); - try { - testFile = new File(absoluteTmpDirPath, "testFile"); - assertTrue(testFile.createNewFile(), "Could not create file " + testFile.getPath()); - } catch (IOException e) { - throw new RuntimeException(e); - } - - newTmpDir = - new File( - System.getProperty("java.io.tmpdir"), LocalBlobFsTest.class.getSimpleName() + "second"); - FileUtils.deleteQuietly(newTmpDir); - assertTrue(newTmpDir.mkdir(), "Could not make directory " + newTmpDir.getPath()); - - nonExistentTmpFolder = - new File( - System.getProperty("java.io.tmpdir"), - LocalBlobFsTest.class.getSimpleName() + "nonExistentParent/nonExistent"); - - absoluteTmpDirPath.deleteOnExit(); - newTmpDir.deleteOnExit(); - nonExistentTmpFolder.deleteOnExit(); - } - - @AfterEach - public void tearDown() { - absoluteTmpDirPath.delete(); - newTmpDir.delete(); - } - - @Test - public void testFS() throws Exception { - LocalBlobFs localBlobFs = new LocalBlobFs(); - URI testFileUri = testFile.toURI(); - // Check whether a directory exists - assertTrue(localBlobFs.exists(absoluteTmpDirPath.toURI())); - assertTrue(localBlobFs.lastModified(absoluteTmpDirPath.toURI()) > 0L); - assertTrue(localBlobFs.isDirectory(absoluteTmpDirPath.toURI())); - // Check whether a file exists - assertTrue(localBlobFs.exists(testFileUri)); - assertFalse(localBlobFs.isDirectory(testFileUri)); - - File file = new File(absoluteTmpDirPath, "secondTestFile"); - URI secondTestFileUri = file.toURI(); - // Check that file does not exist - assertTrue(!localBlobFs.exists(secondTestFileUri)); - - localBlobFs.copy(testFileUri, secondTestFileUri); - assertEquals(2, localBlobFs.listFiles(absoluteTmpDirPath.toURI(), true).length); - - // Check file copy worked when file was not created - assertTrue(localBlobFs.exists(secondTestFileUri)); - - // Create another file in the same path - File thirdTestFile = new File(absoluteTmpDirPath, "thirdTestFile"); - assertTrue(thirdTestFile.createNewFile(), "Could not create file " + thirdTestFile.getPath()); - - File newAbsoluteTempDirPath = new File(absoluteTmpDirPath, "absoluteTwo"); - assertTrue(newAbsoluteTempDirPath.mkdir()); - - // Create a testDir and file underneath directory - File testDir = new File(newAbsoluteTempDirPath, "testDir"); - assertTrue(testDir.mkdir(), "Could not make directory " + testDir.getAbsolutePath()); - File testDirFile = new File(testDir, "testFile"); - // Assert that recursive list files and nonrecursive list files are as expected - assertTrue(testDirFile.createNewFile(), "Could not create file " + testDir.getAbsolutePath()); - assertArrayEquals( - localBlobFs.listFiles(newAbsoluteTempDirPath.toURI(), false), - new String[] {testDir.getAbsolutePath()}); - assertArrayEquals( - localBlobFs.listFiles(newAbsoluteTempDirPath.toURI(), true), - new String[] {testDir.getAbsolutePath(), testDirFile.getAbsolutePath()}); - - // Create another parent dir so we can test recursive move - File newAbsoluteTempDirPath3 = new File(absoluteTmpDirPath, "absoluteThree"); - assertTrue(newAbsoluteTempDirPath3.mkdir()); - assertEquals(newAbsoluteTempDirPath3.listFiles().length, 0); - - localBlobFs.move(newAbsoluteTempDirPath.toURI(), newAbsoluteTempDirPath3.toURI(), true); - assertFalse(localBlobFs.exists(newAbsoluteTempDirPath.toURI())); - assertTrue(localBlobFs.exists(newAbsoluteTempDirPath3.toURI())); - assertTrue(localBlobFs.exists(new File(newAbsoluteTempDirPath3, "testDir").toURI())); - assertTrue( - localBlobFs.exists( - new File(new File(newAbsoluteTempDirPath3, "testDir"), "testFile").toURI())); - - // Check if using a different scheme on URI still works - URI uri = URI.create("hdfs://localhost:9999" + newAbsoluteTempDirPath.getPath()); - localBlobFs.move(newAbsoluteTempDirPath3.toURI(), uri, true); - assertFalse(localBlobFs.exists(newAbsoluteTempDirPath3.toURI())); - assertTrue(localBlobFs.exists(newAbsoluteTempDirPath.toURI())); - assertTrue(localBlobFs.exists(new File(newAbsoluteTempDirPath, "testDir").toURI())); - assertTrue( - localBlobFs.exists( - new File(new File(newAbsoluteTempDirPath, "testDir"), "testFile").toURI())); - - // Check file copy to location where something already exists still works - localBlobFs.copy(testFileUri, thirdTestFile.toURI()); - // Check length of file - assertEquals(0, localBlobFs.length(secondTestFileUri)); - assertTrue(localBlobFs.exists(thirdTestFile.toURI())); - - // Check that method deletes dst directory during move and is successful by overwriting dir - assertTrue(newTmpDir.exists()); - // create a file in the dst folder - File dstFile = new File(newTmpDir.getPath() + "/newFile"); - dstFile.createNewFile(); - - // Expected that a move without overwrite will not succeed - assertFalse(localBlobFs.move(absoluteTmpDirPath.toURI(), newTmpDir.toURI(), false)); - - int files = absoluteTmpDirPath.listFiles().length; - assertTrue(localBlobFs.move(absoluteTmpDirPath.toURI(), newTmpDir.toURI(), true)); - assertEquals(absoluteTmpDirPath.length(), 0); - assertEquals(newTmpDir.listFiles().length, files); - assertFalse(dstFile.exists()); - - // Check that a moving a file to a non-existent destination folder will work - FileUtils.deleteQuietly(nonExistentTmpFolder); - assertFalse(nonExistentTmpFolder.exists()); - File srcFile = new File(absoluteTmpDirPath, "srcFile"); - localBlobFs.mkdir(absoluteTmpDirPath.toURI()); - assertTrue(srcFile.createNewFile()); - dstFile = new File(nonExistentTmpFolder.getPath() + "/newFile"); - assertFalse(dstFile.exists()); - assertTrue( - localBlobFs.move(srcFile.toURI(), dstFile.toURI(), true)); // overwrite flag has no impact - assertFalse(srcFile.exists()); - assertTrue(dstFile.exists()); - - // Check that moving a folder to a non-existent destination folder works - FileUtils.deleteQuietly(nonExistentTmpFolder); - assertFalse(nonExistentTmpFolder.exists()); - srcFile = new File(absoluteTmpDirPath, "srcFile"); - localBlobFs.mkdir(absoluteTmpDirPath.toURI()); - assertTrue(srcFile.createNewFile()); - dstFile = new File(nonExistentTmpFolder.getPath() + "/srcFile"); - assertFalse(dstFile.exists()); - assertTrue( - localBlobFs.move( - absoluteTmpDirPath.toURI(), - nonExistentTmpFolder.toURI(), - true)); // overwrite flag has no impact - assertTrue(dstFile.exists()); - - localBlobFs.delete(secondTestFileUri, true); - // Check deletion from final location worked - assertTrue(!localBlobFs.exists(secondTestFileUri)); - - File firstTempDir = new File(absoluteTmpDirPath, "firstTempDir"); - File secondTempDir = new File(absoluteTmpDirPath, "secondTempDir"); - localBlobFs.mkdir(firstTempDir.toURI()); - assertTrue(firstTempDir.exists(), "Could not make directory " + firstTempDir.getPath()); - - // Check that touching a file works - File nonExistingFile = new File(absoluteTmpDirPath, "nonExistingFile"); - assertFalse(nonExistingFile.exists()); - localBlobFs.touch(nonExistingFile.toURI()); - assertTrue(nonExistingFile.exists()); - long currentTime = System.currentTimeMillis(); - assertTrue(localBlobFs.lastModified(nonExistingFile.toURI()) <= currentTime); - Thread.sleep(1L); - // update last modified. - localBlobFs.touch(nonExistingFile.toURI()); - assertTrue(localBlobFs.lastModified(nonExistingFile.toURI()) > currentTime); - FileUtils.deleteQuietly(nonExistingFile); - - // Check that touch an file in a directory that doesn't exist should throw an exception. - File nonExistingFileUnderNonExistingDir = - new File(absoluteTmpDirPath, "nonExistingDir/nonExistingFile"); - assertFalse(nonExistingFileUnderNonExistingDir.exists()); - try { - localBlobFs.touch(nonExistingFileUnderNonExistingDir.toURI()); - fail("Touch method should throw an IOException"); - } catch (IOException e) { - // Expected. - } - - // Check that directory only copy worked - localBlobFs.copy(firstTempDir.toURI(), secondTempDir.toURI()); - assertTrue(localBlobFs.exists(secondTempDir.toURI())); - - // Copying directory with files to directory with files - File testFile = new File(firstTempDir, "testFile"); - assertTrue(testFile.createNewFile(), "Could not create file " + testFile.getPath()); - File newTestFile = new File(secondTempDir, "newTestFile"); - assertTrue(newTestFile.createNewFile(), "Could not create file " + newTestFile.getPath()); - - localBlobFs.copy(firstTempDir.toURI(), secondTempDir.toURI()); - assertEquals(localBlobFs.listFiles(secondTempDir.toURI(), true).length, 1); - - // Copying directory with files under another directory. - File firstTempDirUnderSecondTempDir = new File(secondTempDir, firstTempDir.getName()); - localBlobFs.copy(firstTempDir.toURI(), firstTempDirUnderSecondTempDir.toURI()); - assertTrue(localBlobFs.exists(firstTempDirUnderSecondTempDir.toURI())); - // There're two files/directories under secondTempDir. - assertEquals(localBlobFs.listFiles(secondTempDir.toURI(), false).length, 2); - // The file under src directory also got copied under dst directory. - assertEquals(localBlobFs.listFiles(firstTempDirUnderSecondTempDir.toURI(), true).length, 1); - - // len of dir = exception - try { - localBlobFs.length(firstTempDir.toURI()); - fail("Exception expected that did not occur"); - } catch (IllegalArgumentException e) { - - } - - assertTrue(testFile.exists()); - - localBlobFs.copyFromLocalFile(testFile, secondTestFileUri); - assertTrue(localBlobFs.exists(secondTestFileUri)); - localBlobFs.copyToLocalFile(testFile.toURI(), new File(secondTestFileUri)); - assertTrue(localBlobFs.exists(secondTestFileUri)); - } -} diff --git a/astra/src/test/java/com/slack/astra/blobfs/s3/S3BlobFsTest.java b/astra/src/test/java/com/slack/astra/blobfs/s3/S3BlobFsTest.java deleted file mode 100644 index 53dd754288..0000000000 --- a/astra/src/test/java/com/slack/astra/blobfs/s3/S3BlobFsTest.java +++ /dev/null @@ -1,353 +0,0 @@ -package com.slack.astra.blobfs.s3; - -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertTrue; - -import com.adobe.testing.s3mock.junit5.S3MockExtension; -import java.io.File; -import java.io.IOException; -import java.io.InputStream; -import java.net.URI; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.UUID; -import org.apache.commons.io.IOUtils; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.RegisterExtension; -import software.amazon.awssdk.core.sync.RequestBody; -import software.amazon.awssdk.services.s3.S3Client; -import software.amazon.awssdk.services.s3.model.CreateBucketRequest; -import software.amazon.awssdk.services.s3.model.HeadObjectResponse; -import software.amazon.awssdk.services.s3.model.ListObjectsV2Response; -import software.amazon.awssdk.services.s3.model.S3Object; - -@Deprecated -public class S3BlobFsTest { - @RegisterExtension - public static final S3MockExtension S3_MOCK_EXTENSION = - S3MockExtension.builder().silent().withSecureConnection(false).build(); - - final String DELIMITER = "/"; - final String SCHEME = "s3"; - final String FILE_FORMAT = "%s://%s/%s"; - final String DIR_FORMAT = "%s://%s"; - - private final S3Client s3Client = S3_MOCK_EXTENSION.createS3ClientV2(); - private String bucket; - private S3BlobFs s3BlobFs; - - @BeforeEach - public void setUp() { - bucket = "test-bucket-" + UUID.randomUUID(); - s3BlobFs = new S3BlobFs(s3Client); - s3Client.createBucket(CreateBucketRequest.builder().bucket(bucket).build()); - } - - @AfterEach - public void tearDown() throws IOException { - if (s3BlobFs != null) { - s3BlobFs.close(); - } - } - - private void createEmptyFile(String folderName, String fileName) { - String fileNameWithFolder = folderName + DELIMITER + fileName; - if (folderName.isEmpty()) { - fileNameWithFolder = fileName; - } - s3Client.putObject( - S3TestUtils.getPutObjectRequest(bucket, fileNameWithFolder), - RequestBody.fromBytes(new byte[0])); - } - - @Test - public void testTouchFileInBucket() throws Exception { - - String[] originalFiles = new String[] {"a-touch.txt", "b-touch.txt", "c-touch.txt"}; - - for (String fileName : originalFiles) { - s3BlobFs.touch(URI.create(String.format(FILE_FORMAT, SCHEME, bucket, fileName))); - } - ListObjectsV2Response listObjectsV2Response = - s3Client.listObjectsV2(S3TestUtils.getListObjectRequest(bucket, "", true)); - - String[] response = - listObjectsV2Response.contents().stream() - .map(S3Object::key) - .filter(x -> x.contains("touch")) - .toArray(String[]::new); - - assertEquals(response.length, originalFiles.length); - assertTrue(Arrays.equals(response, originalFiles)); - } - - @Test - public void testTouchFilesInFolder() throws Exception { - - String folder = "my-files"; - String[] originalFiles = new String[] {"a-touch.txt", "b-touch.txt", "c-touch.txt"}; - - for (String fileName : originalFiles) { - String fileNameWithFolder = folder + DELIMITER + fileName; - s3BlobFs.touch(URI.create(String.format(FILE_FORMAT, SCHEME, bucket, fileNameWithFolder))); - } - ListObjectsV2Response listObjectsV2Response = - s3Client.listObjectsV2(S3TestUtils.getListObjectRequest(bucket, folder, false)); - - String[] response = - listObjectsV2Response.contents().stream() - .map(S3Object::key) - .filter(x -> x.contains("touch")) - .toArray(String[]::new); - assertEquals(response.length, originalFiles.length); - - assertTrue( - Arrays.equals( - response, Arrays.stream(originalFiles).map(x -> folder + DELIMITER + x).toArray())); - } - - @Test - public void testListFilesInBucketNonRecursive() throws Exception { - String[] originalFiles = new String[] {"a-list.txt", "b-list.txt", "c-list.txt"}; - List expectedFileNames = new ArrayList<>(); - - for (String fileName : originalFiles) { - createEmptyFile("", fileName); - expectedFileNames.add(String.format(FILE_FORMAT, SCHEME, bucket, fileName)); - } - - String[] actualFiles = - s3BlobFs.listFiles(URI.create(String.format(DIR_FORMAT, SCHEME, bucket)), false); - - actualFiles = Arrays.stream(actualFiles).filter(x -> x.contains("list")).toArray(String[]::new); - assertEquals(actualFiles.length, originalFiles.length); - - assertTrue(Arrays.equals(actualFiles, expectedFileNames.toArray())); - } - - @Test - public void testListFilesInFolderNonRecursive() throws Exception { - String folder = "list-files"; - String[] originalFiles = new String[] {"a-list-2.txt", "b-list-2.txt", "c-list-2.txt"}; - - for (String fileName : originalFiles) { - createEmptyFile(folder, fileName); - } - - String[] actualFiles = - s3BlobFs.listFiles(URI.create(String.format(FILE_FORMAT, SCHEME, bucket, folder)), false); - - actualFiles = - Arrays.stream(actualFiles).filter(x -> x.contains("list-2")).toArray(String[]::new); - assertEquals(actualFiles.length, originalFiles.length); - - assertTrue( - Arrays.equals( - Arrays.stream(originalFiles) - .map( - fileName -> - String.format(FILE_FORMAT, SCHEME, bucket, folder + DELIMITER + fileName)) - .toArray(), - actualFiles)); - } - - @Test - public void testListFilesInFolderRecursive() throws Exception { - String folder = "list-files-rec"; - String[] nestedFolders = new String[] {"list-files-child-1", "list-files-child-2"}; - String[] originalFiles = new String[] {"a-list-3.txt", "b-list-3.txt", "c-list-3.txt"}; - - List expectedResultList = new ArrayList<>(); - for (String childFolder : nestedFolders) { - String folderName = folder + DELIMITER + childFolder; - for (String fileName : originalFiles) { - createEmptyFile(folderName, fileName); - expectedResultList.add( - String.format(FILE_FORMAT, SCHEME, bucket, folderName + DELIMITER + fileName)); - } - } - String[] actualFiles = - s3BlobFs.listFiles(URI.create(String.format(FILE_FORMAT, SCHEME, bucket, folder)), true); - - actualFiles = - Arrays.stream(actualFiles).filter(x -> x.contains("list-3")).toArray(String[]::new); - assertEquals(actualFiles.length, expectedResultList.size()); - assertTrue(Arrays.equals(expectedResultList.toArray(), actualFiles)); - } - - @Test - public void testDeleteFile() throws Exception { - String[] originalFiles = new String[] {"a-delete.txt", "b-delete.txt", "c-delete.txt"}; - String fileToDelete = "a-delete.txt"; - - List expectedResultList = new ArrayList<>(); - for (String fileName : originalFiles) { - createEmptyFile("", fileName); - if (!fileName.equals(fileToDelete)) { - expectedResultList.add(fileName); - } - } - - s3BlobFs.delete(URI.create(String.format(FILE_FORMAT, SCHEME, bucket, fileToDelete)), false); - - ListObjectsV2Response listObjectsV2Response = - s3Client.listObjectsV2(S3TestUtils.getListObjectRequest(bucket, "", true)); - String[] actualResponse = - listObjectsV2Response.contents().stream() - .map(S3Object::key) - .filter(x -> x.contains("delete")) - .toArray(String[]::new); - - assertEquals(actualResponse.length, 2); - assertTrue(Arrays.equals(actualResponse, expectedResultList.toArray())); - } - - @Test - public void testDeleteFolder() throws Exception { - String[] originalFiles = new String[] {"a-delete-2.txt", "b-delete-2.txt", "c-delete-2.txt"}; - String folderName = "my-files"; - - for (String fileName : originalFiles) { - createEmptyFile(folderName, fileName); - } - - s3BlobFs.delete(URI.create(String.format(FILE_FORMAT, SCHEME, bucket, folderName)), true); - - ListObjectsV2Response listObjectsV2Response = - s3Client.listObjectsV2(S3TestUtils.getListObjectRequest(bucket, "", true)); - String[] actualResponse = - listObjectsV2Response.contents().stream() - .map(S3Object::key) - .filter(x -> x.contains("delete-2")) - .toArray(String[]::new); - - assertEquals(0, actualResponse.length); - } - - @Test - public void testIsDirectory() throws Exception { - String[] originalFiles = new String[] {"a-dir.txt", "b-dir.txt", "c-dir.txt"}; - String folder = "my-files-dir"; - String childFolder = "my-files-dir-child"; - for (String fileName : originalFiles) { - String folderName = folder + DELIMITER + childFolder; - createEmptyFile(folderName, fileName); - } - - boolean isBucketDir = - s3BlobFs.isDirectory(URI.create(String.format(DIR_FORMAT, SCHEME, bucket))); - boolean isDir = - s3BlobFs.isDirectory(URI.create(String.format(FILE_FORMAT, SCHEME, bucket, folder))); - boolean isDirChild = - s3BlobFs.isDirectory( - URI.create( - String.format(FILE_FORMAT, SCHEME, bucket, folder + DELIMITER + childFolder))); - boolean notIsDir = - s3BlobFs.isDirectory( - URI.create( - String.format( - FILE_FORMAT, - SCHEME, - bucket, - folder + DELIMITER + childFolder + DELIMITER + "a-delete.txt"))); - - assertTrue(isBucketDir); - assertTrue(isDir); - assertTrue(isDirChild); - assertFalse(notIsDir); - } - - @Test - public void testExists() throws Exception { - String[] originalFiles = new String[] {"a-ex.txt", "b-ex.txt", "c-ex.txt"}; - String folder = "my-files-dir"; - String childFolder = "my-files-dir-child"; - - for (String fileName : originalFiles) { - String folderName = folder + DELIMITER + childFolder; - createEmptyFile(folderName, fileName); - } - - boolean bucketExists = s3BlobFs.exists(URI.create(String.format(DIR_FORMAT, SCHEME, bucket))); - boolean dirExists = - s3BlobFs.exists(URI.create(String.format(FILE_FORMAT, SCHEME, bucket, folder))); - boolean childDirExists = - s3BlobFs.exists( - URI.create( - String.format(FILE_FORMAT, SCHEME, bucket, folder + DELIMITER + childFolder))); - boolean fileExists = - s3BlobFs.exists( - URI.create( - String.format( - FILE_FORMAT, - SCHEME, - bucket, - folder + DELIMITER + childFolder + DELIMITER + "a-ex.txt"))); - boolean fileNotExists = - s3BlobFs.exists( - URI.create( - String.format( - FILE_FORMAT, - SCHEME, - bucket, - folder + DELIMITER + childFolder + DELIMITER + "d-ex.txt"))); - - assertTrue(bucketExists); - assertTrue(dirExists); - assertTrue(childDirExists); - assertTrue(fileExists); - assertFalse(fileNotExists); - } - - @Test - public void testCopyFromAndToLocal() throws Exception { - String fileName = "copyFile.txt"; - - File fileToCopy = new File(getClass().getClassLoader().getResource(fileName).getFile()); - - s3BlobFs.copyFromLocalFile( - fileToCopy, URI.create(String.format(FILE_FORMAT, SCHEME, bucket, fileName))); - - HeadObjectResponse headObjectResponse = - s3Client.headObject(S3TestUtils.getHeadObjectRequest(bucket, fileName)); - - assertEquals(headObjectResponse.contentLength(), (Long) fileToCopy.length()); - - File fileToDownload = new File("copyFile_download.txt").getAbsoluteFile(); - s3BlobFs.copyToLocalFile( - URI.create(String.format(FILE_FORMAT, SCHEME, bucket, fileName)), fileToDownload); - assertEquals(fileToCopy.length(), fileToDownload.length()); - - fileToDownload.deleteOnExit(); - } - - @Test - public void testOpenFile() throws Exception { - String fileName = "sample.txt"; - String fileContent = "Hello, World"; - - s3Client.putObject( - S3TestUtils.getPutObjectRequest(bucket, fileName), RequestBody.fromString(fileContent)); - - InputStream is = - s3BlobFs.open(URI.create(String.format(FILE_FORMAT, SCHEME, bucket, fileName))); - String actualContents = IOUtils.toString(is, StandardCharsets.UTF_8); - assertEquals(actualContents, fileContent); - } - - @Test - public void testMkdir() throws Exception { - String folderName = "my-test-folder"; - - s3BlobFs.mkdir(URI.create(String.format(FILE_FORMAT, SCHEME, bucket, folderName))); - - HeadObjectResponse headObjectResponse = - s3Client.headObject(S3TestUtils.getHeadObjectRequest(bucket, folderName + DELIMITER)); - assertTrue(headObjectResponse.sdkHttpResponse().isSuccessful()); - } -} diff --git a/astra/src/test/java/com/slack/astra/blobfs/s3/S3CrtBlobFsTest.java b/astra/src/test/java/com/slack/astra/blobfs/s3/S3CrtBlobFsTest.java index 4e5bec6931..1a6cd386c6 100644 --- a/astra/src/test/java/com/slack/astra/blobfs/s3/S3CrtBlobFsTest.java +++ b/astra/src/test/java/com/slack/astra/blobfs/s3/S3CrtBlobFsTest.java @@ -7,17 +7,15 @@ import com.adobe.testing.s3mock.junit5.S3MockExtension; import com.google.common.io.Resources; +import com.slack.astra.blobfs.S3CrtBlobFs; import java.io.File; import java.io.IOException; -import java.io.InputStream; import java.net.URI; -import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.UUID; import java.util.concurrent.ExecutionException; -import org.apache.commons.io.IOUtils; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -25,8 +23,11 @@ import software.amazon.awssdk.core.async.AsyncRequestBody; import software.amazon.awssdk.services.s3.S3AsyncClient; import software.amazon.awssdk.services.s3.model.CreateBucketRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectRequest; import software.amazon.awssdk.services.s3.model.HeadObjectResponse; +import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; import software.amazon.awssdk.services.s3.model.ListObjectsV2Response; +import software.amazon.awssdk.services.s3.model.PutObjectRequest; import software.amazon.awssdk.services.s3.model.S3Object; public class S3CrtBlobFsTest { @@ -66,57 +67,11 @@ private void createEmptyFile(String folderName, String fileName) } s3Client .putObject( - S3TestUtils.getPutObjectRequest(bucket, fileNameWithFolder), + PutObjectRequest.builder().bucket(bucket).key(fileNameWithFolder).build(), AsyncRequestBody.fromBytes(new byte[0])) .get(); } - @Test - public void testTouchFileInBucket() throws Exception { - - String[] originalFiles = new String[] {"a-touch.txt", "b-touch.txt", "c-touch.txt"}; - - for (String fileName : originalFiles) { - s3BlobFs.touch(URI.create(String.format(FILE_FORMAT, SCHEME, bucket, fileName))); - } - ListObjectsV2Response listObjectsV2Response = - s3Client.listObjectsV2(S3TestUtils.getListObjectRequest(bucket, "", true)).get(); - - String[] response = - listObjectsV2Response.contents().stream() - .map(S3Object::key) - .filter(x -> x.contains("touch")) - .toArray(String[]::new); - - assertEquals(response.length, originalFiles.length); - assertTrue(Arrays.equals(response, originalFiles)); - } - - @Test - public void testTouchFilesInFolder() throws Exception { - - String folder = "my-files"; - String[] originalFiles = new String[] {"a-touch.txt", "b-touch.txt", "c-touch.txt"}; - - for (String fileName : originalFiles) { - String fileNameWithFolder = folder + DELIMITER + fileName; - s3BlobFs.touch(URI.create(String.format(FILE_FORMAT, SCHEME, bucket, fileNameWithFolder))); - } - ListObjectsV2Response listObjectsV2Response = - s3Client.listObjectsV2(S3TestUtils.getListObjectRequest(bucket, folder, false)).get(); - - String[] response = - listObjectsV2Response.contents().stream() - .map(S3Object::key) - .filter(x -> x.contains("touch")) - .toArray(String[]::new); - assertEquals(response.length, originalFiles.length); - - assertTrue( - Arrays.equals( - response, Arrays.stream(originalFiles).map(x -> folder + DELIMITER + x).toArray())); - } - @Test public void testListFilesInBucketNonRecursive() throws Exception { String[] originalFiles = new String[] {"a-list.txt", "b-list.txt", "c-list.txt"}; @@ -202,7 +157,7 @@ public void testDeleteFile() throws Exception { s3BlobFs.delete(URI.create(String.format(FILE_FORMAT, SCHEME, bucket, fileToDelete)), false); ListObjectsV2Response listObjectsV2Response = - s3Client.listObjectsV2(S3TestUtils.getListObjectRequest(bucket, "", true)).get(); + s3Client.listObjectsV2(ListObjectsV2Request.builder().bucket(bucket).build()).get(); String[] actualResponse = listObjectsV2Response.contents().stream() .map(S3Object::key) @@ -231,7 +186,7 @@ public void testDeleteFolder() throws Exception { .until( () -> s3Client - .listObjectsV2(S3TestUtils.getListObjectRequest(bucket, "", true)) + .listObjectsV2(ListObjectsV2Request.builder().bucket(bucket).build()) .get() .contents() .stream() @@ -338,7 +293,7 @@ public void testCopyFromAndToLocal() throws Exception { fileToCopy, URI.create(String.format(FILE_FORMAT, SCHEME, bucket, fileName))); HeadObjectResponse headObjectResponse = - s3Client.headObject(S3TestUtils.getHeadObjectRequest(bucket, fileName)).get(); + s3Client.headObject(HeadObjectRequest.builder().bucket(bucket).key(fileName).build()).get(); assertEquals(headObjectResponse.contentLength(), (Long) fileToCopy.length()); @@ -361,7 +316,7 @@ public void testCopyFromAndToLocalDirectory() throws Exception { fileToCopy.getParentFile(), URI.create(String.format(FILE_FORMAT, SCHEME, bucket, ""))); HeadObjectResponse headObjectResponse = - s3Client.headObject(S3TestUtils.getHeadObjectRequest(bucket, fileName)).get(); + s3Client.headObject(HeadObjectRequest.builder().bucket(bucket).key(fileName).build()).get(); assertEquals(headObjectResponse.contentLength(), (Long) fileToCopy.length()); @@ -372,32 +327,4 @@ public void testCopyFromAndToLocalDirectory() throws Exception { fileToDownload.deleteOnExit(); } - - @Test - public void testOpenFile() throws Exception { - String fileName = "sample.txt"; - String fileContent = "Hello, World"; - - s3Client - .putObject( - S3TestUtils.getPutObjectRequest(bucket, fileName), - AsyncRequestBody.fromString(fileContent)) - .get(); - - InputStream is = - s3BlobFs.open(URI.create(String.format(FILE_FORMAT, SCHEME, bucket, fileName))); - String actualContents = IOUtils.toString(is, StandardCharsets.UTF_8); - assertEquals(actualContents, fileContent); - } - - @Test - public void testMkdir() throws Exception { - String folderName = "my-test-folder"; - - s3BlobFs.mkdir(URI.create(String.format(FILE_FORMAT, SCHEME, bucket, folderName))); - - HeadObjectResponse headObjectResponse = - s3Client.headObject(S3TestUtils.getHeadObjectRequest(bucket, folderName + DELIMITER)).get(); - assertTrue(headObjectResponse.sdkHttpResponse().isSuccessful()); - } } diff --git a/astra/src/test/java/com/slack/astra/blobfs/s3/S3TestUtils.java b/astra/src/test/java/com/slack/astra/blobfs/s3/S3TestUtils.java index f6247b2fff..dda3e2ca4b 100644 --- a/astra/src/test/java/com/slack/astra/blobfs/s3/S3TestUtils.java +++ b/astra/src/test/java/com/slack/astra/blobfs/s3/S3TestUtils.java @@ -6,32 +6,8 @@ import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.services.s3.S3AsyncClient; import software.amazon.awssdk.services.s3.crt.S3CrtHttpConfiguration; -import software.amazon.awssdk.services.s3.model.HeadObjectRequest; -import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; -import software.amazon.awssdk.services.s3.model.PutObjectRequest; public class S3TestUtils { - - public static PutObjectRequest getPutObjectRequest(String bucket, String key) { - return PutObjectRequest.builder().bucket(bucket).key(key).build(); - } - - public static HeadObjectRequest getHeadObjectRequest(String bucket, String key) { - return HeadObjectRequest.builder().bucket(bucket).key(key).build(); - } - - public static ListObjectsV2Request getListObjectRequest( - String bucket, String key, boolean isInBucket) { - ListObjectsV2Request.Builder listObjectsV2RequestBuilder = - ListObjectsV2Request.builder().bucket(bucket); - - if (!isInBucket) { - listObjectsV2RequestBuilder = listObjectsV2RequestBuilder.prefix(key); - } - - return listObjectsV2RequestBuilder.build(); - } - /** * Based off of S3_MOCK_EXTENSION.createS3ClientV2(); * diff --git a/astra/src/test/java/com/slack/astra/chunk/IndexingChunkImplTest.java b/astra/src/test/java/com/slack/astra/chunk/IndexingChunkImplTest.java index 0d7abda8fc..1e4f6bd839 100644 --- a/astra/src/test/java/com/slack/astra/chunk/IndexingChunkImplTest.java +++ b/astra/src/test/java/com/slack/astra/chunk/IndexingChunkImplTest.java @@ -17,7 +17,7 @@ import brave.Tracing; import com.adobe.testing.s3mock.junit5.S3MockExtension; -import com.slack.astra.blobfs.s3.S3CrtBlobFs; +import com.slack.astra.blobfs.S3CrtBlobFs; import com.slack.astra.blobfs.s3.S3TestUtils; import com.slack.astra.logstore.LogMessage; import com.slack.astra.logstore.LuceneIndexStoreImpl; @@ -59,6 +59,7 @@ import org.junit.jupiter.api.io.TempDir; import software.amazon.awssdk.services.s3.S3AsyncClient; import software.amazon.awssdk.services.s3.model.CreateBucketRequest; +import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; import software.amazon.awssdk.services.s3.model.ListObjectsV2Response; import software.amazon.awssdk.services.s3.model.S3Object; @@ -713,7 +714,7 @@ public void testSnapshotToS3UsingChunkApi() throws Exception { // Check schema file exists in s3 ListObjectsV2Response objectsResponse = - s3AsyncClient.listObjectsV2(S3TestUtils.getListObjectRequest(bucket, "", true)).get(); + s3AsyncClient.listObjectsV2(ListObjectsV2Request.builder().bucket(bucket).build()).get(); assertThat( objectsResponse.contents().stream() .filter(o -> o.key().equals(SCHEMA_FILE_NAME)) diff --git a/astra/src/test/java/com/slack/astra/chunk/ReadOnlyChunkImplTest.java b/astra/src/test/java/com/slack/astra/chunk/ReadOnlyChunkImplTest.java index 4dc0d2dccb..aa81276741 100644 --- a/astra/src/test/java/com/slack/astra/chunk/ReadOnlyChunkImplTest.java +++ b/astra/src/test/java/com/slack/astra/chunk/ReadOnlyChunkImplTest.java @@ -1,9 +1,9 @@ package com.slack.astra.chunk; +import static com.slack.astra.blobfs.BlobFsUtils.copyToS3; import static com.slack.astra.chunk.ReadOnlyChunkImpl.CHUNK_ASSIGNMENT_TIMER; import static com.slack.astra.chunk.ReadOnlyChunkImpl.CHUNK_EVICTION_TIMER; import static com.slack.astra.chunk.ReadWriteChunk.SCHEMA_FILE_NAME; -import static com.slack.astra.logstore.BlobFsUtils.copyToS3; import static com.slack.astra.logstore.LuceneIndexStoreImpl.COMMITS_TIMER; import static com.slack.astra.logstore.LuceneIndexStoreImpl.MESSAGES_FAILED_COUNTER; import static com.slack.astra.logstore.LuceneIndexStoreImpl.MESSAGES_RECEIVED_COUNTER; @@ -18,8 +18,7 @@ import brave.Tracing; import com.adobe.testing.s3mock.junit5.S3MockExtension; import com.slack.astra.blobfs.ChunkStore; -import com.slack.astra.blobfs.LocalBlobFs; -import com.slack.astra.blobfs.s3.S3CrtBlobFs; +import com.slack.astra.blobfs.S3CrtBlobFs; import com.slack.astra.blobfs.s3.S3TestUtils; import com.slack.astra.logstore.LogMessage; import com.slack.astra.logstore.LuceneIndexStoreImpl; @@ -678,11 +677,7 @@ private void initializeBlobStorageWithIndex(String snapshotId) throws Exception IndexCommit indexCommit = logStore.getIndexCommit(); filesToUpload.addAll(indexCommit.getFileNames()); - LocalBlobFs localBlobFs = new LocalBlobFs(); - - logStore.close(); - assertThat(localBlobFs.listFiles(dirPath.toUri(), false).length) - .isGreaterThanOrEqualTo(filesToUpload.size()); + assertThat(dirPath.toFile().listFiles().length).isGreaterThanOrEqualTo(filesToUpload.size()); // Copy files to S3. copyToS3(dirPath, filesToUpload, TEST_S3_BUCKET, snapshotId, s3CrtBlobFs); diff --git a/astra/src/test/java/com/slack/astra/chunk/RecoveryChunkImplTest.java b/astra/src/test/java/com/slack/astra/chunk/RecoveryChunkImplTest.java index e7eb3c219e..1df5918a1f 100644 --- a/astra/src/test/java/com/slack/astra/chunk/RecoveryChunkImplTest.java +++ b/astra/src/test/java/com/slack/astra/chunk/RecoveryChunkImplTest.java @@ -15,7 +15,7 @@ import brave.Tracing; import com.adobe.testing.s3mock.junit5.S3MockExtension; -import com.slack.astra.blobfs.s3.S3CrtBlobFs; +import com.slack.astra.blobfs.S3CrtBlobFs; import com.slack.astra.blobfs.s3.S3TestUtils; import com.slack.astra.logstore.LogMessage; import com.slack.astra.logstore.LuceneIndexStoreImpl; diff --git a/astra/src/test/java/com/slack/astra/chunkManager/CachingChunkManagerTest.java b/astra/src/test/java/com/slack/astra/chunkManager/CachingChunkManagerTest.java index d7c52e8d56..e681cb41df 100644 --- a/astra/src/test/java/com/slack/astra/chunkManager/CachingChunkManagerTest.java +++ b/astra/src/test/java/com/slack/astra/chunkManager/CachingChunkManagerTest.java @@ -1,9 +1,9 @@ package com.slack.astra.chunkManager; +import static com.slack.astra.blobfs.BlobFsUtils.copyFromS3; +import static com.slack.astra.blobfs.BlobFsUtils.copyToS3; import static com.slack.astra.chunk.ReadWriteChunk.SCHEMA_FILE_NAME; import static com.slack.astra.chunkManager.CachingChunkManager.ASTRA_NG_DYNAMIC_CHUNK_SIZES_FLAG; -import static com.slack.astra.logstore.BlobFsUtils.copyFromS3; -import static com.slack.astra.logstore.BlobFsUtils.copyToS3; import static com.slack.astra.logstore.LuceneIndexStoreImpl.COMMITS_TIMER; import static com.slack.astra.logstore.LuceneIndexStoreImpl.MESSAGES_FAILED_COUNTER; import static com.slack.astra.logstore.LuceneIndexStoreImpl.MESSAGES_RECEIVED_COUNTER; @@ -17,8 +17,7 @@ import com.adobe.testing.s3mock.junit5.S3MockExtension; import com.slack.astra.blobfs.ChunkStore; -import com.slack.astra.blobfs.LocalBlobFs; -import com.slack.astra.blobfs.s3.S3CrtBlobFs; +import com.slack.astra.blobfs.S3CrtBlobFs; import com.slack.astra.blobfs.s3.S3TestUtils; import com.slack.astra.chunk.Chunk; import com.slack.astra.chunk.ReadOnlyChunkImpl; @@ -211,11 +210,8 @@ private void initializeBlobStorageWithIndex(String snapshotId) throws Exception filesToUpload.addAll(indexCommit.getFileNames()); System.out.println(filesToUpload.size()); - LocalBlobFs localBlobFs = new LocalBlobFs(); - logStore.close(); - assertThat(localBlobFs.listFiles(dirPath.toUri(), false).length) - .isGreaterThanOrEqualTo(filesToUpload.size()); + assertThat(dirPath.toFile().listFiles().length).isGreaterThanOrEqualTo(filesToUpload.size()); // Copy files to S3. copyToS3(dirPath, filesToUpload, TEST_S3_BUCKET, snapshotId, s3CrtBlobFs); diff --git a/astra/src/test/java/com/slack/astra/chunkManager/IndexingChunkManagerTest.java b/astra/src/test/java/com/slack/astra/chunkManager/IndexingChunkManagerTest.java index 2d6b4ffe5a..a891a8e18d 100644 --- a/astra/src/test/java/com/slack/astra/chunkManager/IndexingChunkManagerTest.java +++ b/astra/src/test/java/com/slack/astra/chunkManager/IndexingChunkManagerTest.java @@ -33,7 +33,7 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import com.google.protobuf.ByteString; -import com.slack.astra.blobfs.s3.S3CrtBlobFs; +import com.slack.astra.blobfs.S3CrtBlobFs; import com.slack.astra.blobfs.s3.S3TestUtils; import com.slack.astra.chunk.Chunk; import com.slack.astra.chunk.ChunkInfo; diff --git a/astra/src/test/java/com/slack/astra/chunkManager/RecoveryChunkManagerTest.java b/astra/src/test/java/com/slack/astra/chunkManager/RecoveryChunkManagerTest.java index ed9388c163..86ade5a7b9 100644 --- a/astra/src/test/java/com/slack/astra/chunkManager/RecoveryChunkManagerTest.java +++ b/astra/src/test/java/com/slack/astra/chunkManager/RecoveryChunkManagerTest.java @@ -22,7 +22,7 @@ import brave.Tracing; import com.adobe.testing.s3mock.junit5.S3MockExtension; -import com.slack.astra.blobfs.s3.S3CrtBlobFs; +import com.slack.astra.blobfs.S3CrtBlobFs; import com.slack.astra.blobfs.s3.S3TestUtils; import com.slack.astra.chunk.ChunkInfo; import com.slack.astra.chunk.ReadWriteChunk; diff --git a/astra/src/test/java/com/slack/astra/chunkrollover/DiskOrMessageCountBasedRolloverStrategyTest.java b/astra/src/test/java/com/slack/astra/chunkrollover/DiskOrMessageCountBasedRolloverStrategyTest.java index bdabb4e735..62f7782e21 100644 --- a/astra/src/test/java/com/slack/astra/chunkrollover/DiskOrMessageCountBasedRolloverStrategyTest.java +++ b/astra/src/test/java/com/slack/astra/chunkrollover/DiskOrMessageCountBasedRolloverStrategyTest.java @@ -14,7 +14,7 @@ import com.adobe.testing.s3mock.junit5.S3MockExtension; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; -import com.slack.astra.blobfs.s3.S3CrtBlobFs; +import com.slack.astra.blobfs.S3CrtBlobFs; import com.slack.astra.blobfs.s3.S3TestUtils; import com.slack.astra.chunk.SearchContext; import com.slack.astra.chunkManager.IndexingChunkManager; diff --git a/astra/src/test/java/com/slack/astra/clusterManager/SnapshotDeletionServiceTest.java b/astra/src/test/java/com/slack/astra/clusterManager/SnapshotDeletionServiceTest.java index 698ef275a5..ef520f9809 100644 --- a/astra/src/test/java/com/slack/astra/clusterManager/SnapshotDeletionServiceTest.java +++ b/astra/src/test/java/com/slack/astra/clusterManager/SnapshotDeletionServiceTest.java @@ -1,6 +1,6 @@ package com.slack.astra.clusterManager; -import static com.slack.astra.logstore.BlobFsUtils.createURI; +import static com.slack.astra.blobfs.BlobFsUtils.createURI; import static com.slack.astra.proto.metadata.Metadata.IndexType.LOGS_LUCENE9; import static com.slack.astra.server.AstraConfig.DEFAULT_START_STOP_DURATION; import static org.assertj.core.api.Assertions.assertThat; @@ -20,7 +20,7 @@ import brave.Tracing; import com.adobe.testing.s3mock.junit5.S3MockExtension; -import com.slack.astra.blobfs.s3.S3CrtBlobFs; +import com.slack.astra.blobfs.S3CrtBlobFs; import com.slack.astra.blobfs.s3.S3TestUtils; import com.slack.astra.metadata.core.CuratorBuilder; import com.slack.astra.metadata.replica.ReplicaMetadata; diff --git a/astra/src/test/java/com/slack/astra/logstore/LuceneIndexStoreImplTest.java b/astra/src/test/java/com/slack/astra/logstore/LuceneIndexStoreImplTest.java index 0658a4250a..ca79395a9b 100644 --- a/astra/src/test/java/com/slack/astra/logstore/LuceneIndexStoreImplTest.java +++ b/astra/src/test/java/com/slack/astra/logstore/LuceneIndexStoreImplTest.java @@ -1,9 +1,8 @@ package com.slack.astra.logstore; -import static com.slack.astra.logstore.BlobFsUtils.DELIMITER; -import static com.slack.astra.logstore.BlobFsUtils.copyFromS3; -import static com.slack.astra.logstore.BlobFsUtils.copyToLocalPath; -import static com.slack.astra.logstore.BlobFsUtils.copyToS3; +import static com.slack.astra.blobfs.BlobFsUtils.DELIMITER; +import static com.slack.astra.blobfs.BlobFsUtils.copyFromS3; +import static com.slack.astra.blobfs.BlobFsUtils.copyToS3; import static com.slack.astra.logstore.LuceneIndexStoreImpl.COMMITS_TIMER; import static com.slack.astra.logstore.LuceneIndexStoreImpl.MESSAGES_FAILED_COUNTER; import static com.slack.astra.logstore.LuceneIndexStoreImpl.MESSAGES_RECEIVED_COUNTER; @@ -19,8 +18,7 @@ import brave.Tracing; import com.adobe.testing.s3mock.junit5.S3MockExtension; import com.google.protobuf.ByteString; -import com.slack.astra.blobfs.LocalBlobFs; -import com.slack.astra.blobfs.s3.S3CrtBlobFs; +import com.slack.astra.blobfs.S3CrtBlobFs; import com.slack.astra.blobfs.s3.S3TestUtils; import com.slack.astra.logstore.LogMessage.ReservedField; import com.slack.astra.logstore.schema.SchemaAwareLogDocumentBuilderImpl; @@ -51,6 +49,7 @@ import org.junit.jupiter.api.io.TempDir; import software.amazon.awssdk.services.s3.S3AsyncClient; import software.amazon.awssdk.services.s3.model.CreateBucketRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectRequest; import software.amazon.awssdk.services.s3.model.HeadObjectResponse; @SuppressWarnings("unused") @@ -363,14 +362,12 @@ private void testS3Snapshot(String bucket, String prefix) throws Exception { Path dirPath = logStore.getDirectory().getDirectory().toAbsolutePath(); IndexCommit indexCommit = logStore.getIndexCommit(); Collection activeFiles = indexCommit.getFileNames(); - LocalBlobFs localBlobFs = new LocalBlobFs(); logStore.close(); strictLogStore.logSearcher.close(); strictLogStore.logStore = null; strictLogStore.logSearcher = null; - assertThat(localBlobFs.listFiles(dirPath.toUri(), false).length) - .isGreaterThanOrEqualTo(activeFiles.size()); + assertThat(dirPath.toFile().listFiles().length).isGreaterThanOrEqualTo(activeFiles.size()); // create an S3 client S3AsyncClient s3AsyncClient = @@ -386,11 +383,13 @@ private void testS3Snapshot(String bucket, String prefix) throws Exception { HeadObjectResponse headObjectResponse = s3AsyncClient .headObject( - S3TestUtils.getHeadObjectRequest( - bucket, - prefix != null && !prefix.isEmpty() - ? prefix + DELIMITER + fileName - : fileName)) + HeadObjectRequest.builder() + .bucket(bucket) + .key( + prefix != null && !prefix.isEmpty() + ? prefix + DELIMITER + fileName + : fileName) + .build()) .get(); assertThat(headObjectResponse.contentLength()).isEqualTo(fileToCopy.length()); } @@ -430,46 +429,6 @@ private void testS3Snapshot(String bucket, String prefix) throws Exception { newSearcher.close(); s3CrtBlobFs.close(); } - - @Test - public void testLocalSnapshot() throws IOException { - LuceneIndexStoreImpl logStore = strictLogStore.logStore; - addMessages(logStore, 1, 100, true); - Collection results = - findAllMessages( - strictLogStore.logSearcher, MessageUtil.TEST_DATASET_NAME, "Message1", 100); - assertThat(results.size()).isEqualTo(1); - assertThat(getCount(MESSAGES_RECEIVED_COUNTER, strictLogStore.metricsRegistry)) - .isEqualTo(100); - assertThat(getCount(MESSAGES_FAILED_COUNTER, strictLogStore.metricsRegistry)).isEqualTo(0); - assertThat(getTimerCount(REFRESHES_TIMER, strictLogStore.metricsRegistry)).isEqualTo(1); - assertThat(getTimerCount(COMMITS_TIMER, strictLogStore.metricsRegistry)).isEqualTo(1); - - Path dirPath = logStore.getDirectory().getDirectory().toAbsolutePath(); - IndexCommit indexCommit = logStore.getIndexCommit(); - Collection activeFiles = indexCommit.getFileNames(); - LocalBlobFs blobFs = new LocalBlobFs(); - logStore.close(); - strictLogStore.logSearcher.close(); - strictLogStore.logStore = null; - strictLogStore.logSearcher = null; - - assertThat(blobFs.listFiles(dirPath.toUri(), false).length) - .isGreaterThanOrEqualTo(activeFiles.size()); - - copyToLocalPath(dirPath, activeFiles, tmpPath.toAbsolutePath(), blobFs); - - LogIndexSearcherImpl newSearcher = - new LogIndexSearcherImpl( - LogIndexSearcherImpl.searcherManagerFromPath(tmpPath.toAbsolutePath()), - logStore.getSchema()); - - Collection newResults = - findAllMessages(newSearcher, MessageUtil.TEST_DATASET_NAME, "Message1", 100); - assertThat(newResults.size()).isEqualTo(1); - logStore.releaseIndexCommit(indexCommit); - newSearcher.close(); - } } @Nested diff --git a/astra/src/test/java/com/slack/astra/recovery/RecoveryServiceTest.java b/astra/src/test/java/com/slack/astra/recovery/RecoveryServiceTest.java index c15e5adbcc..5c065f3a6e 100644 --- a/astra/src/test/java/com/slack/astra/recovery/RecoveryServiceTest.java +++ b/astra/src/test/java/com/slack/astra/recovery/RecoveryServiceTest.java @@ -25,9 +25,9 @@ import com.adobe.testing.s3mock.junit5.S3MockExtension; import com.google.common.collect.Maps; import com.slack.astra.blobfs.BlobFs; -import com.slack.astra.blobfs.s3.S3CrtBlobFs; +import com.slack.astra.blobfs.BlobFsUtils; +import com.slack.astra.blobfs.S3CrtBlobFs; import com.slack.astra.blobfs.s3.S3TestUtils; -import com.slack.astra.logstore.BlobFsUtils; import com.slack.astra.metadata.core.AstraMetadataTestUtils; import com.slack.astra.metadata.core.CuratorBuilder; import com.slack.astra.metadata.recovery.RecoveryNodeMetadata; diff --git a/astra/src/test/java/com/slack/astra/testlib/ChunkManagerUtil.java b/astra/src/test/java/com/slack/astra/testlib/ChunkManagerUtil.java index 57e06262ed..ff7be2f8ab 100644 --- a/astra/src/test/java/com/slack/astra/testlib/ChunkManagerUtil.java +++ b/astra/src/test/java/com/slack/astra/testlib/ChunkManagerUtil.java @@ -5,7 +5,7 @@ import com.adobe.testing.s3mock.junit5.S3MockExtension; import com.google.common.io.Files; import com.google.common.util.concurrent.MoreExecutors; -import com.slack.astra.blobfs.s3.S3CrtBlobFs; +import com.slack.astra.blobfs.S3CrtBlobFs; import com.slack.astra.blobfs.s3.S3TestUtils; import com.slack.astra.chunk.SearchContext; import com.slack.astra.chunkManager.IndexingChunkManager; From 908e769dee3013d1b418305b3394dbbd2acfdb1b Mon Sep 17 00:00:00 2001 From: Bryan Burkholder Date: Tue, 13 Aug 2024 10:04:25 -0700 Subject: [PATCH 3/7] Add listFiles, delete to chunkStore implementation, remove from blobfs --- .../java/com/slack/astra/blobfs/BlobFs.java | 13 -- .../com/slack/astra/blobfs/ChunkStore.java | 29 +++ .../com/slack/astra/blobfs/S3CrtBlobFs.java | 91 -------- .../SnapshotDeletionService.java | 42 +--- .../metadata/snapshot/SnapshotMetadata.java | 5 +- .../java/com/slack/astra/server/Astra.java | 6 +- .../slack/astra/blobfs/ChunkStoreTest.java | 31 ++- .../astra/blobfs/s3/S3CrtBlobFsTest.java | 59 ----- .../SnapshotDeletionServiceTest.java | 219 +++++++++--------- 9 files changed, 188 insertions(+), 307 deletions(-) diff --git a/astra/src/main/java/com/slack/astra/blobfs/BlobFs.java b/astra/src/main/java/com/slack/astra/blobfs/BlobFs.java index d5e711d7ba..b56bb92f8c 100644 --- a/astra/src/main/java/com/slack/astra/blobfs/BlobFs.java +++ b/astra/src/main/java/com/slack/astra/blobfs/BlobFs.java @@ -22,19 +22,6 @@ @Deprecated public abstract class BlobFs implements Closeable, Serializable { - /** - * Deletes the file at the location provided. If the segmentUri is a directory, it will delete the - * entire directory. - * - * @param segmentUri URI of the segment - * @param forceDelete true if we want the uri and any sub-uris to always be deleted, false if we - * want delete to fail when we want to delete a directory and that directory is not empty - * @return true if delete is successful else false - * @throws IOException on IO failure, e.g Uri is not present or not valid - */ - @Deprecated - public abstract boolean delete(URI segmentUri, boolean forceDelete) throws IOException; - /** * Checks whether the file or directory at the provided location exists. * diff --git a/astra/src/main/java/com/slack/astra/blobfs/ChunkStore.java b/astra/src/main/java/com/slack/astra/blobfs/ChunkStore.java index 806fbfdcd7..bfffa74a91 100644 --- a/astra/src/main/java/com/slack/astra/blobfs/ChunkStore.java +++ b/astra/src/main/java/com/slack/astra/blobfs/ChunkStore.java @@ -3,6 +3,7 @@ import static software.amazon.awssdk.services.s3.model.ListObjectsV2Request.builder; import java.nio.file.Path; +import java.util.ArrayList; import java.util.List; import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicBoolean; @@ -62,6 +63,34 @@ public Path download(String chunkId, Path destination) { } } + public List listFiles(String chunkId) { + ListObjectsV2Request listRequest = builder().bucket(bucketName).prefix(chunkId).build(); + ListObjectsV2Publisher asyncPaginatedListResponse = + s3AsyncClient.listObjectsV2Paginator(listRequest); + + List filesList = new ArrayList<>(); + try { + asyncPaginatedListResponse + .subscribe( + listResponse -> { + listResponse + .contents() + .forEach( + s3Object -> { + filesList.add(String.format("s3://%s/%s", bucketName, s3Object.key())); + }); + }) + .get(); + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException(e); + } + return filesList; + } + + public String getRemotePath(String chunkId) { + return String.format("s3://%s/%s", bucketName, chunkId); + } + public boolean delete(String chunkId) { ListObjectsV2Request listRequest = builder().bucket(bucketName).prefix(chunkId).build(); ListObjectsV2Publisher asyncPaginatedListResponse = diff --git a/astra/src/main/java/com/slack/astra/blobfs/S3CrtBlobFs.java b/astra/src/main/java/com/slack/astra/blobfs/S3CrtBlobFs.java index 5922758329..d4f3ec712d 100644 --- a/astra/src/main/java/com/slack/astra/blobfs/S3CrtBlobFs.java +++ b/astra/src/main/java/com/slack/astra/blobfs/S3CrtBlobFs.java @@ -24,8 +24,6 @@ import software.amazon.awssdk.services.s3.crt.S3CrtHttpConfiguration; import software.amazon.awssdk.services.s3.crt.S3CrtProxyConfiguration; import software.amazon.awssdk.services.s3.crt.S3CrtRetryConfiguration; -import software.amazon.awssdk.services.s3.model.DeleteObjectRequest; -import software.amazon.awssdk.services.s3.model.DeleteObjectResponse; import software.amazon.awssdk.services.s3.model.GetObjectRequest; import software.amazon.awssdk.services.s3.model.HeadObjectRequest; import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; @@ -180,95 +178,6 @@ private boolean existsFile(URI uri) throws IOException { } } - private boolean isEmptyDirectory(URI uri) throws IOException { - if (!isDirectory(uri)) { - return false; - } - String prefix = normalizeToDirectoryPrefix(uri); - boolean isEmpty = true; - ListObjectsV2Response listObjectsV2Response; - ListObjectsV2Request.Builder listObjectsV2RequestBuilder = - ListObjectsV2Request.builder().bucket(uri.getHost()); - - if (!prefix.equals(DELIMITER)) { - listObjectsV2RequestBuilder = listObjectsV2RequestBuilder.prefix(prefix); - } - - ListObjectsV2Request listObjectsV2Request = listObjectsV2RequestBuilder.build(); - try { - listObjectsV2Response = s3AsyncClient.listObjectsV2(listObjectsV2Request).get(); - } catch (InterruptedException | ExecutionException e) { - throw new IOException(e); - } - - for (S3Object s3Object : listObjectsV2Response.contents()) { - if (s3Object.key().equals(prefix)) { - continue; - } else { - isEmpty = false; - break; - } - } - return isEmpty; - } - - @Override - public boolean delete(URI segmentUri, boolean forceDelete) throws IOException { - LOG.debug("Deleting uri {} force {}", segmentUri, forceDelete); - try { - if (isDirectory(segmentUri)) { - if (!forceDelete) { - Preconditions.checkState( - isEmptyDirectory(segmentUri), - "ForceDelete flag is not set and directory '%s' is not empty", - segmentUri); - } - String prefix = normalizeToDirectoryPrefix(segmentUri); - ListObjectsV2Response listObjectsV2Response; - ListObjectsV2Request.Builder listObjectsV2RequestBuilder = - ListObjectsV2Request.builder().bucket(segmentUri.getHost()); - - if (prefix.equals(DELIMITER)) { - ListObjectsV2Request listObjectsV2Request = listObjectsV2RequestBuilder.build(); - listObjectsV2Response = s3AsyncClient.listObjectsV2(listObjectsV2Request).get(); - } else { - ListObjectsV2Request listObjectsV2Request = - listObjectsV2RequestBuilder.prefix(prefix).build(); - listObjectsV2Response = s3AsyncClient.listObjectsV2(listObjectsV2Request).get(); - } - boolean deleteSucceeded = true; - for (S3Object s3Object : listObjectsV2Response.contents()) { - DeleteObjectRequest deleteObjectRequest = - DeleteObjectRequest.builder() - .bucket(segmentUri.getHost()) - .key(s3Object.key()) - .build(); - - DeleteObjectResponse deleteObjectResponse = - s3AsyncClient.deleteObject(deleteObjectRequest).get(); - - deleteSucceeded &= deleteObjectResponse.sdkHttpResponse().isSuccessful(); - } - return deleteSucceeded; - } else { - String prefix = sanitizePath(segmentUri.getPath()); - DeleteObjectRequest deleteObjectRequest = - DeleteObjectRequest.builder().bucket(segmentUri.getHost()).key(prefix).build(); - - DeleteObjectResponse deleteObjectResponse = - s3AsyncClient.deleteObject(deleteObjectRequest).get(); - - return deleteObjectResponse.sdkHttpResponse().isSuccessful(); - } - } catch (NoSuchKeyException e) { - return false; - } catch (S3Exception e) { - throw e; - } catch (Exception e) { - throw new IOException(e); - } - } - @Override public boolean exists(URI fileUri) throws IOException { try { diff --git a/astra/src/main/java/com/slack/astra/clusterManager/SnapshotDeletionService.java b/astra/src/main/java/com/slack/astra/clusterManager/SnapshotDeletionService.java index 7e05973c58..98657b7e38 100644 --- a/astra/src/main/java/com/slack/astra/clusterManager/SnapshotDeletionService.java +++ b/astra/src/main/java/com/slack/astra/clusterManager/SnapshotDeletionService.java @@ -12,7 +12,7 @@ import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.RateLimiter; import com.google.common.util.concurrent.ThreadFactoryBuilder; -import com.slack.astra.blobfs.BlobFs; +import com.slack.astra.blobfs.ChunkStore; import com.slack.astra.metadata.replica.ReplicaMetadataStore; import com.slack.astra.metadata.snapshot.SnapshotMetadata; import com.slack.astra.metadata.snapshot.SnapshotMetadataStore; @@ -20,8 +20,6 @@ import io.micrometer.core.instrument.Counter; import io.micrometer.core.instrument.MeterRegistry; import io.micrometer.core.instrument.Timer; -import java.io.IOException; -import java.net.URI; import java.time.Instant; import java.time.temporal.ChronoUnit; import java.util.List; @@ -54,7 +52,7 @@ public class SnapshotDeletionService extends AbstractScheduledService { private final ReplicaMetadataStore replicaMetadataStore; private final SnapshotMetadataStore snapshotMetadataStore; private final MeterRegistry meterRegistry; - private final BlobFs s3BlobFs; + private final ChunkStore chunkStore; @VisibleForTesting protected int futuresListTimeoutSecs; @@ -79,7 +77,7 @@ public class SnapshotDeletionService extends AbstractScheduledService { public SnapshotDeletionService( ReplicaMetadataStore replicaMetadataStore, SnapshotMetadataStore snapshotMetadataStore, - BlobFs s3BlobFs, + ChunkStore chunkStore, AstraConfigs.ManagerConfig managerConfig, MeterRegistry meterRegistry) { @@ -92,7 +90,7 @@ public SnapshotDeletionService( this.managerConfig = managerConfig; this.replicaMetadataStore = replicaMetadataStore; this.snapshotMetadataStore = snapshotMetadataStore; - this.s3BlobFs = s3BlobFs; + this.chunkStore = chunkStore; this.meterRegistry = meterRegistry; // This functions as the overall "timeout" for deleteExpiredSnapshotsWithoutReplicas, and should @@ -185,35 +183,17 @@ protected int deleteExpiredSnapshotsWithoutReplicas() { () -> { try { // These futures are rate-limited so that we can more evenly - // distribute - // the load to the downstream services (metadata, s3). There is no - // urgency to complete the deletes, so limiting the maximum rate - // allows - // us to avoid unnecessary spikes. + // distribute the load to the downstream services (metadata, s3). + // There is no urgency to complete the deletes, so limiting the + // maximum rate allows us to avoid unnecessary spikes. rateLimiter.acquire(); // First try to delete the object from S3, then delete from metadata - // store. If for some reason the object delete fails, it will leave - // the - // metadata and try again on the next run. - URI snapshotUri = URI.create(snapshotMetadata.snapshotPath); + // store. If for some reason the object delete fails, it will throw + // an exception, leaving the metadata and try again on the next run. LOG.debug("Starting delete of snapshot {}", snapshotMetadata); - if (s3BlobFs.exists(snapshotUri)) { - // Ensure that the file exists before attempting to delete, in case - // the previous run successfully deleted the object but failed the - // metadata delete. Otherwise, this would be expected to perpetually - // fail deleting a non-existing file. - if (s3BlobFs.delete(snapshotUri, true)) { - snapshotMetadataStore.deleteSync(snapshotMetadata); - } else { - throw new IOException( - String.format( - "Failed to delete '%s' from object store", - snapshotMetadata.snapshotPath)); - } - } else { - snapshotMetadataStore.deleteSync(snapshotMetadata); - } + chunkStore.delete(snapshotMetadata.snapshotId); + snapshotMetadataStore.deleteSync(snapshotMetadata); } catch (Exception e) { LOG.error("Exception deleting snapshot", e); throw e; diff --git a/astra/src/main/java/com/slack/astra/metadata/snapshot/SnapshotMetadata.java b/astra/src/main/java/com/slack/astra/metadata/snapshot/SnapshotMetadata.java index 0f0dd25873..89b29c272a 100644 --- a/astra/src/main/java/com/slack/astra/metadata/snapshot/SnapshotMetadata.java +++ b/astra/src/main/java/com/slack/astra/metadata/snapshot/SnapshotMetadata.java @@ -29,7 +29,10 @@ public static boolean isLive(SnapshotMetadata snapshotMetadata) { return snapshotMetadata.snapshotPath.equals(LIVE_SNAPSHOT_PATH); } - public final String snapshotPath; + // snapshot path is deprecated in favor of storing these to a well-known path + // @see ChunkStore.getPath + @Deprecated public final String snapshotPath; + public final String snapshotId; public final long startTimeEpochMs; public final long endTimeEpochMs; diff --git a/astra/src/main/java/com/slack/astra/server/Astra.java b/astra/src/main/java/com/slack/astra/server/Astra.java index efcd37a4a1..6acf8dc697 100644 --- a/astra/src/main/java/com/slack/astra/server/Astra.java +++ b/astra/src/main/java/com/slack/astra/server/Astra.java @@ -335,7 +335,11 @@ private static Set getServices( SnapshotDeletionService snapshotDeletionService = new SnapshotDeletionService( - replicaMetadataStore, snapshotMetadataStore, blobFs, managerConfig, meterRegistry); + replicaMetadataStore, + snapshotMetadataStore, + chunkStore, + managerConfig, + meterRegistry); services.add(snapshotDeletionService); CacheNodeMetadataStore cacheNodeMetadataStore = new CacheNodeMetadataStore(curatorFramework); diff --git a/astra/src/test/java/com/slack/astra/blobfs/ChunkStoreTest.java b/astra/src/test/java/com/slack/astra/blobfs/ChunkStoreTest.java index d3125fed9e..789b4972b4 100644 --- a/astra/src/test/java/com/slack/astra/blobfs/ChunkStoreTest.java +++ b/astra/src/test/java/com/slack/astra/blobfs/ChunkStoreTest.java @@ -1,6 +1,6 @@ package com.slack.astra.blobfs; -import static org.assertj.core.api.AssertionsForClassTypes.assertThat; +import static org.assertj.core.api.Assertions.assertThat; import com.adobe.testing.s3mock.junit5.S3MockExtension; import com.slack.astra.blobfs.s3.S3TestUtils; @@ -9,6 +9,7 @@ import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; +import java.util.List; import java.util.Objects; import java.util.UUID; import java.util.concurrent.ExecutionException; @@ -103,4 +104,32 @@ void testDeleteDoesNotExist() { boolean deleted = chunkStore.delete(UUID.randomUUID().toString()); assertThat(deleted).isFalse(); } + + @Test + void testListFiles() throws IOException { + ChunkStore chunkStore = new ChunkStore(s3Client, TEST_BUCKET); + String chunkId = UUID.randomUUID().toString(); + + assertThat(chunkStore.listFiles(chunkId).size()).isEqualTo(0); + + Path directoryUpload = Files.createTempDirectory(""); + Path foo = Files.createTempFile(directoryUpload, "", ""); + try (FileWriter fileWriter = new FileWriter(foo.toFile())) { + fileWriter.write("Example test 1"); + } + Path bar = Files.createTempFile(directoryUpload, "", ""); + try (FileWriter fileWriter = new FileWriter(bar.toFile())) { + fileWriter.write("Example test 2"); + } + chunkStore.upload(chunkId, directoryUpload); + + assertThat(chunkStore.listFiles(chunkId).size()).isEqualTo(2); + assertThat(chunkStore.listFiles(chunkId)) + .containsExactlyInAnyOrderElementsOf( + List.of( + String.format( + "%s/%s", chunkStore.getRemotePath(chunkId), foo.getFileName().toString()), + String.format( + "%s/%s", chunkStore.getRemotePath(chunkId), bar.getFileName().toString()))); + } } diff --git a/astra/src/test/java/com/slack/astra/blobfs/s3/S3CrtBlobFsTest.java b/astra/src/test/java/com/slack/astra/blobfs/s3/S3CrtBlobFsTest.java index 1a6cd386c6..94d34d1f0e 100644 --- a/astra/src/test/java/com/slack/astra/blobfs/s3/S3CrtBlobFsTest.java +++ b/astra/src/test/java/com/slack/astra/blobfs/s3/S3CrtBlobFsTest.java @@ -25,10 +25,7 @@ import software.amazon.awssdk.services.s3.model.CreateBucketRequest; import software.amazon.awssdk.services.s3.model.HeadObjectRequest; import software.amazon.awssdk.services.s3.model.HeadObjectResponse; -import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; -import software.amazon.awssdk.services.s3.model.ListObjectsV2Response; import software.amazon.awssdk.services.s3.model.PutObjectRequest; -import software.amazon.awssdk.services.s3.model.S3Object; public class S3CrtBlobFsTest { @RegisterExtension @@ -141,62 +138,6 @@ public void testListFilesInFolderRecursive() throws Exception { assertTrue(Arrays.equals(expectedResultList.toArray(), actualFiles)); } - @Test - public void testDeleteFile() throws Exception { - String[] originalFiles = new String[] {"a-delete.txt", "b-delete.txt", "c-delete.txt"}; - String fileToDelete = "a-delete.txt"; - - List expectedResultList = new ArrayList<>(); - for (String fileName : originalFiles) { - createEmptyFile("", fileName); - if (!fileName.equals(fileToDelete)) { - expectedResultList.add(fileName); - } - } - - s3BlobFs.delete(URI.create(String.format(FILE_FORMAT, SCHEME, bucket, fileToDelete)), false); - - ListObjectsV2Response listObjectsV2Response = - s3Client.listObjectsV2(ListObjectsV2Request.builder().bucket(bucket).build()).get(); - String[] actualResponse = - listObjectsV2Response.contents().stream() - .map(S3Object::key) - .filter(x -> x.contains("delete")) - .toArray(String[]::new); - - assertEquals(actualResponse.length, 2); - assertTrue(Arrays.equals(actualResponse, expectedResultList.toArray())); - } - - @Test - public void testDeleteFolder() throws Exception { - String[] originalFiles = new String[] {"a-delete-2.txt", "b-delete-2.txt", "c-delete-2.txt"}; - String folderName = "my-files"; - - for (String fileName : originalFiles) { - createEmptyFile(folderName, fileName); - } - - s3BlobFs.delete(URI.create(String.format(FILE_FORMAT, SCHEME, bucket, folderName)), true); - - // await ignoreExceptions is a workaround due to // - // https://github.com/aws/aws-sdk-java-v2/issues/3658 - await() - .ignoreExceptions() - .until( - () -> - s3Client - .listObjectsV2(ListObjectsV2Request.builder().bucket(bucket).build()) - .get() - .contents() - .stream() - .map(S3Object::key) - .filter(x -> x.contains("delete-2")) - .toArray(String[]::new) - .length - == 0); - } - @Test public void testIsDirectory() throws Exception { String[] originalFiles = new String[] {"a-dir.txt", "b-dir.txt", "c-dir.txt"}; diff --git a/astra/src/test/java/com/slack/astra/clusterManager/SnapshotDeletionServiceTest.java b/astra/src/test/java/com/slack/astra/clusterManager/SnapshotDeletionServiceTest.java index ef520f9809..f0c3020d2d 100644 --- a/astra/src/test/java/com/slack/astra/clusterManager/SnapshotDeletionServiceTest.java +++ b/astra/src/test/java/com/slack/astra/clusterManager/SnapshotDeletionServiceTest.java @@ -1,13 +1,11 @@ package com.slack.astra.clusterManager; -import static com.slack.astra.blobfs.BlobFsUtils.createURI; import static com.slack.astra.proto.metadata.Metadata.IndexType.LOGS_LUCENE9; import static com.slack.astra.server.AstraConfig.DEFAULT_START_STOP_DURATION; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.AssertionsForClassTypes.assertThatExceptionOfType; import static org.awaitility.Awaitility.await; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doCallRealMethod; import static org.mockito.Mockito.doReturn; @@ -20,7 +18,7 @@ import brave.Tracing; import com.adobe.testing.s3mock.junit5.S3MockExtension; -import com.slack.astra.blobfs.S3CrtBlobFs; +import com.slack.astra.blobfs.ChunkStore; import com.slack.astra.blobfs.s3.S3TestUtils; import com.slack.astra.metadata.core.CuratorBuilder; import com.slack.astra.metadata.replica.ReplicaMetadata; @@ -32,11 +30,11 @@ import io.micrometer.core.instrument.MeterRegistry; import io.micrometer.core.instrument.simple.SimpleMeterRegistry; import java.io.IOException; -import java.net.URI; import java.nio.file.Files; import java.nio.file.Path; import java.time.Instant; import java.time.temporal.ChronoUnit; +import java.util.List; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; @@ -69,7 +67,7 @@ public class SnapshotDeletionServiceTest { private SnapshotMetadataStore snapshotMetadataStore; private ReplicaMetadataStore replicaMetadataStore; private S3AsyncClient s3AsyncClient; - private S3CrtBlobFs s3CrtBlobFs; + private ChunkStore chunkStore; @BeforeEach public void setup() throws Exception { @@ -91,7 +89,7 @@ public void setup() throws Exception { replicaMetadataStore = spy(new ReplicaMetadataStore(curatorFramework)); s3AsyncClient = S3TestUtils.createS3CrtClient(S3_MOCK_EXTENSION.getServiceEndpoint()); - s3CrtBlobFs = spy(new S3CrtBlobFs(s3AsyncClient)); + chunkStore = spy(new ChunkStore(s3AsyncClient, S3_TEST_BUCKET)); } @AfterEach @@ -131,7 +129,7 @@ public void shouldThrowOnInvalidSnapshotLifespan() { new SnapshotDeletionService( replicaMetadataStore, snapshotMetadataStore, - s3CrtBlobFs, + chunkStore, managerConfig, meterRegistry)); } @@ -156,14 +154,15 @@ public void shouldDeleteExpiredSnapshotNoReplicas() throws Exception { .setScheduleInitialDelayMins(0) .build(); - Path file = Files.createTempFile("", ""); - URI path = createURI(S3_TEST_BUCKET, "foo", "bar"); - s3CrtBlobFs.copyFromLocalFile(file.toFile(), path); + Path directory = Files.createTempDirectory(""); + Files.createTempFile(directory, "", ""); + String chunkId = UUID.randomUUID().toString(); + chunkStore.upload(chunkId, directory); SnapshotMetadata snapshotMetadata = new SnapshotMetadata( - UUID.randomUUID().toString(), - path.toString(), + chunkId, + chunkStore.getRemotePath(chunkId), Instant.now().minus(11000, ChronoUnit.MINUTES).toEpochMilli(), Instant.now().minus(10900, ChronoUnit.MINUTES).toEpochMilli(), 0, @@ -175,13 +174,13 @@ public void shouldDeleteExpiredSnapshotNoReplicas() throws Exception { SnapshotDeletionService snapshotDeletionService = new SnapshotDeletionService( - replicaMetadataStore, snapshotMetadataStore, s3CrtBlobFs, managerConfig, meterRegistry); + replicaMetadataStore, snapshotMetadataStore, chunkStore, managerConfig, meterRegistry); int deletes = snapshotDeletionService.deleteExpiredSnapshotsWithoutReplicas(); assertThat(deletes).isEqualTo(1); - await().until(() -> snapshotMetadataStore.listSync().size() == 0); - verify(s3CrtBlobFs, times(1)).delete(eq(path), eq(true)); + await().until(() -> snapshotMetadataStore.listSync().isEmpty()); + verify(chunkStore, times(1)).delete(eq(chunkId)); assertThat(MetricsUtil.getCount(SnapshotDeletionService.SNAPSHOT_DELETE_SUCCESS, meterRegistry)) .isEqualTo(1); @@ -212,15 +211,15 @@ public void shouldNotDeleteExpiredSnapshotWithReplicas() throws Exception { .setScheduleInitialDelayMins(0) .build(); - Path file = Files.createTempFile("", ""); - URI filePath = createURI(S3_TEST_BUCKET, "foo", "bar"); - URI directoryPath = URI.create(String.format("s3://%s/%s", S3_TEST_BUCKET, "foo")); - s3CrtBlobFs.copyFromLocalFile(file.toFile(), filePath); + Path directory = Files.createTempDirectory(""); + Files.createTempFile(directory, "", ""); + String chunkId = UUID.randomUUID().toString(); + chunkStore.upload(chunkId, directory); SnapshotMetadata snapshotMetadata = new SnapshotMetadata( - UUID.randomUUID().toString(), - directoryPath.toString(), + chunkId, + chunkStore.getRemotePath(chunkId), Instant.now().minus(11000, ChronoUnit.MINUTES).toEpochMilli(), Instant.now().minus(10900, ChronoUnit.MINUTES).toEpochMilli(), 0, @@ -245,17 +244,18 @@ public void shouldNotDeleteExpiredSnapshotWithReplicas() throws Exception { SnapshotDeletionService snapshotDeletionService = new SnapshotDeletionService( - replicaMetadataStore, snapshotMetadataStore, s3CrtBlobFs, managerConfig, meterRegistry); - String[] s3CrtBlobFsFiles = s3CrtBlobFs.listFiles(directoryPath, true); - assertThat(s3CrtBlobFsFiles.length).isNotEqualTo(0); + replicaMetadataStore, snapshotMetadataStore, chunkStore, managerConfig, meterRegistry); + + List s3CrtBlobFsFiles = chunkStore.listFiles(chunkId); + assertThat(s3CrtBlobFsFiles.size()).isNotEqualTo(0); int deletes = snapshotDeletionService.deleteExpiredSnapshotsWithoutReplicas(); assertThat(deletes).isEqualTo(0); assertThat(snapshotMetadataStore.listSync()).containsExactlyInAnyOrder(snapshotMetadata); assertThat(replicaMetadataStore.listSync()).containsExactlyInAnyOrder(replicaMetadata); - verify(s3CrtBlobFs, times(0)).delete(any(), anyBoolean()); - assertThat(s3CrtBlobFs.listFiles(directoryPath, true)).isEqualTo(s3CrtBlobFsFiles); + verify(chunkStore, times(0)).delete(any()); + assertThat(chunkStore.listFiles(chunkId)).isEqualTo(s3CrtBlobFsFiles); assertThat(MetricsUtil.getCount(SnapshotDeletionService.SNAPSHOT_DELETE_SUCCESS, meterRegistry)) .isEqualTo(0); @@ -288,14 +288,14 @@ public void shouldHandleNoReplicasNoSnapshots() throws IOException { SnapshotDeletionService snapshotDeletionService = new SnapshotDeletionService( - replicaMetadataStore, snapshotMetadataStore, s3CrtBlobFs, managerConfig, meterRegistry); + replicaMetadataStore, snapshotMetadataStore, chunkStore, managerConfig, meterRegistry); int deletes = snapshotDeletionService.deleteExpiredSnapshotsWithoutReplicas(); assertThat(deletes).isEqualTo(0); assertThat(snapshotMetadataStore.listSync().size()).isEqualTo(0); assertThat(replicaMetadataStore.listSync().size()).isEqualTo(0); - verify(s3CrtBlobFs, times(0)).delete(any(), anyBoolean()); + verify(chunkStore, times(0)).delete(any()); assertThat(MetricsUtil.getCount(SnapshotDeletionService.SNAPSHOT_DELETE_SUCCESS, meterRegistry)) .isEqualTo(0); @@ -326,15 +326,15 @@ public void shouldHandleNoReplicasUnexpiredSnapshots() throws Exception { .setScheduleInitialDelayMins(0) .build(); - Path file = Files.createTempFile("", ""); - URI filePath = createURI(S3_TEST_BUCKET, "foo", "bar"); - URI directoryPath = URI.create(String.format("s3://%s/%s", S3_TEST_BUCKET, "foo")); - s3CrtBlobFs.copyFromLocalFile(file.toFile(), filePath); + Path directory = Files.createTempDirectory(""); + Files.createTempFile(directory, "", ""); + String chunkId = UUID.randomUUID().toString(); + chunkStore.upload(chunkId, directory); SnapshotMetadata snapshotMetadata = new SnapshotMetadata( - UUID.randomUUID().toString(), - directoryPath.toString(), + chunkId, + chunkStore.getRemotePath(chunkId), Instant.now().minus(11000, ChronoUnit.MINUTES).toEpochMilli(), Instant.now().minus(500, ChronoUnit.MINUTES).toEpochMilli(), 0, @@ -343,19 +343,19 @@ public void shouldHandleNoReplicasUnexpiredSnapshots() throws Exception { 0); snapshotMetadataStore.createAsync(snapshotMetadata); await().until(() -> snapshotMetadataStore.listSync().size() == 1); - String[] s3CrtBlobFsFiles = s3CrtBlobFs.listFiles(directoryPath, true); - assertThat(s3CrtBlobFsFiles.length).isNotEqualTo(0); + List s3CrtBlobFsFiles = chunkStore.listFiles(chunkId); + assertThat(s3CrtBlobFsFiles.size()).isNotEqualTo(0); SnapshotDeletionService snapshotDeletionService = new SnapshotDeletionService( - replicaMetadataStore, snapshotMetadataStore, s3CrtBlobFs, managerConfig, meterRegistry); + replicaMetadataStore, snapshotMetadataStore, chunkStore, managerConfig, meterRegistry); int deletes = snapshotDeletionService.deleteExpiredSnapshotsWithoutReplicas(); assertThat(deletes).isEqualTo(0); assertThat(snapshotMetadataStore.listSync()).containsExactlyInAnyOrder(snapshotMetadata); - verify(s3CrtBlobFs, times(0)).delete(any(), anyBoolean()); - assertThat(s3CrtBlobFs.listFiles(directoryPath, true)).isEqualTo(s3CrtBlobFsFiles); + verify(chunkStore, times(0)).delete(any()); + assertThat(chunkStore.listFiles(chunkId)).isEqualTo(s3CrtBlobFsFiles); assertThat(MetricsUtil.getCount(SnapshotDeletionService.SNAPSHOT_DELETE_SUCCESS, meterRegistry)) .isEqualTo(0); @@ -386,16 +386,16 @@ public void shouldHandleReplicasWithLongerLifespanThanSnapshots() throws Excepti .setScheduleInitialDelayMins(0) .build(); - Path file = Files.createTempFile("", ""); - URI filePath = createURI(S3_TEST_BUCKET, "foo", "bar"); - URI directoryPath = URI.create(String.format("s3://%s/%s", S3_TEST_BUCKET, "foo")); - s3CrtBlobFs.copyFromLocalFile(file.toFile(), filePath); + Path directory = Files.createTempDirectory(""); + Files.createTempFile(directory, "", ""); + String chunkId = UUID.randomUUID().toString(); + chunkStore.upload(chunkId, directory); // snapshot is expired SnapshotMetadata snapshotMetadata = new SnapshotMetadata( - UUID.randomUUID().toString(), - directoryPath.toString(), + chunkId, + chunkStore.getRemotePath(chunkId), Instant.now().minus(11000, ChronoUnit.MINUTES).toEpochMilli(), Instant.now().minus(10900, ChronoUnit.MINUTES).toEpochMilli(), 0, @@ -418,20 +418,20 @@ public void shouldHandleReplicasWithLongerLifespanThanSnapshots() throws Excepti await().until(() -> snapshotMetadataStore.listSync().size() == 1); await().until(() -> replicaMetadataStore.listSync().size() == 1); - String[] s3CrtBlobFsFiles = s3CrtBlobFs.listFiles(directoryPath, true); - assertThat(s3CrtBlobFsFiles.length).isNotEqualTo(0); + List s3CrtBlobFsFiles = chunkStore.listFiles(chunkId); + assertThat(s3CrtBlobFsFiles.size()).isNotEqualTo(0); SnapshotDeletionService snapshotDeletionService = new SnapshotDeletionService( - replicaMetadataStore, snapshotMetadataStore, s3CrtBlobFs, managerConfig, meterRegistry); + replicaMetadataStore, snapshotMetadataStore, chunkStore, managerConfig, meterRegistry); int deletes = snapshotDeletionService.deleteExpiredSnapshotsWithoutReplicas(); assertThat(deletes).isEqualTo(0); assertThat(snapshotMetadataStore.listSync()).containsExactlyInAnyOrder(snapshotMetadata); assertThat(replicaMetadataStore.listSync()).containsExactlyInAnyOrder(replicaMetadata); - verify(s3CrtBlobFs, times(0)).delete(any(), anyBoolean()); - assertThat(s3CrtBlobFs.listFiles(directoryPath, true)).isEqualTo(s3CrtBlobFsFiles); + verify(chunkStore, times(0)).delete(any()); + assertThat(chunkStore.listFiles(chunkId)).isEqualTo(s3CrtBlobFsFiles); assertThat(MetricsUtil.getCount(SnapshotDeletionService.SNAPSHOT_DELETE_SUCCESS, meterRegistry)) .isEqualTo(0); @@ -462,16 +462,16 @@ public void shouldHandleExceptionalObjectStorageDelete() throws Exception { .setScheduleInitialDelayMins(0) .build(); - Path file = Files.createTempFile("", ""); - URI filePath = createURI(S3_TEST_BUCKET, "foo", "bar"); - URI directoryPath = URI.create(String.format("s3://%s/%s", S3_TEST_BUCKET, "foo")); - s3CrtBlobFs.copyFromLocalFile(file.toFile(), filePath); + Path directory = Files.createTempDirectory(""); + Files.createTempFile(directory, "", ""); + String chunkId = UUID.randomUUID().toString(); + chunkStore.upload(chunkId, directory); // snapshot is expired SnapshotMetadata snapshotMetadata = new SnapshotMetadata( - UUID.randomUUID().toString(), - directoryPath.toString(), + chunkId, + chunkStore.getRemotePath(chunkId), Instant.now().minus(11000, ChronoUnit.MINUTES).toEpochMilli(), Instant.now().minus(10900, ChronoUnit.MINUTES).toEpochMilli(), 0, @@ -484,8 +484,8 @@ public void shouldHandleExceptionalObjectStorageDelete() throws Exception { SnapshotDeletionService snapshotDeletionService = new SnapshotDeletionService( - replicaMetadataStore, snapshotMetadataStore, s3CrtBlobFs, managerConfig, meterRegistry); - doThrow(new IOException()).when(s3CrtBlobFs).delete(any(), anyBoolean()); + replicaMetadataStore, snapshotMetadataStore, chunkStore, managerConfig, meterRegistry); + doThrow(new RuntimeException()).when(chunkStore).delete(any()); int deletes = snapshotDeletionService.deleteExpiredSnapshotsWithoutReplicas(); assertThat(deletes).isEqualTo(0); @@ -521,16 +521,16 @@ public void shouldHandleFailedZkDelete() throws Exception { .setScheduleInitialDelayMins(0) .build(); - Path file = Files.createTempFile("", ""); - URI filePath = createURI(S3_TEST_BUCKET, "foo", "bar"); - URI directoryPath = URI.create(String.format("s3://%s/%s", S3_TEST_BUCKET, "foo")); - s3CrtBlobFs.copyFromLocalFile(file.toFile(), filePath); + Path directory = Files.createTempDirectory(""); + Files.createTempFile(directory, "", ""); + String chunkId = UUID.randomUUID().toString(); + chunkStore.upload(chunkId, directory); // snapshot is expired SnapshotMetadata snapshotMetadata = new SnapshotMetadata( - UUID.randomUUID().toString(), - directoryPath.toString(), + chunkId, + chunkStore.getRemotePath(chunkId), Instant.now().minus(11000, ChronoUnit.MINUTES).toEpochMilli(), Instant.now().minus(10900, ChronoUnit.MINUTES).toEpochMilli(), 0, @@ -542,20 +542,20 @@ public void shouldHandleFailedZkDelete() throws Exception { SnapshotDeletionService snapshotDeletionService = new SnapshotDeletionService( - replicaMetadataStore, snapshotMetadataStore, s3CrtBlobFs, managerConfig, meterRegistry); + replicaMetadataStore, snapshotMetadataStore, chunkStore, managerConfig, meterRegistry); AsyncStage asyncStage = mock(AsyncStage.class); when(asyncStage.toCompletableFuture()) .thenReturn(CompletableFuture.failedFuture(new Exception())); doReturn(asyncStage).when(snapshotMetadataStore).deleteAsync(any(SnapshotMetadata.class)); - assertThat(s3CrtBlobFs.listFiles(directoryPath, true)).isNotEmpty(); + assertThat(chunkStore.listFiles(chunkId)).isNotEmpty(); int deletes = snapshotDeletionService.deleteExpiredSnapshotsWithoutReplicas(); assertThat(deletes).isEqualTo(0); assertThat(snapshotMetadataStore.listSync()).containsExactlyInAnyOrder(snapshotMetadata); - verify(s3CrtBlobFs, times(1)).delete(any(), anyBoolean()); - assertThat(s3CrtBlobFs.listFiles(directoryPath, true)).isEmpty(); + verify(chunkStore, times(1)).delete(any()); + assertThat(chunkStore.listFiles(chunkId)).isEmpty(); assertThat(MetricsUtil.getCount(SnapshotDeletionService.SNAPSHOT_DELETE_SUCCESS, meterRegistry)) .isEqualTo(0); @@ -586,16 +586,16 @@ public void shouldHandleFailedObjectDelete() throws Exception { .setScheduleInitialDelayMins(0) .build(); - Path file = Files.createTempFile("", ""); - URI filePath = createURI(S3_TEST_BUCKET, "foo", "bar"); - URI directoryPath = URI.create(String.format("s3://%s/%s", S3_TEST_BUCKET, "foo")); - s3CrtBlobFs.copyFromLocalFile(file.toFile(), filePath); + Path directory = Files.createTempDirectory(""); + Files.createTempFile(directory, "", ""); + String chunkId = UUID.randomUUID().toString(); + chunkStore.upload(chunkId, directory); // snapshot is expired SnapshotMetadata snapshotMetadata = new SnapshotMetadata( - UUID.randomUUID().toString(), - directoryPath.toString(), + chunkId, + chunkStore.getRemotePath(chunkId), Instant.now().minus(11000, ChronoUnit.MINUTES).toEpochMilli(), Instant.now().minus(10900, ChronoUnit.MINUTES).toEpochMilli(), 0, @@ -607,8 +607,8 @@ public void shouldHandleFailedObjectDelete() throws Exception { SnapshotDeletionService snapshotDeletionService = new SnapshotDeletionService( - replicaMetadataStore, snapshotMetadataStore, s3CrtBlobFs, managerConfig, meterRegistry); - doReturn(false).when(s3CrtBlobFs).delete(any(), anyBoolean()); + replicaMetadataStore, snapshotMetadataStore, chunkStore, managerConfig, meterRegistry); + doThrow(new RuntimeException()).when(chunkStore).delete(any()); int deletes = snapshotDeletionService.deleteExpiredSnapshotsWithoutReplicas(); assertThat(deletes).isEqualTo(0); @@ -644,16 +644,16 @@ public void shouldRetryTimedOutZkDeleteNextRun() throws Exception { .setScheduleInitialDelayMins(0) .build(); - Path file = Files.createTempFile("", ""); - URI filePath = createURI(S3_TEST_BUCKET, "foo", "bar"); - URI directoryPath = URI.create(String.format("s3://%s/%s", S3_TEST_BUCKET, "foo")); - s3CrtBlobFs.copyFromLocalFile(file.toFile(), filePath); + Path directory = Files.createTempDirectory(""); + Files.createTempFile(directory, "", ""); + String chunkId = UUID.randomUUID().toString(); + chunkStore.upload(chunkId, directory); // snapshot is expired SnapshotMetadata snapshotMetadata = new SnapshotMetadata( - UUID.randomUUID().toString(), - directoryPath.toString(), + chunkId, + chunkStore.getRemotePath(chunkId), Instant.now().minus(11000, ChronoUnit.MINUTES).toEpochMilli(), Instant.now().minus(10900, ChronoUnit.MINUTES).toEpochMilli(), 0, @@ -665,10 +665,10 @@ public void shouldRetryTimedOutZkDeleteNextRun() throws Exception { SnapshotDeletionService snapshotDeletionService = new SnapshotDeletionService( - replicaMetadataStore, snapshotMetadataStore, s3CrtBlobFs, managerConfig, meterRegistry); + replicaMetadataStore, snapshotMetadataStore, chunkStore, managerConfig, meterRegistry); snapshotDeletionService.futuresListTimeoutSecs = 2; - String[] s3CrtBlobFsFiles = s3CrtBlobFs.listFiles(directoryPath, true); - assertThat(s3CrtBlobFsFiles.length).isNotEqualTo(0); + List s3CrtBlobFsFiles = chunkStore.listFiles(chunkId); + assertThat(s3CrtBlobFsFiles.size()).isNotEqualTo(0); ExecutorService timeoutServiceExecutor = Executors.newSingleThreadExecutor(); @@ -690,8 +690,8 @@ public void shouldRetryTimedOutZkDeleteNextRun() throws Exception { assertThat(deletes).isEqualTo(0); assertThat(snapshotMetadataStore.listSync()).containsExactlyInAnyOrder(snapshotMetadata); - verify(s3CrtBlobFs, times(1)).delete(any(), anyBoolean()); - assertThat(s3CrtBlobFs.listFiles(directoryPath, true)).isEmpty(); + verify(chunkStore, times(1)).delete(any()); + assertThat(chunkStore.listFiles(chunkId)).isEmpty(); assertThat(MetricsUtil.getCount(SnapshotDeletionService.SNAPSHOT_DELETE_SUCCESS, meterRegistry)) .isEqualTo(0); @@ -706,10 +706,9 @@ public void shouldRetryTimedOutZkDeleteNextRun() throws Exception { int deletesRetry = snapshotDeletionService.deleteExpiredSnapshotsWithoutReplicas(); assertThat(deletesRetry).isEqualTo(1); - await().until(() -> snapshotMetadataStore.listSync().size() == 0); - // delete was called once before - should still be only once - verify(s3CrtBlobFs, times(1)).delete(any(), anyBoolean()); - assertThat(s3CrtBlobFs.listFiles(directoryPath, true)).isEmpty(); + await().until(() -> snapshotMetadataStore.listSync().isEmpty()); + verify(chunkStore, times(2)).delete(any()); + assertThat(chunkStore.listFiles(chunkId)).isEmpty(); assertThat(MetricsUtil.getCount(SnapshotDeletionService.SNAPSHOT_DELETE_SUCCESS, meterRegistry)) .isEqualTo(1); @@ -742,16 +741,16 @@ public void shouldRetryFailedObjectStorageDeleteNextRun() throws Exception { .setScheduleInitialDelayMins(0) .build(); - Path file = Files.createTempFile("", ""); - URI filePath = createURI(S3_TEST_BUCKET, "foo", "bar"); - URI directoryPath = URI.create(String.format("s3://%s/%s", S3_TEST_BUCKET, "foo")); - s3CrtBlobFs.copyFromLocalFile(file.toFile(), filePath); + Path directory = Files.createTempDirectory(""); + Files.createTempFile(directory, "", ""); + String chunkId = UUID.randomUUID().toString(); + chunkStore.upload(chunkId, directory); // snapshot is expired SnapshotMetadata snapshotMetadata = new SnapshotMetadata( - UUID.randomUUID().toString(), - directoryPath.toString(), + chunkId, + chunkStore.getRemotePath(chunkId), Instant.now().minus(11000, ChronoUnit.MINUTES).toEpochMilli(), Instant.now().minus(10900, ChronoUnit.MINUTES).toEpochMilli(), 0, @@ -764,14 +763,14 @@ public void shouldRetryFailedObjectStorageDeleteNextRun() throws Exception { SnapshotDeletionService snapshotDeletionService = new SnapshotDeletionService( - replicaMetadataStore, snapshotMetadataStore, s3CrtBlobFs, managerConfig, meterRegistry); - doThrow(new IOException()).when(s3CrtBlobFs).delete(any(), anyBoolean()); + replicaMetadataStore, snapshotMetadataStore, chunkStore, managerConfig, meterRegistry); + doThrow(new RuntimeException()).when(chunkStore).delete(any()); int deletes = snapshotDeletionService.deleteExpiredSnapshotsWithoutReplicas(); assertThat(deletes).isEqualTo(0); assertThat(snapshotMetadataStore.listSync().size()).isEqualTo(1); - verify(s3CrtBlobFs, times(1)).delete(any(), anyBoolean()); + verify(chunkStore, times(1)).delete(any()); assertThat(MetricsUtil.getCount(SnapshotDeletionService.SNAPSHOT_DELETE_SUCCESS, meterRegistry)) .isEqualTo(0); @@ -781,12 +780,12 @@ public void shouldRetryFailedObjectStorageDeleteNextRun() throws Exception { MetricsUtil.getTimerCount(SnapshotDeletionService.SNAPSHOT_DELETE_TIMER, meterRegistry)) .isEqualTo(1); - doCallRealMethod().when(s3CrtBlobFs).delete(any(), anyBoolean()); + doCallRealMethod().when(chunkStore).delete(any()); int deleteRetry = snapshotDeletionService.deleteExpiredSnapshotsWithoutReplicas(); assertThat(deleteRetry).isEqualTo(1); await().until(() -> snapshotMetadataStore.listSync().size() == 0); - verify(s3CrtBlobFs, times(2)).delete(any(), anyBoolean()); + verify(chunkStore, times(2)).delete(any()); assertThat(MetricsUtil.getCount(SnapshotDeletionService.SNAPSHOT_DELETE_SUCCESS, meterRegistry)) .isEqualTo(1); @@ -817,15 +816,15 @@ public void shouldHandleSnapshotDeleteLifecycle() throws Exception { .setScheduleInitialDelayMins(0) .build(); - Path file = Files.createTempFile("", ""); - URI filePath = createURI(S3_TEST_BUCKET, "foo", "bar"); - URI directoryPath = URI.create(String.format("s3://%s/%s", S3_TEST_BUCKET, "foo")); - s3CrtBlobFs.copyFromLocalFile(file.toFile(), filePath); + Path directory = Files.createTempDirectory(""); + Files.createTempFile(directory, "", ""); + String chunkId = UUID.randomUUID().toString(); + chunkStore.upload(chunkId, directory); SnapshotMetadata snapshotMetadata = new SnapshotMetadata( - UUID.randomUUID().toString(), - directoryPath.toString(), + chunkId, + chunkStore.getRemotePath(chunkId), Instant.now().minus(11000, ChronoUnit.MINUTES).toEpochMilli(), Instant.now().minus(10900, ChronoUnit.MINUTES).toEpochMilli(), 0, @@ -837,12 +836,12 @@ public void shouldHandleSnapshotDeleteLifecycle() throws Exception { SnapshotDeletionService snapshotDeletionService = new SnapshotDeletionService( - replicaMetadataStore, snapshotMetadataStore, s3CrtBlobFs, managerConfig, meterRegistry); + replicaMetadataStore, snapshotMetadataStore, chunkStore, managerConfig, meterRegistry); snapshotDeletionService.startAsync(); snapshotDeletionService.awaitRunning(DEFAULT_START_STOP_DURATION); await().until(() -> snapshotMetadataStore.listSync().size() == 0); - verify(s3CrtBlobFs, times(1)).delete(eq(directoryPath), eq(true)); + verify(chunkStore, times(1)).delete(eq(chunkId)); await() .until( From a542cc3edcbc57e573b49896278abbc50e5b13c9 Mon Sep 17 00:00:00 2001 From: Bryan Burkholder Date: Tue, 13 Aug 2024 12:55:59 -0700 Subject: [PATCH 4/7] Remove use of blobfsutils copyFromS3 test helper --- .../com/slack/astra/blobfs/BlobFsUtils.java | 20 ------- .../chunkManager/CachingChunkManagerTest.java | 16 ++++- .../logstore/LuceneIndexStoreImplTest.java | 58 +++++++------------ 3 files changed, 34 insertions(+), 60 deletions(-) diff --git a/astra/src/main/java/com/slack/astra/blobfs/BlobFsUtils.java b/astra/src/main/java/com/slack/astra/blobfs/BlobFsUtils.java index c46c858f33..186ad4e468 100644 --- a/astra/src/main/java/com/slack/astra/blobfs/BlobFsUtils.java +++ b/astra/src/main/java/com/slack/astra/blobfs/BlobFsUtils.java @@ -4,17 +4,12 @@ import java.io.IOException; import java.net.URI; import java.nio.file.Path; -import java.util.Arrays; import java.util.Collection; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** This class contains static methods that help with blobfs operations. */ @Deprecated public class BlobFsUtils { - private static final Logger LOG = LoggerFactory.getLogger(BlobFsUtils.class); - public static final String SCHEME = "s3"; public static final String DELIMITER = "/"; public static final String FILE_FORMAT = "%s://%s/%s"; @@ -41,19 +36,4 @@ public static URI createURI(String bucket, String prefix, String fileName) { ? URI.create(String.format(FILE_FORMAT, SCHEME, bucket + DELIMITER + prefix, fileName)) : URI.create(String.format(FILE_FORMAT, SCHEME, bucket, fileName)); } - - // TODO: Can we copy files without list files and a prefix only? - // TODO: Take a complete URI as this is the format stored in snapshot data - @Deprecated - public static String[] copyFromS3( - String bucket, String prefix, BlobFs s3BlobFs, Path localDirPath) throws Exception { - LOG.debug("Copying files from bucket={} prefix={} using directory", bucket, prefix); - URI directoryToCopy = createURI(bucket, prefix, ""); - s3BlobFs.copyToLocalFile(directoryToCopy, localDirPath.toFile()); - LOG.debug("Copying S3 files complete"); - return Arrays.stream(localDirPath.toFile().listFiles()) - .map(File::toString) - .distinct() - .toArray(String[]::new); - } } diff --git a/astra/src/test/java/com/slack/astra/chunkManager/CachingChunkManagerTest.java b/astra/src/test/java/com/slack/astra/chunkManager/CachingChunkManagerTest.java index e681cb41df..e12d01da5c 100644 --- a/astra/src/test/java/com/slack/astra/chunkManager/CachingChunkManagerTest.java +++ b/astra/src/test/java/com/slack/astra/chunkManager/CachingChunkManagerTest.java @@ -1,6 +1,5 @@ package com.slack.astra.chunkManager; -import static com.slack.astra.blobfs.BlobFsUtils.copyFromS3; import static com.slack.astra.blobfs.BlobFsUtils.copyToS3; import static com.slack.astra.chunk.ReadWriteChunk.SCHEMA_FILE_NAME; import static com.slack.astra.chunkManager.CachingChunkManager.ASTRA_NG_DYNAMIC_CHUNK_SIZES_FLAG; @@ -44,6 +43,7 @@ import java.time.Duration; import java.util.ArrayList; import java.util.List; +import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -262,7 +262,12 @@ public void testCreatesChunksOnAssignment() throws Exception { .ignoreExceptions() .until( () -> - copyFromS3(TEST_S3_BUCKET, snapshotId, s3CrtBlobFs, Path.of("/tmp/test1")).length + Objects.requireNonNull( + chunkStore + .download(snapshotId, Path.of("/tmp/test1")) + .toFile() + .listFiles()) + .length > 0); initAssignment(snapshotId); @@ -300,7 +305,12 @@ public void testBasicChunkEviction() throws Exception { .ignoreExceptions() .until( () -> - copyFromS3(TEST_S3_BUCKET, snapshotId, s3CrtBlobFs, Path.of("/tmp/test2")).length + Objects.requireNonNull( + chunkStore + .download(snapshotId, Path.of("/tmp/test2")) + .toFile() + .listFiles()) + .length > 0); CacheNodeAssignment assignment = initAssignment(snapshotId); diff --git a/astra/src/test/java/com/slack/astra/logstore/LuceneIndexStoreImplTest.java b/astra/src/test/java/com/slack/astra/logstore/LuceneIndexStoreImplTest.java index ca79395a9b..923aa5a33e 100644 --- a/astra/src/test/java/com/slack/astra/logstore/LuceneIndexStoreImplTest.java +++ b/astra/src/test/java/com/slack/astra/logstore/LuceneIndexStoreImplTest.java @@ -1,8 +1,5 @@ package com.slack.astra.logstore; -import static com.slack.astra.blobfs.BlobFsUtils.DELIMITER; -import static com.slack.astra.blobfs.BlobFsUtils.copyFromS3; -import static com.slack.astra.blobfs.BlobFsUtils.copyToS3; import static com.slack.astra.logstore.LuceneIndexStoreImpl.COMMITS_TIMER; import static com.slack.astra.logstore.LuceneIndexStoreImpl.MESSAGES_FAILED_COUNTER; import static com.slack.astra.logstore.LuceneIndexStoreImpl.MESSAGES_RECEIVED_COUNTER; @@ -18,7 +15,7 @@ import brave.Tracing; import com.adobe.testing.s3mock.junit5.S3MockExtension; import com.google.protobuf.ByteString; -import com.slack.astra.blobfs.S3CrtBlobFs; +import com.slack.astra.blobfs.ChunkStore; import com.slack.astra.blobfs.s3.S3TestUtils; import com.slack.astra.logstore.LogMessage.ReservedField; import com.slack.astra.logstore.schema.SchemaAwareLogDocumentBuilderImpl; @@ -38,6 +35,8 @@ import java.time.temporal.ChronoUnit; import java.util.Collection; import java.util.List; +import java.util.Objects; +import java.util.UUID; import java.util.stream.Collectors; import java.util.stream.IntStream; import org.apache.commons.io.FileUtils; @@ -337,18 +336,9 @@ public class SnapshotTester { public SnapshotTester() throws IOException {} @Test - public void testS3SnapshotWithPrefix() throws Exception { - testS3Snapshot("test-bucket-with-prefix", "snapshot_prefix1"); - } - - @Test - public void testS3SnapshotWithEmptyPrefix() throws Exception { - testS3Snapshot("test-bucket-no-prefix", ""); - } - - private void testS3Snapshot(String bucket, String prefix) throws Exception { + public void testS3Snapshot() throws Exception { LuceneIndexStoreImpl logStore = strictLogStore.logStore; - addMessages(logStore, 1, 100, true); + addMessages(strictLogStore.logStore, 1, 100, true); Collection results = findAllMessages( strictLogStore.logSearcher, MessageUtil.TEST_DATASET_NAME, "Message1", 100); @@ -359,24 +349,27 @@ private void testS3Snapshot(String bucket, String prefix) throws Exception { assertThat(getTimerCount(REFRESHES_TIMER, strictLogStore.metricsRegistry)).isEqualTo(1); assertThat(getTimerCount(COMMITS_TIMER, strictLogStore.metricsRegistry)).isEqualTo(1); - Path dirPath = logStore.getDirectory().getDirectory().toAbsolutePath(); - IndexCommit indexCommit = logStore.getIndexCommit(); + Path dirPath = strictLogStore.logStore.getDirectory().getDirectory().toAbsolutePath(); + IndexCommit indexCommit = strictLogStore.logStore.getIndexCommit(); Collection activeFiles = indexCommit.getFileNames(); - logStore.close(); + strictLogStore.logStore.close(); strictLogStore.logSearcher.close(); strictLogStore.logStore = null; strictLogStore.logSearcher = null; - assertThat(dirPath.toFile().listFiles().length).isGreaterThanOrEqualTo(activeFiles.size()); + + assertThat(Objects.requireNonNull(dirPath.toFile().listFiles()).length) + .isGreaterThanOrEqualTo(activeFiles.size()); // create an S3 client S3AsyncClient s3AsyncClient = S3TestUtils.createS3CrtClient(S3_MOCK_EXTENSION.getServiceEndpoint()); - S3CrtBlobFs s3CrtBlobFs = new S3CrtBlobFs(s3AsyncClient); + String bucket = "snapshot-test"; s3AsyncClient.createBucket(CreateBucketRequest.builder().bucket(bucket).build()).get(); + ChunkStore chunkStore = new ChunkStore(s3AsyncClient, bucket); - // Copy files to S3. - copyToS3(dirPath, activeFiles, bucket, prefix, s3CrtBlobFs); + String chunkId = UUID.randomUUID().toString(); + chunkStore.upload(chunkId, dirPath); for (String fileName : activeFiles) { File fileToCopy = new File(dirPath.toString(), fileName); @@ -385,10 +378,7 @@ private void testS3Snapshot(String bucket, String prefix) throws Exception { .headObject( HeadObjectRequest.builder() .bucket(bucket) - .key( - prefix != null && !prefix.isEmpty() - ? prefix + DELIMITER + fileName - : fileName) + .key(String.format("%s/%s", chunkId, fileName)) .build()) .get(); assertThat(headObjectResponse.contentLength()).isEqualTo(fileToCopy.length()); @@ -404,15 +394,11 @@ private void testS3Snapshot(String bucket, String prefix) throws Exception { // then fail) FileUtils.cleanDirectory(tmpPath.toFile()); // Download files from S3 to local FS. - String[] s3Files = - copyFromS3( - bucket, - prefix, - s3CrtBlobFs, - tmpPath.toAbsolutePath()); // IO java.util.concurrent.ExecutionException: - // software.amazon.awssdk.core.exception.SdkClientException: Unexpected exception - // occurred: s3metaRequest is not initialized yet - return s3Files.length == activeFiles.size(); + chunkStore.download(chunkId, tmpPath.toAbsolutePath()); + // the delta is the presence of the write.lock file, which is released but still in + // the directory + return Objects.requireNonNull(tmpPath.toFile().listFiles()).length + >= activeFiles.size(); }); // Search files in local FS. @@ -425,9 +411,7 @@ private void testS3Snapshot(String bucket, String prefix) throws Exception { assertThat(newResults.size()).isEqualTo(1); // Clean up - logStore.releaseIndexCommit(indexCommit); newSearcher.close(); - s3CrtBlobFs.close(); } } From a50d63e79f7ab0b59bf6f6a1fab71c8b04a0f91d Mon Sep 17 00:00:00 2001 From: Bryan Burkholder Date: Tue, 13 Aug 2024 14:07:06 -0700 Subject: [PATCH 5/7] Remove remaining blobfs implementations --- .../java/com/slack/astra/blobfs/BlobFs.java | 90 ----- .../com/slack/astra/blobfs/BlobFsUtils.java | 39 -- .../com/slack/astra/blobfs/ChunkStore.java | 23 +- .../com/slack/astra/blobfs/S3AsyncUtil.java | 92 +++++ .../com/slack/astra/blobfs/S3CrtBlobFs.java | 368 ------------------ .../com/slack/astra/chunk/ReadWriteChunk.java | 14 +- .../chunkManager/ChunkRolloverFactory.java | 17 +- .../chunkManager/IndexingChunkManager.java | 19 +- .../chunkManager/RecoveryChunkManager.java | 10 +- .../astra/chunkManager/RollOverChunkTask.java | 18 +- .../slack/astra/recovery/RecoveryService.java | 11 +- .../java/com/slack/astra/server/Astra.java | 15 +- .../slack/astra/blobfs/ChunkStoreTest.java | 1 - .../astra/blobfs/{s3 => }/S3TestUtils.java | 2 +- .../astra/blobfs/s3/S3CrtBlobFsTest.java | 271 ------------- .../astra/chunk/IndexingChunkImplTest.java | 19 +- .../astra/chunk/ReadOnlyChunkImplTest.java | 9 +- .../astra/chunk/RecoveryChunkImplTest.java | 22 +- .../chunkManager/CachingChunkManagerTest.java | 9 +- .../IndexingChunkManagerTest.java | 78 ++-- .../RecoveryChunkManagerTest.java | 22 +- ...MessageCountBasedRolloverStrategyTest.java | 21 +- .../SnapshotDeletionServiceTest.java | 2 +- .../logstore/LuceneIndexStoreImplTest.java | 2 +- .../astra/recovery/RecoveryServiceTest.java | 71 ++-- .../com/slack/astra/server/AstraTest.java | 2 +- .../slack/astra/testlib/ChunkManagerUtil.java | 9 +- 27 files changed, 252 insertions(+), 1004 deletions(-) delete mode 100644 astra/src/main/java/com/slack/astra/blobfs/BlobFs.java delete mode 100644 astra/src/main/java/com/slack/astra/blobfs/BlobFsUtils.java create mode 100644 astra/src/main/java/com/slack/astra/blobfs/S3AsyncUtil.java delete mode 100644 astra/src/main/java/com/slack/astra/blobfs/S3CrtBlobFs.java rename astra/src/test/java/com/slack/astra/blobfs/{s3 => }/S3TestUtils.java (96%) delete mode 100644 astra/src/test/java/com/slack/astra/blobfs/s3/S3CrtBlobFsTest.java diff --git a/astra/src/main/java/com/slack/astra/blobfs/BlobFs.java b/astra/src/main/java/com/slack/astra/blobfs/BlobFs.java deleted file mode 100644 index b56bb92f8c..0000000000 --- a/astra/src/main/java/com/slack/astra/blobfs/BlobFs.java +++ /dev/null @@ -1,90 +0,0 @@ -package com.slack.astra.blobfs; - -import java.io.Closeable; -import java.io.File; -import java.io.IOException; -import java.io.Serializable; -import java.net.URI; - -/** - * BlobFs is a restricted FS API that exposes functionality that is required for a store to use - * different FS implementations. The restrictions are in place due to 2 driving factors: 1. Prevent - * unexpected performance hit when a broader API is implemented - especially, we would like to - * reduce calls to remote filesystems that might be needed for a broader API, but not necessarily - * required by blobfs lib(see the documentation for move() method below). 2. Provide an interface - * that is simple to be implemented across different FS types. The contract that developers have to - * adhere to will be simpler. Please read the method level docs carefully to note the exceptions - * while using the APIs. - * - *

NOTE: This code is a fork of PinotFS from Apache Pinot. In future, we will import this code as - * an external lib. - */ -@Deprecated -public abstract class BlobFs implements Closeable, Serializable { - - /** - * Checks whether the file or directory at the provided location exists. - * - * @param fileUri URI of file - * @return true if path exists - * @throws IOException on IO failure - */ - @Deprecated - public abstract boolean exists(URI fileUri) throws IOException; - - /** - * Lists all the files and directories at the location provided. Lists recursively if {@code - * recursive} is set to true. Throws IOException if this abstract pathname is not valid, or if an - * I/O error occurs. - * - * @param fileUri location of file - * @param recursive if we want to list files recursively - * @return an array of strings that contains file paths - * @throws IOException on IO failure. See specific implementation - */ - @Deprecated - public abstract String[] listFiles(URI fileUri, boolean recursive) throws IOException; - - /** - * Copies a file from a remote filesystem to the local one. Keeps the original file. - * - * @param srcUri location of current file on remote filesystem - * @param dstFile location of destination on local filesystem - * @throws Exception if srcUri is not valid or not present, or timeout when downloading file to - * local - */ - @Deprecated - public abstract void copyToLocalFile(URI srcUri, File dstFile) throws Exception; - - /** - * The src file is on the local disk. Add it to filesystem at the given dst name and the source is - * kept intact afterwards. - * - * @param srcFile location of src file on local disk - * @param dstUri location of dst on remote filesystem - * @throws Exception if fileUri is not valid or not present, or timeout when uploading file from - * local - */ - @Deprecated - public abstract void copyFromLocalFile(File srcFile, URI dstUri) throws Exception; - - /** - * Allows us the ability to determine whether the uri is a directory. - * - * @param uri location of file or directory - * @return true if uri is a directory, false otherwise. - * @throws IOException on IO failure, e.g uri is not valid or not present - */ - @Deprecated - public abstract boolean isDirectory(URI uri) throws IOException; - - /** - * For certain filesystems, we may need to close the filesystem and do relevant operations to - * prevent leaks. By default, this method does nothing. - * - * @throws IOException on IO failure - */ - @Override - @Deprecated - public void close() throws IOException {} -} diff --git a/astra/src/main/java/com/slack/astra/blobfs/BlobFsUtils.java b/astra/src/main/java/com/slack/astra/blobfs/BlobFsUtils.java deleted file mode 100644 index 186ad4e468..0000000000 --- a/astra/src/main/java/com/slack/astra/blobfs/BlobFsUtils.java +++ /dev/null @@ -1,39 +0,0 @@ -package com.slack.astra.blobfs; - -import java.io.File; -import java.io.IOException; -import java.net.URI; -import java.nio.file.Path; -import java.util.Collection; - -/** This class contains static methods that help with blobfs operations. */ -@Deprecated -public class BlobFsUtils { - - public static final String SCHEME = "s3"; - public static final String DELIMITER = "/"; - public static final String FILE_FORMAT = "%s://%s/%s"; - - @Deprecated - public static int copyToS3( - Path sourceDirPath, Collection files, String bucket, String prefix, BlobFs blobFs) - throws Exception { - int success = 0; - for (String fileName : files) { - File fileToCopy = new File(sourceDirPath.toString(), fileName); - if (!fileToCopy.exists()) { - throw new IOException("File doesn't exist at path: " + fileToCopy.getAbsolutePath()); - } - blobFs.copyFromLocalFile(fileToCopy, createURI(bucket, prefix, fileName)); - success++; - } - return success; - } - - @Deprecated - public static URI createURI(String bucket, String prefix, String fileName) { - return (prefix != null && !prefix.isEmpty()) - ? URI.create(String.format(FILE_FORMAT, SCHEME, bucket + DELIMITER + prefix, fileName)) - : URI.create(String.format(FILE_FORMAT, SCHEME, bucket, fileName)); - } -} diff --git a/astra/src/main/java/com/slack/astra/blobfs/ChunkStore.java b/astra/src/main/java/com/slack/astra/blobfs/ChunkStore.java index bfffa74a91..ae703c92a0 100644 --- a/astra/src/main/java/com/slack/astra/blobfs/ChunkStore.java +++ b/astra/src/main/java/com/slack/astra/blobfs/ChunkStore.java @@ -14,6 +14,7 @@ import software.amazon.awssdk.services.s3.model.ObjectIdentifier; import software.amazon.awssdk.services.s3.paginators.ListObjectsV2Publisher; import software.amazon.awssdk.transfer.s3.S3TransferManager; +import software.amazon.awssdk.transfer.s3.model.CompletedDirectoryUpload; import software.amazon.awssdk.transfer.s3.model.DownloadDirectoryRequest; import software.amazon.awssdk.transfer.s3.model.UploadDirectoryRequest; @@ -31,15 +32,19 @@ public ChunkStore(S3AsyncClient s3AsyncClient, String bucketName) { public void upload(String chunkId, Path directoryToUpload) { try { - transferManager - .uploadDirectory( - UploadDirectoryRequest.builder() - .source(directoryToUpload) - .s3Prefix(chunkId) - .bucket(bucketName) - .build()) - .completionFuture() - .get(); + CompletedDirectoryUpload upload = + transferManager + .uploadDirectory( + UploadDirectoryRequest.builder() + .source(directoryToUpload) + .s3Prefix(chunkId) + .bucket(bucketName) + .build()) + .completionFuture() + .get(); + if (!upload.failedTransfers().isEmpty()) { + throw new IllegalStateException("Some or all files failed to upload"); + } } catch (ExecutionException | InterruptedException e) { throw new RuntimeException(e); } diff --git a/astra/src/main/java/com/slack/astra/blobfs/S3AsyncUtil.java b/astra/src/main/java/com/slack/astra/blobfs/S3AsyncUtil.java new file mode 100644 index 0000000000..e25b780fac --- /dev/null +++ b/astra/src/main/java/com/slack/astra/blobfs/S3AsyncUtil.java @@ -0,0 +1,92 @@ +package com.slack.astra.blobfs; + +import com.google.common.base.Preconditions; +import com.slack.astra.proto.config.AstraConfigs; +import java.net.URI; +import java.net.URISyntaxException; +import java.time.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; +import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.s3.S3AsyncClient; +import software.amazon.awssdk.services.s3.S3CrtAsyncClientBuilder; +import software.amazon.awssdk.services.s3.crt.S3CrtConnectionHealthConfiguration; +import software.amazon.awssdk.services.s3.crt.S3CrtHttpConfiguration; +import software.amazon.awssdk.services.s3.crt.S3CrtProxyConfiguration; +import software.amazon.awssdk.services.s3.crt.S3CrtRetryConfiguration; +import software.amazon.awssdk.services.s3.model.S3Exception; + +public class S3AsyncUtil { + private static final Logger LOG = LoggerFactory.getLogger(S3AsyncUtil.class); + + public static S3AsyncClient initS3Client(AstraConfigs.S3Config config) { + Preconditions.checkArgument(!isNullOrEmpty(config.getS3Region())); + String region = config.getS3Region(); + + AwsCredentialsProvider awsCredentialsProvider; + try { + + if (!isNullOrEmpty(config.getS3AccessKey()) && !isNullOrEmpty(config.getS3SecretKey())) { + String accessKey = config.getS3AccessKey(); + String secretKey = config.getS3SecretKey(); + AwsBasicCredentials awsBasicCredentials = AwsBasicCredentials.create(accessKey, secretKey); + awsCredentialsProvider = StaticCredentialsProvider.create(awsBasicCredentials); + } else { + awsCredentialsProvider = DefaultCredentialsProvider.create(); + } + + // default to 5% of the heap size for the max crt off-heap or 1GiB (min for client) + long jvmMaxHeapSizeBytes = Runtime.getRuntime().maxMemory(); + long defaultCrtMemoryLimit = Math.max(Math.round(jvmMaxHeapSizeBytes * 0.05), 1073741824); + long maxNativeMemoryLimitBytes = + Long.parseLong( + System.getProperty( + "astra.s3CrtBlobFs.maxNativeMemoryLimitBytes", + String.valueOf(defaultCrtMemoryLimit))); + LOG.info( + "Using a maxNativeMemoryLimitInBytes for the S3AsyncClient of '{}' bytes", + maxNativeMemoryLimitBytes); + S3CrtAsyncClientBuilder s3AsyncClient = + S3AsyncClient.crtBuilder() + .retryConfiguration(S3CrtRetryConfiguration.builder().numRetries(3).build()) + .targetThroughputInGbps(config.getS3TargetThroughputGbps()) + .region(Region.of(region)) + .maxNativeMemoryLimitInBytes(maxNativeMemoryLimitBytes) + .credentialsProvider(awsCredentialsProvider); + + // We add a healthcheck to prevent an error with the CRT client, where it will + // continue to attempt to read data from a socket that is no longer returning data + S3CrtHttpConfiguration.Builder httpConfigurationBuilder = + S3CrtHttpConfiguration.builder() + .proxyConfiguration( + S3CrtProxyConfiguration.builder().useEnvironmentVariableValues(false).build()) + .connectionTimeout(Duration.ofSeconds(5)) + .connectionHealthConfiguration( + S3CrtConnectionHealthConfiguration.builder() + .minimumThroughputTimeout(Duration.ofSeconds(3)) + .minimumThroughputInBps(32000L) + .build()); + s3AsyncClient.httpConfiguration(httpConfigurationBuilder.build()); + + if (!isNullOrEmpty(config.getS3EndPoint())) { + String endpoint = config.getS3EndPoint(); + try { + s3AsyncClient.endpointOverride(new URI(endpoint)); + } catch (URISyntaxException e) { + throw new RuntimeException(e); + } + } + return s3AsyncClient.build(); + } catch (S3Exception e) { + throw new RuntimeException("Could not initialize S3blobFs", e); + } + } + + static boolean isNullOrEmpty(String target) { + return target == null || "".equals(target); + } +} diff --git a/astra/src/main/java/com/slack/astra/blobfs/S3CrtBlobFs.java b/astra/src/main/java/com/slack/astra/blobfs/S3CrtBlobFs.java deleted file mode 100644 index d4f3ec712d..0000000000 --- a/astra/src/main/java/com/slack/astra/blobfs/S3CrtBlobFs.java +++ /dev/null @@ -1,368 +0,0 @@ -package com.slack.astra.blobfs; - -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; -import com.slack.astra.proto.config.AstraConfigs; -import java.io.File; -import java.io.IOException; -import java.net.URI; -import java.net.URISyntaxException; -import java.time.Duration; -import java.util.List; -import java.util.concurrent.ExecutionException; -import org.apache.commons.io.FileUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; -import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; -import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; -import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider; -import software.amazon.awssdk.regions.Region; -import software.amazon.awssdk.services.s3.S3AsyncClient; -import software.amazon.awssdk.services.s3.S3CrtAsyncClientBuilder; -import software.amazon.awssdk.services.s3.crt.S3CrtConnectionHealthConfiguration; -import software.amazon.awssdk.services.s3.crt.S3CrtHttpConfiguration; -import software.amazon.awssdk.services.s3.crt.S3CrtProxyConfiguration; -import software.amazon.awssdk.services.s3.crt.S3CrtRetryConfiguration; -import software.amazon.awssdk.services.s3.model.GetObjectRequest; -import software.amazon.awssdk.services.s3.model.HeadObjectRequest; -import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; -import software.amazon.awssdk.services.s3.model.ListObjectsV2Response; -import software.amazon.awssdk.services.s3.model.NoSuchKeyException; -import software.amazon.awssdk.services.s3.model.PutObjectRequest; -import software.amazon.awssdk.services.s3.model.S3Exception; -import software.amazon.awssdk.services.s3.model.S3Object; -import software.amazon.awssdk.transfer.s3.S3TransferManager; -import software.amazon.awssdk.transfer.s3.model.CompletedDirectoryDownload; -import software.amazon.awssdk.transfer.s3.model.CompletedDirectoryUpload; -import software.amazon.awssdk.transfer.s3.model.DownloadDirectoryRequest; -import software.amazon.awssdk.transfer.s3.model.DownloadFileRequest; -import software.amazon.awssdk.transfer.s3.model.UploadDirectoryRequest; -import software.amazon.awssdk.transfer.s3.model.UploadFileRequest; - -/** - * This class is a duplicate of the original S3BlobFs, but modified to support the new S3 CRT client - * and S3 transfer manager. As part of this all internal api calls to S3 were moved to async, as - * this is the only client type supported by the new CRT code. - * - *

Todo - this class would hugely benefit from a clean sheet rewrite, as a lot of the original - * assumptions this was based on no longer apply. Additionally, several retrofits have been made to - * support new API approaches which has left this overly complex. - */ -@Deprecated -public class S3CrtBlobFs extends BlobFs { - public static final String S3_SCHEME = "s3://"; - private static final Logger LOG = LoggerFactory.getLogger(S3CrtBlobFs.class); - private static final String DELIMITER = "/"; - private static final int LIST_MAX_KEYS = 2500; - - private final S3AsyncClient s3AsyncClient; - private final S3TransferManager transferManager; - - public S3CrtBlobFs(S3AsyncClient s3AsyncClient) { - this.s3AsyncClient = s3AsyncClient; - this.transferManager = S3TransferManager.builder().s3Client(s3AsyncClient).build(); - } - - static boolean isNullOrEmpty(String target) { - return target == null || "".equals(target); - } - - public static S3AsyncClient initS3Client(AstraConfigs.S3Config config) { - Preconditions.checkArgument(!isNullOrEmpty(config.getS3Region())); - String region = config.getS3Region(); - - AwsCredentialsProvider awsCredentialsProvider; - try { - - if (!isNullOrEmpty(config.getS3AccessKey()) && !isNullOrEmpty(config.getS3SecretKey())) { - String accessKey = config.getS3AccessKey(); - String secretKey = config.getS3SecretKey(); - AwsBasicCredentials awsBasicCredentials = AwsBasicCredentials.create(accessKey, secretKey); - awsCredentialsProvider = StaticCredentialsProvider.create(awsBasicCredentials); - } else { - awsCredentialsProvider = DefaultCredentialsProvider.create(); - } - - // default to 5% of the heap size for the max crt off-heap or 1GiB (min for client) - long jvmMaxHeapSizeBytes = Runtime.getRuntime().maxMemory(); - long defaultCrtMemoryLimit = Math.max(Math.round(jvmMaxHeapSizeBytes * 0.05), 1073741824); - long maxNativeMemoryLimitBytes = - Long.parseLong( - System.getProperty( - "astra.s3CrtBlobFs.maxNativeMemoryLimitBytes", - String.valueOf(defaultCrtMemoryLimit))); - LOG.info( - "Using a maxNativeMemoryLimitInBytes for the S3AsyncClient of '{}' bytes", - maxNativeMemoryLimitBytes); - S3CrtAsyncClientBuilder s3AsyncClient = - S3AsyncClient.crtBuilder() - .retryConfiguration(S3CrtRetryConfiguration.builder().numRetries(3).build()) - .targetThroughputInGbps(config.getS3TargetThroughputGbps()) - .region(Region.of(region)) - .maxNativeMemoryLimitInBytes(maxNativeMemoryLimitBytes) - .credentialsProvider(awsCredentialsProvider); - - // We add a healthcheck to prevent an error with the CRT client, where it will - // continue to attempt to read data from a socket that is no longer returning data - S3CrtHttpConfiguration.Builder httpConfigurationBuilder = - S3CrtHttpConfiguration.builder() - .proxyConfiguration( - S3CrtProxyConfiguration.builder().useEnvironmentVariableValues(false).build()) - .connectionTimeout(Duration.ofSeconds(5)) - .connectionHealthConfiguration( - S3CrtConnectionHealthConfiguration.builder() - .minimumThroughputTimeout(Duration.ofSeconds(3)) - .minimumThroughputInBps(32000L) - .build()); - s3AsyncClient.httpConfiguration(httpConfigurationBuilder.build()); - - if (!isNullOrEmpty(config.getS3EndPoint())) { - String endpoint = config.getS3EndPoint(); - try { - s3AsyncClient.endpointOverride(new URI(endpoint)); - } catch (URISyntaxException e) { - throw new RuntimeException(e); - } - } - return s3AsyncClient.build(); - } catch (S3Exception e) { - throw new RuntimeException("Could not initialize S3blobFs", e); - } - } - - private boolean isPathTerminatedByDelimiter(URI uri) { - return uri.getPath().endsWith(DELIMITER); - } - - private String normalizeToDirectoryPrefix(URI uri) throws IOException { - Preconditions.checkNotNull(uri, "uri is null"); - URI strippedUri = getBase(uri).relativize(uri); - if (isPathTerminatedByDelimiter(strippedUri)) { - return sanitizePath(strippedUri.getPath()); - } - return sanitizePath(strippedUri.getPath() + DELIMITER); - } - - private String sanitizePath(String path) { - path = path.replaceAll(DELIMITER + "+", DELIMITER); - if (path.startsWith(DELIMITER) && !path.equals(DELIMITER)) { - path = path.substring(1); - } - return path; - } - - private URI getBase(URI uri) throws IOException { - try { - return new URI(uri.getScheme(), uri.getHost(), null, null); - } catch (URISyntaxException e) { - throw new IOException(e); - } - } - - private boolean existsFile(URI uri) throws IOException { - try { - URI base = getBase(uri); - String path = sanitizePath(base.relativize(uri).getPath()); - HeadObjectRequest headObjectRequest = - HeadObjectRequest.builder().bucket(uri.getHost()).key(path).build(); - - s3AsyncClient.headObject(headObjectRequest).get(); - return true; - } catch (Exception e) { - if (e instanceof ExecutionException && e.getCause() instanceof NoSuchKeyException) { - return false; - } else { - throw new IOException(e); - } - } - } - - @Override - public boolean exists(URI fileUri) throws IOException { - try { - if (isDirectory(fileUri)) { - return true; - } - if (isPathTerminatedByDelimiter(fileUri)) { - return false; - } - return existsFile(fileUri); - } catch (NoSuchKeyException e) { - return false; - } - } - - @Override - public String[] listFiles(URI fileUri, boolean recursive) throws IOException { - try { - ImmutableList.Builder builder = ImmutableList.builder(); - String continuationToken = null; - boolean isDone = false; - String prefix = normalizeToDirectoryPrefix(fileUri); - int fileCount = 0; - while (!isDone) { - ListObjectsV2Request.Builder listObjectsV2RequestBuilder = - ListObjectsV2Request.builder().maxKeys(LIST_MAX_KEYS).bucket(fileUri.getHost()); - if (!prefix.equals(DELIMITER)) { - listObjectsV2RequestBuilder = listObjectsV2RequestBuilder.prefix(prefix); - } - if (!recursive) { - listObjectsV2RequestBuilder = listObjectsV2RequestBuilder.delimiter(DELIMITER); - } - if (continuationToken != null) { - listObjectsV2RequestBuilder.continuationToken(continuationToken); - } - ListObjectsV2Request listObjectsV2Request = listObjectsV2RequestBuilder.build(); - LOG.debug("Trying to send ListObjectsV2Request {}", listObjectsV2Request); - ListObjectsV2Response listObjectsV2Response = - s3AsyncClient.listObjectsV2(listObjectsV2Request).get(); - LOG.debug("Getting ListObjectsV2Response: {}", listObjectsV2Response); - List filesReturned = listObjectsV2Response.contents(); - fileCount += filesReturned.size(); - filesReturned.stream() - .forEach( - object -> { - // Only add files and not directories - if (!object.key().equals(fileUri.getPath()) - && !object.key().endsWith(DELIMITER)) { - String fileKey = object.key(); - if (fileKey.startsWith(DELIMITER)) { - fileKey = fileKey.substring(1); - } - builder.add(S3_SCHEME + fileUri.getHost() + DELIMITER + fileKey); - } - }); - if (fileCount == LIST_MAX_KEYS) { - // check if we reached the max keys returned, if so abort and throw an error message - LOG.error( - "Too many files ({}) returned from S3 when attempting to list object prefixes", - LIST_MAX_KEYS); - throw new IllegalStateException( - String.format( - "Max keys (%s) reached when attempting to list S3 objects", LIST_MAX_KEYS)); - } - isDone = !listObjectsV2Response.isTruncated(); - continuationToken = listObjectsV2Response.nextContinuationToken(); - } - String[] listedFiles = builder.build().toArray(new String[0]); - LOG.debug( - "Listed {} files from URI: {}, is recursive: {}", listedFiles.length, fileUri, recursive); - return listedFiles; - } catch (Throwable t) { - throw new IOException(t); - } - } - - @Override - public void copyToLocalFile(URI srcUri, File dstFile) throws Exception { - LOG.debug("Copy {} to local {}", srcUri, dstFile.getAbsolutePath()); - URI base = getBase(srcUri); - FileUtils.forceMkdir(dstFile.getParentFile()); - String prefix = sanitizePath(base.relativize(srcUri).getPath()); - - if (isDirectory(srcUri)) { - CompletedDirectoryDownload completedDirectoryDownload = - transferManager - .downloadDirectory( - DownloadDirectoryRequest.builder() - .destination(dstFile.toPath()) - .bucket(srcUri.getHost()) - .listObjectsV2RequestTransformer( - builder -> { - builder.maxKeys(LIST_MAX_KEYS); - builder.prefix(prefix); - }) - .build()) - .completionFuture() - .get(); - if (!completedDirectoryDownload.failedTransfers().isEmpty()) { - completedDirectoryDownload - .failedTransfers() - .forEach( - failedFileDownload -> LOG.warn("Failed to download file '{}'", failedFileDownload)); - throw new IllegalStateException( - String.format( - "Was unable to download all files - failed %s", - completedDirectoryDownload.failedTransfers().size())); - } - } else { - GetObjectRequest getObjectRequest = - GetObjectRequest.builder().bucket(srcUri.getHost()).key(prefix).build(); - transferManager - .downloadFile( - DownloadFileRequest.builder() - .getObjectRequest(getObjectRequest) - .destination(dstFile) - .build()) - .completionFuture() - .get(); - } - } - - @Override - public void copyFromLocalFile(File srcFile, URI dstUri) throws Exception { - LOG.debug("Copy {} from local to {}", srcFile.getAbsolutePath(), dstUri); - URI base = getBase(dstUri); - String prefix = sanitizePath(base.relativize(dstUri).getPath()); - - if (srcFile.isDirectory()) { - CompletedDirectoryUpload completedDirectoryUpload = - transferManager - .uploadDirectory( - UploadDirectoryRequest.builder() - .source(srcFile.toPath()) - .bucket(dstUri.getHost()) - .build()) - .completionFuture() - .get(); - - if (!completedDirectoryUpload.failedTransfers().isEmpty()) { - completedDirectoryUpload - .failedTransfers() - .forEach(failedFileUpload -> LOG.warn("Failed to upload file '{}'", failedFileUpload)); - throw new IllegalStateException( - String.format( - "Was unable to upload all files - failed %s", - completedDirectoryUpload.failedTransfers().size())); - } - } else { - PutObjectRequest putObjectRequest = - PutObjectRequest.builder().bucket(dstUri.getHost()).key(prefix).build(); - transferManager - .uploadFile( - UploadFileRequest.builder() - .putObjectRequest(putObjectRequest) - .source(srcFile) - .build()) - .completionFuture() - .get(); - } - } - - @Override - public boolean isDirectory(URI uri) throws IOException { - try { - String prefix = normalizeToDirectoryPrefix(uri); - if (prefix.equals(DELIMITER)) { - return true; - } - - ListObjectsV2Request listObjectsV2Request = - ListObjectsV2Request.builder().bucket(uri.getHost()).prefix(prefix).maxKeys(2).build(); - ListObjectsV2Response listObjectsV2Response = - s3AsyncClient.listObjectsV2(listObjectsV2Request).get(); - return listObjectsV2Response.hasContents(); - } catch (NoSuchKeyException e) { - LOG.error("Could not get directory entry for {}", uri); - return false; - } catch (ExecutionException | InterruptedException e) { - throw new IOException(e); - } - } - - @Override - public void close() throws IOException { - super.close(); - } -} diff --git a/astra/src/main/java/com/slack/astra/chunk/ReadWriteChunk.java b/astra/src/main/java/com/slack/astra/chunk/ReadWriteChunk.java index 7d6ac38080..310527ef35 100644 --- a/astra/src/main/java/com/slack/astra/chunk/ReadWriteChunk.java +++ b/astra/src/main/java/com/slack/astra/chunk/ReadWriteChunk.java @@ -1,12 +1,10 @@ package com.slack.astra.chunk; -import static com.slack.astra.blobfs.BlobFsUtils.copyToS3; -import static com.slack.astra.blobfs.BlobFsUtils.createURI; import static com.slack.astra.chunk.ChunkInfo.toSnapshotMetadata; import static com.slack.astra.writer.SpanFormatter.isValidTimestamp; import com.google.common.annotations.VisibleForTesting; -import com.slack.astra.blobfs.BlobFs; +import com.slack.astra.blobfs.ChunkStore; import com.slack.astra.logstore.LogStore; import com.slack.astra.logstore.LuceneIndexStoreImpl; import com.slack.astra.logstore.search.LogIndexSearcher; @@ -78,7 +76,6 @@ public abstract class ReadWriteChunk implements Chunk { private final Logger logger; private LogIndexSearcher logSearcher; private final Counter fileUploadAttempts; - private final Counter fileUploadFailures; private final MeterRegistry meterRegistry; protected final ChunkInfo chunkInfo; protected final SearchMetadata liveSearchMetadata; @@ -120,7 +117,6 @@ protected ReadWriteChunk( readOnly = false; this.meterRegistry = meterRegistry; fileUploadAttempts = meterRegistry.counter(INDEX_FILES_UPLOAD); - fileUploadFailures = meterRegistry.counter(INDEX_FILES_UPLOAD_FAILED); liveSnapshotMetadata = toSnapshotMetadata(chunkInfo, LIVE_SNAPSHOT_PREFIX); liveSearchMetadata = toSearchMetadata(liveSnapshotMetadata.snapshotId, searchContext); this.searchMetadataStore = searchMetadataStore; @@ -225,7 +221,7 @@ public void preSnapshot() { * * @return true on success, false on failure. */ - public boolean snapshotToS3(String bucket, String prefix, BlobFs blobFs) { + public boolean snapshotToS3(ChunkStore chunkStore) { logger.info("Started RW chunk snapshot to S3 {}", chunkInfo); IndexCommit indexCommit = null; @@ -254,12 +250,10 @@ public boolean snapshotToS3(String bucket, String prefix, BlobFs blobFs) { } this.fileUploadAttempts.increment(filesToUpload.size()); Timer.Sample snapshotTimer = Timer.start(meterRegistry); - - final int success = copyToS3(dirPath, filesToUpload, bucket, prefix, blobFs); + chunkStore.upload(chunkInfo.chunkId, dirPath); snapshotTimer.stop(meterRegistry.timer(SNAPSHOT_TIMER)); - this.fileUploadFailures.increment(filesToUpload.size() - success); - chunkInfo.setSnapshotPath(createURI(bucket, prefix, "").toString()); + chunkInfo.setSnapshotPath(chunkStore.getRemotePath(chunkInfo.chunkId)); chunkInfo.setSizeInBytesOnDisk(totalBytes); logger.info("Finished RW chunk snapshot to S3 {}.", chunkInfo); return true; diff --git a/astra/src/main/java/com/slack/astra/chunkManager/ChunkRolloverFactory.java b/astra/src/main/java/com/slack/astra/chunkManager/ChunkRolloverFactory.java index 623975a123..08bb930db7 100644 --- a/astra/src/main/java/com/slack/astra/chunkManager/ChunkRolloverFactory.java +++ b/astra/src/main/java/com/slack/astra/chunkManager/ChunkRolloverFactory.java @@ -1,6 +1,6 @@ package com.slack.astra.chunkManager; -import com.slack.astra.blobfs.BlobFs; +import com.slack.astra.blobfs.ChunkStore; import com.slack.astra.chunk.ReadWriteChunk; import com.slack.astra.chunkrollover.ChunkRollOverStrategy; import io.micrometer.core.instrument.MeterRegistry; @@ -12,23 +12,18 @@ */ public class ChunkRolloverFactory { private final ChunkRollOverStrategy chunkRollOverStrategy; - private final BlobFs blobFs; - private final String s3Bucket; + private final ChunkStore chunkStore; private final MeterRegistry meterRegistry; public ChunkRolloverFactory( - ChunkRollOverStrategy chunkRollOverStrategy, - BlobFs blobFs, - String s3Bucket, - MeterRegistry registry) { + ChunkRollOverStrategy chunkRollOverStrategy, ChunkStore chunkStore, MeterRegistry registry) { this.chunkRollOverStrategy = chunkRollOverStrategy; - this.blobFs = blobFs; - this.s3Bucket = s3Bucket; + this.chunkStore = chunkStore; this.meterRegistry = registry; } - public RollOverChunkTask getRollOverChunkTask(ReadWriteChunk chunk, String chunkId) { - return new RollOverChunkTask<>(chunk, meterRegistry, blobFs, s3Bucket, chunkId); + public RollOverChunkTask getRollOverChunkTask(ReadWriteChunk chunk) { + return new RollOverChunkTask<>(chunk, meterRegistry, chunkStore); } public ChunkRollOverStrategy getChunkRollOverStrategy() { diff --git a/astra/src/main/java/com/slack/astra/chunkManager/IndexingChunkManager.java b/astra/src/main/java/com/slack/astra/chunkManager/IndexingChunkManager.java index 69eb62500a..2db1ac285d 100644 --- a/astra/src/main/java/com/slack/astra/chunkManager/IndexingChunkManager.java +++ b/astra/src/main/java/com/slack/astra/chunkManager/IndexingChunkManager.java @@ -11,7 +11,7 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; -import com.slack.astra.blobfs.BlobFs; +import com.slack.astra.blobfs.ChunkStore; import com.slack.astra.chunk.Chunk; import com.slack.astra.chunk.ChunkInfo; import com.slack.astra.chunk.IndexingChunkImpl; @@ -60,8 +60,7 @@ public class IndexingChunkManager extends ChunkManagerBase { private final String chunkDataPrefix; - private final BlobFs blobFs; - private final String s3Bucket; + private final ChunkStore chunkStore; private final ChunkRollOverStrategy chunkRollOverStrategy; private final AsyncCuratorFramework curatorFramework; private final SearchContext searchContext; @@ -117,8 +116,7 @@ public IndexingChunkManager( String dataDirectory, ChunkRollOverStrategy chunkRollOverStrategy, MeterRegistry registry, - BlobFs blobFs, - String s3Bucket, + ChunkStore chunkStore, ListeningExecutorService rolloverExecutorService, AsyncCuratorFramework curatorFramework, SearchContext searchContext, @@ -134,8 +132,7 @@ public IndexingChunkManager( liveMessagesIndexedGauge = registry.gauge(LIVE_MESSAGES_INDEXED, new AtomicLong(0)); liveBytesIndexedGauge = registry.gauge(LIVE_BYTES_INDEXED, new AtomicLong(0)); - this.blobFs = blobFs; - this.s3Bucket = s3Bucket; + this.chunkStore = chunkStore; this.rolloverExecutorService = rolloverExecutorService; this.rolloverFuture = null; this.curatorFramework = curatorFramework; @@ -201,8 +198,7 @@ private void doRollover(ReadWriteChunk currentChunk) { currentChunk.info().setChunkLastUpdatedTimeEpochMs(Instant.now().toEpochMilli()); RollOverChunkTask rollOverChunkTask = - new RollOverChunkTask<>( - currentChunk, meterRegistry, blobFs, s3Bucket, currentChunk.info().chunkId); + new RollOverChunkTask<>(currentChunk, meterRegistry, chunkStore); if ((rolloverFuture == null) || rolloverFuture.isDone()) { rolloverFuture = rolloverExecutorService.submit(rollOverChunkTask); @@ -448,7 +444,7 @@ public static IndexingChunkManager fromConfig( MeterRegistry meterRegistry, AsyncCuratorFramework curatorFramework, AstraConfigs.IndexerConfig indexerConfig, - BlobFs blobFs, + ChunkStore chunkStore, AstraConfigs.S3Config s3Config) { ChunkRollOverStrategy chunkRollOverStrategy = @@ -459,8 +455,7 @@ public static IndexingChunkManager fromConfig( indexerConfig.getDataDirectory(), chunkRollOverStrategy, meterRegistry, - blobFs, - s3Config.getS3Bucket(), + chunkStore, makeDefaultRollOverExecutor(), curatorFramework, SearchContext.fromConfig(indexerConfig.getServerConfig()), diff --git a/astra/src/main/java/com/slack/astra/chunkManager/RecoveryChunkManager.java b/astra/src/main/java/com/slack/astra/chunkManager/RecoveryChunkManager.java index 04de7996b7..4e845e175e 100644 --- a/astra/src/main/java/com/slack/astra/chunkManager/RecoveryChunkManager.java +++ b/astra/src/main/java/com/slack/astra/chunkManager/RecoveryChunkManager.java @@ -8,7 +8,7 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; -import com.slack.astra.blobfs.BlobFs; +import com.slack.astra.blobfs.ChunkStore; import com.slack.astra.chunk.Chunk; import com.slack.astra.chunk.ChunkFactory; import com.slack.astra.chunk.ReadWriteChunk; @@ -104,7 +104,7 @@ private void doRollover(ReadWriteChunk currentChunk) { currentChunk.info().setChunkLastUpdatedTimeEpochMs(Instant.now().toEpochMilli()); RollOverChunkTask rollOverChunkTask = - chunkRolloverFactory.getRollOverChunkTask(currentChunk, currentChunk.info().chunkId); + chunkRolloverFactory.getRollOverChunkTask(currentChunk); ListenableFuture rolloverFuture = rolloverExecutorService.submit(rollOverChunkTask); Futures.addCallback( @@ -218,8 +218,7 @@ public static RecoveryChunkManager fromConfig( SearchMetadataStore searchMetadataStore, SnapshotMetadataStore snapshotMetadataStore, AstraConfigs.IndexerConfig indexerConfig, - BlobFs blobFs, - AstraConfigs.S3Config s3Config) + ChunkStore chunkStore) throws Exception { SearchContext searchContext = SearchContext.fromConfig(indexerConfig.getServerConfig()); @@ -234,8 +233,7 @@ public static RecoveryChunkManager fromConfig( searchContext); ChunkRolloverFactory chunkRolloverFactory = - new ChunkRolloverFactory( - new NeverRolloverChunkStrategy(), blobFs, s3Config.getS3Bucket(), meterRegistry); + new ChunkRolloverFactory(new NeverRolloverChunkStrategy(), chunkStore, meterRegistry); return new RecoveryChunkManager<>(recoveryChunkFactory, chunkRolloverFactory, meterRegistry); } diff --git a/astra/src/main/java/com/slack/astra/chunkManager/RollOverChunkTask.java b/astra/src/main/java/com/slack/astra/chunkManager/RollOverChunkTask.java index 04230c3fc1..ab6c0680dd 100644 --- a/astra/src/main/java/com/slack/astra/chunkManager/RollOverChunkTask.java +++ b/astra/src/main/java/com/slack/astra/chunkManager/RollOverChunkTask.java @@ -1,6 +1,6 @@ package com.slack.astra.chunkManager; -import com.slack.astra.blobfs.BlobFs; +import com.slack.astra.blobfs.ChunkStore; import com.slack.astra.chunk.ReadWriteChunk; import io.micrometer.core.instrument.Counter; import io.micrometer.core.instrument.MeterRegistry; @@ -30,20 +30,12 @@ public class RollOverChunkTask implements Callable { private final Timer rollOverTimer; private final ReadWriteChunk chunk; - private final String s3Bucket; - private final String s3BucketPrefix; - private final BlobFs blobFs; + private final ChunkStore chunkStore; public RollOverChunkTask( - ReadWriteChunk chunk, - MeterRegistry meterRegistry, - BlobFs blobFs, - String s3Bucket, - String s3BucketPrefix) { + ReadWriteChunk chunk, MeterRegistry meterRegistry, ChunkStore chunkStore) { this.chunk = chunk; - this.blobFs = blobFs; - this.s3Bucket = s3Bucket; - this.s3BucketPrefix = s3BucketPrefix; + this.chunkStore = chunkStore; rolloversInitiatedCounter = meterRegistry.counter(ROLLOVERS_INITIATED); rolloversCompletedCounter = meterRegistry.counter(ROLLOVERS_COMPLETED); rolloversFailedCounter = meterRegistry.counter(ROLLOVERS_FAILED); @@ -61,7 +53,7 @@ private Boolean doRollover() { rolloversInitiatedCounter.increment(); // Run pre-snapshot and upload chunk to blob store. chunk.preSnapshot(); - if (!chunk.snapshotToS3(s3Bucket, s3BucketPrefix, blobFs)) { + if (!chunk.snapshotToS3(chunkStore)) { LOG.warn("Failed to snapshot the chunk to S3"); rolloversFailedCounter.increment(); return false; diff --git a/astra/src/main/java/com/slack/astra/recovery/RecoveryService.java b/astra/src/main/java/com/slack/astra/recovery/RecoveryService.java index c30c5550ac..e3d37393f5 100644 --- a/astra/src/main/java/com/slack/astra/recovery/RecoveryService.java +++ b/astra/src/main/java/com/slack/astra/recovery/RecoveryService.java @@ -8,7 +8,7 @@ import com.google.common.util.concurrent.AbstractIdleService; import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.protobuf.TextFormat; -import com.slack.astra.blobfs.BlobFs; +import com.slack.astra.blobfs.ChunkStore; import com.slack.astra.chunk.SearchContext; import com.slack.astra.chunkManager.RecoveryChunkManager; import com.slack.astra.logstore.LogMessage; @@ -59,7 +59,7 @@ public class RecoveryService extends AbstractIdleService { private final SearchContext searchContext; private final AsyncCuratorFramework curatorFramework; private final MeterRegistry meterRegistry; - private final BlobFs blobFs; + private final ChunkStore chunkStore; private final AstraConfigs.AstraConfig AstraConfig; private final AdminClient adminClient; @@ -92,12 +92,12 @@ public RecoveryService( AstraConfigs.AstraConfig AstraConfig, AsyncCuratorFramework curatorFramework, MeterRegistry meterRegistry, - BlobFs blobFs) { + ChunkStore chunkStore) { this.curatorFramework = curatorFramework; this.searchContext = SearchContext.fromConfig(AstraConfig.getRecoveryConfig().getServerConfig()); this.meterRegistry = meterRegistry; - this.blobFs = blobFs; + this.chunkStore = chunkStore; this.AstraConfig = AstraConfig; adminClient = @@ -299,8 +299,7 @@ boolean handleRecoveryTask(RecoveryTaskMetadata recoveryTaskMetadata) { searchMetadataStore, snapshotMetadataStore, AstraConfig.getIndexerConfig(), - blobFs, - AstraConfig.getS3Config()); + chunkStore); // Ingest data in parallel LogMessageWriterImpl logMessageWriterImpl = new LogMessageWriterImpl(chunkManager); diff --git a/astra/src/main/java/com/slack/astra/server/Astra.java b/astra/src/main/java/com/slack/astra/server/Astra.java index 6acf8dc697..c2cc690690 100644 --- a/astra/src/main/java/com/slack/astra/server/Astra.java +++ b/astra/src/main/java/com/slack/astra/server/Astra.java @@ -3,9 +3,8 @@ import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.Service; import com.google.common.util.concurrent.ServiceManager; -import com.slack.astra.blobfs.BlobFs; import com.slack.astra.blobfs.ChunkStore; -import com.slack.astra.blobfs.S3CrtBlobFs; +import com.slack.astra.blobfs.S3AsyncUtil; import com.slack.astra.bulkIngestApi.BulkIngestApi; import com.slack.astra.bulkIngestApi.BulkIngestKafkaProducer; import com.slack.astra.bulkIngestApi.DatasetRateLimitingService; @@ -93,7 +92,7 @@ public class Astra { } Astra(AstraConfigs.AstraConfig astraConfig, PrometheusMeterRegistry prometheusMeterRegistry) { - this(astraConfig, S3CrtBlobFs.initS3Client(astraConfig.getS3Config()), prometheusMeterRegistry); + this(astraConfig, S3AsyncUtil.initS3Client(astraConfig.getS3Config()), prometheusMeterRegistry); } public static void main(String[] args) throws Exception { @@ -140,13 +139,10 @@ public void start() throws Exception { curatorFramework = CuratorBuilder.build( prometheusMeterRegistry, astraConfig.getMetadataStoreConfig().getZookeeperConfig()); - - // Initialize blobfs. Only S3 is supported currently. - S3CrtBlobFs s3BlobFs = new S3CrtBlobFs(s3Client); ChunkStore chunkStore = new ChunkStore(s3Client, astraConfig.getS3Config().getS3Bucket()); Set services = - getServices(curatorFramework, astraConfig, s3BlobFs, chunkStore, prometheusMeterRegistry); + getServices(curatorFramework, astraConfig, chunkStore, prometheusMeterRegistry); serviceManager = new ServiceManager(services); serviceManager.addListener(getServiceManagerListener(), MoreExecutors.directExecutor()); @@ -156,7 +152,6 @@ public void start() throws Exception { private static Set getServices( AsyncCuratorFramework curatorFramework, AstraConfigs.AstraConfig astraConfig, - BlobFs blobFs, ChunkStore chunkStore, PrometheusMeterRegistry meterRegistry) throws Exception { @@ -170,7 +165,7 @@ private static Set getServices( meterRegistry, curatorFramework, astraConfig.getIndexerConfig(), - blobFs, + chunkStore, astraConfig.getS3Config()); services.add(chunkManager); @@ -404,7 +399,7 @@ private static Set getServices( services.add(armeriaService); RecoveryService recoveryService = - new RecoveryService(astraConfig, curatorFramework, meterRegistry, blobFs); + new RecoveryService(astraConfig, curatorFramework, meterRegistry, chunkStore); services.add(recoveryService); } diff --git a/astra/src/test/java/com/slack/astra/blobfs/ChunkStoreTest.java b/astra/src/test/java/com/slack/astra/blobfs/ChunkStoreTest.java index 789b4972b4..be05b37013 100644 --- a/astra/src/test/java/com/slack/astra/blobfs/ChunkStoreTest.java +++ b/astra/src/test/java/com/slack/astra/blobfs/ChunkStoreTest.java @@ -3,7 +3,6 @@ import static org.assertj.core.api.Assertions.assertThat; import com.adobe.testing.s3mock.junit5.S3MockExtension; -import com.slack.astra.blobfs.s3.S3TestUtils; import java.io.File; import java.io.FileWriter; import java.io.IOException; diff --git a/astra/src/test/java/com/slack/astra/blobfs/s3/S3TestUtils.java b/astra/src/test/java/com/slack/astra/blobfs/S3TestUtils.java similarity index 96% rename from astra/src/test/java/com/slack/astra/blobfs/s3/S3TestUtils.java rename to astra/src/test/java/com/slack/astra/blobfs/S3TestUtils.java index dda3e2ca4b..9f273fbad8 100644 --- a/astra/src/test/java/com/slack/astra/blobfs/s3/S3TestUtils.java +++ b/astra/src/test/java/com/slack/astra/blobfs/S3TestUtils.java @@ -1,4 +1,4 @@ -package com.slack.astra.blobfs.s3; +package com.slack.astra.blobfs; import java.net.URI; import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; diff --git a/astra/src/test/java/com/slack/astra/blobfs/s3/S3CrtBlobFsTest.java b/astra/src/test/java/com/slack/astra/blobfs/s3/S3CrtBlobFsTest.java deleted file mode 100644 index 94d34d1f0e..0000000000 --- a/astra/src/test/java/com/slack/astra/blobfs/s3/S3CrtBlobFsTest.java +++ /dev/null @@ -1,271 +0,0 @@ -package com.slack.astra.blobfs.s3; - -import static org.awaitility.Awaitility.await; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertTrue; - -import com.adobe.testing.s3mock.junit5.S3MockExtension; -import com.google.common.io.Resources; -import com.slack.astra.blobfs.S3CrtBlobFs; -import java.io.File; -import java.io.IOException; -import java.net.URI; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.UUID; -import java.util.concurrent.ExecutionException; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.RegisterExtension; -import software.amazon.awssdk.core.async.AsyncRequestBody; -import software.amazon.awssdk.services.s3.S3AsyncClient; -import software.amazon.awssdk.services.s3.model.CreateBucketRequest; -import software.amazon.awssdk.services.s3.model.HeadObjectRequest; -import software.amazon.awssdk.services.s3.model.HeadObjectResponse; -import software.amazon.awssdk.services.s3.model.PutObjectRequest; - -public class S3CrtBlobFsTest { - @RegisterExtension - public static final S3MockExtension S3_MOCK_EXTENSION = - S3MockExtension.builder().silent().withSecureConnection(false).build(); - - final String DELIMITER = "/"; - final String SCHEME = "s3"; - final String FILE_FORMAT = "%s://%s/%s"; - final String DIR_FORMAT = "%s://%s"; - - private final S3AsyncClient s3Client = - S3TestUtils.createS3CrtClient(S3_MOCK_EXTENSION.getServiceEndpoint()); - private String bucket; - private S3CrtBlobFs s3BlobFs; - - @BeforeEach - public void setUp() throws ExecutionException, InterruptedException { - bucket = "test-bucket-" + UUID.randomUUID(); - s3BlobFs = new S3CrtBlobFs(s3Client); - s3Client.createBucket(CreateBucketRequest.builder().bucket(bucket).build()).get(); - } - - @AfterEach - public void tearDown() throws IOException { - if (s3BlobFs != null) { - s3BlobFs.close(); - } - } - - private void createEmptyFile(String folderName, String fileName) - throws ExecutionException, InterruptedException { - String fileNameWithFolder = folderName + DELIMITER + fileName; - if (folderName.isEmpty()) { - fileNameWithFolder = fileName; - } - s3Client - .putObject( - PutObjectRequest.builder().bucket(bucket).key(fileNameWithFolder).build(), - AsyncRequestBody.fromBytes(new byte[0])) - .get(); - } - - @Test - public void testListFilesInBucketNonRecursive() throws Exception { - String[] originalFiles = new String[] {"a-list.txt", "b-list.txt", "c-list.txt"}; - List expectedFileNames = new ArrayList<>(); - - for (String fileName : originalFiles) { - createEmptyFile("", fileName); - expectedFileNames.add(String.format(FILE_FORMAT, SCHEME, bucket, fileName)); - } - - String[] actualFiles = - s3BlobFs.listFiles(URI.create(String.format(DIR_FORMAT, SCHEME, bucket)), false); - - actualFiles = Arrays.stream(actualFiles).filter(x -> x.contains("list")).toArray(String[]::new); - assertEquals(actualFiles.length, originalFiles.length); - - assertTrue(Arrays.equals(actualFiles, expectedFileNames.toArray())); - } - - @Test - public void testListFilesInFolderNonRecursive() throws Exception { - String folder = "list-files"; - String[] originalFiles = new String[] {"a-list-2.txt", "b-list-2.txt", "c-list-2.txt"}; - - for (String fileName : originalFiles) { - createEmptyFile(folder, fileName); - } - - String[] actualFiles = - s3BlobFs.listFiles(URI.create(String.format(FILE_FORMAT, SCHEME, bucket, folder)), false); - - actualFiles = - Arrays.stream(actualFiles).filter(x -> x.contains("list-2")).toArray(String[]::new); - assertEquals(actualFiles.length, originalFiles.length); - - assertTrue( - Arrays.equals( - Arrays.stream(originalFiles) - .map( - fileName -> - String.format(FILE_FORMAT, SCHEME, bucket, folder + DELIMITER + fileName)) - .toArray(), - actualFiles)); - } - - @Test - public void testListFilesInFolderRecursive() throws Exception { - String folder = "list-files-rec"; - String[] nestedFolders = new String[] {"list-files-child-1", "list-files-child-2"}; - String[] originalFiles = new String[] {"a-list-3.txt", "b-list-3.txt", "c-list-3.txt"}; - - List expectedResultList = new ArrayList<>(); - for (String childFolder : nestedFolders) { - String folderName = folder + DELIMITER + childFolder; - for (String fileName : originalFiles) { - createEmptyFile(folderName, fileName); - expectedResultList.add( - String.format(FILE_FORMAT, SCHEME, bucket, folderName + DELIMITER + fileName)); - } - } - String[] actualFiles = - s3BlobFs.listFiles(URI.create(String.format(FILE_FORMAT, SCHEME, bucket, folder)), true); - - actualFiles = - Arrays.stream(actualFiles).filter(x -> x.contains("list-3")).toArray(String[]::new); - assertEquals(actualFiles.length, expectedResultList.size()); - assertTrue(Arrays.equals(expectedResultList.toArray(), actualFiles)); - } - - @Test - public void testIsDirectory() throws Exception { - String[] originalFiles = new String[] {"a-dir.txt", "b-dir.txt", "c-dir.txt"}; - String folder = "my-files-dir"; - String childFolder = "my-files-dir-child"; - for (String fileName : originalFiles) { - String folderName = folder + DELIMITER + childFolder; - createEmptyFile(folderName, fileName); - } - - boolean isBucketDir = - s3BlobFs.isDirectory(URI.create(String.format(DIR_FORMAT, SCHEME, bucket))); - boolean isDir = - s3BlobFs.isDirectory(URI.create(String.format(FILE_FORMAT, SCHEME, bucket, folder))); - boolean isDirChild = - s3BlobFs.isDirectory( - URI.create( - String.format(FILE_FORMAT, SCHEME, bucket, folder + DELIMITER + childFolder))); - boolean notIsDir = - s3BlobFs.isDirectory( - URI.create( - String.format( - FILE_FORMAT, - SCHEME, - bucket, - folder + DELIMITER + childFolder + DELIMITER + "a-delete.txt"))); - - assertTrue(isBucketDir); - assertTrue(isDir); - assertTrue(isDirChild); - assertFalse(notIsDir); - } - - @Test - public void testExists() throws Exception { - String[] originalFiles = new String[] {"a-ex.txt", "b-ex.txt", "c-ex.txt"}; - String folder = "my-files-dir"; - String childFolder = "my-files-dir-child"; - - for (String fileName : originalFiles) { - String folderName = folder + DELIMITER + childFolder; - createEmptyFile(folderName, fileName); - } - - // await ignoreExceptions is a workaround due to // - // https://github.com/aws/aws-sdk-java-v2/issues/3658 - await() - .ignoreExceptions() - .until(() -> s3BlobFs.exists(URI.create(String.format(DIR_FORMAT, SCHEME, bucket)))); - await() - .ignoreExceptions() - .until( - () -> s3BlobFs.exists(URI.create(String.format(FILE_FORMAT, SCHEME, bucket, folder)))); - await() - .ignoreExceptions() - .until( - () -> - s3BlobFs.exists( - URI.create( - String.format( - FILE_FORMAT, SCHEME, bucket, folder + DELIMITER + childFolder)))); - await() - .ignoreExceptions() - .until( - () -> - s3BlobFs.exists( - URI.create( - String.format( - FILE_FORMAT, - SCHEME, - bucket, - folder + DELIMITER + childFolder + DELIMITER + "a-ex.txt")))); - - await() - .ignoreExceptions() - .until( - () -> - !s3BlobFs.exists( - URI.create( - String.format( - FILE_FORMAT, - SCHEME, - bucket, - folder + DELIMITER + childFolder + DELIMITER + "d-ex.txt")))); - } - - @Test - public void testCopyFromAndToLocal() throws Exception { - String fileName = "copyFile.txt"; - - File fileToCopy = new File(getClass().getClassLoader().getResource(fileName).getFile()); - - s3BlobFs.copyFromLocalFile( - fileToCopy, URI.create(String.format(FILE_FORMAT, SCHEME, bucket, fileName))); - - HeadObjectResponse headObjectResponse = - s3Client.headObject(HeadObjectRequest.builder().bucket(bucket).key(fileName).build()).get(); - - assertEquals(headObjectResponse.contentLength(), (Long) fileToCopy.length()); - - File fileToDownload = new File("copyFile_download_crt.txt").getAbsoluteFile(); - s3BlobFs.copyToLocalFile( - URI.create(String.format(FILE_FORMAT, SCHEME, bucket, fileName)), fileToDownload); - assertEquals(fileToCopy.length(), fileToDownload.length()); - - fileToDownload.deleteOnExit(); - } - - @Test - public void testCopyFromAndToLocalDirectory() throws Exception { - String fileName = "copyFile.txt"; - - File fileToCopy = - new File(Resources.getResource(String.format("s3CrtBlobFsTest/%s", fileName)).getFile()); - - s3BlobFs.copyFromLocalFile( - fileToCopy.getParentFile(), URI.create(String.format(FILE_FORMAT, SCHEME, bucket, ""))); - - HeadObjectResponse headObjectResponse = - s3Client.headObject(HeadObjectRequest.builder().bucket(bucket).key(fileName).build()).get(); - - assertEquals(headObjectResponse.contentLength(), (Long) fileToCopy.length()); - - File fileToDownload = new File(fileName).getAbsoluteFile(); - s3BlobFs.copyToLocalFile( - URI.create(String.format(FILE_FORMAT, SCHEME, bucket, "")), fileToDownload.getParentFile()); - assertEquals(fileToCopy.length(), fileToDownload.length()); - - fileToDownload.deleteOnExit(); - } -} diff --git a/astra/src/test/java/com/slack/astra/chunk/IndexingChunkImplTest.java b/astra/src/test/java/com/slack/astra/chunk/IndexingChunkImplTest.java index 1e4f6bd839..056ce028d9 100644 --- a/astra/src/test/java/com/slack/astra/chunk/IndexingChunkImplTest.java +++ b/astra/src/test/java/com/slack/astra/chunk/IndexingChunkImplTest.java @@ -17,8 +17,8 @@ import brave.Tracing; import com.adobe.testing.s3mock.junit5.S3MockExtension; -import com.slack.astra.blobfs.S3CrtBlobFs; -import com.slack.astra.blobfs.s3.S3TestUtils; +import com.slack.astra.blobfs.ChunkStore; +import com.slack.astra.blobfs.S3TestUtils; import com.slack.astra.logstore.LogMessage; import com.slack.astra.logstore.LuceneIndexStoreImpl; import com.slack.astra.logstore.schema.SchemaAwareLogDocumentBuilderImpl; @@ -634,10 +634,10 @@ public void testSnapshotToNonExistentS3BucketFails() String bucket = "invalid-bucket"; S3AsyncClient s3AsyncClient = S3TestUtils.createS3CrtClient(S3_MOCK_EXTENSION.getServiceEndpoint()); - S3CrtBlobFs s3CrtBlobFs = new S3CrtBlobFs(s3AsyncClient); + ChunkStore chunkStore = new ChunkStore(s3AsyncClient, bucket); // Snapshot to S3 without creating the s3 bucket. - assertThat(chunk.snapshotToS3(bucket, "", s3CrtBlobFs)).isFalse(); + assertThat(chunk.snapshotToS3(chunkStore)).isFalse(); assertThat(chunk.info().getSnapshotPath()).isEqualTo(SnapshotMetadata.LIVE_SNAPSHOT_PATH); // Metadata checks @@ -698,12 +698,12 @@ public void testSnapshotToS3UsingChunkApi() throws Exception { String bucket = "test-bucket-with-prefix"; S3AsyncClient s3AsyncClient = S3TestUtils.createS3CrtClient(S3_MOCK_EXTENSION.getServiceEndpoint()); - S3CrtBlobFs s3CrtBlobFs = new S3CrtBlobFs(s3AsyncClient); s3AsyncClient.createBucket(CreateBucketRequest.builder().bucket(bucket).build()).get(); + ChunkStore chunkStore = new ChunkStore(s3AsyncClient, bucket); // Snapshot to S3 assertThat(chunk.info().getSnapshotPath()).isEqualTo(SnapshotMetadata.LIVE_SNAPSHOT_PATH); - assertThat(chunk.snapshotToS3(bucket, "", s3CrtBlobFs)).isTrue(); + assertThat(chunk.snapshotToS3(chunkStore)).isTrue(); assertThat(chunk.info().getSnapshotPath()).isNotEmpty(); // depending on heap and CFS files this can be 5 or 19. @@ -714,10 +714,13 @@ public void testSnapshotToS3UsingChunkApi() throws Exception { // Check schema file exists in s3 ListObjectsV2Response objectsResponse = - s3AsyncClient.listObjectsV2(ListObjectsV2Request.builder().bucket(bucket).build()).get(); + s3AsyncClient + .listObjectsV2( + ListObjectsV2Request.builder().bucket(bucket).prefix(chunk.id()).build()) + .get(); assertThat( objectsResponse.contents().stream() - .filter(o -> o.key().equals(SCHEMA_FILE_NAME)) + .filter(o -> o.key().contains(SCHEMA_FILE_NAME)) .count()) .isEqualTo(1); diff --git a/astra/src/test/java/com/slack/astra/chunk/ReadOnlyChunkImplTest.java b/astra/src/test/java/com/slack/astra/chunk/ReadOnlyChunkImplTest.java index aa81276741..d9efa033be 100644 --- a/astra/src/test/java/com/slack/astra/chunk/ReadOnlyChunkImplTest.java +++ b/astra/src/test/java/com/slack/astra/chunk/ReadOnlyChunkImplTest.java @@ -1,6 +1,5 @@ package com.slack.astra.chunk; -import static com.slack.astra.blobfs.BlobFsUtils.copyToS3; import static com.slack.astra.chunk.ReadOnlyChunkImpl.CHUNK_ASSIGNMENT_TIMER; import static com.slack.astra.chunk.ReadOnlyChunkImpl.CHUNK_EVICTION_TIMER; import static com.slack.astra.chunk.ReadWriteChunk.SCHEMA_FILE_NAME; @@ -18,8 +17,7 @@ import brave.Tracing; import com.adobe.testing.s3mock.junit5.S3MockExtension; import com.slack.astra.blobfs.ChunkStore; -import com.slack.astra.blobfs.S3CrtBlobFs; -import com.slack.astra.blobfs.s3.S3TestUtils; +import com.slack.astra.blobfs.S3TestUtils; import com.slack.astra.logstore.LogMessage; import com.slack.astra.logstore.LuceneIndexStoreImpl; import com.slack.astra.logstore.schema.SchemaAwareLogDocumentBuilderImpl; @@ -72,7 +70,6 @@ public class ReadOnlyChunkImplTest { private TestingServer testingServer; private MeterRegistry meterRegistry; - private S3CrtBlobFs s3CrtBlobFs; private ChunkStore chunkStore; @RegisterExtension @@ -91,13 +88,11 @@ public void startup() throws Exception { S3AsyncClient s3AsyncClient = S3TestUtils.createS3CrtClient(S3_MOCK_EXTENSION.getServiceEndpoint()); - s3CrtBlobFs = new S3CrtBlobFs(s3AsyncClient); chunkStore = new ChunkStore(s3AsyncClient, TEST_S3_BUCKET); } @AfterEach public void shutdown() throws IOException { - s3CrtBlobFs.close(); testingServer.close(); meterRegistry.close(); } @@ -680,7 +675,7 @@ private void initializeBlobStorageWithIndex(String snapshotId) throws Exception assertThat(dirPath.toFile().listFiles().length).isGreaterThanOrEqualTo(filesToUpload.size()); // Copy files to S3. - copyToS3(dirPath, filesToUpload, TEST_S3_BUCKET, snapshotId, s3CrtBlobFs); + chunkStore.upload(snapshotId, dirPath); } private void initializeCacheNodeAssignment( diff --git a/astra/src/test/java/com/slack/astra/chunk/RecoveryChunkImplTest.java b/astra/src/test/java/com/slack/astra/chunk/RecoveryChunkImplTest.java index 1df5918a1f..1b93ec09af 100644 --- a/astra/src/test/java/com/slack/astra/chunk/RecoveryChunkImplTest.java +++ b/astra/src/test/java/com/slack/astra/chunk/RecoveryChunkImplTest.java @@ -15,8 +15,8 @@ import brave.Tracing; import com.adobe.testing.s3mock.junit5.S3MockExtension; -import com.slack.astra.blobfs.S3CrtBlobFs; -import com.slack.astra.blobfs.s3.S3TestUtils; +import com.slack.astra.blobfs.ChunkStore; +import com.slack.astra.blobfs.S3TestUtils; import com.slack.astra.logstore.LogMessage; import com.slack.astra.logstore.LuceneIndexStoreImpl; import com.slack.astra.logstore.schema.SchemaAwareLogDocumentBuilderImpl; @@ -35,7 +35,6 @@ import io.micrometer.core.instrument.MeterRegistry; import io.micrometer.core.instrument.simple.SimpleMeterRegistry; import java.io.IOException; -import java.net.URI; import java.nio.file.Path; import java.time.Duration; import java.time.Instant; @@ -524,7 +523,6 @@ public class SnapshotTests { private AsyncCuratorFramework curatorFramework; private SnapshotMetadataStore snapshotMetadataStore; private SearchMetadataStore searchMetadataStore; - private S3CrtBlobFs s3CrtBlobFs; @BeforeEach public void setUp() throws Exception { @@ -577,9 +575,6 @@ public void tearDown() throws IOException, TimeoutException { curatorFramework.unwrap().close(); testingServer.close(); registry.close(); - if (s3CrtBlobFs != null) { - s3CrtBlobFs.close(); - } } @Test @@ -617,14 +612,14 @@ public void testSnapshotToNonExistentS3BucketFails() { assertThat(getCount(INDEX_FILES_UPLOAD_FAILED, registry)).isEqualTo(0); // create an S3 client for test - String bucket = "invalid-bucket"; + String bucket = "invalid-buckets"; S3AsyncClient s3AsyncClient = S3TestUtils.createS3CrtClient(S3_MOCK_EXTENSION.getServiceEndpoint()); - s3CrtBlobFs = new S3CrtBlobFs(s3AsyncClient); + ChunkStore chunkStore = new ChunkStore(s3AsyncClient, bucket); // Snapshot to S3 without creating the s3 bucket. - assertThat(chunk.snapshotToS3(bucket, "", s3CrtBlobFs)).isFalse(); + assertThat(chunk.snapshotToS3(chunkStore)).isFalse(); assertThat(chunk.info().getSnapshotPath()).isEqualTo(SnapshotMetadata.LIVE_SNAPSHOT_PATH); // No live snapshot or search metadata is published since the S3 snapshot failed. @@ -672,12 +667,12 @@ public void testSnapshotToS3UsingChunkApi() throws Exception { String bucket = "test-bucket-with-prefix"; S3AsyncClient s3AsyncClient = S3TestUtils.createS3CrtClient(S3_MOCK_EXTENSION.getServiceEndpoint()); - S3CrtBlobFs s3CrtBlobFs = new S3CrtBlobFs(s3AsyncClient); s3AsyncClient.createBucket(CreateBucketRequest.builder().bucket(bucket).build()).get(); + ChunkStore chunkStore = new ChunkStore(s3AsyncClient, bucket); // Snapshot to S3 assertThat(chunk.info().getSnapshotPath()).isEqualTo(SnapshotMetadata.LIVE_SNAPSHOT_PATH); - assertThat(chunk.snapshotToS3(bucket, "", s3CrtBlobFs)).isTrue(); + assertThat(chunk.snapshotToS3(chunkStore)).isTrue(); assertThat(chunk.info().getSnapshotPath()).isNotEmpty(); // depending on heap and CFS files this can be 5 or 19. @@ -694,7 +689,8 @@ public void testSnapshotToS3UsingChunkApi() throws Exception { AstraMetadataTestUtils.listSyncUncached(snapshotMetadataStore); assertThat(afterSnapshots.size()).isEqualTo(1); assertThat(afterSnapshots).contains(ChunkInfo.toSnapshotMetadata(chunk.info(), "")); - assertThat(s3CrtBlobFs.exists(URI.create(afterSnapshots.get(0).snapshotPath))).isTrue(); + + assertThat(chunkStore.listFiles(afterSnapshots.get(0).snapshotId).size()).isGreaterThan(0); // Only non-live snapshots. No live snapshots. assertThat(afterSnapshots.stream().filter(SnapshotMetadata::isLive).count()).isZero(); // No search nodes are added for recovery chunk. diff --git a/astra/src/test/java/com/slack/astra/chunkManager/CachingChunkManagerTest.java b/astra/src/test/java/com/slack/astra/chunkManager/CachingChunkManagerTest.java index e12d01da5c..713989cb1f 100644 --- a/astra/src/test/java/com/slack/astra/chunkManager/CachingChunkManagerTest.java +++ b/astra/src/test/java/com/slack/astra/chunkManager/CachingChunkManagerTest.java @@ -1,6 +1,5 @@ package com.slack.astra.chunkManager; -import static com.slack.astra.blobfs.BlobFsUtils.copyToS3; import static com.slack.astra.chunk.ReadWriteChunk.SCHEMA_FILE_NAME; import static com.slack.astra.chunkManager.CachingChunkManager.ASTRA_NG_DYNAMIC_CHUNK_SIZES_FLAG; import static com.slack.astra.logstore.LuceneIndexStoreImpl.COMMITS_TIMER; @@ -16,8 +15,7 @@ import com.adobe.testing.s3mock.junit5.S3MockExtension; import com.slack.astra.blobfs.ChunkStore; -import com.slack.astra.blobfs.S3CrtBlobFs; -import com.slack.astra.blobfs.s3.S3TestUtils; +import com.slack.astra.blobfs.S3TestUtils; import com.slack.astra.chunk.Chunk; import com.slack.astra.chunk.ReadOnlyChunkImpl; import com.slack.astra.chunk.SearchContext; @@ -63,7 +61,6 @@ public class CachingChunkManagerTest { private TestingServer testingServer; private MeterRegistry meterRegistry; - private S3CrtBlobFs s3CrtBlobFs; private ChunkStore chunkStore; @RegisterExtension @@ -86,7 +83,6 @@ public void startup() throws Exception { S3AsyncClient s3AsyncClient = S3TestUtils.createS3CrtClient(S3_MOCK_EXTENSION.getServiceEndpoint()); - s3CrtBlobFs = new S3CrtBlobFs(s3AsyncClient); chunkStore = new ChunkStore(s3AsyncClient, TEST_S3_BUCKET); } @@ -99,7 +95,6 @@ public void shutdown() throws IOException, TimeoutException { if (curatorFramework != null) { curatorFramework.unwrap().close(); } - s3CrtBlobFs.close(); testingServer.close(); meterRegistry.close(); disableDynamicChunksFlag(); @@ -214,7 +209,7 @@ private void initializeBlobStorageWithIndex(String snapshotId) throws Exception assertThat(dirPath.toFile().listFiles().length).isGreaterThanOrEqualTo(filesToUpload.size()); // Copy files to S3. - copyToS3(dirPath, filesToUpload, TEST_S3_BUCKET, snapshotId, s3CrtBlobFs); + chunkStore.upload(snapshotId, dirPath); } @Test diff --git a/astra/src/test/java/com/slack/astra/chunkManager/IndexingChunkManagerTest.java b/astra/src/test/java/com/slack/astra/chunkManager/IndexingChunkManagerTest.java index a891a8e18d..e0fc0deade 100644 --- a/astra/src/test/java/com/slack/astra/chunkManager/IndexingChunkManagerTest.java +++ b/astra/src/test/java/com/slack/astra/chunkManager/IndexingChunkManagerTest.java @@ -33,8 +33,8 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import com.google.protobuf.ByteString; -import com.slack.astra.blobfs.S3CrtBlobFs; -import com.slack.astra.blobfs.s3.S3TestUtils; +import com.slack.astra.blobfs.ChunkStore; +import com.slack.astra.blobfs.S3TestUtils; import com.slack.astra.chunk.Chunk; import com.slack.astra.chunk.ChunkInfo; import com.slack.astra.chunk.ReadWriteChunk; @@ -113,7 +113,7 @@ public class IndexingChunkManagerTest { private S3AsyncClient s3AsyncClient; private static final String ZK_PATH_PREFIX = "testZK"; - private S3CrtBlobFs s3CrtBlobFs; + private ChunkStore chunkStore; private TestingServer localZkServer; private AsyncCuratorFramework curatorFramework; private SnapshotMetadataStore snapshotMetadataStore; @@ -125,7 +125,7 @@ public void setUp() throws Exception { metricsRegistry = new SimpleMeterRegistry(); // create an S3 client and a bucket for test s3AsyncClient = S3TestUtils.createS3CrtClient(S3_MOCK_EXTENSION.getServiceEndpoint()); - s3CrtBlobFs = new S3CrtBlobFs(s3AsyncClient); + chunkStore = new ChunkStore(s3AsyncClient, S3_TEST_BUCKET); localZkServer = new TestingServer(); localZkServer.start(); @@ -158,7 +158,7 @@ public void tearDown() throws TimeoutException, IOException { private void initChunkManager( ChunkRollOverStrategy chunkRollOverStrategy, - String s3TestBucket, + ChunkStore chunkStore, ListeningExecutorService listeningExecutorService) throws IOException, TimeoutException { SearchContext searchContext = new SearchContext(TEST_HOST, TEST_PORT); @@ -168,8 +168,7 @@ private void initChunkManager( tmpPath.toFile().getAbsolutePath(), chunkRollOverStrategy, metricsRegistry, - s3CrtBlobFs, - s3TestBucket, + chunkStore, listeningExecutorService, curatorFramework, searchContext, @@ -180,7 +179,7 @@ private void initChunkManager( private void initChunkManager( ChunkRollOverStrategy chunkRollOverStrategy, - String s3TestBucket, + ChunkStore chunkStore, ListeningExecutorService listeningExecutorService, AstraConfigs.IndexerConfig indexerConfig) throws IOException, TimeoutException { @@ -191,8 +190,7 @@ private void initChunkManager( tmpPath.toFile().getAbsolutePath(), chunkRollOverStrategy, metricsRegistry, - s3CrtBlobFs, - s3TestBucket, + chunkStore, listeningExecutorService, curatorFramework, searchContext, @@ -209,10 +207,7 @@ public void testDeleteOverMaxThresholdGreaterThanZero() throws IOException, Time AstraConfigs.IndexerConfig indexerConfig = AstraConfigUtil.makeIndexerConfig(TEST_PORT, 1000, 100, 1, 1_000_000_000L); initChunkManager( - chunkRollOverStrategy, - S3_TEST_BUCKET, - MoreExecutors.newDirectExecutorService(), - indexerConfig); + chunkRollOverStrategy, chunkStore, MoreExecutors.newDirectExecutorService(), indexerConfig); assertThat(chunkManager.getChunkList().isEmpty()).isTrue(); final Instant startTime = @@ -289,10 +284,7 @@ public void testDeleteStaleDataDoesNothingWhenGivenLimitLessThan0() AstraConfigs.IndexerConfig indexerConfig = AstraConfigUtil.makeIndexerConfig(TEST_PORT, 1000, 100, -1, 10_000); initChunkManager( - chunkRollOverStrategy, - S3_TEST_BUCKET, - MoreExecutors.newDirectExecutorService(), - indexerConfig); + chunkRollOverStrategy, chunkStore, MoreExecutors.newDirectExecutorService(), indexerConfig); assertThat(chunkManager.getChunkList().isEmpty()).isTrue(); final Instant startTime = @@ -338,8 +330,7 @@ public void closeDuringCleanerTask() new MessageSizeOrCountBasedRolloverStrategy( metricsRegistry, 10 * 1024 * 1024 * 1024L, 1000000L); - initChunkManager( - chunkRollOverStrategy, S3_TEST_BUCKET, MoreExecutors.newDirectExecutorService()); + initChunkManager(chunkRollOverStrategy, chunkStore, MoreExecutors.newDirectExecutorService()); List messages = SpanUtil.makeSpansWithTimeDifference(1, 11, 1, Instant.now()); int offset = 1; @@ -381,8 +372,7 @@ public void testAddMessage() throws Exception { metricsRegistry, 10 * 1024 * 1024 * 1024L, 1000000L); final String CHUNK_DATA_PREFIX = "testData"; - initChunkManager( - chunkRollOverStrategy, S3_TEST_BUCKET, MoreExecutors.newDirectExecutorService()); + initChunkManager(chunkRollOverStrategy, chunkStore, MoreExecutors.newDirectExecutorService()); List messages = SpanUtil.makeSpansWithTimeDifference(1, 100, 1, Instant.now()); int actualChunkSize = 0; @@ -613,8 +603,7 @@ public void testAddAndSearchMessageInMultipleSlices() throws Exception { ChunkRollOverStrategy chunkRollOverStrategy = new MessageSizeOrCountBasedRolloverStrategy(metricsRegistry, 10 * 1024 * 1024 * 1024L, 10L); - initChunkManager( - chunkRollOverStrategy, S3_TEST_BUCKET, MoreExecutors.newDirectExecutorService()); + initChunkManager(chunkRollOverStrategy, chunkStore, MoreExecutors.newDirectExecutorService()); List messages = SpanUtil.makeSpansWithTimeDifference(1, 15, 1, Instant.now()); int offset = 1; @@ -642,8 +631,7 @@ public void testAddAndSearchMessageInSpecificChunks() throws Exception { ChunkRollOverStrategy chunkRollOverStrategy = new MessageSizeOrCountBasedRolloverStrategy(metricsRegistry, 10 * 1024 * 1024 * 1024L, 10L); - initChunkManager( - chunkRollOverStrategy, S3_TEST_BUCKET, MoreExecutors.newDirectExecutorService()); + initChunkManager(chunkRollOverStrategy, chunkStore, MoreExecutors.newDirectExecutorService()); List messages = SpanUtil.makeSpansWithTimeDifference(1, 15, 1, Instant.now()); int offset = 1; @@ -733,7 +721,7 @@ public void testAddMessageWithPropertyTypeConflicts() throws Exception { new MessageSizeOrCountBasedRolloverStrategy(metricsRegistry, 10 * 1024 * 1024 * 1024L, 10L); ListeningExecutorService rollOverExecutor = IndexingChunkManager.makeDefaultRollOverExecutor(); - initChunkManager(chunkRollOverStrategy, S3_TEST_BUCKET, rollOverExecutor); + initChunkManager(chunkRollOverStrategy, chunkStore, rollOverExecutor); // Add a message int offset = 1; @@ -795,8 +783,7 @@ public void testMessagesAddedToActiveChunks() throws Exception { ChunkRollOverStrategy chunkRollOverStrategy = new DiskOrMessageCountBasedRolloverStrategy(metricsRegistry, 10 * 1024 * 1024 * 1024L, 2L); - initChunkManager( - chunkRollOverStrategy, S3_TEST_BUCKET, MoreExecutors.newDirectExecutorService()); + initChunkManager(chunkRollOverStrategy, chunkStore, MoreExecutors.newDirectExecutorService()); // Add a message List msgs = SpanUtil.makeSpansWithTimeDifference(1, 4, 1000, Instant.now()); @@ -848,7 +835,7 @@ public void testMultiThreadedChunkRollover() throws Exception { new DiskOrMessageCountBasedRolloverStrategy(metricsRegistry, 10 * 1024 * 1024 * 1024L, 10L); ListeningExecutorService rollOverExecutor = IndexingChunkManager.makeDefaultRollOverExecutor(); - initChunkManager(chunkRollOverStrategy, S3_TEST_BUCKET, rollOverExecutor); + initChunkManager(chunkRollOverStrategy, chunkStore, rollOverExecutor); List messages = SpanUtil.makeSpansWithTimeDifference(1, 25, 1, Instant.now()); // Add 11 messages to initiate first roll over. @@ -881,8 +868,7 @@ public void testAddMessagesToChunkWithRollover() throws Exception { ChunkRollOverStrategy chunkRollOverStrategy = new DiskOrMessageCountBasedRolloverStrategy(metricsRegistry, 10 * 1024 * 1024 * 1024L, 10L); - initChunkManager( - chunkRollOverStrategy, S3_TEST_BUCKET, MoreExecutors.newDirectExecutorService()); + initChunkManager(chunkRollOverStrategy, chunkStore, MoreExecutors.newDirectExecutorService()); List messages = SpanUtil.makeSpansWithTimeDifference(1, 25, 1, Instant.now()); // Add 11 messages to initiate first roll over. @@ -972,8 +958,7 @@ public void testAllChunkFailures() throws Exception { ChunkRollOverStrategy chunkRollOverStrategy = new DiskOrMessageCountBasedRolloverStrategy(metricsRegistry, 10 * 1024 * 1024 * 1024L, 10L); - initChunkManager( - chunkRollOverStrategy, S3_TEST_BUCKET, MoreExecutors.newDirectExecutorService()); + initChunkManager(chunkRollOverStrategy, chunkStore, MoreExecutors.newDirectExecutorService()); List messages = SpanUtil.makeSpansWithTimeDifference(1, 25, 1, Instant.now()); // Add 11 messages to initiate first roll over. @@ -1051,8 +1036,7 @@ public void testCommitInvalidChunk() throws Exception { final ChunkRollOverStrategy chunkRollOverStrategy = new DiskOrMessageCountBasedRolloverStrategy(metricsRegistry, 10 * 1024 * 1024 * 1024L, 10L); - initChunkManager( - chunkRollOverStrategy, S3_TEST_BUCKET, MoreExecutors.newDirectExecutorService()); + initChunkManager(chunkRollOverStrategy, chunkStore, MoreExecutors.newDirectExecutorService()); int offset = 1; for (Trace.Span m : messages) { @@ -1088,8 +1072,7 @@ public void testMultiChunkSearch() throws Exception { final ChunkRollOverStrategy chunkRollOverStrategy = new DiskOrMessageCountBasedRolloverStrategy(metricsRegistry, 10 * 1024 * 1024 * 1024L, 10L); - initChunkManager( - chunkRollOverStrategy, S3_TEST_BUCKET, MoreExecutors.newDirectExecutorService()); + initChunkManager(chunkRollOverStrategy, chunkStore, MoreExecutors.newDirectExecutorService()); int offset = 1; for (Trace.Span m : messages) { @@ -1203,7 +1186,7 @@ public void testChunkRollOverInProgressExceptionIsThrown() throws Exception { final ChunkRollOverStrategy chunkRollOverStrategy = new DiskOrMessageCountBasedRolloverStrategy(metricsRegistry, 10 * 1024 * 1024 * 1024L, 10L); initChunkManager( - chunkRollOverStrategy, S3_TEST_BUCKET, IndexingChunkManager.makeDefaultRollOverExecutor()); + chunkRollOverStrategy, chunkStore, IndexingChunkManager.makeDefaultRollOverExecutor()); assertThat(AstraMetadataTestUtils.listSyncUncached(snapshotMetadataStore)).isEmpty(); @@ -1250,7 +1233,7 @@ public void testSuccessfulRollOverFinishesOnClose() throws Exception { final ChunkRollOverStrategy chunkRollOverStrategy = new DiskOrMessageCountBasedRolloverStrategy(metricsRegistry, 10 * 1024 * 1024 * 1024L, 10L); initChunkManager( - chunkRollOverStrategy, S3_TEST_BUCKET, IndexingChunkManager.makeDefaultRollOverExecutor()); + chunkRollOverStrategy, chunkStore, IndexingChunkManager.makeDefaultRollOverExecutor()); // Adding a message and close the chunkManager right away should still finish the failed // rollover. @@ -1300,7 +1283,7 @@ public void testFailedRollOverFinishesOnClose() throws Exception { new DiskOrMessageCountBasedRolloverStrategy(metricsRegistry, 10 * 1024 * 1024 * 1024L, 10L); initChunkManager( chunkRollOverStrategy, - S3_TEST_BUCKET + "Fail", + new ChunkStore(s3AsyncClient, S3_TEST_BUCKET + "Fail"), IndexingChunkManager.makeDefaultRollOverExecutor()); // Adding a message and close the chunkManager right away should still finish the failed @@ -1346,7 +1329,7 @@ public void testRollOverFailure() new DiskOrMessageCountBasedRolloverStrategy(metricsRegistry, 10 * 1024 * 1024 * 1024L, 10L); initChunkManager( chunkRollOverStrategy, - S3_TEST_BUCKET + "Fail", + new ChunkStore(s3AsyncClient, S3_TEST_BUCKET + "Fail"), IndexingChunkManager.makeDefaultRollOverExecutor()); int offset = 1; @@ -1391,7 +1374,9 @@ public void testRollOverFailureWithDirectExecutor() final ChunkRollOverStrategy chunkRollOverStrategy = new DiskOrMessageCountBasedRolloverStrategy(metricsRegistry, 10 * 1024 * 1024 * 1024L, 10L); initChunkManager( - chunkRollOverStrategy, S3_TEST_BUCKET + "Fail", MoreExecutors.newDirectExecutorService()); + chunkRollOverStrategy, + new ChunkStore(s3AsyncClient, S3_TEST_BUCKET + "Fail"), + MoreExecutors.newDirectExecutorService()); // Adding a messages very quickly when running a rollover in background would result in an // exception. @@ -1431,8 +1416,7 @@ public void testNewFieldAddedToSchema() throws IOException, TimeoutException { ChunkRollOverStrategy chunkRollOverStrategy = new DiskOrMessageCountBasedRolloverStrategy( metricsRegistry, 10 * 1024 * 1024 * 1024L, 100L); - initChunkManager( - chunkRollOverStrategy, S3_TEST_BUCKET, MoreExecutors.newDirectExecutorService()); + initChunkManager(chunkRollOverStrategy, chunkStore, MoreExecutors.newDirectExecutorService()); List messages1 = SpanUtil.makeSpansWithTimeDifference(1, 10, 1, Instant.now()); Map schemaBefore = chunkManager.getSchema(); @@ -1481,7 +1465,7 @@ public void testMultipleByteRollOversSuccessfully() new DiskOrMessageCountBasedRolloverStrategy( metricsRegistry, maxBytesPerChunk, msgsPerChunk); initChunkManager( - chunkRollOverStrategy, S3_TEST_BUCKET, IndexingChunkManager.makeDefaultRollOverExecutor()); + chunkRollOverStrategy, chunkStore, IndexingChunkManager.makeDefaultRollOverExecutor()); List messages1 = messages.subList(0, 3); List messages2 = messages.subList(3, 6); @@ -1538,7 +1522,7 @@ public void testMultipleCountRollOversSuccessfully() new DiskOrMessageCountBasedRolloverStrategy( metricsRegistry, 10 * 1024 * 1024 * 1024L, msgsPerChunk); initChunkManager( - chunkRollOverStrategy, S3_TEST_BUCKET, IndexingChunkManager.makeDefaultRollOverExecutor()); + chunkRollOverStrategy, chunkStore, IndexingChunkManager.makeDefaultRollOverExecutor()); List messages1 = messages.subList(0, 10); List messages2 = messages.subList(10, 20); diff --git a/astra/src/test/java/com/slack/astra/chunkManager/RecoveryChunkManagerTest.java b/astra/src/test/java/com/slack/astra/chunkManager/RecoveryChunkManagerTest.java index 86ade5a7b9..e6db8714f4 100644 --- a/astra/src/test/java/com/slack/astra/chunkManager/RecoveryChunkManagerTest.java +++ b/astra/src/test/java/com/slack/astra/chunkManager/RecoveryChunkManagerTest.java @@ -22,8 +22,8 @@ import brave.Tracing; import com.adobe.testing.s3mock.junit5.S3MockExtension; -import com.slack.astra.blobfs.S3CrtBlobFs; -import com.slack.astra.blobfs.s3.S3TestUtils; +import com.slack.astra.blobfs.ChunkStore; +import com.slack.astra.blobfs.S3TestUtils; import com.slack.astra.chunk.ChunkInfo; import com.slack.astra.chunk.ReadWriteChunk; import com.slack.astra.logstore.LogMessage; @@ -83,7 +83,7 @@ public class RecoveryChunkManagerTest { private S3AsyncClient s3AsyncClient; private static final String ZK_PATH_PREFIX = "testZK"; - private S3CrtBlobFs s3CrtBlobFs; + private ChunkStore chunkStore; private TestingServer localZkServer; private AsyncCuratorFramework curatorFramework; private SearchMetadataStore searchMetadataStore; @@ -97,7 +97,7 @@ public void setUp() throws Exception { metricsRegistry = new SimpleMeterRegistry(); // create an S3 client. s3AsyncClient = S3TestUtils.createS3CrtClient(S3_MOCK_EXTENSION.getServiceEndpoint()); - s3CrtBlobFs = new S3CrtBlobFs(s3AsyncClient); + chunkStore = new ChunkStore(s3AsyncClient, S3_TEST_BUCKET); localZkServer = new TestingServer(); localZkServer.start(); @@ -130,8 +130,7 @@ public void tearDown() throws TimeoutException, IOException, InterruptedExceptio localZkServer.stop(); } - private void initChunkManager(String testS3Bucket) throws Exception { - + private void initChunkManager(ChunkStore chunkStore) throws Exception { AstraConfig = AstraConfigUtil.makeAstraConfig( "localhost:9090", @@ -139,7 +138,7 @@ private void initChunkManager(String testS3Bucket) throws Exception { "testKafkaTopic", 0, "astra_test_client", - testS3Bucket, + S3_TEST_BUCKET, 9000 + 1, "localhost:2181", "recoveryZK_", @@ -154,8 +153,7 @@ private void initChunkManager(String testS3Bucket) throws Exception { searchMetadataStore, snapshotMetadataStore, AstraConfig.getIndexerConfig(), - s3CrtBlobFs, - AstraConfig.getS3Config()); + chunkStore); chunkManager.startAsync(); chunkManager.awaitRunning(DEFAULT_START_STOP_DURATION); @@ -163,7 +161,7 @@ private void initChunkManager(String testS3Bucket) throws Exception { @Test public void testAddMessageAndRollover() throws Exception { - initChunkManager(S3_TEST_BUCKET); + initChunkManager(chunkStore); List messages = SpanUtil.makeSpansWithTimeDifference(1, 100, 1, Instant.now()); int actualChunkSize = 0; @@ -367,7 +365,7 @@ private void testChunkManagerSearch( @Test public void testAddMessageWithPropertyTypeConflicts() throws Exception { - initChunkManager(S3_TEST_BUCKET); + initChunkManager(chunkStore); // Add a valid message int offset = 1; @@ -415,7 +413,7 @@ public void testAddMessageWithPropertyTypeConflicts() throws Exception { @Test public void testAddMessagesWithFailedRollOverStopsIngestion() throws Exception { // Use a non-existent bucket to induce roll-over failure. - initChunkManager("fakebucket"); + initChunkManager(new ChunkStore(s3AsyncClient, "fakebucket")); int offset = 1; diff --git a/astra/src/test/java/com/slack/astra/chunkrollover/DiskOrMessageCountBasedRolloverStrategyTest.java b/astra/src/test/java/com/slack/astra/chunkrollover/DiskOrMessageCountBasedRolloverStrategyTest.java index 62f7782e21..9933ac9603 100644 --- a/astra/src/test/java/com/slack/astra/chunkrollover/DiskOrMessageCountBasedRolloverStrategyTest.java +++ b/astra/src/test/java/com/slack/astra/chunkrollover/DiskOrMessageCountBasedRolloverStrategyTest.java @@ -14,8 +14,8 @@ import com.adobe.testing.s3mock.junit5.S3MockExtension; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; -import com.slack.astra.blobfs.S3CrtBlobFs; -import com.slack.astra.blobfs.s3.S3TestUtils; +import com.slack.astra.blobfs.ChunkStore; +import com.slack.astra.blobfs.S3TestUtils; import com.slack.astra.chunk.SearchContext; import com.slack.astra.chunkManager.IndexingChunkManager; import com.slack.astra.chunkManager.RollOverChunkTask; @@ -69,7 +69,7 @@ public class DiskOrMessageCountBasedRolloverStrategyTest { private SimpleMeterRegistry metricsRegistry; private S3AsyncClient s3AsyncClient; private static final String ZK_PATH_PREFIX = "testZK"; - private S3CrtBlobFs s3CrtBlobFs; + private ChunkStore chunkStore; private TestingServer localZkServer; private AsyncCuratorFramework curatorFramework; @@ -90,7 +90,7 @@ public void setUp() throws Exception { metricsRegistry = new SimpleMeterRegistry(); s3AsyncClient = S3TestUtils.createS3CrtClient(S3_MOCK_EXTENSION.getServiceEndpoint()); - s3CrtBlobFs = new S3CrtBlobFs(s3AsyncClient); + chunkStore = new ChunkStore(s3AsyncClient, S3_TEST_BUCKET); localZkServer = new TestingServer(); localZkServer.start(); @@ -127,7 +127,6 @@ public void tearDown() throws TimeoutException, IOException { private void initChunkManager( ChunkRollOverStrategy chunkRollOverStrategy, - String s3TestBucket, ListeningExecutorService listeningExecutorService) throws IOException, TimeoutException { SearchContext searchContext = new SearchContext(TEST_HOST, TEST_PORT); @@ -137,8 +136,7 @@ private void initChunkManager( tmpPath.toFile().getAbsolutePath(), chunkRollOverStrategy, metricsRegistry, - s3CrtBlobFs, - s3TestBucket, + chunkStore, listeningExecutorService, curatorFramework, searchContext, @@ -167,8 +165,7 @@ public void testDiskBasedRolloverWithMaxBytes() throws Exception { new DiskOrMessageCountBasedRolloverStrategy( metricsRegistry, MAX_BYTES_PER_CHUNK, 1_000_000_000); - initChunkManager( - chunkRollOverStrategy, S3_TEST_BUCKET, MoreExecutors.newDirectExecutorService()); + initChunkManager(chunkRollOverStrategy, MoreExecutors.newDirectExecutorService()); final Instant startTime = LocalDateTime.of(2020, 10, 1, 10, 10, 0).atZone(ZoneOffset.UTC).toInstant(); @@ -255,8 +252,7 @@ public void testRolloverBasedOnMaxTime() throws Exception { new DiskOrMessageCountBasedRolloverStrategy( metricsRegistry, Long.MAX_VALUE, Long.MAX_VALUE, 2); - initChunkManager( - chunkRollOverStrategy, S3_TEST_BUCKET, MoreExecutors.newDirectExecutorService()); + initChunkManager(chunkRollOverStrategy, MoreExecutors.newDirectExecutorService()); // add 1 message so that new chunk is created // wait for 2+ seconds so that the chunk rollover code will get triggered @@ -281,8 +277,7 @@ public void testDiskBasedRolloverWithMaxMessages() throws Exception { ChunkRollOverStrategy chunkRollOverStrategy = new DiskOrMessageCountBasedRolloverStrategy(metricsRegistry, Long.MAX_VALUE, 4); - initChunkManager( - chunkRollOverStrategy, S3_TEST_BUCKET, MoreExecutors.newDirectExecutorService()); + initChunkManager(chunkRollOverStrategy, MoreExecutors.newDirectExecutorService()); final Instant startTime = Instant.now(); diff --git a/astra/src/test/java/com/slack/astra/clusterManager/SnapshotDeletionServiceTest.java b/astra/src/test/java/com/slack/astra/clusterManager/SnapshotDeletionServiceTest.java index f0c3020d2d..0fc1d401b2 100644 --- a/astra/src/test/java/com/slack/astra/clusterManager/SnapshotDeletionServiceTest.java +++ b/astra/src/test/java/com/slack/astra/clusterManager/SnapshotDeletionServiceTest.java @@ -19,7 +19,7 @@ import brave.Tracing; import com.adobe.testing.s3mock.junit5.S3MockExtension; import com.slack.astra.blobfs.ChunkStore; -import com.slack.astra.blobfs.s3.S3TestUtils; +import com.slack.astra.blobfs.S3TestUtils; import com.slack.astra.metadata.core.CuratorBuilder; import com.slack.astra.metadata.replica.ReplicaMetadata; import com.slack.astra.metadata.replica.ReplicaMetadataStore; diff --git a/astra/src/test/java/com/slack/astra/logstore/LuceneIndexStoreImplTest.java b/astra/src/test/java/com/slack/astra/logstore/LuceneIndexStoreImplTest.java index 923aa5a33e..34bfac8a66 100644 --- a/astra/src/test/java/com/slack/astra/logstore/LuceneIndexStoreImplTest.java +++ b/astra/src/test/java/com/slack/astra/logstore/LuceneIndexStoreImplTest.java @@ -16,7 +16,7 @@ import com.adobe.testing.s3mock.junit5.S3MockExtension; import com.google.protobuf.ByteString; import com.slack.astra.blobfs.ChunkStore; -import com.slack.astra.blobfs.s3.S3TestUtils; +import com.slack.astra.blobfs.S3TestUtils; import com.slack.astra.logstore.LogMessage.ReservedField; import com.slack.astra.logstore.schema.SchemaAwareLogDocumentBuilderImpl; import com.slack.astra.logstore.search.LogIndexSearcherImpl; diff --git a/astra/src/test/java/com/slack/astra/recovery/RecoveryServiceTest.java b/astra/src/test/java/com/slack/astra/recovery/RecoveryServiceTest.java index 5c065f3a6e..60596058d8 100644 --- a/astra/src/test/java/com/slack/astra/recovery/RecoveryServiceTest.java +++ b/astra/src/test/java/com/slack/astra/recovery/RecoveryServiceTest.java @@ -24,10 +24,8 @@ import brave.Tracing; import com.adobe.testing.s3mock.junit5.S3MockExtension; import com.google.common.collect.Maps; -import com.slack.astra.blobfs.BlobFs; -import com.slack.astra.blobfs.BlobFsUtils; -import com.slack.astra.blobfs.S3CrtBlobFs; -import com.slack.astra.blobfs.s3.S3TestUtils; +import com.slack.astra.blobfs.ChunkStore; +import com.slack.astra.blobfs.S3TestUtils; import com.slack.astra.metadata.core.AstraMetadataTestUtils; import com.slack.astra.metadata.core.CuratorBuilder; import com.slack.astra.metadata.recovery.RecoveryNodeMetadata; @@ -43,7 +41,6 @@ import com.slack.astra.writer.kafka.AstraKafkaConsumer; import io.micrometer.core.instrument.MeterRegistry; import io.micrometer.core.instrument.simple.SimpleMeterRegistry; -import java.net.URI; import java.time.Duration; import java.time.Instant; import java.time.LocalDateTime; @@ -83,7 +80,7 @@ public class RecoveryServiceTest { private TestingServer zkServer; private MeterRegistry meterRegistry; - private BlobFs blobFs; + private ChunkStore chunkStore; private TestKafkaServer kafkaServer; private S3AsyncClient s3AsyncClient; private RecoveryService recoveryService; @@ -96,7 +93,7 @@ public void setup() throws Exception { meterRegistry = new SimpleMeterRegistry(); zkServer = new TestingServer(); s3AsyncClient = S3TestUtils.createS3CrtClient(S3_MOCK_EXTENSION.getServiceEndpoint()); - blobFs = new S3CrtBlobFs(s3AsyncClient); + chunkStore = new ChunkStore(s3AsyncClient, TEST_S3_BUCKET); } @AfterEach @@ -108,9 +105,6 @@ public void shutdown() throws Exception { if (curatorFramework != null) { curatorFramework.unwrap().close(); } - if (blobFs != null) { - blobFs.close(); - } if (kafkaServer != null) { kafkaServer.close(); } @@ -126,20 +120,19 @@ public void shutdown() throws Exception { } @SuppressWarnings("OptionalGetWithoutIsPresent") - private AstraConfigs.AstraConfig makeAstraConfig(String testS3Bucket) { - return makeAstraConfig(kafkaServer, testS3Bucket, RecoveryServiceTest.TEST_KAFKA_TOPIC_1); + private AstraConfigs.AstraConfig makeAstraConfig() { + return makeAstraConfig(kafkaServer, RecoveryServiceTest.TEST_KAFKA_TOPIC_1); } @SuppressWarnings("OptionalGetWithoutIsPresent") - private AstraConfigs.AstraConfig makeAstraConfig( - TestKafkaServer testKafkaServer, String testS3Bucket, String topic) { + private AstraConfigs.AstraConfig makeAstraConfig(TestKafkaServer testKafkaServer, String topic) { return AstraConfigUtil.makeAstraConfig( "localhost:" + testKafkaServer.getBroker().getKafkaPort().get(), 9000, topic, 0, RecoveryServiceTest.ASTRA_TEST_CLIENT_1, - testS3Bucket, + TEST_S3_BUCKET, 9000 + 1, zkServer.getConnectString(), "recoveryZK_", @@ -151,12 +144,12 @@ private AstraConfigs.AstraConfig makeAstraConfig( @Test public void testShouldHandleRecoveryTask() throws Exception { - AstraConfigs.AstraConfig astraCfg = makeAstraConfig(TEST_S3_BUCKET); + AstraConfigs.AstraConfig astraCfg = makeAstraConfig(); curatorFramework = CuratorBuilder.build(meterRegistry, astraCfg.getMetadataStoreConfig().getZookeeperConfig()); // Start recovery service - recoveryService = new RecoveryService(astraCfg, curatorFramework, meterRegistry, blobFs); + recoveryService = new RecoveryService(astraCfg, curatorFramework, meterRegistry, chunkStore); recoveryService.startAsync(); recoveryService.awaitRunning(DEFAULT_START_STOP_DURATION); @@ -173,10 +166,7 @@ public void testShouldHandleRecoveryTask() throws Exception { List snapshots = AstraMetadataTestUtils.listSyncUncached(snapshotMetadataStore); assertThat(snapshots.size()).isEqualTo(1); - assertThat(blobFs.listFiles(BlobFsUtils.createURI(TEST_S3_BUCKET, "/", ""), true)).isNotEmpty(); - assertThat(blobFs.exists(URI.create(snapshots.get(0).snapshotPath))).isTrue(); - assertThat(blobFs.listFiles(URI.create(snapshots.get(0).snapshotPath), false).length) - .isGreaterThan(1); + assertThat(chunkStore.listFiles(snapshots.get(0).snapshotId).size()).isGreaterThan(0); assertThat(getCount(MESSAGES_RECEIVED_COUNTER, meterRegistry)).isEqualTo(31); assertThat(getCount(MESSAGES_FAILED_COUNTER, meterRegistry)).isEqualTo(0); assertThat(getCount(ROLLOVERS_INITIATED, meterRegistry)).isEqualTo(1); @@ -189,7 +179,7 @@ public void testShouldHandleRecoveryTaskWithCompletelyUnavailableOffsets() throw final TopicPartition topicPartition = new TopicPartition(TestKafkaServer.TEST_KAFKA_TOPIC, 0); TestKafkaServer.KafkaComponents components = getKafkaTestServer(S3_MOCK_EXTENSION); AstraConfigs.AstraConfig astraCfg = - makeAstraConfig(components.testKafkaServer, TEST_S3_BUCKET, topicPartition.topic()); + makeAstraConfig(components.testKafkaServer, topicPartition.topic()); curatorFramework = CuratorBuilder.build(meterRegistry, astraCfg.getMetadataStoreConfig().getZookeeperConfig()); @@ -242,7 +232,7 @@ public void testShouldHandleRecoveryTaskWithCompletelyUnavailableOffsets() throw // Start recovery service recoveryService = - new RecoveryService(astraCfg, curatorFramework, components.meterRegistry, blobFs); + new RecoveryService(astraCfg, curatorFramework, components.meterRegistry, chunkStore); recoveryService.startAsync(); recoveryService.awaitRunning(DEFAULT_START_STOP_DURATION); long startOffset = 1; @@ -261,7 +251,6 @@ public void testShouldHandleRecoveryTaskWithCompletelyUnavailableOffsets() throw List snapshots = AstraMetadataTestUtils.listSyncUncached(snapshotMetadataStore); assertThat(snapshots.size()).isEqualTo(0); - assertThat(blobFs.listFiles(BlobFsUtils.createURI(TEST_S3_BUCKET, "/", ""), true)).isEmpty(); assertThat(getCount(MESSAGES_FAILED_COUNTER, meterRegistry)).isEqualTo(0); assertThat(getCount(ROLLOVERS_INITIATED, meterRegistry)).isEqualTo(0); assertThat(getCount(ROLLOVERS_COMPLETED, meterRegistry)).isEqualTo(0); @@ -273,7 +262,7 @@ public void testShouldHandleRecoveryTaskWithPartiallyUnavailableOffsets() throws final TopicPartition topicPartition = new TopicPartition(TestKafkaServer.TEST_KAFKA_TOPIC, 0); TestKafkaServer.KafkaComponents components = getKafkaTestServer(S3_MOCK_EXTENSION); AstraConfigs.AstraConfig astraCfg = - makeAstraConfig(components.testKafkaServer, TEST_S3_BUCKET, topicPartition.topic()); + makeAstraConfig(components.testKafkaServer, topicPartition.topic()); curatorFramework = CuratorBuilder.build(meterRegistry, astraCfg.getMetadataStoreConfig().getZookeeperConfig()); @@ -325,7 +314,7 @@ public void testShouldHandleRecoveryTaskWithPartiallyUnavailableOffsets() throws // Start recovery service recoveryService = - new RecoveryService(astraCfg, curatorFramework, components.meterRegistry, blobFs); + new RecoveryService(astraCfg, curatorFramework, components.meterRegistry, chunkStore); recoveryService.startAsync(); recoveryService.awaitRunning(DEFAULT_START_STOP_DURATION); @@ -345,10 +334,7 @@ public void testShouldHandleRecoveryTaskWithPartiallyUnavailableOffsets() throws List snapshots = AstraMetadataTestUtils.listSyncUncached(snapshotMetadataStore); assertThat(snapshots.size()).isEqualTo(1); - assertThat(blobFs.listFiles(BlobFsUtils.createURI(TEST_S3_BUCKET, "/", ""), true)).isNotEmpty(); - assertThat(blobFs.exists(URI.create(snapshots.get(0).snapshotPath))).isTrue(); - assertThat(blobFs.listFiles(URI.create(snapshots.get(0).snapshotPath), false).length) - .isGreaterThan(1); + assertThat(chunkStore.listFiles(snapshots.get(0).snapshotId).size()).isGreaterThan(0); assertThat(getCount(MESSAGES_FAILED_COUNTER, meterRegistry)).isEqualTo(0); assertThat(getCount(ROLLOVERS_INITIATED, meterRegistry)).isEqualTo(0); assertThat(getCount(ROLLOVERS_COMPLETED, meterRegistry)).isEqualTo(0); @@ -358,12 +344,14 @@ public void testShouldHandleRecoveryTaskWithPartiallyUnavailableOffsets() throws @Test public void testShouldHandleRecoveryTaskFailure() throws Exception { String fakeS3Bucket = "fakeBucket"; - AstraConfigs.AstraConfig astraCfg = makeAstraConfig(fakeS3Bucket); + AstraConfigs.AstraConfig astraCfg = makeAstraConfig(); curatorFramework = CuratorBuilder.build(meterRegistry, astraCfg.getMetadataStoreConfig().getZookeeperConfig()); // Start recovery service - recoveryService = new RecoveryService(astraCfg, curatorFramework, meterRegistry, blobFs); + recoveryService = + new RecoveryService( + astraCfg, curatorFramework, meterRegistry, new ChunkStore(s3AsyncClient, fakeS3Bucket)); recoveryService.startAsync(); recoveryService.awaitRunning(DEFAULT_START_STOP_DURATION); @@ -397,12 +385,12 @@ public void testShouldHandleRecoveryTaskFailure() throws Exception { @Test public void testShouldHandleRecoveryTaskAssignmentSuccess() throws Exception { - AstraConfigs.AstraConfig astraCfg = makeAstraConfig(TEST_S3_BUCKET); + AstraConfigs.AstraConfig astraCfg = makeAstraConfig(); curatorFramework = CuratorBuilder.build(meterRegistry, astraCfg.getMetadataStoreConfig().getZookeeperConfig()); // Start recovery service - recoveryService = new RecoveryService(astraCfg, curatorFramework, meterRegistry, blobFs); + recoveryService = new RecoveryService(astraCfg, curatorFramework, meterRegistry, chunkStore); recoveryService.startAsync(); recoveryService.awaitRunning(DEFAULT_START_STOP_DURATION); @@ -467,10 +455,7 @@ public void testShouldHandleRecoveryTaskAssignmentSuccess() throws Exception { List snapshots = AstraMetadataTestUtils.listSyncUncached(snapshotMetadataStore); assertThat(AstraMetadataTestUtils.listSyncUncached(snapshotMetadataStore).size()).isEqualTo(1); - assertThat(blobFs.exists(URI.create(snapshots.get(0).snapshotPath))).isTrue(); - assertThat(blobFs.listFiles(URI.create(snapshots.get(0).snapshotPath), false).length) - .isGreaterThan(1); - + assertThat(chunkStore.listFiles(snapshots.get(0).snapshotId).size()).isGreaterThan(0); assertThat(getCount(MESSAGES_RECEIVED_COUNTER, meterRegistry)).isEqualTo(31); assertThat(getCount(MESSAGES_FAILED_COUNTER, meterRegistry)).isEqualTo(0); assertThat(getCount(ROLLOVERS_INITIATED, meterRegistry)).isEqualTo(1); @@ -481,12 +466,14 @@ public void testShouldHandleRecoveryTaskAssignmentSuccess() throws Exception { @Test public void testShouldHandleRecoveryTaskAssignmentFailure() throws Exception { String fakeS3Bucket = "fakeS3Bucket"; - AstraConfigs.AstraConfig astraCfg = makeAstraConfig(fakeS3Bucket); + AstraConfigs.AstraConfig astraCfg = makeAstraConfig(); curatorFramework = CuratorBuilder.build(meterRegistry, astraCfg.getMetadataStoreConfig().getZookeeperConfig()); // Start recovery service - recoveryService = new RecoveryService(astraCfg, curatorFramework, meterRegistry, blobFs); + recoveryService = + new RecoveryService( + astraCfg, curatorFramework, meterRegistry, new ChunkStore(s3AsyncClient, fakeS3Bucket)); recoveryService.startAsync(); recoveryService.awaitRunning(DEFAULT_START_STOP_DURATION); @@ -656,12 +643,12 @@ public void testValidateOffsetsWhenRecoveryTaskOverlapsWithEndOfKafkaRange() { @Test public void shouldHandleInvalidRecoveryTasks() throws Exception { - AstraConfigs.AstraConfig astraCfg = makeAstraConfig(TEST_S3_BUCKET); + AstraConfigs.AstraConfig astraCfg = makeAstraConfig(); curatorFramework = CuratorBuilder.build(meterRegistry, astraCfg.getMetadataStoreConfig().getZookeeperConfig()); // Start recovery service - recoveryService = new RecoveryService(astraCfg, curatorFramework, meterRegistry, blobFs); + recoveryService = new RecoveryService(astraCfg, curatorFramework, meterRegistry, chunkStore); recoveryService.startAsync(); recoveryService.awaitRunning(DEFAULT_START_STOP_DURATION); diff --git a/astra/src/test/java/com/slack/astra/server/AstraTest.java b/astra/src/test/java/com/slack/astra/server/AstraTest.java index a932c88c6c..12a7f0b85e 100644 --- a/astra/src/test/java/com/slack/astra/server/AstraTest.java +++ b/astra/src/test/java/com/slack/astra/server/AstraTest.java @@ -13,7 +13,7 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; -import com.slack.astra.blobfs.s3.S3TestUtils; +import com.slack.astra.blobfs.S3TestUtils; import com.slack.astra.chunkManager.RollOverChunkTask; import com.slack.astra.metadata.core.AstraMetadataTestUtils; import com.slack.astra.metadata.core.CuratorBuilder; diff --git a/astra/src/test/java/com/slack/astra/testlib/ChunkManagerUtil.java b/astra/src/test/java/com/slack/astra/testlib/ChunkManagerUtil.java index ff7be2f8ab..789e3bad85 100644 --- a/astra/src/test/java/com/slack/astra/testlib/ChunkManagerUtil.java +++ b/astra/src/test/java/com/slack/astra/testlib/ChunkManagerUtil.java @@ -5,8 +5,8 @@ import com.adobe.testing.s3mock.junit5.S3MockExtension; import com.google.common.io.Files; import com.google.common.util.concurrent.MoreExecutors; -import com.slack.astra.blobfs.S3CrtBlobFs; -import com.slack.astra.blobfs.s3.S3TestUtils; +import com.slack.astra.blobfs.ChunkStore; +import com.slack.astra.blobfs.S3TestUtils; import com.slack.astra.chunk.SearchContext; import com.slack.astra.chunkManager.IndexingChunkManager; import com.slack.astra.chunkrollover.ChunkRollOverStrategy; @@ -89,7 +89,7 @@ public ChunkManagerUtil( tempFolder = Files.createTempDir(); // TODO: don't use beta func. s3AsyncClient = S3TestUtils.createS3CrtClient(s3MockExtension.getServiceEndpoint()); - S3CrtBlobFs s3CrtBlobFs = new S3CrtBlobFs(s3AsyncClient); + ChunkStore chunkStore = new ChunkStore(s3AsyncClient, s3Bucket); this.zkServer = zkServer; // noop if zk has already been started by the caller @@ -107,8 +107,7 @@ public ChunkManagerUtil( tempFolder.getAbsolutePath(), chunkRollOverStrategy, meterRegistry, - s3CrtBlobFs, - s3Bucket, + chunkStore, MoreExecutors.newDirectExecutorService(), curatorFramework, searchContext, From 04090c52c641b1e13308263a4e65d303274790b1 Mon Sep 17 00:00:00 2001 From: Bryan Burkholder Date: Wed, 14 Aug 2024 12:57:57 -0700 Subject: [PATCH 6/7] Remove snapshotpath, index type from snapshots --- .../java/com/slack/astra/chunk/ChunkInfo.java | 27 +- .../slack/astra/chunk/IndexingChunkImpl.java | 7 +- .../slack/astra/chunk/ReadOnlyChunkImpl.java | 8 +- .../com/slack/astra/chunk/ReadWriteChunk.java | 6 +- .../astra/chunkManager/ChunkManagerBase.java | 1 - .../ReplicaCreationService.java | 6 +- .../SnapshotDeletionService.java | 3 +- .../metadata/cache/CacheSlotMetadata.java | 14 +- .../cache/CacheSlotMetadataSerializer.java | 2 - .../cache/CacheSlotMetadataStore.java | 1 - .../metadata/replica/ReplicaMetadata.java | 12 +- .../replica/ReplicaMetadataSerializer.java | 4 +- .../metadata/snapshot/SnapshotMetadata.java | 84 ++---- .../snapshot/SnapshotMetadataSerializer.java | 4 - astra/src/main/proto/metadata.proto | 11 +- .../com/slack/astra/chunk/ChunkInfoTest.java | 51 +--- .../astra/chunk/IndexingChunkImplTest.java | 18 +- .../astra/chunk/ReadOnlyChunkImplTest.java | 7 +- .../astra/chunk/RecoveryChunkImplTest.java | 3 - .../chunkManager/CachingChunkManagerTest.java | 4 +- .../IndexingChunkManagerTest.java | 6 +- .../CacheNodeAssignmentServiceTest.java | 85 ++---- .../ClusterHpaMetricServiceTest.java | 56 ++-- .../ReplicaAssignmentServiceTest.java | 155 ++-------- .../ReplicaCreationServiceTest.java | 92 +----- .../ReplicaDeletionServiceTest.java | 32 +- .../ReplicaEvictionServiceTest.java | 126 +------- .../ReplicaRestoreServiceTest.java | 13 +- .../SnapshotDeletionServiceTest.java | 41 +-- .../AstraDistributedQueryServiceTest.java | 5 - .../CacheSlotMetadataSerializerTest.java | 13 +- .../cache/CacheSlotMetadataStoreTest.java | 40 +-- .../metadata/cache/CacheSlotMetadataTest.java | 81 +---- .../ReplicaMetadataSerializerTest.java | 11 +- .../metadata/replica/ReplicaMetadataTest.java | 74 +---- .../SnapshotMetadataSerializerTest.java | 12 +- .../snapshot/SnapshotMetadataTest.java | 71 +---- .../slack/astra/server/AstraIndexerTest.java | 144 ++------- .../astra/server/ManagerApiGrpcTest.java | 54 ++-- .../astra/server/RecoveryTaskCreatorTest.java | 282 +++--------------- .../slack/astra/testlib/ChunkManagerUtil.java | 2 +- .../com/slack/astra/util/SnapshotUtil.java | 18 -- 42 files changed, 284 insertions(+), 1402 deletions(-) delete mode 100644 astra/src/test/java/com/slack/astra/util/SnapshotUtil.java diff --git a/astra/src/main/java/com/slack/astra/chunk/ChunkInfo.java b/astra/src/main/java/com/slack/astra/chunk/ChunkInfo.java index b7c09e598f..81fe0ddba7 100644 --- a/astra/src/main/java/com/slack/astra/chunk/ChunkInfo.java +++ b/astra/src/main/java/com/slack/astra/chunk/ChunkInfo.java @@ -3,7 +3,6 @@ import static com.slack.astra.util.ArgValidationUtils.ensureTrue; import com.slack.astra.metadata.snapshot.SnapshotMetadata; -import com.slack.astra.proto.metadata.Metadata; import java.util.Objects; /** @@ -31,19 +30,16 @@ public static ChunkInfo fromSnapshotMetadata(SnapshotMetadata snapshotMetadata) snapshotMetadata.endTimeEpochMs, snapshotMetadata.maxOffset, snapshotMetadata.partitionId, - snapshotMetadata.snapshotPath, snapshotMetadata.sizeInBytesOnDisk); } public static SnapshotMetadata toSnapshotMetadata(ChunkInfo chunkInfo, String chunkPrefix) { return new SnapshotMetadata( chunkPrefix + chunkInfo.chunkId, - chunkInfo.snapshotPath, chunkInfo.getDataStartTimeEpochMs(), chunkInfo.getDataEndTimeEpochMs(), chunkInfo.maxOffset, chunkInfo.kafkaPartitionId, - Metadata.IndexType.LOGS_LUCENE9, chunkInfo.sizeInBytesOnDisk); } @@ -77,14 +73,10 @@ public static SnapshotMetadata toSnapshotMetadata(ChunkInfo chunkInfo, String ch // indexing and snapshotting and is not useful afterwards. private long chunkSnapshotTimeEpochMs; - // Path to S3 snapshot. - private String snapshotPath; - // Size of chunk on disk in bytes private long sizeInBytesOnDisk; - public ChunkInfo( - String chunkId, long chunkCreationTimeEpochMs, String kafkaPartitionId, String snapshotPath) { + public ChunkInfo(String chunkId, long chunkCreationTimeEpochMs, String kafkaPartitionId) { // TODO: Should we set the snapshot time to creation time also? this( chunkId, @@ -95,7 +87,6 @@ public ChunkInfo( 0, DEFAULT_MAX_OFFSET, kafkaPartitionId, - snapshotPath, 0); } @@ -108,7 +99,6 @@ public ChunkInfo( long chunkSnapshotTimeEpochMs, long maxOffset, String kafkaPartitionId, - String snapshotPath, long sizeInBytesOnDisk) { ensureTrue(chunkId != null && !chunkId.isEmpty(), "Invalid chunk dataset name " + chunkId); ensureTrue( @@ -124,7 +114,6 @@ public ChunkInfo( this.chunkSnapshotTimeEpochMs = chunkSnapshotTimeEpochMs; this.maxOffset = maxOffset; this.kafkaPartitionId = kafkaPartitionId; - this.snapshotPath = snapshotPath; this.sizeInBytesOnDisk = sizeInBytesOnDisk; } @@ -164,14 +153,6 @@ public void setChunkLastUpdatedTimeEpochMs(long chunkLastUpdatedTimeEpochMs) { this.chunkLastUpdatedTimeEpochMs = chunkLastUpdatedTimeEpochMs; } - public void setSnapshotPath(String snapshotPath) { - this.snapshotPath = snapshotPath; - } - - public String getSnapshotPath() { - return snapshotPath; - } - public long getSizeInBytesOnDisk() { return sizeInBytesOnDisk; } @@ -238,8 +219,6 @@ public String toString() { + dataEndTimeEpochMs + ", chunkSnapshotTimeEpochMs=" + chunkSnapshotTimeEpochMs - + ", snapshotPath='" - + snapshotPath + ", sizeInBytesOnDisk='" + sizeInBytesOnDisk + '}'; @@ -258,7 +237,6 @@ public boolean equals(Object o) { && chunkSnapshotTimeEpochMs == chunkInfo.chunkSnapshotTimeEpochMs && Objects.equals(chunkId, chunkInfo.chunkId) && Objects.equals(kafkaPartitionId, chunkInfo.kafkaPartitionId) - && Objects.equals(snapshotPath, chunkInfo.snapshotPath) && sizeInBytesOnDisk == chunkInfo.sizeInBytesOnDisk; } @@ -272,7 +250,6 @@ public int hashCode() { chunkLastUpdatedTimeEpochMs, dataStartTimeEpochMs, dataEndTimeEpochMs, - chunkSnapshotTimeEpochMs, - snapshotPath); + chunkSnapshotTimeEpochMs); } } diff --git a/astra/src/main/java/com/slack/astra/chunk/IndexingChunkImpl.java b/astra/src/main/java/com/slack/astra/chunk/IndexingChunkImpl.java index 70fc76e74d..718a6a4a46 100644 --- a/astra/src/main/java/com/slack/astra/chunk/IndexingChunkImpl.java +++ b/astra/src/main/java/com/slack/astra/chunk/IndexingChunkImpl.java @@ -6,7 +6,6 @@ import com.slack.astra.metadata.search.SearchMetadataStore; import com.slack.astra.metadata.snapshot.SnapshotMetadata; import com.slack.astra.metadata.snapshot.SnapshotMetadataStore; -import com.slack.astra.proto.metadata.Metadata; import io.micrometer.core.instrument.MeterRegistry; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -66,18 +65,16 @@ public void postSnapshot() { SnapshotMetadata nonLiveSnapshotMetadata = toSnapshotMetadata(chunkInfo, ""); snapshotMetadataStore.createSync(nonLiveSnapshotMetadata); - // Update the live snapshot. Keep the same snapshotId and snapshotPath to + // Update the live snapshot. Keep the same snapshotId to // ensure it's a live snapshot. SnapshotMetadata updatedSnapshotMetadata = new SnapshotMetadata( liveSnapshotMetadata.snapshotId, - liveSnapshotMetadata.snapshotPath, chunkInfo.getDataStartTimeEpochMs(), chunkInfo.getDataEndTimeEpochMs(), chunkInfo.getMaxOffset(), chunkInfo.getKafkaPartitionId(), - Metadata.IndexType.LOGS_LUCENE9, - chunkInfo.getSizeInBytesOnDisk()); + liveSnapshotMetadata.sizeInBytesOnDisk); snapshotMetadataStore.updateSync(updatedSnapshotMetadata); liveSnapshotMetadata = updatedSnapshotMetadata; diff --git a/astra/src/main/java/com/slack/astra/chunk/ReadOnlyChunkImpl.java b/astra/src/main/java/com/slack/astra/chunk/ReadOnlyChunkImpl.java index 8dc1b5bd3c..4ec25ef396 100644 --- a/astra/src/main/java/com/slack/astra/chunk/ReadOnlyChunkImpl.java +++ b/astra/src/main/java/com/slack/astra/chunk/ReadOnlyChunkImpl.java @@ -29,7 +29,6 @@ import java.nio.file.Files; import java.nio.file.Path; import java.time.Instant; -import java.util.List; import java.util.Map; import java.util.Objects; import java.util.UUID; @@ -152,7 +151,6 @@ public ReadOnlyChunkImpl( Metadata.CacheSlotMetadata.CacheSlotState.FREE, "", Instant.now().toEpochMilli(), - List.of(Metadata.IndexType.LOGS_LUCENE9), searchContext.hostname, replicaSet); cacheSlotMetadataStore.createSync(cacheSlotMetadata); @@ -265,9 +263,8 @@ public void downloadChunkData() { long durationNanos = assignmentTimer.stop(chunkAssignmentTimerSuccess); LOG.info( - "Downloaded chunk with snapshot id '{}' at path '{}' in {} seconds, was {}", + "Downloaded chunk with snapshot id '{}' in {} seconds, was {}", snapshotMetadata.snapshotId, - snapshotMetadata.snapshotPath, TimeUnit.SECONDS.convert(durationNanos, TimeUnit.NANOSECONDS), FileUtils.byteCountToDisplaySize(FileUtils.sizeOfDirectory(dataDirectory.toFile()))); } catch (Exception e) { @@ -408,9 +405,8 @@ private void handleChunkAssignment(CacheSlotMetadata cacheSlotMetadata) { long durationNanos = assignmentTimer.stop(chunkAssignmentTimerSuccess); LOG.debug( - "Downloaded chunk with snapshot id '{}' at path '{}' in {} seconds, was {}", + "Downloaded chunk with snapshot id '{}' in {} seconds, was {}", snapshotMetadata.snapshotId, - snapshotMetadata.snapshotPath, TimeUnit.SECONDS.convert(durationNanos, TimeUnit.NANOSECONDS), FileUtils.byteCountToDisplaySize(FileUtils.sizeOfDirectory(dataDirectory.toFile()))); } catch (Exception e) { diff --git a/astra/src/main/java/com/slack/astra/chunk/ReadWriteChunk.java b/astra/src/main/java/com/slack/astra/chunk/ReadWriteChunk.java index 310527ef35..d91a6c33be 100644 --- a/astra/src/main/java/com/slack/astra/chunk/ReadWriteChunk.java +++ b/astra/src/main/java/com/slack/astra/chunk/ReadWriteChunk.java @@ -68,7 +68,7 @@ public abstract class ReadWriteChunk implements Chunk { public static final String INDEX_FILES_UPLOAD = "index_files_upload"; public static final String INDEX_FILES_UPLOAD_FAILED = "index_files_upload_failed"; public static final String SNAPSHOT_TIMER = "snapshot.timer"; - public static final String LIVE_SNAPSHOT_PREFIX = SnapshotMetadata.LIVE_SNAPSHOT_PATH + "_"; + public static final String LIVE_SNAPSHOT_PREFIX = "LIVE_"; public static final String SCHEMA_FILE_NAME = "schema.json"; private final LogStore logStore; @@ -111,8 +111,7 @@ protected ReadWriteChunk( new ChunkInfo( chunkDataPrefix + "_" + chunkCreationTime.getEpochSecond() + "_" + logStoreId, chunkCreationTime.toEpochMilli(), - kafkaPartitionId, - SnapshotMetadata.LIVE_SNAPSHOT_PATH); + kafkaPartitionId); readOnly = false; this.meterRegistry = meterRegistry; @@ -253,7 +252,6 @@ public boolean snapshotToS3(ChunkStore chunkStore) { chunkStore.upload(chunkInfo.chunkId, dirPath); snapshotTimer.stop(meterRegistry.timer(SNAPSHOT_TIMER)); - chunkInfo.setSnapshotPath(chunkStore.getRemotePath(chunkInfo.chunkId)); chunkInfo.setSizeInBytesOnDisk(totalBytes); logger.info("Finished RW chunk snapshot to S3 {}.", chunkInfo); return true; diff --git a/astra/src/main/java/com/slack/astra/chunkManager/ChunkManagerBase.java b/astra/src/main/java/com/slack/astra/chunkManager/ChunkManagerBase.java index 0d7b75f320..77fd4c2996 100644 --- a/astra/src/main/java/com/slack/astra/chunkManager/ChunkManagerBase.java +++ b/astra/src/main/java/com/slack/astra/chunkManager/ChunkManagerBase.java @@ -98,7 +98,6 @@ public SearchResult query(SearchQuery query, Duration queryTimeout) { Tracing.currentTracer() .startScopedSpan("ChunkManagerBase.chunkQuery"); span.tag("chunkId", chunk.id()); - span.tag("chunkSnapshotPath", chunk.info().getSnapshotPath()); concurrentQueries.acquire(); try { return chunk.query(query); diff --git a/astra/src/main/java/com/slack/astra/clusterManager/ReplicaCreationService.java b/astra/src/main/java/com/slack/astra/clusterManager/ReplicaCreationService.java index 48ace911a6..e603b25490 100644 --- a/astra/src/main/java/com/slack/astra/clusterManager/ReplicaCreationService.java +++ b/astra/src/main/java/com/slack/astra/clusterManager/ReplicaCreationService.java @@ -18,7 +18,6 @@ import com.slack.astra.metadata.snapshot.SnapshotMetadata; import com.slack.astra.metadata.snapshot.SnapshotMetadataStore; import com.slack.astra.proto.config.AstraConfigs; -import com.slack.astra.proto.metadata.Metadata; import io.micrometer.core.instrument.Counter; import io.micrometer.core.instrument.MeterRegistry; import io.micrometer.core.instrument.Timer; @@ -177,7 +176,7 @@ protected Map createReplicasForUnassignedSnapshots() { .filter( snapshotMetadata -> snapshotMetadata.endTimeEpochMs > snapshotExpiration - && !SnapshotMetadata.isLive(snapshotMetadata) + && !snapshotMetadata.isLive() && !existingReplicas.contains(snapshotMetadata.snapshotId)) .map( (snapshotMetadata) -> { @@ -249,7 +248,6 @@ public static ReplicaMetadata replicaMetadataFromSnapshotId( replicaSet, Instant.now().toEpochMilli(), expireAfter.toEpochMilli(), - isRestored, - Metadata.IndexType.LOGS_LUCENE9); + isRestored); } } diff --git a/astra/src/main/java/com/slack/astra/clusterManager/SnapshotDeletionService.java b/astra/src/main/java/com/slack/astra/clusterManager/SnapshotDeletionService.java index 98657b7e38..d9676c7d5e 100644 --- a/astra/src/main/java/com/slack/astra/clusterManager/SnapshotDeletionService.java +++ b/astra/src/main/java/com/slack/astra/clusterManager/SnapshotDeletionService.java @@ -14,7 +14,6 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.slack.astra.blobfs.ChunkStore; import com.slack.astra.metadata.replica.ReplicaMetadataStore; -import com.slack.astra.metadata.snapshot.SnapshotMetadata; import com.slack.astra.metadata.snapshot.SnapshotMetadataStore; import com.slack.astra.proto.config.AstraConfigs; import io.micrometer.core.instrument.Counter; @@ -175,7 +174,7 @@ protected int deleteExpiredSnapshotsWithoutReplicas() { // served from the indexers. To avoid the whole headache of managing all the // different states we could be in, we should just disable the deletion of live // snapshots whole-cloth. We clean those up when a node boots anyhow - .filter(snapshotMetadata -> !SnapshotMetadata.isLive(snapshotMetadata)) + .filter(snapshotMetadata -> !snapshotMetadata.isLive()) .map( snapshotMetadata -> { ListenableFuture future = diff --git a/astra/src/main/java/com/slack/astra/metadata/cache/CacheSlotMetadata.java b/astra/src/main/java/com/slack/astra/metadata/cache/CacheSlotMetadata.java index cc7db5c06d..c06b07cf88 100644 --- a/astra/src/main/java/com/slack/astra/metadata/cache/CacheSlotMetadata.java +++ b/astra/src/main/java/com/slack/astra/metadata/cache/CacheSlotMetadata.java @@ -4,8 +4,6 @@ import com.slack.astra.metadata.core.AstraPartitionedMetadata; import com.slack.astra.proto.metadata.Metadata; -import java.util.Collections; -import java.util.List; import java.util.Objects; /** @@ -18,23 +16,18 @@ public class CacheSlotMetadata extends AstraPartitionedMetadata { public final Metadata.CacheSlotMetadata.CacheSlotState cacheSlotState; public final String replicaId; public final long updatedTimeEpochMs; - public final List supportedIndexTypes; public CacheSlotMetadata( String name, Metadata.CacheSlotMetadata.CacheSlotState cacheSlotState, String replicaId, long updatedTimeEpochMs, - List supportedIndexTypes, String hostname, String replicaSet) { super(name); checkArgument(hostname != null && !hostname.isEmpty(), "Hostname cannot be null or empty"); checkArgument(cacheSlotState != null, "Cache slot state cannot be null"); checkArgument(updatedTimeEpochMs > 0, "Updated time must be greater than 0"); - checkArgument( - supportedIndexTypes != null && !supportedIndexTypes.isEmpty(), - "supported index types shouldn't be empty"); if (cacheSlotState.equals(Metadata.CacheSlotMetadata.CacheSlotState.FREE)) { checkArgument( replicaId != null && replicaId.isEmpty(), @@ -50,7 +43,6 @@ public CacheSlotMetadata( this.cacheSlotState = cacheSlotState; this.replicaId = replicaId; this.updatedTimeEpochMs = updatedTimeEpochMs; - this.supportedIndexTypes = Collections.unmodifiableList(supportedIndexTypes); } public String getHostname() { @@ -67,8 +59,7 @@ public boolean equals(Object o) { if (!hostname.equals(that.hostname)) return false; if (!Objects.equals(replicaSet, that.replicaSet)) return false; if (cacheSlotState != that.cacheSlotState) return false; - if (!replicaId.equals(that.replicaId)) return false; - return supportedIndexTypes.equals(that.supportedIndexTypes); + return replicaId.equals(that.replicaId); } @Override @@ -79,7 +70,6 @@ public int hashCode() { result = 31 * result + cacheSlotState.hashCode(); result = 31 * result + replicaId.hashCode(); result = 31 * result + (int) (updatedTimeEpochMs ^ (updatedTimeEpochMs >>> 32)); - result = 31 * result + supportedIndexTypes.hashCode(); return result; } @@ -99,8 +89,6 @@ public String toString() { + '\'' + ", updatedTimeEpochMs=" + updatedTimeEpochMs - + ", supportedIndexTypes=" - + supportedIndexTypes + ", name='" + name + '\'' diff --git a/astra/src/main/java/com/slack/astra/metadata/cache/CacheSlotMetadataSerializer.java b/astra/src/main/java/com/slack/astra/metadata/cache/CacheSlotMetadataSerializer.java index 67b263d289..900f512568 100644 --- a/astra/src/main/java/com/slack/astra/metadata/cache/CacheSlotMetadataSerializer.java +++ b/astra/src/main/java/com/slack/astra/metadata/cache/CacheSlotMetadataSerializer.java @@ -14,7 +14,6 @@ private static CacheSlotMetadata fromCacheSlotMetadataProto( cacheSlotMetadataProto.getCacheSlotState().name()), cacheSlotMetadataProto.getReplicaId(), cacheSlotMetadataProto.getUpdatedTimeEpochMs(), - cacheSlotMetadataProto.getSupportedIndexTypesList(), cacheSlotMetadataProto.getHostname(), cacheSlotMetadataProto.getReplicaSet()); } @@ -25,7 +24,6 @@ private static Metadata.CacheSlotMetadata toCacheSlotMetadataProto(CacheSlotMeta .setReplicaId(metadata.replicaId) .setCacheSlotState(metadata.cacheSlotState) .setUpdatedTimeEpochMs(metadata.updatedTimeEpochMs) - .addAllSupportedIndexTypes(metadata.supportedIndexTypes) .setHostname(metadata.hostname) .setReplicaSet(metadata.replicaSet) .build(); diff --git a/astra/src/main/java/com/slack/astra/metadata/cache/CacheSlotMetadataStore.java b/astra/src/main/java/com/slack/astra/metadata/cache/CacheSlotMetadataStore.java index eb331082db..ad870148ca 100644 --- a/astra/src/main/java/com/slack/astra/metadata/cache/CacheSlotMetadataStore.java +++ b/astra/src/main/java/com/slack/astra/metadata/cache/CacheSlotMetadataStore.java @@ -50,7 +50,6 @@ public ListenableFuture updateCacheSlotStateStateWithReplicaId( newState, replicaId, Instant.now().toEpochMilli(), - cacheSlotMetadata.supportedIndexTypes, cacheSlotMetadata.hostname, cacheSlotMetadata.replicaSet); // todo - consider refactoring this to return a completable future instead diff --git a/astra/src/main/java/com/slack/astra/metadata/replica/ReplicaMetadata.java b/astra/src/main/java/com/slack/astra/metadata/replica/ReplicaMetadata.java index 2ac71627ad..c1c163ab20 100644 --- a/astra/src/main/java/com/slack/astra/metadata/replica/ReplicaMetadata.java +++ b/astra/src/main/java/com/slack/astra/metadata/replica/ReplicaMetadata.java @@ -3,7 +3,6 @@ import static com.google.common.base.Preconditions.checkArgument; import com.slack.astra.metadata.core.AstraPartitionedMetadata; -import com.slack.astra.proto.metadata.Metadata; import java.time.Instant; import java.time.ZoneOffset; import java.time.ZonedDateTime; @@ -23,7 +22,6 @@ public class ReplicaMetadata extends AstraPartitionedMetadata { public final long createdTimeEpochMs; public final long expireAfterEpochMs; public boolean isRestored; - public final Metadata.IndexType indexType; public ReplicaMetadata( String name, @@ -31,8 +29,7 @@ public ReplicaMetadata( String replicaSet, long createdTimeEpochMs, long expireAfterEpochMs, - boolean isRestored, - Metadata.IndexType indexType) { + boolean isRestored) { super(name); checkArgument(createdTimeEpochMs > 0, "Created time must be greater than 0"); checkArgument(expireAfterEpochMs >= 0, "Expiration time must be greater than or equal to 0"); @@ -44,7 +41,6 @@ public ReplicaMetadata( this.createdTimeEpochMs = createdTimeEpochMs; this.expireAfterEpochMs = expireAfterEpochMs; this.isRestored = isRestored; - this.indexType = indexType; } public String getSnapshotId() { @@ -77,8 +73,7 @@ public boolean equals(Object o) { if (expireAfterEpochMs != that.expireAfterEpochMs) return false; if (isRestored != that.isRestored) return false; if (!snapshotId.equals(that.snapshotId)) return false; - if (!replicaSet.equals(that.replicaSet)) return false; - return indexType == that.indexType; + return replicaSet.equals(that.replicaSet); } @Override @@ -89,7 +84,6 @@ public int hashCode() { result = 31 * result + (int) (createdTimeEpochMs ^ (createdTimeEpochMs >>> 32)); result = 31 * result + (int) (expireAfterEpochMs ^ (expireAfterEpochMs >>> 32)); result = 31 * result + (isRestored ? 1 : 0); - result = 31 * result + indexType.hashCode(); return result; } @@ -108,8 +102,6 @@ public String toString() { + expireAfterEpochMs + ", isRestored=" + isRestored - + ", indexType=" - + indexType + ", name='" + name + '\'' diff --git a/astra/src/main/java/com/slack/astra/metadata/replica/ReplicaMetadataSerializer.java b/astra/src/main/java/com/slack/astra/metadata/replica/ReplicaMetadataSerializer.java index 773e2ea514..66d0a9cf12 100644 --- a/astra/src/main/java/com/slack/astra/metadata/replica/ReplicaMetadataSerializer.java +++ b/astra/src/main/java/com/slack/astra/metadata/replica/ReplicaMetadataSerializer.java @@ -14,8 +14,7 @@ private static ReplicaMetadata fromReplicaMetadataProto( replicaMetadataProto.getReplicaSet(), replicaMetadataProto.getCreatedTimeEpochMs(), replicaMetadataProto.getExpireAfterEpochMs(), - replicaMetadataProto.getIsRestored(), - Metadata.IndexType.LOGS_LUCENE9); + replicaMetadataProto.getIsRestored()); } private static Metadata.ReplicaMetadata toReplicaMetadataProto(ReplicaMetadata metadata) { @@ -26,7 +25,6 @@ private static Metadata.ReplicaMetadata toReplicaMetadataProto(ReplicaMetadata m .setCreatedTimeEpochMs(metadata.createdTimeEpochMs) .setExpireAfterEpochMs(metadata.expireAfterEpochMs) .setIsRestored(metadata.isRestored) - .setIndexType(metadata.indexType) .build(); } diff --git a/astra/src/main/java/com/slack/astra/metadata/snapshot/SnapshotMetadata.java b/astra/src/main/java/com/slack/astra/metadata/snapshot/SnapshotMetadata.java index 89b29c272a..0e21374230 100644 --- a/astra/src/main/java/com/slack/astra/metadata/snapshot/SnapshotMetadata.java +++ b/astra/src/main/java/com/slack/astra/metadata/snapshot/SnapshotMetadata.java @@ -3,7 +3,6 @@ import static com.google.common.base.Preconditions.checkArgument; import com.slack.astra.metadata.core.AstraPartitionedMetadata; -import com.slack.astra.proto.metadata.Metadata; import java.time.Instant; import java.time.ZoneOffset; import java.time.ZonedDateTime; @@ -23,55 +22,38 @@ * here. */ public class SnapshotMetadata extends AstraPartitionedMetadata { - public static final String LIVE_SNAPSHOT_PATH = "LIVE"; - - public static boolean isLive(SnapshotMetadata snapshotMetadata) { - return snapshotMetadata.snapshotPath.equals(LIVE_SNAPSHOT_PATH); - } - - // snapshot path is deprecated in favor of storing these to a well-known path - // @see ChunkStore.getPath - @Deprecated public final String snapshotPath; - public final String snapshotId; public final long startTimeEpochMs; public final long endTimeEpochMs; public final long maxOffset; public final String partitionId; - public final Metadata.IndexType indexType; public long sizeInBytesOnDisk; public SnapshotMetadata( String snapshotId, - String snapshotPath, long startTimeEpochMs, long endTimeEpochMs, long maxOffset, String partitionId, - Metadata.IndexType indexType, long sizeInBytesOnDisk) { this( snapshotId, - snapshotPath, snapshotId, startTimeEpochMs, endTimeEpochMs, maxOffset, partitionId, - sizeInBytesOnDisk, - indexType); + sizeInBytesOnDisk); } private SnapshotMetadata( String name, - String snapshotPath, String snapshotId, long startTimeEpochMs, long endTimeEpochMs, long maxOffset, String partitionId, - long sizeInBytesOnDisk, - Metadata.IndexType indexType) { + long sizeInBytesOnDisk) { super(name); checkArgument(snapshotId != null && !snapshotId.isEmpty(), "snapshotId can't be null or empty"); checkArgument(startTimeEpochMs > 0, "start time should be greater than zero."); @@ -82,65 +64,45 @@ private SnapshotMetadata( checkArgument(maxOffset >= 0, "max offset should be greater than or equal to zero."); checkArgument( partitionId != null && !partitionId.isEmpty(), "partitionId can't be null or empty"); - checkArgument( - snapshotPath != null && !snapshotPath.isEmpty(), "snapshotPath can't be null or empty"); - this.snapshotPath = snapshotPath; this.snapshotId = snapshotId; this.startTimeEpochMs = startTimeEpochMs; this.endTimeEpochMs = endTimeEpochMs; this.maxOffset = maxOffset; this.partitionId = partitionId; - this.indexType = indexType; this.sizeInBytesOnDisk = sizeInBytesOnDisk; } @Override - public boolean equals(Object o) { + public final boolean equals(Object o) { if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (!(o instanceof SnapshotMetadata that)) return false; if (!super.equals(o)) return false; - SnapshotMetadata that = (SnapshotMetadata) o; - - if (startTimeEpochMs != that.startTimeEpochMs) return false; - if (endTimeEpochMs != that.endTimeEpochMs) return false; - if (maxOffset != that.maxOffset) return false; - if (snapshotPath != null ? !snapshotPath.equals(that.snapshotPath) : that.snapshotPath != null) - return false; - if (snapshotId != null ? !snapshotId.equals(that.snapshotId) : that.snapshotId != null) - return false; - if (partitionId != null ? !partitionId.equals(that.partitionId) : that.partitionId != null) - return false; - if (sizeInBytesOnDisk != that.sizeInBytesOnDisk) return false; - return indexType == that.indexType; + return startTimeEpochMs == that.startTimeEpochMs + && endTimeEpochMs == that.endTimeEpochMs + && maxOffset == that.maxOffset + && sizeInBytesOnDisk == that.sizeInBytesOnDisk + && snapshotId.equals(that.snapshotId) + && partitionId.equals(that.partitionId); } @Override public int hashCode() { int result = super.hashCode(); - result = 31 * result + (snapshotPath != null ? snapshotPath.hashCode() : 0); - result = 31 * result + (snapshotId != null ? snapshotId.hashCode() : 0); - result = 31 * result + (int) (startTimeEpochMs ^ (startTimeEpochMs >>> 32)); - result = 31 * result + (int) (endTimeEpochMs ^ (endTimeEpochMs >>> 32)); - result = 31 * result + (int) (maxOffset ^ (maxOffset >>> 32)); - result = 31 * result + (partitionId != null ? partitionId.hashCode() : 0); - result = 31 * result + (indexType != null ? indexType.hashCode() : 0); + result = 31 * result + snapshotId.hashCode(); + result = 31 * result + Long.hashCode(startTimeEpochMs); + result = 31 * result + Long.hashCode(endTimeEpochMs); + result = 31 * result + Long.hashCode(maxOffset); + result = 31 * result + partitionId.hashCode(); result = 31 * result + Long.hashCode(sizeInBytesOnDisk); return result; } @Override public String toString() { - // Include name from super class in the toString method to simplify debugging. return "SnapshotMetadata{" - + "name='" - + name - + '\'' - + ", snapshotPath='" - + snapshotPath - + '\'' - + ", snapshotId='" + + "snapshotId='" + snapshotId + '\'' + ", startTimeEpochMs=" @@ -152,16 +114,17 @@ public String toString() { + ", partitionId='" + partitionId + '\'' - + ", indexType=" - + indexType + ", sizeInBytesOnDisk=" + sizeInBytesOnDisk + + ", name='" + + name + + '\'' + '}'; } @Override public String getPartition() { - if (isLive(this)) { + if (isLive()) { // this keeps all the live snapshots in a single partition - this is important as their stored // startTimeEpochMs is not stable, and will be updated. This would cause an update to a live // node to fail with a partitioned metadata store as it cannot change the path of the znode. @@ -174,4 +137,11 @@ public String getPartition() { snapshotTime.getLong(ChronoField.HOUR_OF_DAY)); } } + + // todo - this is better than the previous version of storing a static "LIVE" string to a path + // variable but not by a lot. The "isLive" functionality should be reconsidered more broadly. + // The ideal way is likely to reconsider the ZK type for "LIVE" snapshots + public boolean isLive() { + return this.sizeInBytesOnDisk == 0; + } } diff --git a/astra/src/main/java/com/slack/astra/metadata/snapshot/SnapshotMetadataSerializer.java b/astra/src/main/java/com/slack/astra/metadata/snapshot/SnapshotMetadataSerializer.java index f2224adfb3..3fb5918e46 100644 --- a/astra/src/main/java/com/slack/astra/metadata/snapshot/SnapshotMetadataSerializer.java +++ b/astra/src/main/java/com/slack/astra/metadata/snapshot/SnapshotMetadataSerializer.java @@ -11,12 +11,10 @@ private static Metadata.SnapshotMetadata toSnapshotMetadataProto( return Metadata.SnapshotMetadata.newBuilder() .setName(snapshotMetadata.name) .setSnapshotId(snapshotMetadata.snapshotId) - .setSnapshotPath(snapshotMetadata.snapshotPath) .setStartTimeEpochMs(snapshotMetadata.startTimeEpochMs) .setEndTimeEpochMs(snapshotMetadata.endTimeEpochMs) .setPartitionId(snapshotMetadata.partitionId) .setMaxOffset(snapshotMetadata.maxOffset) - .setIndexType(snapshotMetadata.indexType) .setSizeInBytes(snapshotMetadata.sizeInBytesOnDisk) .build(); } @@ -25,12 +23,10 @@ private static SnapshotMetadata fromSnapshotMetadataProto( Metadata.SnapshotMetadata protoSnapshotMetadata) { return new SnapshotMetadata( protoSnapshotMetadata.getSnapshotId(), - protoSnapshotMetadata.getSnapshotPath(), protoSnapshotMetadata.getStartTimeEpochMs(), protoSnapshotMetadata.getEndTimeEpochMs(), protoSnapshotMetadata.getMaxOffset(), protoSnapshotMetadata.getPartitionId(), - Metadata.IndexType.LOGS_LUCENE9, protoSnapshotMetadata.getSizeInBytes()); } diff --git a/astra/src/main/proto/metadata.proto b/astra/src/main/proto/metadata.proto index a8e6f2b247..180d539b64 100644 --- a/astra/src/main/proto/metadata.proto +++ b/astra/src/main/proto/metadata.proto @@ -4,11 +4,6 @@ package slack.proto.astra; option java_package = "com.slack.astra.proto.metadata"; -// Type of index used to store the data. -enum IndexType { - LOGS_LUCENE9 = 0; -}; - message CacheSlotMetadata { enum CacheSlotState { FREE = 0; @@ -32,7 +27,7 @@ message CacheSlotMetadata { int64 updated_time_epoch_ms = 4; // Index types supported by cache slot. - repeated IndexType supported_index_types = 5; + // **removed Aug 2024** repeated IndexType supported_index_types = 5; // Unique string identifying the host string hostname = 6; @@ -59,7 +54,7 @@ message ReplicaMetadata { bool isRestored = 6; - IndexType index_type = 7; + // **removed Aug 2024** IndexType index_type = 7; } message SnapshotMetadata { @@ -83,7 +78,7 @@ message SnapshotMetadata { int64 max_offset = 6; // The type of index used to store this data. - IndexType index_type = 8; + // **removed Aug 2024** IndexType index_type = 8; // Size of the snapshot in bytes int64 sizeInBytes = 9; diff --git a/astra/src/test/java/com/slack/astra/chunk/ChunkInfoTest.java b/astra/src/test/java/com/slack/astra/chunk/ChunkInfoTest.java index c547d6993b..c75a4e2e3d 100644 --- a/astra/src/test/java/com/slack/astra/chunk/ChunkInfoTest.java +++ b/astra/src/test/java/com/slack/astra/chunk/ChunkInfoTest.java @@ -15,13 +15,11 @@ public class ChunkInfoTest { private static final String TEST_KAFKA_PARTITION_ID = "10"; private static final String TEST_CHUNK_NAME = "testChunkInfo1"; - private static final String TEST_SNAPSHOT_PATH = "testSnapshotPath"; @Test public void testChunkInfoCreation() { final long chunkCreationTime = 1000; - final ChunkInfo info = - new ChunkInfo(TEST_CHUNK_NAME, 1000, TEST_KAFKA_PARTITION_ID, TEST_SNAPSHOT_PATH); + final ChunkInfo info = new ChunkInfo(TEST_CHUNK_NAME, 1000, TEST_KAFKA_PARTITION_ID); assertThat(info.getChunkCreationTimeEpochMs()).isEqualTo(chunkCreationTime); assertThat(info.getChunkLastUpdatedTimeEpochMs()).isEqualTo(chunkCreationTime); assertThat(info.getDataStartTimeEpochMs()).isEqualTo(chunkCreationTime); @@ -29,7 +27,6 @@ public void testChunkInfoCreation() { assertThat(info.getChunkSnapshotTimeEpochMs()).isEqualTo(0); assertThat(info.getMaxOffset()).isEqualTo(DEFAULT_MAX_OFFSET); assertThat(info.getKafkaPartitionId()).isEqualTo(TEST_KAFKA_PARTITION_ID); - assertThat(info.getSnapshotPath()).isEqualTo(TEST_SNAPSHOT_PATH); } @Test @@ -37,11 +34,7 @@ public void testChunkDataTimeRange() { final LocalDateTime startTime = LocalDateTime.of(2020, 10, 1, 10, 10, 0); final long chunkCreationTimeEpochMilli = startTime.toInstant(ZoneOffset.UTC).toEpochMilli(); final ChunkInfo info = - new ChunkInfo( - TEST_CHUNK_NAME, - chunkCreationTimeEpochMilli, - TEST_KAFKA_PARTITION_ID, - TEST_SNAPSHOT_PATH); + new ChunkInfo(TEST_CHUNK_NAME, chunkCreationTimeEpochMilli, TEST_KAFKA_PARTITION_ID); assertThat(info.getChunkCreationTimeEpochMs()).isEqualTo(chunkCreationTimeEpochMilli); assertThat(info.getChunkLastUpdatedTimeEpochMs()).isEqualTo(chunkCreationTimeEpochMilli); assertThat(info.getDataStartTimeEpochMs()).isEqualTo(chunkCreationTimeEpochMilli); @@ -49,7 +42,6 @@ public void testChunkDataTimeRange() { assertThat(info.getChunkSnapshotTimeEpochMs()).isEqualTo(0); assertThat(info.getMaxOffset()).isEqualTo(DEFAULT_MAX_OFFSET); assertThat(info.getKafkaPartitionId()).isEqualTo(TEST_KAFKA_PARTITION_ID); - assertThat(info.getSnapshotPath()).isEqualTo(TEST_SNAPSHOT_PATH); // Add message with same time range. info.updateDataTimeRange(chunkCreationTimeEpochMilli); @@ -115,8 +107,7 @@ public void testUnInitializedChunkDataInRange() { final LocalDateTime startTime = LocalDateTime.of(2020, 10, 1, 10, 10, 0); final long chunkCreationTimeSecs = startTime.toInstant(ZoneOffset.UTC).toEpochMilli(); final ChunkInfo info = - new ChunkInfo( - TEST_CHUNK_NAME, chunkCreationTimeSecs, TEST_KAFKA_PARTITION_ID, TEST_SNAPSHOT_PATH); + new ChunkInfo(TEST_CHUNK_NAME, chunkCreationTimeSecs, TEST_KAFKA_PARTITION_ID); assertThat(info.getChunkCreationTimeEpochMs()).isEqualTo(chunkCreationTimeSecs); assertThat(info.getChunkLastUpdatedTimeEpochMs()).isEqualTo(chunkCreationTimeSecs); assertThat(info.getDataStartTimeEpochMs()).isEqualTo(chunkCreationTimeSecs); @@ -133,7 +124,6 @@ public void testUnInitializedChunkDataInRange() { assertThat(info.containsDataInTimeRange(1000, chunkCreationTimeSecs + 1)).isTrue(); assertThat(info.getMaxOffset()).isEqualTo(DEFAULT_MAX_OFFSET); assertThat(info.getKafkaPartitionId()).isEqualTo(TEST_KAFKA_PARTITION_ID); - assertThat(info.getSnapshotPath()).isEqualTo(TEST_SNAPSHOT_PATH); } @Test @@ -141,8 +131,7 @@ public void testChunkDataInRange() { final LocalDateTime startTime = LocalDateTime.of(2020, 10, 1, 10, 10, 0); final long chunkCreationTimeMs = startTime.toInstant(ZoneOffset.UTC).toEpochMilli(); final ChunkInfo info = - new ChunkInfo( - TEST_CHUNK_NAME, chunkCreationTimeMs, TEST_KAFKA_PARTITION_ID, TEST_SNAPSHOT_PATH); + new ChunkInfo(TEST_CHUNK_NAME, chunkCreationTimeMs, TEST_KAFKA_PARTITION_ID); assertThat(info.getChunkCreationTimeEpochMs()).isEqualTo(chunkCreationTimeMs); assertThat(info.getChunkLastUpdatedTimeEpochMs()).isEqualTo(chunkCreationTimeMs); assertThat(info.getDataStartTimeEpochMs()).isEqualTo(chunkCreationTimeMs); @@ -256,8 +245,7 @@ public void testChunkDataInRange() { @Test public void testNegativeStartTimeInDateRange() { - final ChunkInfo info = - new ChunkInfo(TEST_CHUNK_NAME, 1000, TEST_KAFKA_PARTITION_ID, TEST_SNAPSHOT_PATH); + final ChunkInfo info = new ChunkInfo(TEST_CHUNK_NAME, 1000, TEST_KAFKA_PARTITION_ID); info.updateDataTimeRange(980); info.updateDataTimeRange(1020); @@ -267,8 +255,7 @@ public void testNegativeStartTimeInDateRange() { @Test public void testNegativeEndTimeInDateRange() { - final ChunkInfo info = - new ChunkInfo(TEST_CHUNK_NAME, 1000, TEST_KAFKA_PARTITION_ID, TEST_SNAPSHOT_PATH); + final ChunkInfo info = new ChunkInfo(TEST_CHUNK_NAME, 1000, TEST_KAFKA_PARTITION_ID); info.updateDataTimeRange(980); info.updateDataTimeRange(1020); @@ -278,8 +265,7 @@ public void testNegativeEndTimeInDateRange() { @Test public void testNegativeIntervalInDateRange() { - final ChunkInfo info = - new ChunkInfo(TEST_CHUNK_NAME, 1000, TEST_KAFKA_PARTITION_ID, TEST_SNAPSHOT_PATH); + final ChunkInfo info = new ChunkInfo(TEST_CHUNK_NAME, 1000, TEST_KAFKA_PARTITION_ID); info.updateDataTimeRange(980); info.updateDataTimeRange(1020); @@ -290,32 +276,30 @@ public void testNegativeIntervalInDateRange() { @Test public void testInvalidChunkName() { assertThatExceptionOfType(IllegalArgumentException.class) - .isThrownBy(() -> new ChunkInfo(null, 100, TEST_KAFKA_PARTITION_ID, TEST_SNAPSHOT_PATH)); + .isThrownBy(() -> new ChunkInfo(null, 100, TEST_KAFKA_PARTITION_ID)); } @Test public void testEmptyChunkName() { assertThatExceptionOfType(IllegalArgumentException.class) - .isThrownBy(() -> new ChunkInfo("", 100, TEST_KAFKA_PARTITION_ID, TEST_SNAPSHOT_PATH)); + .isThrownBy(() -> new ChunkInfo("", 100, TEST_KAFKA_PARTITION_ID)); } @Test public void testNegativeChunkCreationTime() { assertThatExceptionOfType(IllegalArgumentException.class) - .isThrownBy( - () -> new ChunkInfo(TEST_CHUNK_NAME, -1, TEST_KAFKA_PARTITION_ID, TEST_SNAPSHOT_PATH)); + .isThrownBy(() -> new ChunkInfo(TEST_CHUNK_NAME, -1, TEST_KAFKA_PARTITION_ID)); } @Test public void testEmptyKafkaPartitionId() { assertThatExceptionOfType(IllegalArgumentException.class) - .isThrownBy(() -> new ChunkInfo(TEST_CHUNK_NAME, 100, "", TEST_SNAPSHOT_PATH)); + .isThrownBy(() -> new ChunkInfo(TEST_CHUNK_NAME, 100, "")); } @Test public void testMaxOffset() { - ChunkInfo chunkInfo = - new ChunkInfo(TEST_CHUNK_NAME, 100, TEST_KAFKA_PARTITION_ID, TEST_SNAPSHOT_PATH); + ChunkInfo chunkInfo = new ChunkInfo(TEST_CHUNK_NAME, 100, TEST_KAFKA_PARTITION_ID); assertThat(chunkInfo.getMaxOffset()).isEqualTo(DEFAULT_MAX_OFFSET); chunkInfo.updateMaxOffset(100); assertThat(chunkInfo.getMaxOffset()).isEqualTo(100); @@ -333,16 +317,6 @@ public void testMaxOffset() { assertThat(chunkInfo.getMaxOffset()).isEqualTo(104); } - @Test - public void testSnapshotPathCanBeUpdated() { - ChunkInfo chunkInfo = - new ChunkInfo(TEST_CHUNK_NAME, 10000, TEST_KAFKA_PARTITION_ID, TEST_SNAPSHOT_PATH); - assertThat(chunkInfo.getSnapshotPath()).isEqualTo(TEST_SNAPSHOT_PATH); - String testPath = "/path"; - chunkInfo.setSnapshotPath(testPath); - assertThat(chunkInfo.getSnapshotPath()).isEqualTo(testPath); - } - @Test public void snapshotMetadataConversion() { long dataStart = 101; @@ -357,7 +331,6 @@ public void snapshotMetadataConversion() { dataEnd, 1000, TEST_KAFKA_PARTITION_ID, - TEST_SNAPSHOT_PATH, 0); assertThat(fromSnapshotMetadata(toSnapshotMetadata(chunkInfo, ""))).isEqualTo(chunkInfo); } diff --git a/astra/src/test/java/com/slack/astra/chunk/IndexingChunkImplTest.java b/astra/src/test/java/com/slack/astra/chunk/IndexingChunkImplTest.java index 056ce028d9..dc254480f8 100644 --- a/astra/src/test/java/com/slack/astra/chunk/IndexingChunkImplTest.java +++ b/astra/src/test/java/com/slack/astra/chunk/IndexingChunkImplTest.java @@ -82,8 +82,6 @@ private static void testBeforeSnapshotState( assertThat(beforeSearchNodes.size()).isEqualTo(1); assertThat(beforeSearchNodes.get(0).url).contains(TEST_HOST); assertThat(beforeSearchNodes.get(0).url).contains(String.valueOf(TEST_PORT)); - - assertThat(beforeSearchNodes.get(0).snapshotName).contains(SnapshotMetadata.LIVE_SNAPSHOT_PATH); } @Nested @@ -638,7 +636,6 @@ public void testSnapshotToNonExistentS3BucketFails() // Snapshot to S3 without creating the s3 bucket. assertThat(chunk.snapshotToS3(chunkStore)).isFalse(); - assertThat(chunk.info().getSnapshotPath()).isEqualTo(SnapshotMetadata.LIVE_SNAPSHOT_PATH); // Metadata checks List afterSnapshots = @@ -647,15 +644,11 @@ public void testSnapshotToNonExistentS3BucketFails() assertThat(afterSnapshots.get(0).partitionId).isEqualTo(TEST_KAFKA_PARTITION_ID); assertThat(afterSnapshots.get(0).maxOffset).isEqualTo(0); - assertThat(afterSnapshots.get(0).snapshotPath).isEqualTo(SnapshotMetadata.LIVE_SNAPSHOT_PATH); - List afterSearchNodes = AstraMetadataTestUtils.listSyncUncached(searchMetadataStore); assertThat(afterSearchNodes.size()).isEqualTo(1); assertThat(afterSearchNodes.get(0).url).contains(TEST_HOST); assertThat(afterSearchNodes.get(0).url).contains(String.valueOf(TEST_PORT)); - assertThat(afterSearchNodes.get(0).snapshotName) - .contains(SnapshotMetadata.LIVE_SNAPSHOT_PATH); } @SuppressWarnings("OptionalGetWithoutIsPresent") @@ -702,9 +695,7 @@ public void testSnapshotToS3UsingChunkApi() throws Exception { ChunkStore chunkStore = new ChunkStore(s3AsyncClient, bucket); // Snapshot to S3 - assertThat(chunk.info().getSnapshotPath()).isEqualTo(SnapshotMetadata.LIVE_SNAPSHOT_PATH); assertThat(chunk.snapshotToS3(chunkStore)).isTrue(); - assertThat(chunk.info().getSnapshotPath()).isNotEmpty(); // depending on heap and CFS files this can be 5 or 19. assertThat(getCount(INDEX_FILES_UPLOAD, registry)).isGreaterThan(5); @@ -733,21 +724,16 @@ public void testSnapshotToS3UsingChunkApi() throws Exception { assertThat(afterSnapshots.size()).isEqualTo(2); assertThat(afterSnapshots).contains(ChunkInfo.toSnapshotMetadata(chunk.info(), "")); SnapshotMetadata liveSnapshot = - afterSnapshots.stream() - .filter(s -> s.snapshotPath.equals(SnapshotMetadata.LIVE_SNAPSHOT_PATH)) - .findFirst() - .get(); + afterSnapshots.stream().filter(SnapshotMetadata::isLive).findFirst().get(); assertThat(liveSnapshot.partitionId).isEqualTo(TEST_KAFKA_PARTITION_ID); assertThat(liveSnapshot.maxOffset).isEqualTo(offset - 1); - assertThat(liveSnapshot.snapshotPath).isEqualTo(SnapshotMetadata.LIVE_SNAPSHOT_PATH); + assertThat(liveSnapshot.isLive()).isTrue(); List afterSearchNodes = AstraMetadataTestUtils.listSyncUncached(searchMetadataStore); assertThat(afterSearchNodes.size()).isEqualTo(1); assertThat(afterSearchNodes.get(0).url).contains(TEST_HOST); assertThat(afterSearchNodes.get(0).url).contains(String.valueOf(TEST_PORT)); - assertThat(afterSearchNodes.get(0).snapshotName) - .contains(SnapshotMetadata.LIVE_SNAPSHOT_PATH); // Check total size of objects uploaded was correctly tracked assertThat(chunk.info().getSizeInBytesOnDisk()) diff --git a/astra/src/test/java/com/slack/astra/chunk/ReadOnlyChunkImplTest.java b/astra/src/test/java/com/slack/astra/chunk/ReadOnlyChunkImplTest.java index d9efa033be..da0f6bbc0d 100644 --- a/astra/src/test/java/com/slack/astra/chunk/ReadOnlyChunkImplTest.java +++ b/astra/src/test/java/com/slack/astra/chunk/ReadOnlyChunkImplTest.java @@ -7,7 +7,6 @@ import static com.slack.astra.logstore.LuceneIndexStoreImpl.MESSAGES_FAILED_COUNTER; import static com.slack.astra.logstore.LuceneIndexStoreImpl.MESSAGES_RECEIVED_COUNTER; import static com.slack.astra.logstore.LuceneIndexStoreImpl.REFRESHES_TIMER; -import static com.slack.astra.proto.metadata.Metadata.IndexType.LOGS_LUCENE9; import static com.slack.astra.testlib.MetricsUtil.getCount; import static com.slack.astra.testlib.MetricsUtil.getTimerCount; import static com.slack.astra.testlib.TemporaryLogStoreAndSearcherExtension.addMessages; @@ -606,7 +605,6 @@ private void assignReplicaToChunk( Metadata.CacheSlotMetadata.CacheSlotState.ASSIGNED, replicaId, Instant.now().toEpochMilli(), - List.of(LOGS_LUCENE9), readOnlyChunk.searchContext.hostname, "rep1"); cacheSlotMetadataStore.updateAsync(updatedCacheSlotMetadata); @@ -619,12 +617,10 @@ private void initializeZkSnapshot( snapshotMetadataStore.createSync( new SnapshotMetadata( snapshotId, - "path", Instant.now().minus(1, ChronoUnit.MINUTES).toEpochMilli(), Instant.now().toEpochMilli(), 1, "partitionId", - LOGS_LUCENE9, sizeInBytesOnDisk)); } @@ -639,8 +635,7 @@ private void initializeZkReplica( "rep1", Instant.now().toEpochMilli(), Instant.now().plusSeconds(60).toEpochMilli(), - false, - LOGS_LUCENE9)); + false)); } private void initializeBlobStorageWithIndex(String snapshotId) throws Exception { diff --git a/astra/src/test/java/com/slack/astra/chunk/RecoveryChunkImplTest.java b/astra/src/test/java/com/slack/astra/chunk/RecoveryChunkImplTest.java index 1b93ec09af..d9710d4350 100644 --- a/astra/src/test/java/com/slack/astra/chunk/RecoveryChunkImplTest.java +++ b/astra/src/test/java/com/slack/astra/chunk/RecoveryChunkImplTest.java @@ -620,7 +620,6 @@ public void testSnapshotToNonExistentS3BucketFails() { // Snapshot to S3 without creating the s3 bucket. assertThat(chunk.snapshotToS3(chunkStore)).isFalse(); - assertThat(chunk.info().getSnapshotPath()).isEqualTo(SnapshotMetadata.LIVE_SNAPSHOT_PATH); // No live snapshot or search metadata is published since the S3 snapshot failed. assertThat(AstraMetadataTestUtils.listSyncUncached(snapshotMetadataStore)).isEmpty(); @@ -671,9 +670,7 @@ public void testSnapshotToS3UsingChunkApi() throws Exception { ChunkStore chunkStore = new ChunkStore(s3AsyncClient, bucket); // Snapshot to S3 - assertThat(chunk.info().getSnapshotPath()).isEqualTo(SnapshotMetadata.LIVE_SNAPSHOT_PATH); assertThat(chunk.snapshotToS3(chunkStore)).isTrue(); - assertThat(chunk.info().getSnapshotPath()).isNotEmpty(); // depending on heap and CFS files this can be 5 or 19. assertThat(getCount(INDEX_FILES_UPLOAD, registry)).isGreaterThan(5); diff --git a/astra/src/test/java/com/slack/astra/chunkManager/CachingChunkManagerTest.java b/astra/src/test/java/com/slack/astra/chunkManager/CachingChunkManagerTest.java index 713989cb1f..67420041d8 100644 --- a/astra/src/test/java/com/slack/astra/chunkManager/CachingChunkManagerTest.java +++ b/astra/src/test/java/com/slack/astra/chunkManager/CachingChunkManagerTest.java @@ -159,9 +159,7 @@ private CachingChunkManager initChunkManager() throws TimeoutExcepti private CacheNodeAssignment initAssignment(String snapshotId) throws Exception { cacheNodeAssignmentStore = new CacheNodeAssignmentStore(curatorFramework); snapshotMetadataStore = new SnapshotMetadataStore(curatorFramework); - snapshotMetadataStore.createSync( - new SnapshotMetadata( - snapshotId, TEST_S3_BUCKET, 1, 1, 0, "abcd", Metadata.IndexType.LOGS_LUCENE9, 29)); + snapshotMetadataStore.createSync(new SnapshotMetadata(snapshotId, 1, 1, 0, "abcd", 29)); CacheNodeAssignment newAssignment = new CacheNodeAssignment( "abcd", diff --git a/astra/src/test/java/com/slack/astra/chunkManager/IndexingChunkManagerTest.java b/astra/src/test/java/com/slack/astra/chunkManager/IndexingChunkManagerTest.java index e0fc0deade..e162601d8c 100644 --- a/astra/src/test/java/com/slack/astra/chunkManager/IndexingChunkManagerTest.java +++ b/astra/src/test/java/com/slack/astra/chunkManager/IndexingChunkManagerTest.java @@ -430,10 +430,9 @@ public void testAddMessage() throws Exception { List liveSnapshots = fetchLiveSnapshot(snapshots); assertThat(liveSnapshots.size()).isEqualTo(1); assertThat(fetchNonLiveSnapshot(snapshots)).isEmpty(); - assertThat(snapshots.get(0).snapshotPath).startsWith(SnapshotMetadata.LIVE_SNAPSHOT_PATH); + assertThat(snapshots.get(0).isLive()).isTrue(); assertThat(snapshots.get(0).maxOffset).isEqualTo(0); assertThat(snapshots.get(0).partitionId).isEqualTo(TEST_KAFKA_PARTITION_ID); - assertThat(snapshots.get(0).snapshotId).startsWith(SnapshotMetadata.LIVE_SNAPSHOT_PATH); assertThat(snapshots.get(0).startTimeEpochMs) .isCloseTo(creationTime.toEpochMilli(), Offset.offset(5000L)); assertThat(snapshots.get(0).endTimeEpochMs).isEqualTo(MAX_FUTURE_TIME); @@ -442,7 +441,6 @@ public void testAddMessage() throws Exception { assertThat(searchNodes.size()).isEqualTo(1); assertThat(searchNodes.get(0).url).contains(TEST_HOST); assertThat(searchNodes.get(0).url).contains(String.valueOf(TEST_PORT)); - assertThat(searchNodes.get(0).snapshotName).contains(SnapshotMetadata.LIVE_SNAPSHOT_PATH); // Add a message with a very high offset. final int veryHighOffset = 1000; @@ -1267,7 +1265,7 @@ public void testSuccessfulRollOverFinishesOnClose() throws Exception { assertThat(fetchLiveSnapshot(snapshots)).isEmpty(); assertThat(snapshots.get(0).maxOffset).isEqualTo(offset - 1); assertThat(snapshots.get(0).endTimeEpochMs).isLessThan(MAX_FUTURE_TIME); - assertThat(snapshots.get(0).snapshotId).doesNotContain(SnapshotMetadata.LIVE_SNAPSHOT_PATH); + assertThat(snapshots.get(0).isLive()).isFalse(); searchMetadataStore.close(); snapshotMetadataStore.close(); } diff --git a/astra/src/test/java/com/slack/astra/clusterManager/CacheNodeAssignmentServiceTest.java b/astra/src/test/java/com/slack/astra/clusterManager/CacheNodeAssignmentServiceTest.java index 994cec8021..c67e4c3b53 100644 --- a/astra/src/test/java/com/slack/astra/clusterManager/CacheNodeAssignmentServiceTest.java +++ b/astra/src/test/java/com/slack/astra/clusterManager/CacheNodeAssignmentServiceTest.java @@ -2,7 +2,6 @@ import static com.slack.astra.clusterManager.CacheNodeAssignmentService.assign; import static com.slack.astra.clusterManager.CacheNodeAssignmentService.sortSnapshotsByReplicaCreationTime; -import static com.slack.astra.proto.metadata.Metadata.IndexType.LOGS_LUCENE9; import static org.assertj.core.api.Assertions.assertThat; import static org.awaitility.Awaitility.await; import static org.mockito.Mockito.spy; @@ -103,8 +102,7 @@ public void testBasicLifecycle() throws Exception { cacheNodeMetadataStore.createSync(cacheNodeMetadata); SnapshotMetadata snapshotMetadata = - new SnapshotMetadata( - "snapshot" + i, "snapshot" + i, 1L, 2L, 10L, "abcd", LOGS_LUCENE9, 5); + new SnapshotMetadata("snapshot" + i, 1L, 2L, 10L, "abcd", 5); snapshotMetadataStore.createSync(snapshotMetadata); ReplicaMetadata replicaMetadata = @@ -114,8 +112,7 @@ public void testBasicLifecycle() throws Exception { "rep1", 1L, Instant.now().plus(15, ChronoUnit.MINUTES).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataStore.createSync(replicaMetadata); } CacheNodeAssignmentService cacheNodeAssignmentService = @@ -161,15 +158,7 @@ public void testExpiredReplicasMarkedForEvictionLifecycle() throws TimeoutExcept cacheNodeAssignmentStore.createSync(newAssignment); SnapshotMetadata snapshotMetadata = - new SnapshotMetadata( - String.format(snapshotKey, i), - String.format(snapshotKey, i), - 1L, - 2L, - 10L, - "abcd", - LOGS_LUCENE9, - 5); + new SnapshotMetadata(String.format(snapshotKey, i), 1L, 2L, 10L, "abcd", 5); snapshotMetadataStore.createSync(snapshotMetadata); ReplicaMetadata replicaMetadata = @@ -179,8 +168,7 @@ public void testExpiredReplicasMarkedForEvictionLifecycle() throws TimeoutExcept replicaSet, 1L, Instant.now().minusSeconds(120).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataStore.createSync(replicaMetadata); } @@ -225,15 +213,7 @@ public void testEvictExpiredReplicasOnly() { cacheNodeAssignmentStore.createSync(newAssignment); SnapshotMetadata snapshotMetadata = - new SnapshotMetadata( - String.format(snapshotKey, i), - String.format(snapshotKey, i), - 1L, - 2L, - 10L, - "abcd", - LOGS_LUCENE9, - 5); + new SnapshotMetadata(String.format(snapshotKey, i), 1L, 2L, 10L, "abcd", 5); snapshotMetadataStore.createSync(snapshotMetadata); ReplicaMetadata replicaMetadata = @@ -243,8 +223,7 @@ public void testEvictExpiredReplicasOnly() { replicaSet, 1L, Instant.now().minusSeconds(120).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataStore.createSync(replicaMetadata); } @@ -297,20 +276,13 @@ public void testShouldntCreateDuplicateAssignments() throws TimeoutException { cacheNodeAssignmentStore.createSync(existingAssignment); // Create snapshot1 and store it in the store - SnapshotMetadata snapshotMetadata = - new SnapshotMetadata(snapshotId, snapshotId, 1L, 2L, 10L, "abcd", LOGS_LUCENE9, 5); + SnapshotMetadata snapshotMetadata = new SnapshotMetadata(snapshotId, 1L, 2L, 10L, "abcd", 5); snapshotMetadataStore.createSync(snapshotMetadata); // Create a replica for snapshot1 ReplicaMetadata replicaMetadata = new ReplicaMetadata( - "replica1", - snapshotId, - replicaSet, - 1L, - Instant.now().toEpochMilli(), - false, - LOGS_LUCENE9); + "replica1", snapshotId, replicaSet, 1L, Instant.now().toEpochMilli(), false); replicaMetadataStore.createSync(replicaMetadata); CacheNodeAssignmentService cacheNodeAssignmentService = @@ -505,8 +477,7 @@ public void testComplyWithMaxConcurrentAssignments() { cacheNodeMetadataStore.createSync(cacheNodeMetadata); SnapshotMetadata snapshotMetadata = - new SnapshotMetadata( - "snapshot" + i, "snapshot" + i, 1L, 2L, 10L, "abcd", LOGS_LUCENE9, 1); + new SnapshotMetadata("snapshot" + i, 1L, 2L, 10L, "abcd", 1); snapshotMetadataStore.createSync(snapshotMetadata); ReplicaMetadata replicaMetadata = @@ -516,8 +487,7 @@ public void testComplyWithMaxConcurrentAssignments() { "rep1", 1L, Instant.now().plus(15, ChronoUnit.MINUTES).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataStore.createSync(replicaMetadata); } CacheNodeAssignmentService cacheNodeAssignmentService = @@ -561,15 +531,7 @@ public void testAssignMostRecentSnapshotsFirst() { Instant now = Instant.now(); for (int i = 0; i < 6; i++) { SnapshotMetadata snapshotMetadata = - new SnapshotMetadata( - String.format(SNAPSHOT_ID_KEY, i), - String.format(SNAPSHOT_ID_KEY, i), - 1L, - 2L, - 5L, - "abcd", - LOGS_LUCENE9, - 10); + new SnapshotMetadata(String.format(SNAPSHOT_ID_KEY, i), 1L, 2L, 5L, "abcd", 10); snapshotMetadataStore.createSync(snapshotMetadata); ReplicaMetadata replicaMetadata = @@ -579,8 +541,7 @@ public void testAssignMostRecentSnapshotsFirst() { "rep1", now.plus(15 * i, ChronoUnit.MINUTES).toEpochMilli(), now.plus(15, ChronoUnit.MINUTES).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataStore.createSync(replicaMetadata); } @@ -624,8 +585,7 @@ public void testSortSnapshotsByReplicaCreationTime() { for (int i = 0; i < 4; i++) { String snapshotId = String.format(SNAPSHOT_ID_KEY, i); - SnapshotMetadata snapshotMetadata = - new SnapshotMetadata(snapshotId, snapshotId, 1L, 2L, 10L, "abcd", LOGS_LUCENE9, 1); + SnapshotMetadata snapshotMetadata = new SnapshotMetadata(snapshotId, 1L, 2L, 10L, "abcd", 1); snapshots.add(snapshotMetadata); ReplicaMetadata replicaMetadata = @@ -635,8 +595,7 @@ public void testSortSnapshotsByReplicaCreationTime() { "rep1", 10 + i, Instant.now().plus(15, ChronoUnit.MINUTES).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicas.put(snapshotId, replicaMetadata); } Collections.shuffle(snapshots); @@ -667,15 +626,7 @@ public void testPacksMostEmptyBinsFirst() { Instant now = Instant.now(); for (int i = 0; i < 3; i++) { SnapshotMetadata snapshotMetadata = - new SnapshotMetadata( - String.format(SNAPSHOT_ID_KEY, i), - String.format(SNAPSHOT_ID_KEY, i), - 1L, - 2L, - 5L, - "abcd", - LOGS_LUCENE9, - 10); + new SnapshotMetadata(String.format(SNAPSHOT_ID_KEY, i), 1L, 2L, 5L, "abcd", 10); snapshotMetadataStore.createSync(snapshotMetadata); ReplicaMetadata replicaMetadata = @@ -685,8 +636,7 @@ public void testPacksMostEmptyBinsFirst() { "rep1", now.plus(15 * i, ChronoUnit.MINUTES).toEpochMilli(), now.plus(15, ChronoUnit.MINUTES).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataStore.createSync(replicaMetadata); } @@ -734,8 +684,7 @@ private static List makeSnapshotsWithSizes(List sizes List snapshots = new ArrayList<>(); for (int i = 0; i < sizes.size(); i++) { Integer size = sizes.get(i); - snapshots.add( - new SnapshotMetadata("snapshot" + i, "/" + i, 1, 2 * 1000, 3, "a", LOGS_LUCENE9, size)); + snapshots.add(new SnapshotMetadata("snapshot" + i, 1, 2 * 1000, 3, "a", size)); } return snapshots; } diff --git a/astra/src/test/java/com/slack/astra/clusterManager/ClusterHpaMetricServiceTest.java b/astra/src/test/java/com/slack/astra/clusterManager/ClusterHpaMetricServiceTest.java index 4241cde782..def9aac1a6 100644 --- a/astra/src/test/java/com/slack/astra/clusterManager/ClusterHpaMetricServiceTest.java +++ b/astra/src/test/java/com/slack/astra/clusterManager/ClusterHpaMetricServiceTest.java @@ -1,7 +1,6 @@ package com.slack.astra.clusterManager; import static com.slack.astra.clusterManager.ClusterHpaMetricService.CACHE_HPA_METRIC_NAME; -import static com.slack.astra.proto.metadata.Metadata.IndexType.LOGS_LUCENE9; import static org.assertj.core.api.Assertions.assertThat; import static org.awaitility.Awaitility.await; import static org.mockito.Mockito.spy; @@ -120,9 +119,9 @@ void oneReplicasetScaledown() { when(replicaMetadataStore.listSync()) .thenReturn( List.of( - new ReplicaMetadata("foo", "foo", "rep1", 1L, 0L, false, LOGS_LUCENE9), - new ReplicaMetadata("bar", "bar", "rep2", 1L, 0L, false, LOGS_LUCENE9), - new ReplicaMetadata("baz", "baz", "rep1", 1L, 0L, false, LOGS_LUCENE9))); + new ReplicaMetadata("foo", "foo", "rep1", 1L, 0L, false), + new ReplicaMetadata("bar", "bar", "rep2", 1L, 0L, false), + new ReplicaMetadata("baz", "baz", "rep1", 1L, 0L, false))); when(cacheSlotMetadataStore.listSync()) .thenReturn( @@ -132,7 +131,6 @@ void oneReplicasetScaledown() { Metadata.CacheSlotMetadata.CacheSlotState.FREE, "", 1, - List.of(LOGS_LUCENE9), "localhost", "rep1"), new CacheSlotMetadata( @@ -140,7 +138,6 @@ void oneReplicasetScaledown() { Metadata.CacheSlotMetadata.CacheSlotState.FREE, "", 1, - List.of(LOGS_LUCENE9), "localhost", "rep1"), new CacheSlotMetadata( @@ -148,7 +145,6 @@ void oneReplicasetScaledown() { Metadata.CacheSlotMetadata.CacheSlotState.FREE, "", 1, - List.of(LOGS_LUCENE9), "localhost", "rep2"), new CacheSlotMetadata( @@ -156,7 +152,6 @@ void oneReplicasetScaledown() { Metadata.CacheSlotMetadata.CacheSlotState.FREE, "", 1, - List.of(LOGS_LUCENE9), "localhost", "rep1"))); @@ -206,8 +201,8 @@ void twoReplicasetScaledown() { when(replicaMetadataStore.listSync()) .thenReturn( List.of( - new ReplicaMetadata("foo", "foo", "rep1", 1L, 0L, false, LOGS_LUCENE9), - new ReplicaMetadata("bar", "bar", "rep2", 1L, 0L, false, LOGS_LUCENE9))); + new ReplicaMetadata("foo", "foo", "rep1", 1L, 0L, false), + new ReplicaMetadata("bar", "bar", "rep2", 1L, 0L, false))); when(cacheSlotMetadataStore.listSync()) .thenReturn( @@ -217,7 +212,6 @@ void twoReplicasetScaledown() { Metadata.CacheSlotMetadata.CacheSlotState.FREE, "", 1, - List.of(LOGS_LUCENE9), "localhost", "rep1"), new CacheSlotMetadata( @@ -225,7 +219,6 @@ void twoReplicasetScaledown() { Metadata.CacheSlotMetadata.CacheSlotState.FREE, "", 1, - List.of(LOGS_LUCENE9), "localhost", "rep1"), new CacheSlotMetadata( @@ -233,7 +226,6 @@ void twoReplicasetScaledown() { Metadata.CacheSlotMetadata.CacheSlotState.FREE, "", 1, - List.of(LOGS_LUCENE9), "localhost", "rep2"), new CacheSlotMetadata( @@ -241,7 +233,6 @@ void twoReplicasetScaledown() { Metadata.CacheSlotMetadata.CacheSlotState.FREE, "", 1, - List.of(LOGS_LUCENE9), "localhost", "rep2"))); @@ -294,10 +285,10 @@ void twoReplicasetScaleup() { when(replicaMetadataStore.listSync()) .thenReturn( List.of( - new ReplicaMetadata("foo", "foo", "rep1", 1L, 0L, false, LOGS_LUCENE9), - new ReplicaMetadata("bar", "bar", "rep1", 1L, 0L, false, LOGS_LUCENE9), - new ReplicaMetadata("baz", "bar", "rep2", 1L, 0L, false, LOGS_LUCENE9), - new ReplicaMetadata("bal", "bar", "rep2", 1L, 0L, false, LOGS_LUCENE9))); + new ReplicaMetadata("foo", "foo", "rep1", 1L, 0L, false), + new ReplicaMetadata("bar", "bar", "rep1", 1L, 0L, false), + new ReplicaMetadata("baz", "bar", "rep2", 1L, 0L, false), + new ReplicaMetadata("bal", "bar", "rep2", 1L, 0L, false))); when(cacheSlotMetadataStore.listSync()) .thenReturn( @@ -307,7 +298,6 @@ void twoReplicasetScaleup() { Metadata.CacheSlotMetadata.CacheSlotState.FREE, "", 1, - List.of(LOGS_LUCENE9), "localhost", "rep1"))); @@ -364,17 +354,15 @@ public void testHpaScaleUp() throws Exception { when(snapshotMetadataStore.listSync()) .thenReturn( List.of( - new SnapshotMetadata( - "snapshot2", "snapshot2", 1L, 2L, 5L, "abcd", LOGS_LUCENE9, 10), - new SnapshotMetadata( - "snapshot1", "snapshot1", 1L, 2L, 5L, "abcd", LOGS_LUCENE9, 5))); + new SnapshotMetadata("snapshot2", 1L, 2L, 5L, "abcd", 10), + new SnapshotMetadata("snapshot1", 1L, 2L, 5L, "abcd", 5))); // Register 1 replica associated with the snapshot when(replicaMetadataStore.listSync()) .thenReturn( List.of( - new ReplicaMetadata("replica2", "snapshot2", "rep1", 1L, 2L, false, LOGS_LUCENE9), - new ReplicaMetadata("replica1", "snapshot1", "rep1", 1L, 2L, false, LOGS_LUCENE9))); + new ReplicaMetadata("replica2", "snapshot2", "rep1", 1L, 2L, false), + new ReplicaMetadata("replica1", "snapshot1", "rep1", 1L, 2L, false))); // Register 2 cache nodes with lots of capacity when(cacheNodeMetadataStore.listSync()) @@ -423,16 +411,11 @@ public void testHpaScaleDown() throws Exception { // Register 1 snapshot when(snapshotMetadataStore.listSync()) - .thenReturn( - List.of( - new SnapshotMetadata( - "snapshot1", "snapshot1", 1L, 2L, 5L, "abcd", LOGS_LUCENE9, 5))); + .thenReturn(List.of(new SnapshotMetadata("snapshot1", 1L, 2L, 5L, "abcd", 5))); // Register 1 replica associated with the snapshot when(replicaMetadataStore.listSync()) - .thenReturn( - List.of( - new ReplicaMetadata("replica1", "snapshot1", "rep1", 1L, 2L, false, LOGS_LUCENE9))); + .thenReturn(List.of(new ReplicaMetadata("replica1", "snapshot1", "rep1", 1L, 2L, false))); // Register 2 cache nodes with lots of capacity when(cacheNodeMetadataStore.listSync()) @@ -476,17 +459,14 @@ public void testHpaReplicaSetFiltering() throws Exception { // Register snapshot of size 15 when(snapshotMetadataStore.listSync()) - .thenReturn( - List.of( - new SnapshotMetadata( - "snapshot1", "snapshot1", 1L, 2L, 5L, "abcd", LOGS_LUCENE9, 15))); + .thenReturn(List.of(new SnapshotMetadata("snapshot1", 1L, 2L, 5L, "abcd", 15))); // Register 2 replicas for rep1 and rep2 when(replicaMetadataStore.listSync()) .thenReturn( List.of( - new ReplicaMetadata("replica2", "snapshot1", "rep1", 1L, 2L, false, LOGS_LUCENE9), - new ReplicaMetadata("replica1", "snapshot1", "rep2", 1L, 2L, false, LOGS_LUCENE9))); + new ReplicaMetadata("replica2", "snapshot1", "rep1", 1L, 2L, false), + new ReplicaMetadata("replica1", "snapshot1", "rep2", 1L, 2L, false))); // Register 2 cache nodes (rep1, rep2), of size 10 each when(cacheNodeMetadataStore.listSync()) diff --git a/astra/src/test/java/com/slack/astra/clusterManager/ReplicaAssignmentServiceTest.java b/astra/src/test/java/com/slack/astra/clusterManager/ReplicaAssignmentServiceTest.java index 6712329334..13e5921f30 100644 --- a/astra/src/test/java/com/slack/astra/clusterManager/ReplicaAssignmentServiceTest.java +++ b/astra/src/test/java/com/slack/astra/clusterManager/ReplicaAssignmentServiceTest.java @@ -2,7 +2,6 @@ import static com.slack.astra.clusterManager.ReplicaAssignmentService.REPLICA_ASSIGN_PENDING; import static com.slack.astra.clusterManager.ReplicaAssignmentService.REPLICA_ASSIGN_TIMER; -import static com.slack.astra.proto.metadata.Metadata.IndexType.LOGS_LUCENE9; import static com.slack.astra.server.AstraConfig.DEFAULT_START_STOP_DURATION; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.AssertionsForClassTypes.assertThatExceptionOfType; @@ -43,8 +42,6 @@ import org.junit.jupiter.api.Test; public class ReplicaAssignmentServiceTest { - - private static final List SUPPORTED_INDEX_TYPES = List.of(LOGS_LUCENE9); public static final String HOSTNAME = "hostname"; public static final String REPLICA_SET = "rep1"; @@ -220,8 +217,7 @@ public void shouldHandleNoAvailableSlots() { REPLICA_SET, Instant.now().toEpochMilli(), Instant.now().plusSeconds(60).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataList.add(replicaMetadata); replicaMetadataStore.createAsync(replicaMetadata); } @@ -278,7 +274,6 @@ public void shouldHandleNoAvailableReplicas() { Metadata.CacheSlotMetadata.CacheSlotState.FREE, "", Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, HOSTNAME, REPLICA_SET); cacheSlotMetadataList.add(cacheSlotMetadata); @@ -338,8 +333,7 @@ public void shouldHandleAllReplicasAlreadyAssigned() { REPLICA_SET, Instant.now().toEpochMilli(), Instant.now().plusSeconds(60).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataList.add(replicaMetadata); replicaMetadataStore.createAsync(replicaMetadata); } @@ -352,7 +346,6 @@ public void shouldHandleAllReplicasAlreadyAssigned() { Metadata.CacheSlotMetadata.CacheSlotState.ASSIGNED, replicaMetadataList.get(i).name, Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, HOSTNAME, REPLICA_SET); cacheSlotMetadataList.add(cacheSlotMetadata); @@ -366,7 +359,6 @@ public void shouldHandleAllReplicasAlreadyAssigned() { Metadata.CacheSlotMetadata.CacheSlotState.FREE, "", Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, HOSTNAME, REPLICA_SET); cacheSlotMetadataList.add(cacheSlotMetadata); @@ -430,8 +422,7 @@ public void shouldHandleMixOfSlotStates() { REPLICA_SET, Instant.now().toEpochMilli(), Instant.now().plusSeconds(60).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataList.add(replicaMetadata); replicaMetadataStore.createAsync(replicaMetadata); } @@ -443,7 +434,6 @@ public void shouldHandleMixOfSlotStates() { Metadata.CacheSlotMetadata.CacheSlotState.ASSIGNED, replicaMetadataList.get(0).name, Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, HOSTNAME, REPLICA_SET); unmutatedSlots.add(cacheSlotWithAssignment); @@ -455,7 +445,6 @@ public void shouldHandleMixOfSlotStates() { Metadata.CacheSlotMetadata.CacheSlotState.LIVE, replicaMetadataList.get(1).name, Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, HOSTNAME, REPLICA_SET); unmutatedSlots.add(cacheSlotLive); @@ -467,7 +456,6 @@ public void shouldHandleMixOfSlotStates() { Metadata.CacheSlotMetadata.CacheSlotState.EVICTING, replicaMetadataList.get(2).name, Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, HOSTNAME, REPLICA_SET); unmutatedSlots.add(cacheSlotEvicting); @@ -479,7 +467,6 @@ public void shouldHandleMixOfSlotStates() { Metadata.CacheSlotMetadata.CacheSlotState.FREE, "", Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, HOSTNAME, REPLICA_SET); cacheSlotMetadataStore.createAsync(cacheSlotFree); @@ -547,8 +534,7 @@ public void shouldHandleAllSlotsAlreadyAssigned() { REPLICA_SET, Instant.now().toEpochMilli(), Instant.now().plusSeconds(60).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataList.add(replicaMetadata); replicaMetadataStore.createAsync(replicaMetadata); } @@ -561,7 +547,6 @@ public void shouldHandleAllSlotsAlreadyAssigned() { Metadata.CacheSlotMetadata.CacheSlotState.ASSIGNED, replicaMetadataList.get(i).name, Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, HOSTNAME, REPLICA_SET); cacheSlotMetadataList.add(cacheSlotMetadata); @@ -625,8 +610,7 @@ public void shouldHandleExpiredReplicas() { REPLICA_SET, Instant.now().minus(1500, ChronoUnit.MINUTES).toEpochMilli(), Instant.now().minusSeconds(60).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataExpiredList.add(replicaMetadata); replicaMetadataStore.createAsync(replicaMetadata); } @@ -639,8 +623,7 @@ public void shouldHandleExpiredReplicas() { REPLICA_SET, Instant.now().minus(1500, ChronoUnit.MINUTES).toEpochMilli(), 0, - false, - LOGS_LUCENE9); + false); replicaMetadataExpiredList.add(replicaMetadataZero); replicaMetadataStore.createAsync(replicaMetadataZero); @@ -652,8 +635,7 @@ public void shouldHandleExpiredReplicas() { REPLICA_SET, Instant.now().toEpochMilli(), Instant.now().plusSeconds(60).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataStore.createAsync(replicaMetadata); } @@ -664,7 +646,6 @@ public void shouldHandleExpiredReplicas() { Metadata.CacheSlotMetadata.CacheSlotState.FREE, "", Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, HOSTNAME, REPLICA_SET); cacheSlotMetadataStore.createAsync(cacheSlotMetadata); @@ -745,8 +726,7 @@ public void shouldRetryFailedAssignmentOnFollowingRun() { REPLICA_SET, Instant.now().toEpochMilli(), Instant.now().plusSeconds(60).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataStore.createAsync(replicaMetadata); } @@ -757,7 +737,6 @@ public void shouldRetryFailedAssignmentOnFollowingRun() { Metadata.CacheSlotMetadata.CacheSlotState.FREE, "", Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, HOSTNAME, REPLICA_SET); cacheSlotMetadataStore.createAsync(cacheSlotMetadata); @@ -870,8 +849,7 @@ public void shouldHandleTimedOutFutures() { REPLICA_SET, Instant.now().toEpochMilli(), Instant.now().plusSeconds(60).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataStore.createAsync(replicaMetadata); } @@ -882,7 +860,6 @@ public void shouldHandleTimedOutFutures() { Metadata.CacheSlotMetadata.CacheSlotState.FREE, "", Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, HOSTNAME, REPLICA_SET); cacheSlotMetadataStore.createAsync(cacheSlotMetadata); @@ -971,8 +948,7 @@ public void shouldHandleExceptionalFutures() { REPLICA_SET, Instant.now().toEpochMilli(), Instant.now().plusSeconds(60).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataStore.createAsync(replicaMetadata); } @@ -983,7 +959,6 @@ public void shouldHandleExceptionalFutures() { Metadata.CacheSlotMetadata.CacheSlotState.FREE, "", Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, HOSTNAME, REPLICA_SET); cacheSlotMetadataStore.createAsync(cacheSlotMetadata); @@ -1058,7 +1033,6 @@ public void shouldHandleSlotsAvailableFirstLifecycle() throws Exception { Metadata.CacheSlotMetadata.CacheSlotState.FREE, "", Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, HOSTNAME, REPLICA_SET); cacheSlotMetadataStore.createAsync(cacheSlotMetadata); @@ -1086,8 +1060,7 @@ public void shouldHandleSlotsAvailableFirstLifecycle() throws Exception { REPLICA_SET, Instant.now().toEpochMilli(), Instant.now().plusSeconds(60).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataStore.createAsync(replicaMetadata); } @@ -1156,8 +1129,7 @@ public void shouldHandleReplicasAvailableFirstLifecycle() throws Exception { REPLICA_SET, Instant.now().toEpochMilli(), Instant.now().plusSeconds(60).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataList.add(replicaMetadata); replicaMetadataStore.createAsync(replicaMetadata); } @@ -1174,7 +1146,6 @@ public void shouldHandleReplicasAvailableFirstLifecycle() throws Exception { Metadata.CacheSlotMetadata.CacheSlotState.FREE, "", Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, HOSTNAME, REPLICA_SET); cacheSlotMetadataStore.createAsync(cacheSlotMetadata); @@ -1249,81 +1220,7 @@ public void shouldPreferNewerReplicasIfLackingCapacity() throws Exception { REPLICA_SET, now.minus(1, ChronoUnit.HOURS).toEpochMilli(), now.plusSeconds(60).toEpochMilli(), - false, - LOGS_LUCENE9); - replicaMetadataStore.createAsync(olderReplicaMetadata); - - ReplicaMetadata newerReplicaMetadata = - new ReplicaMetadata( - UUID.randomUUID().toString(), - UUID.randomUUID().toString(), - REPLICA_SET, - now.toEpochMilli(), - now.plusSeconds(60).toEpochMilli(), - false, - LOGS_LUCENE9); - replicaMetadataStore.createAsync(newerReplicaMetadata); - - await().until(() -> replicaMetadataStore.listSync().size() == 2); - - replicaAssignmentService.startAsync(); - replicaAssignmentService.awaitRunning(DEFAULT_START_STOP_DURATION); - - CacheSlotMetadata cacheSlotMetadata = - new CacheSlotMetadata( - UUID.randomUUID().toString(), - Metadata.CacheSlotMetadata.CacheSlotState.FREE, - "", - Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, - HOSTNAME, - REPLICA_SET); - - cacheSlotMetadataStore.createAsync(cacheSlotMetadata); - await() - .until( - () -> { - List cacheSlotMetadataList = cacheSlotMetadataStore.listSync(); - return cacheSlotMetadataList.size() == 1 - && cacheSlotMetadataList.get(0).cacheSlotState - == Metadata.CacheSlotMetadata.CacheSlotState.ASSIGNED; - }); - - List assignedCacheSlot = - AstraMetadataTestUtils.listSyncUncached(cacheSlotMetadataStore); - assertThat(assignedCacheSlot.get(0).replicaId).isEqualTo(newerReplicaMetadata.name); - assertThat(assignedCacheSlot.get(0).supportedIndexTypes).containsAll(SUPPORTED_INDEX_TYPES); - } - - @Test - public void assignmentPreservesSupportedIndexTypes() throws Exception { - AstraConfigs.ManagerConfig.ReplicaAssignmentServiceConfig replicaAssignmentServiceConfig = - AstraConfigs.ManagerConfig.ReplicaAssignmentServiceConfig.newBuilder() - .setSchedulePeriodMins(1) - .addAllReplicaSets(List.of(REPLICA_SET)) - .setMaxConcurrentPerNode(2) - .build(); - AstraConfigs.ManagerConfig managerConfig = - AstraConfigs.ManagerConfig.newBuilder() - .setEventAggregationSecs(2) - .setScheduleInitialDelayMins(1) - .setReplicaAssignmentServiceConfig(replicaAssignmentServiceConfig) - .build(); - - ReplicaAssignmentService replicaAssignmentService = - new ReplicaAssignmentService( - cacheSlotMetadataStore, replicaMetadataStore, managerConfig, meterRegistry); - - Instant now = Instant.now(); - ReplicaMetadata olderReplicaMetadata = - new ReplicaMetadata( - UUID.randomUUID().toString(), - UUID.randomUUID().toString(), - REPLICA_SET, - now.minus(1, ChronoUnit.HOURS).toEpochMilli(), - now.plusSeconds(60).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataStore.createAsync(olderReplicaMetadata); ReplicaMetadata newerReplicaMetadata = @@ -1333,8 +1230,7 @@ public void assignmentPreservesSupportedIndexTypes() throws Exception { REPLICA_SET, now.toEpochMilli(), now.plusSeconds(60).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataStore.createAsync(newerReplicaMetadata); await().until(() -> replicaMetadataStore.listSync().size() == 2); @@ -1342,14 +1238,12 @@ public void assignmentPreservesSupportedIndexTypes() throws Exception { replicaAssignmentService.startAsync(); replicaAssignmentService.awaitRunning(DEFAULT_START_STOP_DURATION); - final List suppportedIndexTypes = List.of(LOGS_LUCENE9, LOGS_LUCENE9); CacheSlotMetadata cacheSlotMetadata = new CacheSlotMetadata( UUID.randomUUID().toString(), Metadata.CacheSlotMetadata.CacheSlotState.FREE, "", Instant.now().toEpochMilli(), - suppportedIndexTypes, HOSTNAME, REPLICA_SET); @@ -1366,9 +1260,6 @@ public void assignmentPreservesSupportedIndexTypes() throws Exception { List assignedCacheSlot = AstraMetadataTestUtils.listSyncUncached(cacheSlotMetadataStore); assertThat(assignedCacheSlot.get(0).replicaId).isEqualTo(newerReplicaMetadata.name); - assertThat(assignedCacheSlot.get(0).supportedIndexTypes) - .containsExactlyInAnyOrderElementsOf(suppportedIndexTypes); - assertThat(assignedCacheSlot.size()).isEqualTo(1); } @Test @@ -1403,8 +1294,7 @@ public void shouldNotAssignIfAlreadyLoading() throws Exception { REPLICA_SET, now.minus(1, ChronoUnit.HOURS).toEpochMilli(), now.plusSeconds(60).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataStore.createAsync(expectedUnassignedMetadata); ReplicaMetadata loadingMetadata = @@ -1414,8 +1304,7 @@ public void shouldNotAssignIfAlreadyLoading() throws Exception { REPLICA_SET, now.minus(1, ChronoUnit.HOURS).toEpochMilli(), now.plusSeconds(60).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataStore.createAsync(loadingMetadata); await().until(() -> replicaMetadataStore.listSync().size() == 2); @@ -1426,7 +1315,6 @@ public void shouldNotAssignIfAlreadyLoading() throws Exception { Metadata.CacheSlotMetadata.CacheSlotState.LOADING, loadingMetadata.snapshotId, Instant.now().toEpochMilli(), - List.of(LOGS_LUCENE9, LOGS_LUCENE9), HOSTNAME, REPLICA_SET); cacheSlotMetadataStore.createAsync(cacheSlotMetadata); @@ -1437,7 +1325,6 @@ public void shouldNotAssignIfAlreadyLoading() throws Exception { Metadata.CacheSlotMetadata.CacheSlotState.FREE, "", Instant.now().toEpochMilli(), - List.of(LOGS_LUCENE9, LOGS_LUCENE9), HOSTNAME, REPLICA_SET); cacheSlotMetadataStore.createAsync(freeCacheSlot); @@ -1484,8 +1371,7 @@ public void shouldPreventConcurrentAssignmentsExceedingLimit() throws Exception REPLICA_SET, now.minus(1, ChronoUnit.HOURS).toEpochMilli(), now.plusSeconds(60).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataStore.createAsync(expectedAssignedMetadata1); ReplicaMetadata expectedAssignedMetadata2 = @@ -1495,8 +1381,7 @@ public void shouldPreventConcurrentAssignmentsExceedingLimit() throws Exception REPLICA_SET, now.minus(1, ChronoUnit.HOURS).toEpochMilli(), now.plusSeconds(60).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataStore.createAsync(expectedAssignedMetadata2); ReplicaMetadata expectedUnassignedMetadata = @@ -1506,8 +1391,7 @@ public void shouldPreventConcurrentAssignmentsExceedingLimit() throws Exception REPLICA_SET, now.toEpochMilli(), now.plusSeconds(60).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataStore.createAsync(expectedUnassignedMetadata); await().until(() -> replicaMetadataStore.listSync().size() == 3); @@ -1520,7 +1404,6 @@ public void shouldPreventConcurrentAssignmentsExceedingLimit() throws Exception Metadata.CacheSlotMetadata.CacheSlotState.FREE, "", Instant.now().toEpochMilli(), - List.of(LOGS_LUCENE9, LOGS_LUCENE9), HOSTNAME, REPLICA_SET); cacheSlotMetadataStore.createAsync(cacheSlotMetadata); diff --git a/astra/src/test/java/com/slack/astra/clusterManager/ReplicaCreationServiceTest.java b/astra/src/test/java/com/slack/astra/clusterManager/ReplicaCreationServiceTest.java index 9a11a7b2e1..43ea2bfe84 100644 --- a/astra/src/test/java/com/slack/astra/clusterManager/ReplicaCreationServiceTest.java +++ b/astra/src/test/java/com/slack/astra/clusterManager/ReplicaCreationServiceTest.java @@ -1,6 +1,5 @@ package com.slack.astra.clusterManager; -import static com.slack.astra.proto.metadata.Metadata.IndexType.LOGS_LUCENE9; import static com.slack.astra.server.AstraConfig.DEFAULT_START_STOP_DURATION; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.AssertionsForClassTypes.assertThatExceptionOfType; @@ -89,14 +88,7 @@ public void shouldDoNothingIfReplicasAlreadyExist() throws Exception { SnapshotMetadata snapshotA = new SnapshotMetadata( - "a", - "a", - Instant.now().toEpochMilli() - 1, - Instant.now().toEpochMilli(), - 0, - "a", - LOGS_LUCENE9, - 0); + "a", Instant.now().toEpochMilli() - 1, Instant.now().toEpochMilli(), 0, "a", 0); snapshotMetadataStore.createSync(snapshotA); replicaMetadataStore.createSync( @@ -140,14 +132,7 @@ public void shouldDoNothingIfReplicasAlreadyExist() throws Exception { public void shouldCreateZeroReplicasNoneConfigured() throws Exception { SnapshotMetadata snapshotA = new SnapshotMetadata( - "a", - "a", - Instant.now().toEpochMilli() - 1, - Instant.now().toEpochMilli(), - 0, - "a", - LOGS_LUCENE9, - 0); + "a", Instant.now().toEpochMilli() - 1, Instant.now().toEpochMilli(), 0, "a", 0); snapshotMetadataStore.createSync(snapshotA); AstraConfigs.ManagerConfig.ReplicaCreationServiceConfig replicaCreationServiceConfig = @@ -184,14 +169,7 @@ public void shouldCreateZeroReplicasNoneConfigured() throws Exception { public void shouldCreateFourReplicasIfNoneExist() throws Exception { SnapshotMetadata snapshotA = new SnapshotMetadata( - "a", - "a", - Instant.now().toEpochMilli() - 1, - Instant.now().toEpochMilli(), - 0, - "a", - LOGS_LUCENE9, - 0); + "a", Instant.now().toEpochMilli() - 1, Instant.now().toEpochMilli(), 0, "a", 100); snapshotMetadataStore.createSync(snapshotA); AstraConfigs.ManagerConfig.ReplicaCreationServiceConfig replicaCreationServiceConfig = @@ -237,26 +215,12 @@ public void shouldCreateFourReplicasIfNoneExist() throws Exception { public void shouldNotCreateReplicasForLiveSnapshots() { SnapshotMetadata snapshotNotLive = new SnapshotMetadata( - "a", - "a", - Instant.now().toEpochMilli() - 1, - Instant.now().toEpochMilli(), - 0, - "a", - LOGS_LUCENE9, - 0); + "a", Instant.now().toEpochMilli() - 1, Instant.now().toEpochMilli(), 0, "a", 100); snapshotMetadataStore.createSync(snapshotNotLive); SnapshotMetadata snapshotLive = new SnapshotMetadata( - "b", - SnapshotMetadata.LIVE_SNAPSHOT_PATH, - Instant.now().toEpochMilli() - 1, - Instant.now().toEpochMilli(), - 0, - "b", - LOGS_LUCENE9, - 0); + "b", Instant.now().toEpochMilli() - 1, Instant.now().toEpochMilli(), 0, "b", 0); snapshotMetadataStore.createSync(snapshotLive); AstraConfigs.ManagerConfig.ReplicaCreationServiceConfig replicaCreationServiceConfig = @@ -334,14 +298,12 @@ public void shouldHandleVeryLargeListOfIneligibleSnapshots() { String snapshotId = UUID.randomUUID().toString(); SnapshotMetadata snapshot = new SnapshotMetadata( - snapshotId, snapshotId, Instant.now().minus(1450, ChronoUnit.MINUTES).toEpochMilli(), Instant.now().minus(1441, ChronoUnit.MINUTES).toEpochMilli(), 0, snapshotId, - LOGS_LUCENE9, - 0); + 100); snapshotList.add(snapshot); }); @@ -352,12 +314,10 @@ public void shouldHandleVeryLargeListOfIneligibleSnapshots() { SnapshotMetadata snapshot = new SnapshotMetadata( snapshotId, - SnapshotMetadata.LIVE_SNAPSHOT_PATH, Instant.now().toEpochMilli() - 1, Instant.now().toEpochMilli(), 0, snapshotId, - LOGS_LUCENE9, 0); snapshotList.add(snapshot); }); @@ -369,14 +329,12 @@ public void shouldHandleVeryLargeListOfIneligibleSnapshots() { String snapshotId = UUID.randomUUID().toString(); SnapshotMetadata snapshot = new SnapshotMetadata( - snapshotId, snapshotId, Instant.now().toEpochMilli() - 1, Instant.now().toEpochMilli(), 0, snapshotId, - LOGS_LUCENE9, - 0); + 100); eligibleSnapshots.add(snapshot); }); snapshotList.addAll(eligibleSnapshots); @@ -456,14 +414,7 @@ public void shouldCreateReplicaWhenSnapshotAddedAfterRunning() throws Exception // EventAggregationSecs duration, attempt to create the replicas SnapshotMetadata snapshotA = new SnapshotMetadata( - "a", - "a", - Instant.now().toEpochMilli() - 1, - Instant.now().toEpochMilli(), - 0, - "a", - LOGS_LUCENE9, - 0); + "a", Instant.now().toEpochMilli() - 1, Instant.now().toEpochMilli(), 0, "a", 100); snapshotMetadataStore.createSync(snapshotA); await().until(() -> replicaMetadataStore.listSync().size() == 2); @@ -529,14 +480,7 @@ public void shouldStillCreateReplicaIfFirstAttemptFails() throws Exception { // attempt to create the replicas SnapshotMetadata snapshotA = new SnapshotMetadata( - "a", - "a", - Instant.now().toEpochMilli() - 1, - Instant.now().toEpochMilli(), - 0, - "a", - LOGS_LUCENE9, - 0); + "a", Instant.now().toEpochMilli() - 1, Instant.now().toEpochMilli(), 0, "a", 100); snapshotMetadataStore.createSync(snapshotA); await() @@ -582,14 +526,7 @@ public void shouldStillCreateReplicaIfFirstAttemptFails() throws Exception { public void shouldHandleFailedCreateFutures() { SnapshotMetadata snapshotA = new SnapshotMetadata( - "a", - "a", - Instant.now().toEpochMilli() - 1, - Instant.now().toEpochMilli(), - 0, - "a", - LOGS_LUCENE9, - 0); + "a", Instant.now().toEpochMilli() - 1, Instant.now().toEpochMilli(), 0, "a", 100); snapshotMetadataStore.createSync(snapshotA); await().until(() -> snapshotMetadataStore.listSync().size() == 1); @@ -632,14 +569,7 @@ public void shouldHandleFailedCreateFutures() { public void shouldHandleMixOfSuccessfulFailedZkFutures() { SnapshotMetadata snapshotA = new SnapshotMetadata( - "a", - "a", - Instant.now().toEpochMilli() - 1, - Instant.now().toEpochMilli(), - 0, - "a", - LOGS_LUCENE9, - 0); + "a", Instant.now().toEpochMilli() - 1, Instant.now().toEpochMilli(), 0, "a", 100); snapshotMetadataStore.createSync(snapshotA); await().until(() -> snapshotMetadataStore.listSync().size() == 1); diff --git a/astra/src/test/java/com/slack/astra/clusterManager/ReplicaDeletionServiceTest.java b/astra/src/test/java/com/slack/astra/clusterManager/ReplicaDeletionServiceTest.java index 5175f0ffac..eb13a32cf7 100644 --- a/astra/src/test/java/com/slack/astra/clusterManager/ReplicaDeletionServiceTest.java +++ b/astra/src/test/java/com/slack/astra/clusterManager/ReplicaDeletionServiceTest.java @@ -1,6 +1,5 @@ package com.slack.astra.clusterManager; -import static com.slack.astra.proto.metadata.Metadata.IndexType.LOGS_LUCENE9; import static com.slack.astra.server.AstraConfig.DEFAULT_START_STOP_DURATION; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.AssertionsForClassTypes.assertThatExceptionOfType; @@ -41,7 +40,6 @@ import org.junit.jupiter.api.Test; public class ReplicaDeletionServiceTest { - public static final List SUPPORTED_INDEX_TYPES = List.of(LOGS_LUCENE9); public static final String HOSTNAME = "hostname"; private static final String REPLICA_SET = "rep1"; private TestingServer testingServer; @@ -161,8 +159,7 @@ public void shouldDeleteExpiredReplicaWithoutAssignment() { REPLICA_SET, Instant.now().minusSeconds(30).toEpochMilli(), Instant.now().minusSeconds(10).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataStore.createAsync(replicaMetadata); CacheSlotMetadata cacheSlotMetadata = @@ -171,7 +168,6 @@ public void shouldDeleteExpiredReplicaWithoutAssignment() { Metadata.CacheSlotMetadata.CacheSlotState.FREE, "", Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, HOSTNAME, REPLICA_SET); cacheSlotMetadataStore.createAsync(cacheSlotMetadata); @@ -224,8 +220,7 @@ public void shouldNotDeleteExpiredReplicasWithAssignments() { REPLICA_SET, Instant.now().minusSeconds(30).toEpochMilli(), Instant.now().minusSeconds(10).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataList.add(replicaMetadata); replicaMetadataStore.createAsync(replicaMetadata); } @@ -236,7 +231,6 @@ public void shouldNotDeleteExpiredReplicasWithAssignments() { Metadata.CacheSlotMetadata.CacheSlotState.ASSIGNED, replicaMetadataList.get(0).name, Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, HOSTNAME, REPLICA_SET); cacheSlotMetadataStore.createAsync(cacheSlotMetadataAssigned); @@ -247,7 +241,6 @@ public void shouldNotDeleteExpiredReplicasWithAssignments() { Metadata.CacheSlotMetadata.CacheSlotState.EVICT, replicaMetadataList.get(1).name, Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, HOSTNAME, REPLICA_SET); cacheSlotMetadataStore.createAsync(cacheSlotMetadataEvict); @@ -258,7 +251,6 @@ public void shouldNotDeleteExpiredReplicasWithAssignments() { Metadata.CacheSlotMetadata.CacheSlotState.EVICTING, replicaMetadataList.get(2).name, Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, HOSTNAME, REPLICA_SET); cacheSlotMetadataStore.createAsync(cacheSlotMetadataEvicting); @@ -312,8 +304,7 @@ public void shouldNotDeleteUnexpiredReplicaWithoutAssignment() { REPLICA_SET, Instant.now().minusSeconds(30).toEpochMilli(), Instant.now().plusSeconds(30).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataStore.createAsync(replicaMetadata); CacheSlotMetadata cacheSlotMetadata = @@ -322,7 +313,6 @@ public void shouldNotDeleteUnexpiredReplicaWithoutAssignment() { Metadata.CacheSlotMetadata.CacheSlotState.FREE, "", Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, HOSTNAME, REPLICA_SET); cacheSlotMetadataStore.createAsync(cacheSlotMetadata); @@ -373,8 +363,7 @@ public void shouldRetryFailedDeleteAttempt() { REPLICA_SET, Instant.now().minusSeconds(30).toEpochMilli(), Instant.now().minusSeconds(10).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataStore.createAsync(replicaMetadata); CacheSlotMetadata cacheSlotMetadata = @@ -383,7 +372,6 @@ public void shouldRetryFailedDeleteAttempt() { Metadata.CacheSlotMetadata.CacheSlotState.FREE, "", Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, HOSTNAME, REPLICA_SET); cacheSlotMetadataStore.createAsync(cacheSlotMetadata); @@ -458,8 +446,7 @@ public void shouldHandleMixOfZkSuccessFailure() { REPLICA_SET, Instant.now().minusSeconds(30).toEpochMilli(), Instant.now().minusSeconds(10).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataStore.createAsync(replicaMetadata); } @@ -471,7 +458,6 @@ public void shouldHandleMixOfZkSuccessFailure() { Metadata.CacheSlotMetadata.CacheSlotState.FREE, "", Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, HOSTNAME, REPLICA_SET); cacheSlotMetadataList.add(cacheSlotMetadata); @@ -564,8 +550,7 @@ public void shouldHandleDeletionLifecycle() throws TimeoutException { REPLICA_SET, Instant.now().minusSeconds(30).toEpochMilli(), Instant.now().minusSeconds(10).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataStore.createAsync(replicaMetadataUnassigned); ReplicaMetadata replicaMetadataAssigned = @@ -575,8 +560,7 @@ public void shouldHandleDeletionLifecycle() throws TimeoutException { REPLICA_SET, Instant.now().minusSeconds(30).toEpochMilli(), Instant.now().minusSeconds(10).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataStore.createAsync(replicaMetadataAssigned); CacheSlotMetadata cacheSlotMetadataUnassigned = @@ -585,7 +569,6 @@ public void shouldHandleDeletionLifecycle() throws TimeoutException { Metadata.CacheSlotMetadata.CacheSlotState.FREE, "", Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, HOSTNAME, REPLICA_SET); cacheSlotMetadataStore.createAsync(cacheSlotMetadataUnassigned); @@ -596,7 +579,6 @@ public void shouldHandleDeletionLifecycle() throws TimeoutException { Metadata.CacheSlotMetadata.CacheSlotState.ASSIGNED, replicaMetadataAssigned.name, Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, HOSTNAME, REPLICA_SET); cacheSlotMetadataStore.createAsync(cacheSlotMetadataAssigned); diff --git a/astra/src/test/java/com/slack/astra/clusterManager/ReplicaEvictionServiceTest.java b/astra/src/test/java/com/slack/astra/clusterManager/ReplicaEvictionServiceTest.java index 3259fa7ab2..2b87db2ca6 100644 --- a/astra/src/test/java/com/slack/astra/clusterManager/ReplicaEvictionServiceTest.java +++ b/astra/src/test/java/com/slack/astra/clusterManager/ReplicaEvictionServiceTest.java @@ -1,6 +1,5 @@ package com.slack.astra.clusterManager; -import static com.slack.astra.proto.metadata.Metadata.IndexType.LOGS_LUCENE9; import static com.slack.astra.server.AstraConfig.DEFAULT_START_STOP_DURATION; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.AssertionsForClassTypes.assertThatExceptionOfType; @@ -41,7 +40,6 @@ import org.junit.jupiter.api.Test; public class ReplicaEvictionServiceTest { - private static final List SUPPORTED_INDEX_TYPES = List.of(LOGS_LUCENE9); public static final String HOSTNAME = "hostname"; public static final String REPLICA_SET = "rep1"; private TestingServer testingServer; @@ -164,8 +162,7 @@ public void shouldHandleNoExpiredCacheSlots() { REPLICA_SET, Instant.now().toEpochMilli(), Instant.now().plusSeconds(60).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicas.add(replicaMetadata); replicaMetadataStore.createAsync(replicaMetadata); } @@ -177,7 +174,6 @@ public void shouldHandleNoExpiredCacheSlots() { Metadata.CacheSlotMetadata.CacheSlotState.ASSIGNED, replicas.get(0).name, Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, HOSTNAME, REPLICA_SET); cacheSlots.add(cacheSlotAssigned); @@ -189,7 +185,6 @@ public void shouldHandleNoExpiredCacheSlots() { Metadata.CacheSlotMetadata.CacheSlotState.LIVE, replicas.get(1).name, Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, HOSTNAME, REPLICA_SET); cacheSlots.add(cacheSlotLive); @@ -201,7 +196,6 @@ public void shouldHandleNoExpiredCacheSlots() { Metadata.CacheSlotMetadata.CacheSlotState.LOADING, replicas.get(2).name, Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, HOSTNAME, REPLICA_SET); cacheSlots.add(cacheSlotLoading); @@ -229,88 +223,6 @@ public void shouldHandleNoExpiredCacheSlots() { .isEqualTo(1); } - @Test - public void shouldPreserveSupportedIndexTypesOnEviction() { - AstraConfigs.ManagerConfig.ReplicaEvictionServiceConfig replicaEvictionServiceConfig = - AstraConfigs.ManagerConfig.ReplicaEvictionServiceConfig.newBuilder() - .setSchedulePeriodMins(10) - .build(); - - AstraConfigs.ManagerConfig managerConfig = - AstraConfigs.ManagerConfig.newBuilder() - .setReplicaEvictionServiceConfig(replicaEvictionServiceConfig) - .setEventAggregationSecs(10) - .setScheduleInitialDelayMins(10) - .build(); - - ReplicaEvictionService replicaEvictionService = - new ReplicaEvictionService( - cacheSlotMetadataStore, replicaMetadataStore, managerConfig, meterRegistry); - - ReplicaMetadata replicaMetadata = - new ReplicaMetadata( - UUID.randomUUID().toString(), - UUID.randomUUID().toString(), - REPLICA_SET, - Instant.now().toEpochMilli(), - 0, - false, - LOGS_LUCENE9); - replicaMetadataStore.createAsync(replicaMetadata); - - // TODO: Update list with different index types when we have more types. - final List supportedIndexTypes = List.of(LOGS_LUCENE9, LOGS_LUCENE9); - CacheSlotMetadata cacheSlotMetadata = - new CacheSlotMetadata( - UUID.randomUUID().toString(), - Metadata.CacheSlotMetadata.CacheSlotState.LIVE, - replicaMetadata.name, - Instant.now().toEpochMilli(), - supportedIndexTypes, - HOSTNAME, - REPLICA_SET); - cacheSlotMetadataStore.createAsync(cacheSlotMetadata); - assertThat(cacheSlotMetadata.supportedIndexTypes.size()).isEqualTo(2); - - await().until(() -> replicaMetadataStore.listSync().size() == 1); - await().until(() -> cacheSlotMetadataStore.listSync().size() == 1); - - int replicasMarked = replicaEvictionService.markReplicasForEviction(Instant.now()); - assertThat(replicasMarked).isEqualTo(1); - - assertThat(replicaMetadataStore.listSync()).containsExactly(replicaMetadata); - await() - .until( - () -> - cacheSlotMetadataStore.listSync().stream() - .allMatch( - cacheSlot -> - cacheSlot.cacheSlotState.equals( - Metadata.CacheSlotMetadata.CacheSlotState.EVICT))); - - CacheSlotMetadata updatedCacheSlot = cacheSlotMetadataStore.listSync().get(0); - assertThat(updatedCacheSlot.updatedTimeEpochMs) - .isGreaterThan(cacheSlotMetadata.updatedTimeEpochMs); - assertThat(updatedCacheSlot.cacheSlotState) - .isEqualTo(Metadata.CacheSlotMetadata.CacheSlotState.EVICT); - assertThat(updatedCacheSlot.name).isEqualTo(cacheSlotMetadata.name); - assertThat(updatedCacheSlot.replicaId).isEqualTo(cacheSlotMetadata.replicaId); - assertThat(updatedCacheSlot.supportedIndexTypes) - .containsExactlyInAnyOrderElementsOf(cacheSlotMetadata.supportedIndexTypes); - - assertThat( - MetricsUtil.getCount( - ReplicaEvictionService.REPLICA_MARK_EVICT_SUCCEEDED, meterRegistry)) - .isEqualTo(1); - assertThat( - MetricsUtil.getCount(ReplicaEvictionService.REPLICA_MARK_EVICT_FAILED, meterRegistry)) - .isEqualTo(0); - assertThat( - MetricsUtil.getTimerCount( - ReplicaEvictionService.REPLICA_MARK_EVICT_TIMER, meterRegistry)) - .isEqualTo(1); - } - @Test public void shouldEvictReplicaWithEmptyExpiration() { AstraConfigs.ManagerConfig.ReplicaEvictionServiceConfig replicaEvictionServiceConfig = @@ -336,8 +248,7 @@ public void shouldEvictReplicaWithEmptyExpiration() { REPLICA_SET, Instant.now().toEpochMilli(), 0, - false, - LOGS_LUCENE9); + false); replicaMetadataStore.createAsync(replicaMetadata); CacheSlotMetadata cacheSlotMetadata = @@ -346,7 +257,6 @@ public void shouldEvictReplicaWithEmptyExpiration() { Metadata.CacheSlotMetadata.CacheSlotState.LIVE, replicaMetadata.name, Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, HOSTNAME, REPLICA_SET); cacheSlotMetadataStore.createAsync(cacheSlotMetadata); @@ -413,8 +323,7 @@ public void shouldNotMutateReplicaAlreadyMarkedForEviction() { REPLICA_SET, Instant.now().toEpochMilli(), Instant.now().minusSeconds(60).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataStore.createAsync(replicaMetadata); CacheSlotMetadata cacheSlotMetadata = @@ -423,7 +332,6 @@ public void shouldNotMutateReplicaAlreadyMarkedForEviction() { Metadata.CacheSlotMetadata.CacheSlotState.EVICT, replicaMetadata.name, Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, HOSTNAME, REPLICA_SET); cacheSlotMetadataStore.createAsync(cacheSlotMetadata); @@ -475,8 +383,7 @@ public void shouldNotMutateReplicaAlreadyEvicting() { REPLICA_SET, Instant.now().toEpochMilli(), Instant.now().minusSeconds(60).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataStore.createAsync(replicaMetadata); CacheSlotMetadata cacheSlotMetadata = @@ -485,7 +392,6 @@ public void shouldNotMutateReplicaAlreadyEvicting() { Metadata.CacheSlotMetadata.CacheSlotState.EVICTING, replicaMetadata.name, Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, HOSTNAME, REPLICA_SET); cacheSlotMetadataStore.createAsync(cacheSlotMetadata); @@ -537,8 +443,7 @@ public void shouldRetryFailedEvictionOnNextRun() { REPLICA_SET, Instant.now().toEpochMilli(), Instant.now().minusSeconds(60).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataStore.createAsync(replicaMetadata); CacheSlotMetadata cacheSlotMetadata = @@ -547,7 +452,6 @@ public void shouldRetryFailedEvictionOnNextRun() { Metadata.CacheSlotMetadata.CacheSlotState.LIVE, replicaMetadata.name, Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, HOSTNAME, REPLICA_SET); cacheSlotMetadataStore.createAsync(cacheSlotMetadata); @@ -644,8 +548,7 @@ public void shouldHandleMixOfZkSuccessFailures() { REPLICA_SET, Instant.now().toEpochMilli(), Instant.now().minusSeconds(60).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicas.add(replicaMetadata); replicaMetadataStore.createAsync(replicaMetadata); } @@ -657,7 +560,6 @@ public void shouldHandleMixOfZkSuccessFailures() { Metadata.CacheSlotMetadata.CacheSlotState.LIVE, replicas.get(0).name, Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, HOSTNAME, REPLICA_SET); cacheSlotMetadataStore.createAsync(cacheSlotMetadata); @@ -774,8 +676,7 @@ public void shouldHandleMixOfExpiredAndUnexpiredLifecycle() throws TimeoutExcept REPLICA_SET, Instant.now().toEpochMilli(), Instant.now().minusSeconds(60).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataStore.createAsync(replicaMetadataExpiredOne); CacheSlotMetadata cacheSlotReplicaExpiredOne = new CacheSlotMetadata( @@ -783,7 +684,6 @@ public void shouldHandleMixOfExpiredAndUnexpiredLifecycle() throws TimeoutExcept Metadata.CacheSlotMetadata.CacheSlotState.LIVE, replicaMetadataExpiredOne.name, Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, HOSTNAME, REPLICA_SET); cacheSlotMetadataStore.createAsync(cacheSlotReplicaExpiredOne); @@ -795,8 +695,7 @@ public void shouldHandleMixOfExpiredAndUnexpiredLifecycle() throws TimeoutExcept REPLICA_SET, Instant.now().toEpochMilli(), Instant.now().minusSeconds(60).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataStore.createAsync(replicaMetadataExpiredTwo); CacheSlotMetadata cacheSlotReplicaExpireTwo = new CacheSlotMetadata( @@ -804,7 +703,6 @@ public void shouldHandleMixOfExpiredAndUnexpiredLifecycle() throws TimeoutExcept Metadata.CacheSlotMetadata.CacheSlotState.EVICT, replicaMetadataExpiredTwo.name, Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, HOSTNAME, REPLICA_SET); cacheSlotMetadataStore.createAsync(cacheSlotReplicaExpireTwo); @@ -816,8 +714,7 @@ public void shouldHandleMixOfExpiredAndUnexpiredLifecycle() throws TimeoutExcept REPLICA_SET, Instant.now().toEpochMilli(), Instant.now().plusSeconds(360).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataStore.createAsync(replicaMetadataUnexpiredOne); CacheSlotMetadata cacheSlotReplicaUnexpiredOne = new CacheSlotMetadata( @@ -825,7 +722,6 @@ public void shouldHandleMixOfExpiredAndUnexpiredLifecycle() throws TimeoutExcept Metadata.CacheSlotMetadata.CacheSlotState.LIVE, replicaMetadataUnexpiredOne.name, Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, HOSTNAME, REPLICA_SET); cacheSlotMetadataStore.createAsync(cacheSlotReplicaUnexpiredOne); @@ -837,8 +733,7 @@ public void shouldHandleMixOfExpiredAndUnexpiredLifecycle() throws TimeoutExcept REPLICA_SET, Instant.now().toEpochMilli(), Instant.now().plusSeconds(360).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataStore.createAsync(replicaMetadataUnexpiredTwo); CacheSlotMetadata cacheSlotFree = new CacheSlotMetadata( @@ -846,7 +741,6 @@ public void shouldHandleMixOfExpiredAndUnexpiredLifecycle() throws TimeoutExcept Metadata.CacheSlotMetadata.CacheSlotState.FREE, "", Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, HOSTNAME, REPLICA_SET); cacheSlotMetadataStore.createAsync(cacheSlotFree); diff --git a/astra/src/test/java/com/slack/astra/clusterManager/ReplicaRestoreServiceTest.java b/astra/src/test/java/com/slack/astra/clusterManager/ReplicaRestoreServiceTest.java index f81fc5a494..b3a6f793a2 100644 --- a/astra/src/test/java/com/slack/astra/clusterManager/ReplicaRestoreServiceTest.java +++ b/astra/src/test/java/com/slack/astra/clusterManager/ReplicaRestoreServiceTest.java @@ -1,6 +1,5 @@ package com.slack.astra.clusterManager; -import static com.slack.astra.proto.metadata.Metadata.IndexType.LOGS_LUCENE9; import static org.assertj.core.api.AssertionsForClassTypes.assertThat; import static org.assertj.core.api.AssertionsForClassTypes.assertThatExceptionOfType; import static org.assertj.core.api.AssertionsForClassTypes.fail; @@ -96,8 +95,7 @@ public void shouldHandleDrainingAndAdding() throws Exception { for (int i = 0; i < 10; i++) { long now = Instant.now().toEpochMilli(); String id = "loop" + i; - SnapshotMetadata snapshotIncluded = - new SnapshotMetadata(id, id, now + 10, now + 15, 0, id, LOGS_LUCENE9, 0); + SnapshotMetadata snapshotIncluded = new SnapshotMetadata(id, now + 10, now + 15, 0, id, 0); replicaRestoreService.queueSnapshotsForRestoration(List.of(snapshotIncluded)); Thread.sleep(300); } @@ -140,7 +138,7 @@ public void shouldHandleMultipleSimultaneousRequests() { long now = Instant.now().toEpochMilli(); String id = "loop" + UUID.randomUUID(); SnapshotMetadata snapshotIncluded = - new SnapshotMetadata(id, id, now + 10, now + 15, 0, id, LOGS_LUCENE9, 0); + new SnapshotMetadata(id, now + 10, now + 15, 0, id, 0); try { replicaRestoreService.queueSnapshotsForRestoration(List.of(snapshotIncluded)); Thread.sleep(300); @@ -191,8 +189,7 @@ public void shouldRemoveDuplicates() throws Exception { List duplicateSnapshots = new ArrayList<>(); for (int i = 0; i < 10; i++) { String id = "duplicate"; - duplicateSnapshots.add( - new SnapshotMetadata(id, id, now + 10, now + 15, 0, id, LOGS_LUCENE9, 0)); + duplicateSnapshots.add(new SnapshotMetadata(id, now + 10, now + 15, 0, id, 0)); } replicaRestoreService.queueSnapshotsForRestoration(duplicateSnapshots); @@ -211,7 +208,7 @@ public void shouldRemoveDuplicates() throws Exception { for (int i = 0; i < 3; i++) { now = Instant.now().toEpochMilli(); String id = "loop" + i; - snapshots.add(new SnapshotMetadata(id, id, now + 10, now + 15, 0, id, LOGS_LUCENE9, 0)); + snapshots.add(new SnapshotMetadata(id, now + 10, now + 15, 0, id, 0)); } replicaRestoreService.queueSnapshotsForRestoration(snapshots); @@ -248,7 +245,7 @@ public void shouldNotQueueIfFull() { for (int i = 0; i < MAX_QUEUE_SIZE; i++) { long now = Instant.now().toEpochMilli(); String id = "loop" + i; - snapshots.add(new SnapshotMetadata(id, id, now + 10, now + 15, 0, id, LOGS_LUCENE9, 0)); + snapshots.add(new SnapshotMetadata(id, now + 10, now + 15, 0, id, 0)); } assertThatExceptionOfType(SizeLimitExceededException.class) diff --git a/astra/src/test/java/com/slack/astra/clusterManager/SnapshotDeletionServiceTest.java b/astra/src/test/java/com/slack/astra/clusterManager/SnapshotDeletionServiceTest.java index 0fc1d401b2..d97f661e9c 100644 --- a/astra/src/test/java/com/slack/astra/clusterManager/SnapshotDeletionServiceTest.java +++ b/astra/src/test/java/com/slack/astra/clusterManager/SnapshotDeletionServiceTest.java @@ -1,6 +1,5 @@ package com.slack.astra.clusterManager; -import static com.slack.astra.proto.metadata.Metadata.IndexType.LOGS_LUCENE9; import static com.slack.astra.server.AstraConfig.DEFAULT_START_STOP_DURATION; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.AssertionsForClassTypes.assertThatExceptionOfType; @@ -162,13 +161,11 @@ public void shouldDeleteExpiredSnapshotNoReplicas() throws Exception { SnapshotMetadata snapshotMetadata = new SnapshotMetadata( chunkId, - chunkStore.getRemotePath(chunkId), Instant.now().minus(11000, ChronoUnit.MINUTES).toEpochMilli(), Instant.now().minus(10900, ChronoUnit.MINUTES).toEpochMilli(), 0, "1", - LOGS_LUCENE9, - 0); + 100); snapshotMetadataStore.createAsync(snapshotMetadata); await().until(() -> snapshotMetadataStore.listSync().size() == 1); @@ -219,12 +216,10 @@ public void shouldNotDeleteExpiredSnapshotWithReplicas() throws Exception { SnapshotMetadata snapshotMetadata = new SnapshotMetadata( chunkId, - chunkStore.getRemotePath(chunkId), Instant.now().minus(11000, ChronoUnit.MINUTES).toEpochMilli(), Instant.now().minus(10900, ChronoUnit.MINUTES).toEpochMilli(), 0, "1", - LOGS_LUCENE9, 0); snapshotMetadataStore.createAsync(snapshotMetadata); @@ -235,8 +230,7 @@ public void shouldNotDeleteExpiredSnapshotWithReplicas() throws Exception { "rep1", Instant.now().minus(10900, ChronoUnit.MINUTES).toEpochMilli(), Instant.now().minus(500, ChronoUnit.MINUTES).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataStore.createAsync(replicaMetadata); await().until(() -> snapshotMetadataStore.listSync().size() == 1); @@ -334,12 +328,10 @@ public void shouldHandleNoReplicasUnexpiredSnapshots() throws Exception { SnapshotMetadata snapshotMetadata = new SnapshotMetadata( chunkId, - chunkStore.getRemotePath(chunkId), Instant.now().minus(11000, ChronoUnit.MINUTES).toEpochMilli(), Instant.now().minus(500, ChronoUnit.MINUTES).toEpochMilli(), 0, "1", - LOGS_LUCENE9, 0); snapshotMetadataStore.createAsync(snapshotMetadata); await().until(() -> snapshotMetadataStore.listSync().size() == 1); @@ -395,12 +387,10 @@ public void shouldHandleReplicasWithLongerLifespanThanSnapshots() throws Excepti SnapshotMetadata snapshotMetadata = new SnapshotMetadata( chunkId, - chunkStore.getRemotePath(chunkId), Instant.now().minus(11000, ChronoUnit.MINUTES).toEpochMilli(), Instant.now().minus(10900, ChronoUnit.MINUTES).toEpochMilli(), 0, "1", - LOGS_LUCENE9, 0); snapshotMetadataStore.createAsync(snapshotMetadata); @@ -412,8 +402,7 @@ public void shouldHandleReplicasWithLongerLifespanThanSnapshots() throws Excepti "rep1", Instant.now().minus(11000, ChronoUnit.MINUTES).toEpochMilli(), Instant.now().minus(10900, ChronoUnit.MINUTES).toEpochMilli(), - false, - LOGS_LUCENE9); + false); replicaMetadataStore.createAsync(replicaMetadata); await().until(() -> snapshotMetadataStore.listSync().size() == 1); @@ -471,13 +460,11 @@ public void shouldHandleExceptionalObjectStorageDelete() throws Exception { SnapshotMetadata snapshotMetadata = new SnapshotMetadata( chunkId, - chunkStore.getRemotePath(chunkId), Instant.now().minus(11000, ChronoUnit.MINUTES).toEpochMilli(), Instant.now().minus(10900, ChronoUnit.MINUTES).toEpochMilli(), 0, "1", - LOGS_LUCENE9, - 0); + 100); snapshotMetadataStore.createAsync(snapshotMetadata); await().until(() -> snapshotMetadataStore.listSync().size() == 1); @@ -530,13 +517,11 @@ public void shouldHandleFailedZkDelete() throws Exception { SnapshotMetadata snapshotMetadata = new SnapshotMetadata( chunkId, - chunkStore.getRemotePath(chunkId), Instant.now().minus(11000, ChronoUnit.MINUTES).toEpochMilli(), Instant.now().minus(10900, ChronoUnit.MINUTES).toEpochMilli(), 0, "1", - LOGS_LUCENE9, - 0); + 100); snapshotMetadataStore.createAsync(snapshotMetadata); await().until(() -> snapshotMetadataStore.listSync().size() == 1); @@ -595,13 +580,11 @@ public void shouldHandleFailedObjectDelete() throws Exception { SnapshotMetadata snapshotMetadata = new SnapshotMetadata( chunkId, - chunkStore.getRemotePath(chunkId), Instant.now().minus(11000, ChronoUnit.MINUTES).toEpochMilli(), Instant.now().minus(10900, ChronoUnit.MINUTES).toEpochMilli(), 0, "1", - LOGS_LUCENE9, - 0); + 100); snapshotMetadataStore.createAsync(snapshotMetadata); await().until(() -> snapshotMetadataStore.listSync().size() == 1); @@ -653,13 +636,11 @@ public void shouldRetryTimedOutZkDeleteNextRun() throws Exception { SnapshotMetadata snapshotMetadata = new SnapshotMetadata( chunkId, - chunkStore.getRemotePath(chunkId), Instant.now().minus(11000, ChronoUnit.MINUTES).toEpochMilli(), Instant.now().minus(10900, ChronoUnit.MINUTES).toEpochMilli(), 0, "1", - LOGS_LUCENE9, - 0); + 100); snapshotMetadataStore.createAsync(snapshotMetadata); await().until(() -> snapshotMetadataStore.listSync().size() == 1); @@ -750,13 +731,11 @@ public void shouldRetryFailedObjectStorageDeleteNextRun() throws Exception { SnapshotMetadata snapshotMetadata = new SnapshotMetadata( chunkId, - chunkStore.getRemotePath(chunkId), Instant.now().minus(11000, ChronoUnit.MINUTES).toEpochMilli(), Instant.now().minus(10900, ChronoUnit.MINUTES).toEpochMilli(), 0, "1", - LOGS_LUCENE9, - 0); + 100); snapshotMetadataStore.createAsync(snapshotMetadata); await().until(() -> snapshotMetadataStore.listSync().size() == 1); @@ -824,13 +803,11 @@ public void shouldHandleSnapshotDeleteLifecycle() throws Exception { SnapshotMetadata snapshotMetadata = new SnapshotMetadata( chunkId, - chunkStore.getRemotePath(chunkId), Instant.now().minus(11000, ChronoUnit.MINUTES).toEpochMilli(), Instant.now().minus(10900, ChronoUnit.MINUTES).toEpochMilli(), 0, "1", - LOGS_LUCENE9, - 0); + 100); snapshotMetadataStore.createAsync(snapshotMetadata); await().until(() -> snapshotMetadataStore.listSync().size() == 1); diff --git a/astra/src/test/java/com/slack/astra/logstore/search/AstraDistributedQueryServiceTest.java b/astra/src/test/java/com/slack/astra/logstore/search/AstraDistributedQueryServiceTest.java index 0831b04c16..9707cae7e8 100644 --- a/astra/src/test/java/com/slack/astra/logstore/search/AstraDistributedQueryServiceTest.java +++ b/astra/src/test/java/com/slack/astra/logstore/search/AstraDistributedQueryServiceTest.java @@ -6,7 +6,6 @@ import static com.slack.astra.logstore.search.AstraDistributedQueryService.getMatchingSearchMetadata; import static com.slack.astra.logstore.search.AstraDistributedQueryService.getMatchingSnapshots; import static com.slack.astra.logstore.search.AstraDistributedQueryService.getNodesAndSnapshotsToQuery; -import static com.slack.astra.metadata.snapshot.SnapshotMetadata.LIVE_SNAPSHOT_PATH; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.fail; import static org.awaitility.Awaitility.await; @@ -31,7 +30,6 @@ import com.slack.astra.metadata.snapshot.SnapshotMetadata; import com.slack.astra.metadata.snapshot.SnapshotMetadataStore; import com.slack.astra.proto.config.AstraConfigs; -import com.slack.astra.proto.metadata.Metadata; import com.slack.astra.proto.schema.Schema; import com.slack.astra.proto.service.AstraSearch; import com.slack.astra.proto.service.AstraServiceGrpc; @@ -835,12 +833,10 @@ public void testSchema() { List.of( new SnapshotMetadata( "snapshot1", - "/1", endTime.minus(30, ChronoUnit.MINUTES).toEpochMilli(), endTime.toEpochMilli(), 10, "1", - Metadata.IndexType.LOGS_LUCENE9, 0))); DatasetMetadataStore datasetMetadataStoreMock = mock(DatasetMetadataStore.class); when(datasetMetadataStoreMock.listSync()) @@ -969,7 +965,6 @@ private SnapshotMetadata createSnapshot( chunkEndTime.toEpochMilli(), 1234, partition, - isLive ? LIVE_SNAPSHOT_PATH : "cacheSnapshotPath", 0); SnapshotMetadata snapshotMetadata = toSnapshotMetadata(chunkInfo, isLive ? LIVE_SNAPSHOT_PREFIX : ""); diff --git a/astra/src/test/java/com/slack/astra/metadata/cache/CacheSlotMetadataSerializerTest.java b/astra/src/test/java/com/slack/astra/metadata/cache/CacheSlotMetadataSerializerTest.java index e7269f585b..6a43364f5e 100644 --- a/astra/src/test/java/com/slack/astra/metadata/cache/CacheSlotMetadataSerializerTest.java +++ b/astra/src/test/java/com/slack/astra/metadata/cache/CacheSlotMetadataSerializerTest.java @@ -1,13 +1,11 @@ package com.slack.astra.metadata.cache; -import static com.slack.astra.proto.metadata.Metadata.IndexType.LOGS_LUCENE9; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.catchThrowable; import com.google.protobuf.InvalidProtocolBufferException; import com.slack.astra.proto.metadata.Metadata; import java.time.Instant; -import java.util.List; import org.junit.jupiter.api.Test; public class CacheSlotMetadataSerializerTest { @@ -22,17 +20,10 @@ public void testCacheSlotMetadataSerializer() throws InvalidProtocolBufferExcept Metadata.CacheSlotMetadata.CacheSlotState.ASSIGNED; String replicaId = "123"; long updatedTimeEpochMs = Instant.now().toEpochMilli(); - List supportedIndexTypes = List.of(LOGS_LUCENE9, LOGS_LUCENE9); CacheSlotMetadata cacheSlotMetadata = new CacheSlotMetadata( - name, - cacheSlotState, - replicaId, - updatedTimeEpochMs, - supportedIndexTypes, - hostname, - replicaSet); + name, cacheSlotState, replicaId, updatedTimeEpochMs, hostname, replicaSet); String serializedCacheSlotMetadata = serDe.toJsonStr(cacheSlotMetadata); assertThat(serializedCacheSlotMetadata).isNotEmpty(); @@ -47,8 +38,6 @@ public void testCacheSlotMetadataSerializer() throws InvalidProtocolBufferExcept assertThat(deserializedCacheSlotMetadata.cacheSlotState).isEqualTo(cacheSlotState); assertThat(deserializedCacheSlotMetadata.replicaId).isEqualTo(replicaId); assertThat(deserializedCacheSlotMetadata.updatedTimeEpochMs).isEqualTo(updatedTimeEpochMs); - assertThat(deserializedCacheSlotMetadata.supportedIndexTypes) - .containsExactlyInAnyOrderElementsOf(supportedIndexTypes); } @Test diff --git a/astra/src/test/java/com/slack/astra/metadata/cache/CacheSlotMetadataStoreTest.java b/astra/src/test/java/com/slack/astra/metadata/cache/CacheSlotMetadataStoreTest.java index dbc0051f73..dc64dd5b0c 100644 --- a/astra/src/test/java/com/slack/astra/metadata/cache/CacheSlotMetadataStoreTest.java +++ b/astra/src/test/java/com/slack/astra/metadata/cache/CacheSlotMetadataStoreTest.java @@ -1,7 +1,6 @@ package com.slack.astra.metadata.cache; import static com.slack.astra.proto.metadata.Metadata.CacheSlotMetadata.CacheSlotState; -import static com.slack.astra.proto.metadata.Metadata.IndexType.LOGS_LUCENE9; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatIllegalArgumentException; import static org.awaitility.Awaitility.await; @@ -14,7 +13,6 @@ import io.micrometer.core.instrument.simple.SimpleMeterRegistry; import java.io.IOException; import java.time.Instant; -import java.util.List; import java.util.concurrent.TimeUnit; import org.apache.curator.test.TestingServer; import org.apache.curator.x.async.AsyncCuratorFramework; @@ -23,8 +21,6 @@ import org.junit.jupiter.api.Test; public class CacheSlotMetadataStoreTest { - private static final List SUPPORTED_INDEX_TYPES = List.of(LOGS_LUCENE9); - private TestingServer testingServer; private AsyncCuratorFramework curatorFramework; @@ -68,18 +64,11 @@ public void testUpdateNonFreeCacheSlotStateSync() throws Exception { final CacheSlotMetadata cacheSlotMetadata = new CacheSlotMetadata( - name, - cacheSlotState, - replicaId, - updatedTimeEpochMs, - SUPPORTED_INDEX_TYPES, - hostname, - replicaSet); + name, cacheSlotState, replicaId, updatedTimeEpochMs, hostname, replicaSet); assertThat(cacheSlotMetadata.name).isEqualTo(name); assertThat(cacheSlotMetadata.cacheSlotState).isEqualTo(cacheSlotState); assertThat(cacheSlotMetadata.replicaId).isEqualTo(replicaId); assertThat(cacheSlotMetadata.updatedTimeEpochMs).isEqualTo(updatedTimeEpochMs); - assertThat(cacheSlotMetadata.supportedIndexTypes).isEqualTo(SUPPORTED_INDEX_TYPES); assertThat(cacheSlotMetadata.hostname).isEqualTo(hostname); store.createSync(cacheSlotMetadata); @@ -98,7 +87,6 @@ public void testUpdateNonFreeCacheSlotStateSync() throws Exception { assertThat(liveNode.cacheSlotState).isEqualTo(CacheSlotState.LIVE); assertThat(liveNode.replicaId).isEqualTo(replicaId); assertThat(liveNode.updatedTimeEpochMs).isGreaterThan(cacheSlotMetadata.updatedTimeEpochMs); - assertThat(liveNode.supportedIndexTypes).isEqualTo(SUPPORTED_INDEX_TYPES); store .updateNonFreeCacheSlotState(cacheSlotMetadata, CacheSlotState.EVICT) @@ -113,7 +101,6 @@ public void testUpdateNonFreeCacheSlotStateSync() throws Exception { assertThat(evictNode.cacheSlotState).isEqualTo(CacheSlotState.EVICT); assertThat(evictNode.replicaId).isEqualTo(replicaId); assertThat(evictNode.updatedTimeEpochMs).isGreaterThan(liveNode.updatedTimeEpochMs); - assertThat(evictNode.supportedIndexTypes).isEqualTo(SUPPORTED_INDEX_TYPES); store .updateNonFreeCacheSlotState(cacheSlotMetadata, CacheSlotState.FREE) @@ -128,7 +115,6 @@ public void testUpdateNonFreeCacheSlotStateSync() throws Exception { assertThat(freeNode.cacheSlotState).isEqualTo(CacheSlotState.FREE); assertThat(freeNode.replicaId).isEmpty(); assertThat(freeNode.updatedTimeEpochMs).isGreaterThan(evictNode.updatedTimeEpochMs); - assertThat(freeNode.supportedIndexTypes).isEqualTo(SUPPORTED_INDEX_TYPES); // Only non-free states can be set. assertThatIllegalArgumentException() @@ -150,18 +136,11 @@ public void testNonFreeCacheSlotState() throws Exception { final CacheSlotMetadata cacheSlotMetadata = new CacheSlotMetadata( - name, - cacheSlotState, - replicaId, - updatedTimeEpochMs, - SUPPORTED_INDEX_TYPES, - hostname, - replicaSet); + name, cacheSlotState, replicaId, updatedTimeEpochMs, hostname, replicaSet); assertThat(cacheSlotMetadata.name).isEqualTo(name); assertThat(cacheSlotMetadata.cacheSlotState).isEqualTo(cacheSlotState); assertThat(cacheSlotMetadata.replicaId).isEqualTo(replicaId); assertThat(cacheSlotMetadata.updatedTimeEpochMs).isEqualTo(updatedTimeEpochMs); - assertThat(cacheSlotMetadata.supportedIndexTypes).isEqualTo(SUPPORTED_INDEX_TYPES); assertThat(cacheSlotMetadata.hostname).isEqualTo(hostname); store.createSync(cacheSlotMetadata); @@ -180,7 +159,6 @@ public void testNonFreeCacheSlotState() throws Exception { assertThat(liveNode.cacheSlotState).isEqualTo(CacheSlotState.LIVE); assertThat(liveNode.replicaId).isEqualTo(replicaId); assertThat(liveNode.updatedTimeEpochMs).isGreaterThan(cacheSlotMetadata.updatedTimeEpochMs); - assertThat(liveNode.supportedIndexTypes).isEqualTo(SUPPORTED_INDEX_TYPES); store.updateNonFreeCacheSlotState(liveNode, CacheSlotState.EVICT).get(1, TimeUnit.SECONDS); await() @@ -193,7 +171,6 @@ public void testNonFreeCacheSlotState() throws Exception { assertThat(evictNode.cacheSlotState).isEqualTo(CacheSlotState.EVICT); assertThat(evictNode.replicaId).isEqualTo(replicaId); assertThat(evictNode.updatedTimeEpochMs).isGreaterThan(liveNode.updatedTimeEpochMs); - assertThat(evictNode.supportedIndexTypes).isEqualTo(SUPPORTED_INDEX_TYPES); store.updateNonFreeCacheSlotState(evictNode, CacheSlotState.FREE).get(1, TimeUnit.SECONDS); await() @@ -206,7 +183,6 @@ public void testNonFreeCacheSlotState() throws Exception { assertThat(freeNode.cacheSlotState).isEqualTo(CacheSlotState.FREE); assertThat(freeNode.replicaId).isEmpty(); assertThat(freeNode.updatedTimeEpochMs).isGreaterThan(evictNode.updatedTimeEpochMs); - assertThat(freeNode.supportedIndexTypes).isEqualTo(SUPPORTED_INDEX_TYPES); // Only non-free states can be set. assertThatIllegalArgumentException() @@ -229,18 +205,11 @@ public void testCacheSlotStateWithReplica() throws Exception { final CacheSlotMetadata cacheSlotMetadata = new CacheSlotMetadata( - name, - cacheSlotState, - emptyReplicaId, - updatedTimeEpochMs, - SUPPORTED_INDEX_TYPES, - hostname, - replicaSet); + name, cacheSlotState, emptyReplicaId, updatedTimeEpochMs, hostname, replicaSet); assertThat(cacheSlotMetadata.name).isEqualTo(name); assertThat(cacheSlotMetadata.cacheSlotState).isEqualTo(cacheSlotState); assertThat(cacheSlotMetadata.replicaId).isEqualTo(emptyReplicaId); assertThat(cacheSlotMetadata.updatedTimeEpochMs).isEqualTo(updatedTimeEpochMs); - assertThat(cacheSlotMetadata.supportedIndexTypes).isEqualTo(SUPPORTED_INDEX_TYPES); assertThat(cacheSlotMetadata.hostname).isEqualTo(hostname); store.createSync(cacheSlotMetadata); @@ -261,7 +230,6 @@ public void testCacheSlotStateWithReplica() throws Exception { assertThat(freeNode.cacheSlotState).isEqualTo(cacheSlotState); assertThat(freeNode.replicaId).isEqualTo(emptyReplicaId); assertThat(freeNode.updatedTimeEpochMs).isGreaterThanOrEqualTo(updatedTimeEpochMs); - assertThat(freeNode.supportedIndexTypes).isEqualTo(SUPPORTED_INDEX_TYPES); final String replicaId = "1234"; store @@ -281,7 +249,6 @@ public void testCacheSlotStateWithReplica() throws Exception { .isEqualTo(Metadata.CacheSlotMetadata.CacheSlotState.ASSIGNED); assertThat(assignedNode.replicaId).isEqualTo(replicaId); assertThat(assignedNode.updatedTimeEpochMs).isGreaterThan(freeNode.updatedTimeEpochMs); - assertThat(assignedNode.supportedIndexTypes).isEqualTo(SUPPORTED_INDEX_TYPES); store .updateCacheSlotStateStateWithReplicaId( @@ -300,6 +267,5 @@ public void testCacheSlotStateWithReplica() throws Exception { .isEqualTo(Metadata.CacheSlotMetadata.CacheSlotState.EVICT); assertThat(evictedNode.replicaId).isEqualTo(replicaId); assertThat(evictedNode.updatedTimeEpochMs).isGreaterThan(freeNode.updatedTimeEpochMs); - assertThat(evictedNode.supportedIndexTypes).isEqualTo(SUPPORTED_INDEX_TYPES); } } diff --git a/astra/src/test/java/com/slack/astra/metadata/cache/CacheSlotMetadataTest.java b/astra/src/test/java/com/slack/astra/metadata/cache/CacheSlotMetadataTest.java index 0fab4427e4..837a491df5 100644 --- a/astra/src/test/java/com/slack/astra/metadata/cache/CacheSlotMetadataTest.java +++ b/astra/src/test/java/com/slack/astra/metadata/cache/CacheSlotMetadataTest.java @@ -1,19 +1,14 @@ package com.slack.astra.metadata.cache; -import static com.slack.astra.proto.metadata.Metadata.IndexType.LOGS_LUCENE9; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatIllegalArgumentException; import com.slack.astra.proto.metadata.Metadata; import java.time.Instant; -import java.util.Collections; -import java.util.List; import org.junit.jupiter.api.Test; public class CacheSlotMetadataTest { - private static final List SUPPORTED_INDEX_TYPES = List.of(LOGS_LUCENE9); - @Test public void testCacheSlotMetadata() { String hostname = "hostname"; @@ -26,20 +21,13 @@ public void testCacheSlotMetadata() { CacheSlotMetadata cacheSlotMetadata = new CacheSlotMetadata( - name, - cacheSlotState, - replicaId, - updatedTimeEpochMs, - SUPPORTED_INDEX_TYPES, - hostname, - replicaSet); + name, cacheSlotState, replicaId, updatedTimeEpochMs, hostname, replicaSet); assertThat(cacheSlotMetadata.name).isEqualTo(name); assertThat(cacheSlotMetadata.hostname).isEqualTo(hostname); assertThat(cacheSlotMetadata.replicaSet).isEqualTo(replicaSet); assertThat(cacheSlotMetadata.cacheSlotState).isEqualTo(cacheSlotState); assertThat(cacheSlotMetadata.replicaId).isEqualTo(replicaId); assertThat(cacheSlotMetadata.updatedTimeEpochMs).isEqualTo(updatedTimeEpochMs); - assertThat(cacheSlotMetadata.supportedIndexTypes).isEqualTo(SUPPORTED_INDEX_TYPES); } @Test @@ -54,77 +42,34 @@ public void testCacheSlotEqualsHashcode() { CacheSlotMetadata cacheSlot = new CacheSlotMetadata( - name, - cacheSlotState, - replicaId, - updatedTimeEpochMs, - SUPPORTED_INDEX_TYPES, - hostname, - replicaSet); + name, cacheSlotState, replicaId, updatedTimeEpochMs, hostname, replicaSet); CacheSlotMetadata cacheSlotDuplicate = new CacheSlotMetadata( - name, - cacheSlotState, - replicaId, - updatedTimeEpochMs, - SUPPORTED_INDEX_TYPES, - hostname, - replicaSet); + name, cacheSlotState, replicaId, updatedTimeEpochMs, hostname, replicaSet); CacheSlotMetadata cacheSlotDifferentState = new CacheSlotMetadata( name, Metadata.CacheSlotMetadata.CacheSlotState.EVICT, replicaId, updatedTimeEpochMs, - SUPPORTED_INDEX_TYPES, hostname, replicaSet); CacheSlotMetadata cacheSlotDifferentReplicaId = new CacheSlotMetadata( - name, - cacheSlotState, - "321", - updatedTimeEpochMs, - SUPPORTED_INDEX_TYPES, - hostname, - replicaSet); + name, cacheSlotState, "321", updatedTimeEpochMs, hostname, replicaSet); CacheSlotMetadata cacheSlotDifferentUpdatedTime = new CacheSlotMetadata( - name, - cacheSlotState, - replicaId, - updatedTimeEpochMs + 1, - SUPPORTED_INDEX_TYPES, - hostname, - replicaSet); + name, cacheSlotState, replicaId, updatedTimeEpochMs + 1, hostname, replicaSet); CacheSlotMetadata cacheSlotDifferentSupportedIndexType = new CacheSlotMetadata( - name, - cacheSlotState, - replicaId, - updatedTimeEpochMs + 1, - List.of(LOGS_LUCENE9, LOGS_LUCENE9), - hostname, - replicaSet); + name, cacheSlotState, replicaId, updatedTimeEpochMs + 1, hostname, replicaSet); CacheSlotMetadata cacheSlotDifferentHostname = new CacheSlotMetadata( - name, - cacheSlotState, - replicaId, - updatedTimeEpochMs, - SUPPORTED_INDEX_TYPES, - "hostname2", - replicaSet); + name, cacheSlotState, replicaId, updatedTimeEpochMs, "hostname2", replicaSet); CacheSlotMetadata cacheSlotDifferentReplicaPartition = new CacheSlotMetadata( - name, - cacheSlotState, - replicaId, - updatedTimeEpochMs, - SUPPORTED_INDEX_TYPES, - hostname, - "rep2"); + name, cacheSlotState, replicaId, updatedTimeEpochMs, hostname, "rep2"); assertThat(cacheSlot.hashCode()).isEqualTo(cacheSlotDuplicate.hashCode()); assertThat(cacheSlot).isEqualTo(cacheSlotDuplicate); @@ -153,7 +98,6 @@ public void invalidArgumentsShouldThrow() { Metadata.CacheSlotMetadata.CacheSlotState.FREE, "123", Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, "hostname", "rep1")); assertThatIllegalArgumentException() @@ -164,7 +108,6 @@ public void invalidArgumentsShouldThrow() { Metadata.CacheSlotMetadata.CacheSlotState.ASSIGNED, "", Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, "hostname", "rep1")); assertThatIllegalArgumentException() @@ -175,7 +118,6 @@ public void invalidArgumentsShouldThrow() { Metadata.CacheSlotMetadata.CacheSlotState.EVICT, "", Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, "hostname", "rep1")); assertThatIllegalArgumentException() @@ -186,7 +128,6 @@ public void invalidArgumentsShouldThrow() { Metadata.CacheSlotMetadata.CacheSlotState.EVICTING, "", Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, "hostname", "rep1")); assertThatIllegalArgumentException() @@ -197,7 +138,6 @@ public void invalidArgumentsShouldThrow() { Metadata.CacheSlotMetadata.CacheSlotState.LOADING, "", Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, "hostname", "rep1")); assertThatIllegalArgumentException() @@ -208,7 +148,6 @@ public void invalidArgumentsShouldThrow() { Metadata.CacheSlotMetadata.CacheSlotState.LIVE, "", Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, "hostname", "rep1")); assertThatIllegalArgumentException() @@ -219,7 +158,6 @@ public void invalidArgumentsShouldThrow() { Metadata.CacheSlotMetadata.CacheSlotState.FREE, "", 0, - SUPPORTED_INDEX_TYPES, "hostname", "rep1")); assertThatIllegalArgumentException() @@ -230,7 +168,6 @@ public void invalidArgumentsShouldThrow() { Metadata.CacheSlotMetadata.CacheSlotState.FREE, null, Instant.now().toEpochMilli(), - SUPPORTED_INDEX_TYPES, "hostname", "rep1")); assertThatIllegalArgumentException() @@ -241,7 +178,6 @@ public void invalidArgumentsShouldThrow() { Metadata.CacheSlotMetadata.CacheSlotState.FREE, "123", 100000, - Collections.emptyList(), "hostname", "rep1"); }); @@ -253,7 +189,6 @@ public void invalidArgumentsShouldThrow() { Metadata.CacheSlotMetadata.CacheSlotState.FREE, "123", 100000, - SUPPORTED_INDEX_TYPES, "hostname", "rep1"); }); diff --git a/astra/src/test/java/com/slack/astra/metadata/replica/ReplicaMetadataSerializerTest.java b/astra/src/test/java/com/slack/astra/metadata/replica/ReplicaMetadataSerializerTest.java index 3b4db81d7a..16e02e1b08 100644 --- a/astra/src/test/java/com/slack/astra/metadata/replica/ReplicaMetadataSerializerTest.java +++ b/astra/src/test/java/com/slack/astra/metadata/replica/ReplicaMetadataSerializerTest.java @@ -1,6 +1,5 @@ package com.slack.astra.metadata.replica; -import static com.slack.astra.proto.metadata.Metadata.IndexType.LOGS_LUCENE9; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.catchThrowable; @@ -21,13 +20,7 @@ public void testReplicaMetadataSerializer() throws InvalidProtocolBufferExceptio ReplicaMetadata replicaMetadata = new ReplicaMetadata( - name, - snapshotId, - replicaSet, - createdTimeEpochMs, - expireAfterEpochMs, - true, - LOGS_LUCENE9); + name, snapshotId, replicaSet, createdTimeEpochMs, expireAfterEpochMs, true); String serializedReplicaMetadata = serDe.toJsonStr(replicaMetadata); assertThat(serializedReplicaMetadata).isNotEmpty(); @@ -41,7 +34,6 @@ public void testReplicaMetadataSerializer() throws InvalidProtocolBufferExceptio assertThat(deserializedReplicaMetadata.createdTimeEpochMs).isEqualTo(createdTimeEpochMs); assertThat(deserializedReplicaMetadata.expireAfterEpochMs).isEqualTo(expireAfterEpochMs); assertThat(deserializedReplicaMetadata.isRestored).isTrue(); - assertThat(deserializedReplicaMetadata.indexType).isEqualTo(LOGS_LUCENE9); } @Test @@ -63,7 +55,6 @@ public void shouldHandleEmptyExpirationAndRestore() throws InvalidProtocolBuffer assertThat(deserializedReplicaMetadata.createdTimeEpochMs).isEqualTo(1639677020380L); assertThat(deserializedReplicaMetadata.expireAfterEpochMs).isEqualTo(0L); assertThat(deserializedReplicaMetadata.isRestored).isFalse(); - assertThat(deserializedReplicaMetadata.indexType).isEqualTo(LOGS_LUCENE9); } @Test diff --git a/astra/src/test/java/com/slack/astra/metadata/replica/ReplicaMetadataTest.java b/astra/src/test/java/com/slack/astra/metadata/replica/ReplicaMetadataTest.java index 602e46926e..090637e0d5 100644 --- a/astra/src/test/java/com/slack/astra/metadata/replica/ReplicaMetadataTest.java +++ b/astra/src/test/java/com/slack/astra/metadata/replica/ReplicaMetadataTest.java @@ -1,6 +1,5 @@ package com.slack.astra.metadata.replica; -import static com.slack.astra.proto.metadata.Metadata.IndexType.LOGS_LUCENE9; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatIllegalArgumentException; @@ -19,13 +18,7 @@ public void testReplicaMetadata() { ReplicaMetadata replicaMetadata = new ReplicaMetadata( - name, - snapshotId, - replicaSet, - createdTimeEpochMs, - expireAfterEpochMs, - false, - LOGS_LUCENE9); + name, snapshotId, replicaSet, createdTimeEpochMs, expireAfterEpochMs, false); assertThat(replicaMetadata.name).isEqualTo(name); assertThat(replicaMetadata.snapshotId).isEqualTo(snapshotId); @@ -33,17 +26,10 @@ public void testReplicaMetadata() { assertThat(replicaMetadata.createdTimeEpochMs).isEqualTo(createdTimeEpochMs); assertThat(replicaMetadata.expireAfterEpochMs).isEqualTo(expireAfterEpochMs); assertThat(replicaMetadata.isRestored).isFalse(); - assertThat(replicaMetadata.indexType).isEqualTo(LOGS_LUCENE9); ReplicaMetadata restoredReplicaMetadata = new ReplicaMetadata( - name, - snapshotId, - replicaSet, - createdTimeEpochMs, - expireAfterEpochMs, - true, - LOGS_LUCENE9); + name, snapshotId, replicaSet, createdTimeEpochMs, expireAfterEpochMs, true); assertThat(restoredReplicaMetadata.name).isEqualTo(name); assertThat(restoredReplicaMetadata.snapshotId).isEqualTo(snapshotId); @@ -51,7 +37,6 @@ public void testReplicaMetadata() { assertThat(restoredReplicaMetadata.createdTimeEpochMs).isEqualTo(createdTimeEpochMs); assertThat(restoredReplicaMetadata.expireAfterEpochMs).isEqualTo(expireAfterEpochMs); assertThat(restoredReplicaMetadata.isRestored).isTrue(); - assertThat(restoredReplicaMetadata.indexType).isEqualTo(LOGS_LUCENE9); } @Test @@ -64,52 +49,21 @@ public void testReplicaMetadataEqualsHashcode() { ReplicaMetadata replicaMetadataA = new ReplicaMetadata( - name, - snapshotId, - replicaSet, - createdTimeEpochMs, - expireAfterEpochMs, - true, - LOGS_LUCENE9); + name, snapshotId, replicaSet, createdTimeEpochMs, expireAfterEpochMs, true); ReplicaMetadata replicaMetadataB = new ReplicaMetadata( - name, - snapshotId, - replicaSet, - createdTimeEpochMs, - expireAfterEpochMs, - true, - LOGS_LUCENE9); + name, snapshotId, replicaSet, createdTimeEpochMs, expireAfterEpochMs, true); ReplicaMetadata replicaMetadataC = new ReplicaMetadata( - "nameC", - snapshotId, - replicaSet, - createdTimeEpochMs, - expireAfterEpochMs, - true, - LOGS_LUCENE9); + "nameC", snapshotId, replicaSet, createdTimeEpochMs, expireAfterEpochMs, true); ReplicaMetadata replicaMetadataD = new ReplicaMetadata( - name, - snapshotId, - replicaSet, - createdTimeEpochMs + 1, - expireAfterEpochMs, - false, - LOGS_LUCENE9); + name, snapshotId, replicaSet, createdTimeEpochMs + 1, expireAfterEpochMs, false); ReplicaMetadata replicaMetadataE = new ReplicaMetadata( - name, - snapshotId, - replicaSet, - createdTimeEpochMs, - expireAfterEpochMs + 1, - false, - LOGS_LUCENE9); + name, snapshotId, replicaSet, createdTimeEpochMs, expireAfterEpochMs + 1, false); ReplicaMetadata replicaMetadataF = - new ReplicaMetadata( - name, snapshotId, "rep2", createdTimeEpochMs, expireAfterEpochMs, true, LOGS_LUCENE9); + new ReplicaMetadata(name, snapshotId, "rep2", createdTimeEpochMs, expireAfterEpochMs, true); assertThat(replicaMetadataA).isEqualTo(replicaMetadataB); assertThat(replicaMetadataA).isNotEqualTo(replicaMetadataC); @@ -135,8 +89,7 @@ public void invalidArgumentsShouldThrow() { "rep1", Instant.now().toEpochMilli(), Instant.now().toEpochMilli(), - false, - LOGS_LUCENE9)); + false)); assertThatIllegalArgumentException() .isThrownBy( () -> @@ -146,17 +99,14 @@ public void invalidArgumentsShouldThrow() { "rep1", Instant.now().toEpochMilli(), Instant.now().toEpochMilli(), - true, - LOGS_LUCENE9)); + true)); assertThatIllegalArgumentException() .isThrownBy( () -> - new ReplicaMetadata( - "name", "123", "rep1", 0, Instant.now().toEpochMilli(), false, LOGS_LUCENE9)); + new ReplicaMetadata("name", "123", "rep1", 0, Instant.now().toEpochMilli(), false)); assertThatIllegalArgumentException() .isThrownBy( () -> - new ReplicaMetadata( - "name", "123", "rep1", Instant.now().toEpochMilli(), -1, true, LOGS_LUCENE9)); + new ReplicaMetadata("name", "123", "rep1", Instant.now().toEpochMilli(), -1, true)); } } diff --git a/astra/src/test/java/com/slack/astra/metadata/snapshot/SnapshotMetadataSerializerTest.java b/astra/src/test/java/com/slack/astra/metadata/snapshot/SnapshotMetadataSerializerTest.java index 2d8987e124..ac0ee8ad7c 100644 --- a/astra/src/test/java/com/slack/astra/metadata/snapshot/SnapshotMetadataSerializerTest.java +++ b/astra/src/test/java/com/slack/astra/metadata/snapshot/SnapshotMetadataSerializerTest.java @@ -1,6 +1,5 @@ package com.slack.astra.metadata.snapshot; -import static com.slack.astra.proto.metadata.Metadata.IndexType.LOGS_LUCENE9; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.AssertionsForClassTypes.assertThatExceptionOfType; @@ -14,7 +13,6 @@ public class SnapshotMetadataSerializerTest { @Test public void testSnapshotMetadataSerializer() throws InvalidProtocolBufferException { final String name = "testSnapshotId"; - final String path = "/testPath_" + name; final long startTime = 1; final long endTime = 100; final long maxOffset = 123; @@ -22,8 +20,7 @@ public void testSnapshotMetadataSerializer() throws InvalidProtocolBufferExcepti final long sizeInBytes = 0; SnapshotMetadata snapshotMetadata = - new SnapshotMetadata( - name, path, startTime, endTime, maxOffset, partitionId, LOGS_LUCENE9, sizeInBytes); + new SnapshotMetadata(name, startTime, endTime, maxOffset, partitionId, sizeInBytes); String serializedSnapshot = serDe.toJsonStr(snapshotMetadata); assertThat(serializedSnapshot).isNotEmpty(); @@ -32,20 +29,17 @@ public void testSnapshotMetadataSerializer() throws InvalidProtocolBufferExcepti assertThat(deserializedSnapshotMetadata).isEqualTo(snapshotMetadata); assertThat(deserializedSnapshotMetadata.name).isEqualTo(name); - assertThat(deserializedSnapshotMetadata.snapshotPath).isEqualTo(path); assertThat(deserializedSnapshotMetadata.snapshotId).isEqualTo(name); assertThat(deserializedSnapshotMetadata.startTimeEpochMs).isEqualTo(startTime); assertThat(deserializedSnapshotMetadata.endTimeEpochMs).isEqualTo(endTime); assertThat(deserializedSnapshotMetadata.maxOffset).isEqualTo(maxOffset); assertThat(deserializedSnapshotMetadata.partitionId).isEqualTo(partitionId); - assertThat(deserializedSnapshotMetadata.indexType).isEqualTo(LOGS_LUCENE9); assertThat(deserializedSnapshotMetadata.sizeInBytesOnDisk).isEqualTo(sizeInBytes); } @Test public void testDeserializingWithoutSizeField() throws InvalidProtocolBufferException { final String name = "testSnapshotId"; - final String path = "/testPath_" + name; final long startTime = 1; final long endTime = 100; final long maxOffset = 123; @@ -54,13 +48,11 @@ public void testDeserializingWithoutSizeField() throws InvalidProtocolBufferExce Metadata.SnapshotMetadata protoSnapshotMetadata = Metadata.SnapshotMetadata.newBuilder() .setName(name) - .setSnapshotPath(path) .setSnapshotId(name) .setStartTimeEpochMs(startTime) .setEndTimeEpochMs(endTime) .setMaxOffset(maxOffset) .setPartitionId(partitionId) - .setIndexType(LOGS_LUCENE9) // leaving out the `size` field .build(); @@ -72,13 +64,11 @@ public void testDeserializingWithoutSizeField() throws InvalidProtocolBufferExce // Assert everything else is deserialized correctly assertThat(deserializedSnapshotMetadata.name).isEqualTo(name); - assertThat(deserializedSnapshotMetadata.snapshotPath).isEqualTo(path); assertThat(deserializedSnapshotMetadata.snapshotId).isEqualTo(name); assertThat(deserializedSnapshotMetadata.startTimeEpochMs).isEqualTo(startTime); assertThat(deserializedSnapshotMetadata.endTimeEpochMs).isEqualTo(endTime); assertThat(deserializedSnapshotMetadata.maxOffset).isEqualTo(maxOffset); assertThat(deserializedSnapshotMetadata.partitionId).isEqualTo(partitionId); - assertThat(deserializedSnapshotMetadata.indexType).isEqualTo(LOGS_LUCENE9); } @Test diff --git a/astra/src/test/java/com/slack/astra/metadata/snapshot/SnapshotMetadataTest.java b/astra/src/test/java/com/slack/astra/metadata/snapshot/SnapshotMetadataTest.java index 585a2f09ad..66ae0ca697 100644 --- a/astra/src/test/java/com/slack/astra/metadata/snapshot/SnapshotMetadataTest.java +++ b/astra/src/test/java/com/slack/astra/metadata/snapshot/SnapshotMetadataTest.java @@ -1,6 +1,5 @@ package com.slack.astra.metadata.snapshot; -import static com.slack.astra.proto.metadata.Metadata.IndexType.LOGS_LUCENE9; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatIllegalArgumentException; @@ -13,41 +12,34 @@ public class SnapshotMetadataTest { @Test public void testSnapshotMetadata() { final String name = "testSnapshotId"; - final String path = "/testPath_" + name; final long startTime = 1; final long endTime = 100; final long maxOffset = 123; final String partitionId = "1"; SnapshotMetadata snapshotMetadata = - new SnapshotMetadata( - name, path, startTime, endTime, maxOffset, partitionId, LOGS_LUCENE9, 0); + new SnapshotMetadata(name, startTime, endTime, maxOffset, partitionId, 0); assertThat(snapshotMetadata.name).isEqualTo(name); - assertThat(snapshotMetadata.snapshotPath).isEqualTo(path); assertThat(snapshotMetadata.snapshotId).isEqualTo(name); assertThat(snapshotMetadata.startTimeEpochMs).isEqualTo(startTime); assertThat(snapshotMetadata.endTimeEpochMs).isEqualTo(endTime); assertThat(snapshotMetadata.maxOffset).isEqualTo(maxOffset); assertThat(snapshotMetadata.partitionId).isEqualTo(partitionId); - assertThat(snapshotMetadata.indexType).isEqualTo(LOGS_LUCENE9); } @Test public void testEqualsAndHashCode() { final String name = "testSnapshotId"; - final String path = "/testPath_" + name; final long startTime = 1; final long endTime = 100; final long maxOffset = 0; final String partitionId = "1"; SnapshotMetadata snapshot1 = - new SnapshotMetadata( - name, path, startTime, endTime, maxOffset, partitionId, LOGS_LUCENE9, 0); + new SnapshotMetadata(name, startTime, endTime, maxOffset, partitionId, 0); SnapshotMetadata snapshot2 = - new SnapshotMetadata( - name + "2", path, startTime, endTime, maxOffset, partitionId, LOGS_LUCENE9, 0); + new SnapshotMetadata(name + "2", startTime, endTime, maxOffset, partitionId, 0); // Ensure the name field from super class is included. assertThat(snapshot1).isNotEqualTo(snapshot2); @@ -61,87 +53,52 @@ public void testEqualsAndHashCode() { @Test public void ensureValidSnapshotData() { final String name = "testSnapshotId"; - final String path = "/testPath_" + name; final long startTime = 1; final long endTime = 100; final long maxOffset = 123; final String partitionId = "1"; assertThatIllegalArgumentException() - .isThrownBy( - () -> - new SnapshotMetadata( - "", path, startTime, endTime, maxOffset, partitionId, LOGS_LUCENE9, 0)); - - assertThatIllegalArgumentException() - .isThrownBy( - () -> - new SnapshotMetadata( - name, "", startTime, endTime, maxOffset, partitionId, LOGS_LUCENE9, 0)); + .isThrownBy(() -> new SnapshotMetadata("", startTime, endTime, maxOffset, partitionId, 0)); assertThatIllegalArgumentException() - .isThrownBy( - () -> - new SnapshotMetadata( - name, path, 0, endTime, maxOffset, partitionId, LOGS_LUCENE9, 0)); + .isThrownBy(() -> new SnapshotMetadata(name, 0, endTime, maxOffset, partitionId, 0)); assertThatIllegalArgumentException() - .isThrownBy( - () -> - new SnapshotMetadata( - name, path, startTime, 0, maxOffset, partitionId, LOGS_LUCENE9, 0)); + .isThrownBy(() -> new SnapshotMetadata(name, startTime, 0, maxOffset, partitionId, 0)); // Start time < end time assertThatIllegalArgumentException() .isThrownBy( - () -> - new SnapshotMetadata( - name, path, endTime, startTime, maxOffset, partitionId, LOGS_LUCENE9, 0)); + () -> new SnapshotMetadata(name, endTime, startTime, maxOffset, partitionId, 0)); // Start time same as end time. assertThat( - new SnapshotMetadata( - name, path, startTime, startTime, maxOffset, partitionId, LOGS_LUCENE9, 0) + new SnapshotMetadata(name, startTime, startTime, maxOffset, partitionId, 0) .endTimeEpochMs) .isEqualTo(startTime); assertThatIllegalArgumentException() - .isThrownBy( - () -> - new SnapshotMetadata( - name, path, startTime, endTime, -1, partitionId, LOGS_LUCENE9, 0)); + .isThrownBy(() -> new SnapshotMetadata(name, startTime, endTime, -1, partitionId, 0)); assertThatIllegalArgumentException() - .isThrownBy( - () -> - new SnapshotMetadata( - name, path, startTime, endTime, maxOffset, "", LOGS_LUCENE9, 0)); + .isThrownBy(() -> new SnapshotMetadata(name, startTime, endTime, maxOffset, "", 0)); } @Test public void testLive() { final String name = "testSnapshotId"; - final String path = "/testPath_" + name; final long startTime = 1; final long endTime = 100; final long maxOffset = 123; final String partitionId = "1"; SnapshotMetadata nonLiveSnapshot = - new SnapshotMetadata( - name, path, startTime, endTime, maxOffset, partitionId, LOGS_LUCENE9, 0); - assertThat(SnapshotMetadata.isLive(nonLiveSnapshot)).isFalse(); + new SnapshotMetadata(name, startTime, endTime, maxOffset, partitionId, 100); + assertThat(nonLiveSnapshot.isLive()).isFalse(); SnapshotMetadata liveSnapshot = - new SnapshotMetadata( - name, - SnapshotMetadata.LIVE_SNAPSHOT_PATH, - startTime, - endTime, - maxOffset, - partitionId, - LOGS_LUCENE9, - 0); - assertThat(SnapshotMetadata.isLive(liveSnapshot)).isTrue(); + new SnapshotMetadata(name, startTime, endTime, maxOffset, partitionId, 0); + assertThat(liveSnapshot.isLive()).isTrue(); } } diff --git a/astra/src/test/java/com/slack/astra/server/AstraIndexerTest.java b/astra/src/test/java/com/slack/astra/server/AstraIndexerTest.java index 978d57378e..5a5c7d36a1 100644 --- a/astra/src/test/java/com/slack/astra/server/AstraIndexerTest.java +++ b/astra/src/test/java/com/slack/astra/server/AstraIndexerTest.java @@ -2,8 +2,6 @@ import static com.slack.astra.logstore.LuceneIndexStoreImpl.MESSAGES_FAILED_COUNTER; import static com.slack.astra.logstore.LuceneIndexStoreImpl.MESSAGES_RECEIVED_COUNTER; -import static com.slack.astra.metadata.snapshot.SnapshotMetadata.LIVE_SNAPSHOT_PATH; -import static com.slack.astra.proto.metadata.Metadata.IndexType.LOGS_LUCENE9; import static com.slack.astra.server.AstraConfig.DEFAULT_START_STOP_DURATION; import static com.slack.astra.testlib.AstraConfigUtil.makeIndexerConfig; import static com.slack.astra.testlib.AstraConfigUtil.makeKafkaConfig; @@ -199,15 +197,7 @@ public void testDeleteStaleSnapshotAndStartConsumerKafkaSearchViaGrpcSearchApi() final long endTimeMs = 100; final long maxOffset = 50; SnapshotMetadata livePartition1 = - new SnapshotMetadata( - name + "live1", - LIVE_SNAPSHOT_PATH, - startTimeMs, - endTimeMs, - maxOffset, - "0", - LOGS_LUCENE9, - 0); + new SnapshotMetadata(name + "live1", startTimeMs, endTimeMs, maxOffset, "0", 0); snapshotMetadataStore.createSync(livePartition1); assertThat(AstraMetadataTestUtils.listSyncUncached(snapshotMetadataStore)) .containsOnly(livePartition1); @@ -246,27 +236,11 @@ public void testExceptionOnIndexerStartup() throws Exception { final long endTimeMs = 100; final long maxOffset = 50; SnapshotMetadata livePartition0 = - new SnapshotMetadata( - name + "live0", - LIVE_SNAPSHOT_PATH, - startTimeMs, - endTimeMs, - maxOffset, - "0", - LOGS_LUCENE9, - 0); + new SnapshotMetadata(name + "live0", startTimeMs, endTimeMs, maxOffset, "0", 0); snapshotMetadataStore.createSync(livePartition0); SnapshotMetadata livePartition1 = - new SnapshotMetadata( - name + "live1", - LIVE_SNAPSHOT_PATH, - startTimeMs, - endTimeMs, - maxOffset, - "1", - LOGS_LUCENE9, - 0); + new SnapshotMetadata(name + "live1", startTimeMs, endTimeMs, maxOffset, "1", 0); snapshotMetadataStore.createSync(livePartition1); assertThat(AstraMetadataTestUtils.listSyncUncached(snapshotMetadataStore)) .containsOnly(livePartition1, livePartition0); @@ -300,27 +274,11 @@ public void testWithMultipleLiveSnapshotsOnIndexerStart() throws Exception { final long endTimeMs = 100; final long maxOffset = 50; SnapshotMetadata livePartition0 = - new SnapshotMetadata( - name + "live0", - LIVE_SNAPSHOT_PATH, - startTimeMs, - endTimeMs, - maxOffset, - "0", - LOGS_LUCENE9, - 0); + new SnapshotMetadata(name + "live0", startTimeMs, endTimeMs, maxOffset, "0", 0); snapshotMetadataStore.createSync(livePartition0); SnapshotMetadata livePartition1 = - new SnapshotMetadata( - name + "live1", - LIVE_SNAPSHOT_PATH, - startTimeMs, - endTimeMs, - maxOffset, - "1", - LOGS_LUCENE9, - 0); + new SnapshotMetadata(name + "live1", startTimeMs, endTimeMs, maxOffset, "1", 0); snapshotMetadataStore.createSync(livePartition1); assertThat(AstraMetadataTestUtils.listSyncUncached(snapshotMetadataStore)) .containsOnly(livePartition1, livePartition0); @@ -355,36 +313,19 @@ public void testIndexerStartsWithPreviousOffset() throws Exception { // Create a live partition for this partiton final String name = "testSnapshotId"; - final String path = "/testPath_" + name; final long startTimeMs = 1; final long endTimeMs = 100; final long maxOffset = 50; SnapshotMetadata livePartition0 = - new SnapshotMetadata( - name + "live0", - LIVE_SNAPSHOT_PATH, - startTimeMs, - endTimeMs, - maxOffset, - "0", - LOGS_LUCENE9, - 0); + new SnapshotMetadata(name + "live0", startTimeMs, endTimeMs, maxOffset, "0", 0); snapshotMetadataStore.createSync(livePartition0); SnapshotMetadata livePartition1 = - new SnapshotMetadata( - name + "live1", - LIVE_SNAPSHOT_PATH, - startTimeMs, - endTimeMs, - maxOffset, - "1", - LOGS_LUCENE9, - 0); + new SnapshotMetadata(name + "live1", startTimeMs, endTimeMs, maxOffset, "1", 0); snapshotMetadataStore.createSync(livePartition1); final SnapshotMetadata partition0 = - new SnapshotMetadata(name, path, startTimeMs, endTimeMs, maxOffset, "0", LOGS_LUCENE9, 0); + new SnapshotMetadata(name, startTimeMs, endTimeMs, maxOffset, "0", 100); snapshotMetadataStore.createSync(partition0); assertThat(AstraMetadataTestUtils.listSyncUncached(snapshotMetadataStore)) @@ -422,36 +363,19 @@ public void testIndexerCreatesRecoveryTask() throws Exception { // Create a live partition for this partiton final String name = "testSnapshotId"; - final String path = "/testPath_" + name; final long startTimeMs = 1; final long endTimeMs = 100; final long maxOffset = 30; SnapshotMetadata livePartition0 = - new SnapshotMetadata( - name + "live0", - LIVE_SNAPSHOT_PATH, - startTimeMs, - endTimeMs, - maxOffset, - "0", - LOGS_LUCENE9, - 0); + new SnapshotMetadata(name + "live0", startTimeMs, endTimeMs, maxOffset, "0", 0); snapshotMetadataStore.createSync(livePartition0); SnapshotMetadata livePartition1 = - new SnapshotMetadata( - name + "live1", - LIVE_SNAPSHOT_PATH, - startTimeMs, - endTimeMs, - maxOffset, - "1", - LOGS_LUCENE9, - 0); + new SnapshotMetadata(name + "live1", startTimeMs, endTimeMs, maxOffset, "1", 0); snapshotMetadataStore.createSync(livePartition1); final SnapshotMetadata partition0 = - new SnapshotMetadata(name, path, startTimeMs, endTimeMs, maxOffset, "0", LOGS_LUCENE9, 0); + new SnapshotMetadata(name, startTimeMs, endTimeMs, maxOffset, "0", 100); snapshotMetadataStore.createSync(partition0); assertThat(AstraMetadataTestUtils.listSyncUncached(snapshotMetadataStore)) @@ -497,36 +421,19 @@ public void testIndexerShutdownTwice() throws Exception { // Create a live partition for this partiton final String name = "testSnapshotId"; - final String path = "/testPath_" + name; final long startTimeMs = 1; final long endTimeMs = 100; final long maxOffset = 30; SnapshotMetadata livePartition0 = - new SnapshotMetadata( - name + "live0", - LIVE_SNAPSHOT_PATH, - startTimeMs, - endTimeMs, - maxOffset, - "0", - LOGS_LUCENE9, - 0); + new SnapshotMetadata(name + "live0", startTimeMs, endTimeMs, maxOffset, "0", 0); snapshotMetadataStore.createSync(livePartition0); SnapshotMetadata livePartition1 = - new SnapshotMetadata( - name + "live1", - LIVE_SNAPSHOT_PATH, - startTimeMs, - endTimeMs, - maxOffset, - "1", - LOGS_LUCENE9, - 0); + new SnapshotMetadata(name + "live1", startTimeMs, endTimeMs, maxOffset, "1", 0); snapshotMetadataStore.createSync(livePartition1); final SnapshotMetadata partition0 = - new SnapshotMetadata(name, path, startTimeMs, endTimeMs, maxOffset, "0", LOGS_LUCENE9, 0); + new SnapshotMetadata(name, startTimeMs, endTimeMs, maxOffset, "0", 100); snapshotMetadataStore.createSync(partition0); assertThat(AstraMetadataTestUtils.listSyncUncached(snapshotMetadataStore)) @@ -576,36 +483,19 @@ public void testIndexerRestart() throws Exception { // Create a live partition for this partiton final String name = "testSnapshotId"; - final String path = "/testPath_" + name; final long startTimeMs = 1; final long endTimeMs = 100; final long maxOffset = 30; SnapshotMetadata livePartition0 = - new SnapshotMetadata( - name + "live0", - LIVE_SNAPSHOT_PATH, - startTimeMs, - endTimeMs, - maxOffset, - "0", - LOGS_LUCENE9, - 0); + new SnapshotMetadata(name + "live0", startTimeMs, endTimeMs, maxOffset, "0", 0); snapshotMetadataStore.createSync(livePartition0); SnapshotMetadata livePartition1 = - new SnapshotMetadata( - name + "live1", - LIVE_SNAPSHOT_PATH, - startTimeMs, - endTimeMs, - maxOffset, - "1", - LOGS_LUCENE9, - 0); + new SnapshotMetadata(name + "live1", startTimeMs, endTimeMs, maxOffset, "1", 0); snapshotMetadataStore.createSync(livePartition1); final SnapshotMetadata partition0 = - new SnapshotMetadata(name, path, startTimeMs, endTimeMs, maxOffset, "0", LOGS_LUCENE9, 0); + new SnapshotMetadata(name, startTimeMs, endTimeMs, maxOffset, "0", 100); snapshotMetadataStore.createSync(partition0); assertThat(AstraMetadataTestUtils.listSyncUncached(snapshotMetadataStore)) diff --git a/astra/src/test/java/com/slack/astra/server/ManagerApiGrpcTest.java b/astra/src/test/java/com/slack/astra/server/ManagerApiGrpcTest.java index 848f513f00..5d9e2d90ba 100644 --- a/astra/src/test/java/com/slack/astra/server/ManagerApiGrpcTest.java +++ b/astra/src/test/java/com/slack/astra/server/ManagerApiGrpcTest.java @@ -587,39 +587,29 @@ public void shouldFetchSnapshotsWithinTimeframeAndPartition() { long end = startTime + 10; SnapshotMetadata overlapsStartTimeIncluded = - new SnapshotMetadata( - "a", "a", startTime, startTime + 6, 0, "a", Metadata.IndexType.LOGS_LUCENE9, 0); + new SnapshotMetadata("a", startTime, startTime + 6, 0, "a", 0); SnapshotMetadata overlapsStartTimeExcluded = - new SnapshotMetadata( - "b", "b", startTime, startTime + 6, 0, "b", Metadata.IndexType.LOGS_LUCENE9, 0); + new SnapshotMetadata("b", startTime, startTime + 6, 0, "b", 0); SnapshotMetadata fullyOverlapsStartEndTimeIncluded = - new SnapshotMetadata( - "c", "c", startTime + 4, startTime + 11, 0, "a", Metadata.IndexType.LOGS_LUCENE9, 0); + new SnapshotMetadata("c", startTime + 4, startTime + 11, 0, "a", 0); SnapshotMetadata fullyOverlapsStartEndTimeExcluded = - new SnapshotMetadata( - "d", "d", startTime + 4, startTime + 11, 0, "b", Metadata.IndexType.LOGS_LUCENE9, 0); + new SnapshotMetadata("d", startTime + 4, startTime + 11, 0, "b", 0); SnapshotMetadata partiallyOverlapsStartEndTimeIncluded = - new SnapshotMetadata( - "e", "e", startTime + 4, startTime + 5, 0, "a", Metadata.IndexType.LOGS_LUCENE9, 0); + new SnapshotMetadata("e", startTime + 4, startTime + 5, 0, "a", 0); SnapshotMetadata partiallyOverlapsStartEndTimeExcluded = - new SnapshotMetadata( - "f", "f", startTime + 4, startTime + 5, 0, "b", Metadata.IndexType.LOGS_LUCENE9, 0); + new SnapshotMetadata("f", startTime + 4, startTime + 5, 0, "b", 0); SnapshotMetadata overlapsEndTimeIncluded = - new SnapshotMetadata( - "g", "g", startTime + 10, startTime + 15, 0, "a", Metadata.IndexType.LOGS_LUCENE9, 0); + new SnapshotMetadata("g", startTime + 10, startTime + 15, 0, "a", 0); SnapshotMetadata overlapsEndTimeExcluded = - new SnapshotMetadata( - "h", "h", startTime + 10, startTime + 15, 0, "b", Metadata.IndexType.LOGS_LUCENE9, 0); + new SnapshotMetadata("h", startTime + 10, startTime + 15, 0, "b", 0); SnapshotMetadata notWithinStartEndTimeExcluded1 = - new SnapshotMetadata( - "i", "i", startTime, startTime + 4, 0, "a", Metadata.IndexType.LOGS_LUCENE9, 0); + new SnapshotMetadata("i", startTime, startTime + 4, 0, "a", 0); SnapshotMetadata notWithinStartEndTimeExcluded2 = - new SnapshotMetadata( - "j", "j", startTime + 11, startTime + 15, 0, "a", Metadata.IndexType.LOGS_LUCENE9, 0); + new SnapshotMetadata("j", startTime + 11, startTime + 15, 0, "a", 0); DatasetMetadata datasetWithDataInPartitionA = new DatasetMetadata( @@ -669,11 +659,9 @@ public void shouldRestoreReplicaSinglePartition() { long end = startTime + 10; SnapshotMetadata snapshotIncluded = - new SnapshotMetadata( - "g", "g", startTime + 10, startTime + 15, 0, "a", Metadata.IndexType.LOGS_LUCENE9, 0); + new SnapshotMetadata("g", startTime + 10, startTime + 15, 0, "a", 0); SnapshotMetadata snapshotExcluded = - new SnapshotMetadata( - "h", "h", startTime + 10, startTime + 15, 0, "b", Metadata.IndexType.LOGS_LUCENE9, 0); + new SnapshotMetadata("h", startTime + 10, startTime + 15, 0, "b", 0); snapshotMetadataStore.createSync(snapshotIncluded); snapshotMetadataStore.createSync(snapshotExcluded); @@ -713,14 +701,11 @@ public void shouldRestoreReplicasMultiplePartitions() { long end = startTime + 10; SnapshotMetadata snapshotIncluded = - new SnapshotMetadata( - "a", "a", startTime + 10, startTime + 15, 0, "a", Metadata.IndexType.LOGS_LUCENE9, 0); + new SnapshotMetadata("a", startTime + 10, startTime + 15, 0, "a", 0); SnapshotMetadata snapshotIncluded2 = - new SnapshotMetadata( - "b", "b", startTime + 10, startTime + 15, 0, "b", Metadata.IndexType.LOGS_LUCENE9, 0); + new SnapshotMetadata("b", startTime + 10, startTime + 15, 0, "b", 0); SnapshotMetadata snapshotExcluded = - new SnapshotMetadata( - "c", "c", startTime + 10, startTime + 15, 0, "c", Metadata.IndexType.LOGS_LUCENE9, 0); + new SnapshotMetadata("c", startTime + 10, startTime + 15, 0, "c", 0); snapshotMetadataStore.createSync(snapshotIncluded); snapshotMetadataStore.createSync(snapshotIncluded2); @@ -763,14 +748,11 @@ public void shouldRestoreGivenSnapshotIds() { long startTime = Instant.now().toEpochMilli(); SnapshotMetadata snapshotFoo = - new SnapshotMetadata( - "foo", "a", startTime + 10, startTime + 15, 0, "a", Metadata.IndexType.LOGS_LUCENE9, 0); + new SnapshotMetadata("foo", startTime + 10, startTime + 15, 0, "a", 0); SnapshotMetadata snapshotBar = - new SnapshotMetadata( - "bar", "b", startTime + 10, startTime + 15, 0, "b", Metadata.IndexType.LOGS_LUCENE9, 0); + new SnapshotMetadata("bar", startTime + 10, startTime + 15, 0, "b", 0); SnapshotMetadata snapshotBaz = - new SnapshotMetadata( - "baz", "c", startTime + 10, startTime + 15, 0, "c", Metadata.IndexType.LOGS_LUCENE9, 0); + new SnapshotMetadata("baz", startTime + 10, startTime + 15, 0, "c", 0); snapshotMetadataStore.createSync(snapshotFoo); snapshotMetadataStore.createSync(snapshotBar); diff --git a/astra/src/test/java/com/slack/astra/server/RecoveryTaskCreatorTest.java b/astra/src/test/java/com/slack/astra/server/RecoveryTaskCreatorTest.java index c356332dd6..de0d7bad41 100644 --- a/astra/src/test/java/com/slack/astra/server/RecoveryTaskCreatorTest.java +++ b/astra/src/test/java/com/slack/astra/server/RecoveryTaskCreatorTest.java @@ -1,7 +1,5 @@ package com.slack.astra.server; -import static com.slack.astra.metadata.snapshot.SnapshotMetadata.LIVE_SNAPSHOT_PATH; -import static com.slack.astra.proto.metadata.Metadata.IndexType.LOGS_LUCENE9; import static com.slack.astra.server.RecoveryTaskCreator.RECOVERY_TASKS_CREATED; import static com.slack.astra.server.RecoveryTaskCreator.STALE_SNAPSHOT_DELETE_SUCCESS; import static com.slack.astra.server.RecoveryTaskCreator.getHighestDurableOffsetForPartition; @@ -21,6 +19,7 @@ import static org.mockito.Mockito.when; import brave.Tracing; +import com.slack.astra.chunk.ChunkInfo; import com.slack.astra.metadata.core.AstraMetadataTestUtils; import com.slack.astra.metadata.core.CuratorBuilder; import com.slack.astra.metadata.core.InternalMetadataStoreException; @@ -109,39 +108,22 @@ public void shutdown() throws IOException { @Test public void testStaleSnapshotDetection() { final String name = "testSnapshotId"; - final String path = "/testPath_" + name; final long startTime = 1; final long endTime = 100; final long maxOffset = 123; SnapshotMetadata partition1 = - new SnapshotMetadata( - name, path, startTime, endTime, maxOffset, partitionId, LOGS_LUCENE9, 0); + new SnapshotMetadata(name, startTime, endTime, maxOffset, partitionId, 100); SnapshotMetadata livePartition1 = new SnapshotMetadata( - name + "1", - LIVE_SNAPSHOT_PATH, - startTime, - endTime, - maxOffset, - partitionId, - LOGS_LUCENE9, - 0); + name + "1", startTime, ChunkInfo.MAX_FUTURE_TIME, maxOffset, partitionId, 0); SnapshotMetadata livePartition11 = new SnapshotMetadata( - name + "11", - LIVE_SNAPSHOT_PATH, - startTime, - endTime, - maxOffset, - partitionId, - LOGS_LUCENE9, - 0); + name + "11", startTime, ChunkInfo.MAX_FUTURE_TIME, maxOffset, partitionId, 0); SnapshotMetadata livePartition2 = - new SnapshotMetadata( - name + "2", LIVE_SNAPSHOT_PATH, startTime, endTime, maxOffset, "2", LOGS_LUCENE9, 0); + new SnapshotMetadata(name + "2", startTime, ChunkInfo.MAX_FUTURE_TIME, maxOffset, "2", 0); SnapshotMetadata partition2 = - new SnapshotMetadata(name + "3", path, startTime, endTime, maxOffset, "2", LOGS_LUCENE9, 0); + new SnapshotMetadata(name + "3", startTime, endTime, maxOffset, "2", 100); assertThat(getStaleLiveSnapshots(List.of(partition1), partitionId)).isEmpty(); assertThat(getStaleLiveSnapshots(List.of(partition2), partitionId)).isEmpty(); @@ -171,39 +153,22 @@ public void testStaleSnapshotDetection() { @Test public void testDeleteStaleSnapshotDeletion() { final String name = "testSnapshotId"; - final String path = "/testPath_" + name; final long startTime = 1; final long endTime = 100; final long maxOffset = 123; SnapshotMetadata partition1 = - new SnapshotMetadata( - name, path, startTime, endTime, maxOffset, partitionId, LOGS_LUCENE9, 0); + new SnapshotMetadata(name, startTime, endTime, maxOffset, partitionId, 100); SnapshotMetadata livePartition1 = new SnapshotMetadata( - name + "1", - LIVE_SNAPSHOT_PATH, - startTime, - endTime, - maxOffset, - partitionId, - LOGS_LUCENE9, - 0); + name + "1", startTime, ChunkInfo.MAX_FUTURE_TIME, maxOffset, partitionId, 0); SnapshotMetadata livePartition11 = new SnapshotMetadata( - name + "11", - LIVE_SNAPSHOT_PATH, - startTime, - endTime, - maxOffset, - partitionId, - LOGS_LUCENE9, - 0); + name + "11", startTime, ChunkInfo.MAX_FUTURE_TIME, maxOffset, partitionId, 0); SnapshotMetadata livePartition2 = - new SnapshotMetadata( - name + "2", LIVE_SNAPSHOT_PATH, startTime, endTime, maxOffset, "2", LOGS_LUCENE9, 0); + new SnapshotMetadata(name + "2", startTime, ChunkInfo.MAX_FUTURE_TIME, maxOffset, "2", 0); SnapshotMetadata partition2 = - new SnapshotMetadata(name + "3", path, startTime, endTime, maxOffset, "2", LOGS_LUCENE9, 0); + new SnapshotMetadata(name + "3", startTime, endTime, maxOffset, "2", 100); testDeleteSnapshots(List.of(partition1), 0, List.of(partition1)); testDeleteSnapshots(List.of(partition2), 0, List.of(partition2)); @@ -254,39 +219,22 @@ private void testDeleteSnapshots( @Test public void shouldStaleDeletionShouldHandleTimeouts() { final String name = "testSnapshotId"; - final String path = "/testPath_" + name; final long startTime = 1; final long endTime = 100; final long maxOffset = 123; SnapshotMetadata partition1 = - new SnapshotMetadata( - name, path, startTime, endTime, maxOffset, partitionId, LOGS_LUCENE9, 0); + new SnapshotMetadata(name, startTime, endTime, maxOffset, partitionId, 100); SnapshotMetadata livePartition1 = new SnapshotMetadata( - name + "1", - LIVE_SNAPSHOT_PATH, - startTime, - endTime, - maxOffset, - partitionId, - LOGS_LUCENE9, - 0); + name + "1", startTime, ChunkInfo.MAX_FUTURE_TIME, maxOffset, partitionId, 0); SnapshotMetadata livePartition11 = new SnapshotMetadata( - name + "11", - LIVE_SNAPSHOT_PATH, - startTime, - endTime, - maxOffset, - partitionId, - LOGS_LUCENE9, - 0); + name + "11", startTime, ChunkInfo.MAX_FUTURE_TIME, maxOffset, partitionId, 0); SnapshotMetadata livePartition2 = - new SnapshotMetadata( - name + "2", LIVE_SNAPSHOT_PATH, startTime, endTime, maxOffset, "2", LOGS_LUCENE9, 0); + new SnapshotMetadata(name + "2", startTime, ChunkInfo.MAX_FUTURE_TIME, maxOffset, "2", 0); SnapshotMetadata partition2 = - new SnapshotMetadata(name + "3", path, startTime, endTime, maxOffset, "2", LOGS_LUCENE9, 0); + new SnapshotMetadata(name + "3", startTime, endTime, maxOffset, "2", 100); testDeleteSnapshotsTimeouts(List.of(partition1), List.of(partition1), false); testDeleteSnapshotsTimeouts(List.of(livePartition1), List.of(livePartition1), true); @@ -348,39 +296,22 @@ private void testDeleteSnapshotsTimeouts( @Test public void shouldStaleDeletionShouldHandleExceptions() { final String name = "testSnapshotId"; - final String path = "/testPath_" + name; final long startTime = 1; final long endTime = 100; final long maxOffset = 123; final SnapshotMetadata partition1 = - new SnapshotMetadata( - name, path, startTime, endTime, maxOffset, partitionId, LOGS_LUCENE9, 0); + new SnapshotMetadata(name, startTime, endTime, maxOffset, partitionId, 100); final SnapshotMetadata livePartition1 = new SnapshotMetadata( - name + "1", - LIVE_SNAPSHOT_PATH, - startTime, - endTime, - maxOffset, - partitionId, - LOGS_LUCENE9, - 0); + name + "1", startTime, ChunkInfo.MAX_FUTURE_TIME, maxOffset, partitionId, 0); final SnapshotMetadata livePartition11 = new SnapshotMetadata( - name + "11", - LIVE_SNAPSHOT_PATH, - startTime, - endTime, - maxOffset, - partitionId, - LOGS_LUCENE9, - 0); + name + "11", startTime, ChunkInfo.MAX_FUTURE_TIME, maxOffset, partitionId, 0); final SnapshotMetadata livePartition2 = - new SnapshotMetadata( - name + "2", LIVE_SNAPSHOT_PATH, startTime, endTime, maxOffset, "2", LOGS_LUCENE9, 0); + new SnapshotMetadata(name + "2", startTime, endTime, maxOffset, "2", 0); final SnapshotMetadata partition2 = - new SnapshotMetadata(name + "3", path, startTime, endTime, maxOffset, "2", LOGS_LUCENE9, 0); + new SnapshotMetadata(name + "3", startTime, endTime, maxOffset, "2", 100); List snapshots = List.of(partition1, livePartition1, livePartition11, partition2, livePartition2); @@ -445,60 +376,28 @@ public void shouldStaleDeletionShouldHandleExceptions() { @Test public void testMaxOffset() { final String name = "testSnapshotId"; - final String path = "/testPath_" + name; final long startTime = 1; final long endTime = 100; final long maxOffset = 100; final SnapshotMetadata partition1 = - new SnapshotMetadata( - name, path, startTime, endTime, maxOffset, partitionId, LOGS_LUCENE9, 0); + new SnapshotMetadata(name, startTime, endTime, maxOffset, partitionId, 0); final SnapshotMetadata partition11 = - new SnapshotMetadata( - name + "1", - path, - endTime + 1, - endTime * 2, - maxOffset * 2, - partitionId, - LOGS_LUCENE9, - 0); + new SnapshotMetadata(name + "1", endTime + 1, endTime * 2, maxOffset * 2, partitionId, 0); final SnapshotMetadata partition12 = new SnapshotMetadata( - name + "12", - path, - endTime * 2 + 1, - endTime * 3, - maxOffset * 3, - partitionId, - LOGS_LUCENE9, - 0); + name + "12", endTime * 2 + 1, endTime * 3, maxOffset * 3, partitionId, 0); final String partitionId2 = "2"; final long partition2Offset = maxOffset * 10; final SnapshotMetadata partition2 = - new SnapshotMetadata( - name + "2", path, startTime, endTime, partition2Offset, partitionId2, LOGS_LUCENE9, 0); + new SnapshotMetadata(name + "2", startTime, endTime, partition2Offset, partitionId2, 0); final SnapshotMetadata partition21 = new SnapshotMetadata( - name + "21", - path, - endTime + 1, - endTime * 2, - partition2Offset * 2, - partitionId2, - LOGS_LUCENE9, - 0); + name + "21", endTime + 1, endTime * 2, partition2Offset * 2, partitionId2, 0); final SnapshotMetadata partition22 = new SnapshotMetadata( - name + "22", - path, - endTime * 2 + 1, - endTime * 3, - partition2Offset * 3, - partitionId2, - LOGS_LUCENE9, - 0); + name + "22", endTime * 2 + 1, endTime * 3, partition2Offset * 3, partitionId2, 0); // empty results assertThat( @@ -768,13 +667,12 @@ public void testInit() { // Data exists for not for this partition. final String name = "testSnapshotId"; - final String path = "/testPath_" + name; final long startTime = 1; final long endTime = 100; final long maxOffset = 100; final SnapshotMetadata partition1 = - new SnapshotMetadata(name, path, startTime, endTime, maxOffset, "2", LOGS_LUCENE9, 0); + new SnapshotMetadata(name, startTime, endTime, maxOffset, "2", 0); snapshotMetadataStore.createSync(partition1); await().until(() -> snapshotMetadataStore.listSync().contains(partition1)); @@ -782,8 +680,7 @@ public void testInit() { .isEqualTo(1); final SnapshotMetadata partition11 = - new SnapshotMetadata( - name + "1", path, endTime + 1, endTime * 2, maxOffset * 2, "2", LOGS_LUCENE9, 0); + new SnapshotMetadata(name + "1", endTime + 1, endTime * 2, maxOffset * 2, "2", 0); snapshotMetadataStore.createSync(partition11); await().until(() -> snapshotMetadataStore.listSync().contains(partition11)); assertThat(AstraMetadataTestUtils.listSyncUncached(snapshotMetadataStore)) @@ -827,20 +724,18 @@ public void testDetermineStartOffsetReturnsNegativeWhenNoOffset() { // Data exists for not for this partition. final String name = "testSnapshotId"; - final String path = "/testPath_" + name; final long startTime = 1; final long endTime = 100; final long maxOffset = 100; final SnapshotMetadata partition1 = - new SnapshotMetadata(name, path, startTime, endTime, maxOffset, "2", LOGS_LUCENE9, 0); + new SnapshotMetadata(name, startTime, endTime, maxOffset, "2", 0); snapshotMetadataStore.createSync(partition1); await().until(() -> snapshotMetadataStore.listSync().contains(partition1)); assertThat(recoveryTaskCreator.determineStartingOffset(0, 0, indexerConfig)).isNegative(); final SnapshotMetadata partition11 = - new SnapshotMetadata( - name + "1", path, endTime + 1, endTime * 2, maxOffset * 2, "2", LOGS_LUCENE9, 0); + new SnapshotMetadata(name + "1", endTime + 1, endTime * 2, maxOffset * 2, "2", 0); snapshotMetadataStore.createSync(partition11); await().until(() -> snapshotMetadataStore.listSync().contains(partition11)); assertThat(AstraMetadataTestUtils.listSyncUncached(snapshotMetadataStore)) @@ -1110,14 +1005,12 @@ public void testDetermineStartingOffsetOnlySnapshotsNoDelay() { assertThat(recoveryTaskCreator.determineStartingOffset(1000, 0, indexerConfig)).isNegative(); final String name = "testSnapshotId"; - final String path = "/testPath_" + name; final long startTime = 1; final long endTime = 100; final long maxOffset = 100; final SnapshotMetadata partition1 = - new SnapshotMetadata( - name, path, startTime, endTime, maxOffset, partitionId, LOGS_LUCENE9, 0); + new SnapshotMetadata(name, startTime, endTime, maxOffset, partitionId, 100); snapshotMetadataStore.createSync(partition1); await().until(() -> snapshotMetadataStore.listSync().contains(partition1)); assertThat( @@ -1133,14 +1026,7 @@ public void testDetermineStartingOffsetOnlySnapshotsNoDelay() { final SnapshotMetadata partition11 = new SnapshotMetadata( - name + "11", - path, - endTime + 1, - endTime * 2, - maxOffset * 2, - partitionId, - LOGS_LUCENE9, - 0); + name + "11", endTime + 1, endTime * 2, maxOffset * 2, partitionId, 100); snapshotMetadataStore.createSync(partition11); await().until(() -> snapshotMetadataStore.listSync().contains(partition11)); @@ -1157,14 +1043,7 @@ public void testDetermineStartingOffsetOnlySnapshotsNoDelay() { // Live partition is cleaned up, no delay. SnapshotMetadata livePartition1 = new SnapshotMetadata( - name + "live1", - LIVE_SNAPSHOT_PATH, - startTime, - endTime, - maxOffset, - partitionId, - LOGS_LUCENE9, - 0); + name + "live1", startTime, ChunkInfo.MAX_FUTURE_TIME, maxOffset, partitionId, 0); snapshotMetadataStore.createSync(livePartition1); await().until(() -> snapshotMetadataStore.listSync().contains(livePartition1)); assertThat(AstraMetadataTestUtils.listSyncUncached(snapshotMetadataStore)) @@ -1180,14 +1059,7 @@ public void testDetermineStartingOffsetOnlySnapshotsNoDelay() { snapshotMetadataStore.createSync(livePartition1); SnapshotMetadata livePartition11 = new SnapshotMetadata( - name + "live11", - LIVE_SNAPSHOT_PATH, - startTime, - endTime, - maxOffset, - partitionId, - LOGS_LUCENE9, - 0); + name + "live11", startTime, ChunkInfo.MAX_FUTURE_TIME, maxOffset, partitionId, 0); snapshotMetadataStore.createSync(livePartition11); await().until(() -> snapshotMetadataStore.listSync().contains(livePartition11)); assertThat(AstraMetadataTestUtils.listSyncUncached(snapshotMetadataStore)) @@ -1204,14 +1076,7 @@ public void testDetermineStartingOffsetOnlySnapshotsNoDelay() { snapshotMetadataStore.createSync(livePartition11); SnapshotMetadata livePartition2 = new SnapshotMetadata( - name + "2", - LIVE_SNAPSHOT_PATH, - startTime, - endTime, - maxOffset * 5, - "2", - LOGS_LUCENE9, - 0); + name + "2", startTime, ChunkInfo.MAX_FUTURE_TIME, maxOffset * 5, "2", 0); snapshotMetadataStore.createSync(livePartition2); await() .until( @@ -1232,8 +1097,7 @@ public void testDetermineStartingOffsetOnlySnapshotsNoDelay() { snapshotMetadataStore.createSync(livePartition1); snapshotMetadataStore.createSync(livePartition11); SnapshotMetadata partition2 = - new SnapshotMetadata( - name + "3", path, startTime, endTime, maxOffset * 3, "2", LOGS_LUCENE9, 0); + new SnapshotMetadata(name + "3", startTime, endTime, maxOffset * 3, "2", 0); snapshotMetadataStore.createSync(partition2); await() .until( @@ -1269,14 +1133,12 @@ public void testDetermineStartingOffsetOnlySnapshotsWithDelay() { assertThat(recoveryTaskCreator.determineStartingOffset(1000, 0, indexerConfig)).isNegative(); final String name = "testSnapshotId"; - final String path = "/testPath_" + name; final long startTime = 1; final long endTime = 100; final long maxOffset = 100; final SnapshotMetadata partition1 = - new SnapshotMetadata( - name, path, startTime, endTime, maxOffset, partitionId, LOGS_LUCENE9, 0); + new SnapshotMetadata(name, startTime, endTime, maxOffset, partitionId, 100); snapshotMetadataStore.createSync(partition1); await().until(() -> snapshotMetadataStore.listSync().contains(partition1)); assertThat( @@ -1302,14 +1164,7 @@ public void testDetermineStartingOffsetOnlySnapshotsWithDelay() { final SnapshotMetadata partition11 = new SnapshotMetadata( - name + "11", - path, - endTime + 1, - endTime * 2, - maxOffset * 2, - partitionId, - LOGS_LUCENE9, - 0); + name + "11", endTime + 1, endTime * 2, maxOffset * 2, partitionId, 100); snapshotMetadataStore.createSync(partition11); await().until(() -> snapshotMetadataStore.listSync().contains(partition11)); @@ -1338,14 +1193,7 @@ public void testDetermineStartingOffsetOnlySnapshotsWithDelay() { // Live partition is cleaned up, new recovery task is created. SnapshotMetadata livePartition1 = new SnapshotMetadata( - name + "live1", - LIVE_SNAPSHOT_PATH, - startTime, - endTime, - maxOffset, - partitionId, - LOGS_LUCENE9, - 0); + name + "live1", startTime, ChunkInfo.MAX_FUTURE_TIME, maxOffset, partitionId, 0); snapshotMetadataStore.createSync(livePartition1); await() .until( @@ -1379,14 +1227,7 @@ public void testDetermineStartingOffsetOnlySnapshotsWithDelay() { snapshotMetadataStore.createSync(livePartition1); SnapshotMetadata livePartition11 = new SnapshotMetadata( - name + "live11", - LIVE_SNAPSHOT_PATH, - startTime, - endTime, - maxOffset, - partitionId, - LOGS_LUCENE9, - 0); + name + "live11", startTime, ChunkInfo.MAX_FUTURE_TIME, maxOffset, partitionId, 0); snapshotMetadataStore.createSync(livePartition11); await() .until( @@ -1425,14 +1266,7 @@ public void testDetermineStartingOffsetOnlySnapshotsWithDelay() { snapshotMetadataStore.createSync(livePartition11); SnapshotMetadata livePartition2 = new SnapshotMetadata( - name + "2", - LIVE_SNAPSHOT_PATH, - startTime, - endTime, - maxOffset * 5, - "2", - LOGS_LUCENE9, - 0); + name + "2", startTime, ChunkInfo.MAX_FUTURE_TIME, maxOffset * 5, "2", 0); snapshotMetadataStore.createSync(livePartition2); await() .until( @@ -1470,8 +1304,7 @@ public void testDetermineStartingOffsetOnlySnapshotsWithDelay() { snapshotMetadataStore.createSync(livePartition1); snapshotMetadataStore.createSync(livePartition11); SnapshotMetadata partition2 = - new SnapshotMetadata( - name + "3", path, startTime, endTime, maxOffset * 3, "2", LOGS_LUCENE9, 0); + new SnapshotMetadata(name + "3", startTime, endTime, maxOffset * 3, "2", 100); snapshotMetadataStore.createSync(partition2); await() .until( @@ -1531,14 +1364,12 @@ public void testRecoveryTaskCreationFailureFailsDetermineStartOffset() { assertThat(recoveryTaskCreator.determineStartingOffset(1000, 0, indexerConfig)).isNegative(); final String name = "testSnapshotId"; - final String path = "/testPath_" + name; final long startTime = 1; final long endTime = 100; final long maxOffset = 100; final SnapshotMetadata partition1 = - new SnapshotMetadata( - name, path, startTime, endTime, maxOffset, partitionId, LOGS_LUCENE9, 0); + new SnapshotMetadata(name, startTime, endTime, maxOffset, partitionId, 100); snapshotMetadataStore.createSync(partition1); await().until(() -> snapshotMetadataStore.listSync().contains(partition1)); assertThat( @@ -1584,14 +1415,12 @@ public void testSnapshotListFailureFailsDetermineStartOffset() { assertThat(recoveryTaskCreator.determineStartingOffset(1000, 0, indexerConfig)).isNegative(); final String name = "testSnapshotId"; - final String path = "/testPath_" + name; final long startTime = 1; final long endTime = 100; final long maxOffset = 100; final SnapshotMetadata partition1 = - new SnapshotMetadata( - name, path, startTime, endTime, maxOffset, partitionId, LOGS_LUCENE9, 0); + new SnapshotMetadata(name, startTime, endTime, maxOffset, partitionId, 100); snapshotMetadataStore.createSync(partition1); await().until(() -> snapshotMetadataStore.listSync().contains(partition1)); assertThat( @@ -1641,14 +1470,12 @@ public void testRecoveryListFailureFailsDetermineStartOffset() { assertThat(recoveryTaskCreator.determineStartingOffset(1000, 0, indexerConfig)).isNegative(); final String name = "testSnapshotId"; - final String path = "/testPath_" + name; final long startTime = 1; final long endTime = 100; final long maxOffset = 100; final SnapshotMetadata partition1 = - new SnapshotMetadata( - name, path, startTime, endTime, maxOffset, partitionId, LOGS_LUCENE9, 0); + new SnapshotMetadata(name, startTime, endTime, maxOffset, partitionId, 100); snapshotMetadataStore.createSync(partition1); await().until(() -> snapshotMetadataStore.listSync().contains(partition1)); assertThat( @@ -1699,14 +1526,12 @@ public void testFailureToDeleteStaleSnapshotsFailsDetermineStartOffset() assertThat(recoveryTaskCreator.determineStartingOffset(1000, 0, indexerConfig)).isNegative(); final String name = "testSnapshotId"; - final String path = "/testPath_" + name; final long startTime = 1; final long endTime = 100; final long maxOffset = 100; final SnapshotMetadata partition1 = - new SnapshotMetadata( - name, path, startTime, endTime, maxOffset, partitionId, LOGS_LUCENE9, 0); + new SnapshotMetadata(name, startTime, endTime, maxOffset, partitionId, 100); snapshotMetadataStore.createSync(partition1); await().until(() -> snapshotMetadataStore.listSync().contains(partition1)); assertThat( @@ -1728,14 +1553,7 @@ public void testFailureToDeleteStaleSnapshotsFailsDetermineStartOffset() // Add a live partition to be deleted. SnapshotMetadata livePartition1 = new SnapshotMetadata( - name + "1", - LIVE_SNAPSHOT_PATH, - startTime, - endTime, - maxOffset, - partitionId, - LOGS_LUCENE9, - 0); + name + "1", startTime, ChunkInfo.MAX_FUTURE_TIME, maxOffset, partitionId, 0); snapshotMetadataStore.createSync(livePartition1); await().until(() -> snapshotMetadataStore.listSync().contains(livePartition1)); assertThat(AstraMetadataTestUtils.listSyncUncached(snapshotMetadataStore)) @@ -1867,14 +1685,12 @@ public void testMultipleRecoveryTaskCreationWithSnapshotDelay() { assertThat(AstraMetadataTestUtils.listSyncUncached(recoveryTaskStore)).isEmpty(); final String name = "testSnapshotId"; - final String path = "/testPath_" + name; final long startTime = 1; final long endTime = 100; final long maxOffset = 100; final SnapshotMetadata partition1 = - new SnapshotMetadata( - name, path, startTime, endTime, maxOffset, partitionId, LOGS_LUCENE9, 0); + new SnapshotMetadata(name, startTime, endTime, maxOffset, partitionId, 100); snapshotMetadataStore.createSync(partition1); await().until(() -> snapshotMetadataStore.listSync().contains(partition1)); assertThat( diff --git a/astra/src/test/java/com/slack/astra/testlib/ChunkManagerUtil.java b/astra/src/test/java/com/slack/astra/testlib/ChunkManagerUtil.java index 789e3bad85..d4ae4b3a4f 100644 --- a/astra/src/test/java/com/slack/astra/testlib/ChunkManagerUtil.java +++ b/astra/src/test/java/com/slack/astra/testlib/ChunkManagerUtil.java @@ -124,7 +124,7 @@ public void close() throws IOException, TimeoutException { } public static List fetchNonLiveSnapshot(List snapshots) { - Predicate nonLiveSnapshotPredicate = s -> !SnapshotMetadata.isLive(s); + Predicate nonLiveSnapshotPredicate = s -> !s.isLive(); return fetchSnapshotMatching(snapshots, nonLiveSnapshotPredicate); } diff --git a/astra/src/test/java/com/slack/astra/util/SnapshotUtil.java b/astra/src/test/java/com/slack/astra/util/SnapshotUtil.java deleted file mode 100644 index d74a373348..0000000000 --- a/astra/src/test/java/com/slack/astra/util/SnapshotUtil.java +++ /dev/null @@ -1,18 +0,0 @@ -package com.slack.astra.util; - -import com.slack.astra.metadata.snapshot.SnapshotMetadata; -import com.slack.astra.proto.metadata.Metadata; - -public class SnapshotUtil { - public static SnapshotMetadata makeSnapshot(String name) { - return new SnapshotMetadata( - name + "snapshotId", - "/testPath_" + name, - 1, - 100, - 1, - "1", - Metadata.IndexType.LOGS_LUCENE9, - 0); - } -} From 3d7fae1714579456c7dd37b209bb0374895e95ae Mon Sep 17 00:00:00 2001 From: Bryan Burkholder Date: Tue, 20 Aug 2024 09:51:31 -0700 Subject: [PATCH 7/7] Refactoring, add documentation, add tests --- .../{ChunkStore.java => BlobStore.java} | 92 +++++--- .../com/slack/astra/blobfs/S3AsyncUtil.java | 12 +- .../slack/astra/chunk/ReadOnlyChunkImpl.java | 16 +- .../com/slack/astra/chunk/ReadWriteChunk.java | 6 +- .../chunkManager/CachingChunkManager.java | 16 +- .../chunkManager/ChunkRolloverFactory.java | 10 +- .../chunkManager/IndexingChunkManager.java | 14 +- .../chunkManager/RecoveryChunkManager.java | 6 +- .../astra/chunkManager/RollOverChunkTask.java | 10 +- .../SnapshotDeletionService.java | 10 +- .../slack/astra/recovery/RecoveryService.java | 10 +- .../java/com/slack/astra/server/Astra.java | 20 +- .../com/slack/astra/blobfs/BlobStoreTest.java | 218 ++++++++++++++++++ .../slack/astra/blobfs/ChunkStoreTest.java | 134 ----------- .../astra/chunk/IndexingChunkImplTest.java | 10 +- .../astra/chunk/ReadOnlyChunkImplTest.java | 18 +- .../astra/chunk/RecoveryChunkImplTest.java | 12 +- .../chunkManager/CachingChunkManagerTest.java | 37 ++- .../IndexingChunkManagerTest.java | 56 ++--- .../RecoveryChunkManagerTest.java | 16 +- ...MessageCountBasedRolloverStrategyTest.java | 8 +- .../SnapshotDeletionServiceTest.java | 102 ++++---- .../logstore/LuceneIndexStoreImplTest.java | 8 +- .../astra/recovery/RecoveryServiceTest.java | 26 +-- .../slack/astra/testlib/ChunkManagerUtil.java | 6 +- 25 files changed, 495 insertions(+), 378 deletions(-) rename astra/src/main/java/com/slack/astra/blobfs/{ChunkStore.java => BlobStore.java} (57%) create mode 100644 astra/src/test/java/com/slack/astra/blobfs/BlobStoreTest.java delete mode 100644 astra/src/test/java/com/slack/astra/blobfs/ChunkStoreTest.java diff --git a/astra/src/main/java/com/slack/astra/blobfs/ChunkStore.java b/astra/src/main/java/com/slack/astra/blobfs/BlobStore.java similarity index 57% rename from astra/src/main/java/com/slack/astra/blobfs/ChunkStore.java rename to astra/src/main/java/com/slack/astra/blobfs/BlobStore.java index ae703c92a0..6bbc052a70 100644 --- a/astra/src/main/java/com/slack/astra/blobfs/ChunkStore.java +++ b/astra/src/main/java/com/slack/astra/blobfs/BlobStore.java @@ -5,6 +5,7 @@ import java.nio.file.Path; import java.util.ArrayList; import java.util.List; +import java.util.Objects; import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicBoolean; import software.amazon.awssdk.services.s3.S3AsyncClient; @@ -18,58 +19,98 @@ import software.amazon.awssdk.transfer.s3.model.DownloadDirectoryRequest; import software.amazon.awssdk.transfer.s3.model.UploadDirectoryRequest; -public class ChunkStore { - +/** + * Blob store abstraction for basic operations on chunk/snapshots on remote storage. All operations + * are invoked with a prefix, which can be considered analogous to the "folder," and all objects are + * stored to a consistent location of "{prefix}/{filename}". + */ +public class BlobStore { private final String bucketName; private final S3AsyncClient s3AsyncClient; private final S3TransferManager transferManager; - public ChunkStore(S3AsyncClient s3AsyncClient, String bucketName) { + public BlobStore(S3AsyncClient s3AsyncClient, String bucketName) { this.bucketName = bucketName; this.s3AsyncClient = s3AsyncClient; this.transferManager = S3TransferManager.builder().s3Client(s3AsyncClient).build(); } - public void upload(String chunkId, Path directoryToUpload) { + /** + * Uploads a directory to the object store by prefix + * + * @param prefix Prefix to store (ie, chunk id) + * @param directoryToUpload Directory to upload + * @throws IllegalStateException Thrown if any files fail to upload + * @throws RuntimeException Thrown when error is considered generally non-retryable + */ + public void upload(String prefix, Path directoryToUpload) { + assert prefix != null && !prefix.isEmpty(); + assert directoryToUpload.toFile().isDirectory(); + assert Objects.requireNonNull(directoryToUpload.toFile().listFiles()).length > 0; + try { CompletedDirectoryUpload upload = transferManager .uploadDirectory( UploadDirectoryRequest.builder() .source(directoryToUpload) - .s3Prefix(chunkId) + .s3Prefix(prefix) .bucket(bucketName) .build()) .completionFuture() .get(); if (!upload.failedTransfers().isEmpty()) { - throw new IllegalStateException("Some or all files failed to upload"); + throw new IllegalStateException( + String.format( + "Some files failed to upload - attempted to upload %s files, failed %s.", + Objects.requireNonNull(directoryToUpload.toFile().listFiles()).length, + upload.failedTransfers().size())); } } catch (ExecutionException | InterruptedException e) { throw new RuntimeException(e); } } - // todo - make destination optional - public Path download(String chunkId, Path destination) { + /** + * Downloads a directory from the object store by prefix + * + * @param prefix Prefix to store (ie, chunk id) + * @param destinationDirectory Directory to download to + * @throws RuntimeException Thrown when error is considered generally non-retryable + */ + public void download(String prefix, Path destinationDirectory) { + assert prefix != null && !prefix.isEmpty(); + if (destinationDirectory.toFile().exists()) { + assert destinationDirectory.toFile().isDirectory(); + } + try { transferManager .downloadDirectory( DownloadDirectoryRequest.builder() .bucket(bucketName) - .destination(destination) - .listObjectsV2RequestTransformer(l -> l.prefix(chunkId)) + .destination(destinationDirectory) + .listObjectsV2RequestTransformer(l -> l.prefix(prefix)) .build()) .completionFuture() .get(); - return destination; } catch (ExecutionException | InterruptedException e) { throw new RuntimeException(e); } } - public List listFiles(String chunkId) { - ListObjectsV2Request listRequest = builder().bucket(bucketName).prefix(chunkId).build(); + /** + * Lists all files found on object store by the complete object key (including prefix). This would + * included what is generally considered the directory (ie foo/bar.example) + * + * @param prefix Prefix to list (ie, chunk id) + * @return List of file names by complete object key + * @throws RuntimeException Thrown when error is considered generally non-retryable + */ + public List listFiles(String prefix) { + assert prefix != null && !prefix.isEmpty(); + + ListObjectsV2Request listRequest = builder().bucket(bucketName).prefix(prefix).build(); ListObjectsV2Publisher asyncPaginatedListResponse = s3AsyncClient.listObjectsV2Paginator(listRequest); @@ -77,14 +118,8 @@ public List listFiles(String chunkId) { try { asyncPaginatedListResponse .subscribe( - listResponse -> { - listResponse - .contents() - .forEach( - s3Object -> { - filesList.add(String.format("s3://%s/%s", bucketName, s3Object.key())); - }); - }) + listResponse -> + listResponse.contents().forEach(s3Object -> filesList.add(s3Object.key()))) .get(); } catch (InterruptedException | ExecutionException e) { throw new RuntimeException(e); @@ -92,12 +127,17 @@ public List listFiles(String chunkId) { return filesList; } - public String getRemotePath(String chunkId) { - return String.format("s3://%s/%s", bucketName, chunkId); - } + /** + * Deletes a chunk off of object storage by chunk id. If object was not found returns false. + * + * @param prefix Prefix to delete (ie, chunk id) + * @return boolean if any files at the prefix/directory were deleted + * @throws RuntimeException Thrown when error is considered generally non-retryable + */ + public boolean delete(String prefix) { + assert prefix != null && !prefix.isEmpty(); - public boolean delete(String chunkId) { - ListObjectsV2Request listRequest = builder().bucket(bucketName).prefix(chunkId).build(); + ListObjectsV2Request listRequest = builder().bucket(bucketName).prefix(prefix).build(); ListObjectsV2Publisher asyncPaginatedListResponse = s3AsyncClient.listObjectsV2Paginator(listRequest); diff --git a/astra/src/main/java/com/slack/astra/blobfs/S3AsyncUtil.java b/astra/src/main/java/com/slack/astra/blobfs/S3AsyncUtil.java index e25b780fac..beba2ef778 100644 --- a/astra/src/main/java/com/slack/astra/blobfs/S3AsyncUtil.java +++ b/astra/src/main/java/com/slack/astra/blobfs/S3AsyncUtil.java @@ -23,14 +23,16 @@ public class S3AsyncUtil { private static final Logger LOG = LoggerFactory.getLogger(S3AsyncUtil.class); + private S3AsyncUtil() {} + public static S3AsyncClient initS3Client(AstraConfigs.S3Config config) { - Preconditions.checkArgument(!isNullOrEmpty(config.getS3Region())); + Preconditions.checkArgument(notNullOrEmpty(config.getS3Region())); String region = config.getS3Region(); AwsCredentialsProvider awsCredentialsProvider; try { - if (!isNullOrEmpty(config.getS3AccessKey()) && !isNullOrEmpty(config.getS3SecretKey())) { + if (notNullOrEmpty(config.getS3AccessKey()) && notNullOrEmpty(config.getS3SecretKey())) { String accessKey = config.getS3AccessKey(); String secretKey = config.getS3SecretKey(); AwsBasicCredentials awsBasicCredentials = AwsBasicCredentials.create(accessKey, secretKey); @@ -72,7 +74,7 @@ public static S3AsyncClient initS3Client(AstraConfigs.S3Config config) { .build()); s3AsyncClient.httpConfiguration(httpConfigurationBuilder.build()); - if (!isNullOrEmpty(config.getS3EndPoint())) { + if (notNullOrEmpty(config.getS3EndPoint())) { String endpoint = config.getS3EndPoint(); try { s3AsyncClient.endpointOverride(new URI(endpoint)); @@ -86,7 +88,7 @@ public static S3AsyncClient initS3Client(AstraConfigs.S3Config config) { } } - static boolean isNullOrEmpty(String target) { - return target == null || "".equals(target); + static boolean notNullOrEmpty(String target) { + return target != null && !target.isEmpty(); } } diff --git a/astra/src/main/java/com/slack/astra/chunk/ReadOnlyChunkImpl.java b/astra/src/main/java/com/slack/astra/chunk/ReadOnlyChunkImpl.java index 4ec25ef396..02ba0107a7 100644 --- a/astra/src/main/java/com/slack/astra/chunk/ReadOnlyChunkImpl.java +++ b/astra/src/main/java/com/slack/astra/chunk/ReadOnlyChunkImpl.java @@ -4,7 +4,7 @@ import static com.slack.astra.server.AstraConfig.DEFAULT_ZK_TIMEOUT_SECS; import com.google.common.annotations.VisibleForTesting; -import com.slack.astra.blobfs.ChunkStore; +import com.slack.astra.blobfs.BlobStore; import com.slack.astra.logstore.search.LogIndexSearcher; import com.slack.astra.logstore.search.LogIndexSearcherImpl; import com.slack.astra.logstore.search.SearchQuery; @@ -71,7 +71,7 @@ public class ReadOnlyChunkImpl implements Chunk { private final SearchMetadataStore searchMetadataStore; private CacheNodeAssignmentStore cacheNodeAssignmentStore; private final MeterRegistry meterRegistry; - private final ChunkStore chunkStore; + private final BlobStore blobStore; public static final String CHUNK_ASSIGNMENT_TIMER = "chunk_assignment_timer"; public static final String CHUNK_EVICTION_TIMER = "chunk_eviction_timer"; @@ -89,7 +89,7 @@ public class ReadOnlyChunkImpl implements Chunk { public ReadOnlyChunkImpl( AsyncCuratorFramework curatorFramework, MeterRegistry meterRegistry, - ChunkStore chunkStore, + BlobStore blobStore, SearchContext searchContext, String s3Bucket, String dataDirectoryPrefix, @@ -105,7 +105,7 @@ public ReadOnlyChunkImpl( this( curatorFramework, meterRegistry, - chunkStore, + blobStore, searchContext, s3Bucket, dataDirectoryPrefix, @@ -123,7 +123,7 @@ public ReadOnlyChunkImpl( public ReadOnlyChunkImpl( AsyncCuratorFramework curatorFramework, MeterRegistry meterRegistry, - ChunkStore chunkStore, + BlobStore blobStore, SearchContext searchContext, String s3Bucket, String dataDirectoryPrefix, @@ -134,7 +134,7 @@ public ReadOnlyChunkImpl( SearchMetadataStore searchMetadataStore) throws Exception { this.meterRegistry = meterRegistry; - this.chunkStore = chunkStore; + this.blobStore = blobStore; this.dataDirectoryPrefix = dataDirectoryPrefix; this.searchContext = searchContext; this.slotId = UUID.randomUUID().toString(); @@ -232,7 +232,7 @@ public void downloadChunkData() { } } - chunkStore.download(snapshotMetadata.snapshotId, dataDirectory); + blobStore.download(snapshotMetadata.snapshotId, dataDirectory); try (Stream fileList = Files.list(dataDirectory)) { if (fileList.findAny().isEmpty()) { throw new IOException("No files found on blob storage, released slot for re-assignment"); @@ -374,7 +374,7 @@ private void handleChunkAssignment(CacheSlotMetadata cacheSlotMetadata) { } SnapshotMetadata snapshotMetadata = getSnapshotMetadata(cacheSlotMetadata.replicaId); - chunkStore.download(snapshotMetadata.snapshotId, dataDirectory); + blobStore.download(snapshotMetadata.snapshotId, dataDirectory); try (Stream fileList = Files.list(dataDirectory)) { if (fileList.findAny().isEmpty()) { throw new IOException("No files found on blob storage, released slot for re-assignment"); diff --git a/astra/src/main/java/com/slack/astra/chunk/ReadWriteChunk.java b/astra/src/main/java/com/slack/astra/chunk/ReadWriteChunk.java index d91a6c33be..2881a3df1c 100644 --- a/astra/src/main/java/com/slack/astra/chunk/ReadWriteChunk.java +++ b/astra/src/main/java/com/slack/astra/chunk/ReadWriteChunk.java @@ -4,7 +4,7 @@ import static com.slack.astra.writer.SpanFormatter.isValidTimestamp; import com.google.common.annotations.VisibleForTesting; -import com.slack.astra.blobfs.ChunkStore; +import com.slack.astra.blobfs.BlobStore; import com.slack.astra.logstore.LogStore; import com.slack.astra.logstore.LuceneIndexStoreImpl; import com.slack.astra.logstore.search.LogIndexSearcher; @@ -220,7 +220,7 @@ public void preSnapshot() { * * @return true on success, false on failure. */ - public boolean snapshotToS3(ChunkStore chunkStore) { + public boolean snapshotToS3(BlobStore blobStore) { logger.info("Started RW chunk snapshot to S3 {}", chunkInfo); IndexCommit indexCommit = null; @@ -249,7 +249,7 @@ public boolean snapshotToS3(ChunkStore chunkStore) { } this.fileUploadAttempts.increment(filesToUpload.size()); Timer.Sample snapshotTimer = Timer.start(meterRegistry); - chunkStore.upload(chunkInfo.chunkId, dirPath); + blobStore.upload(chunkInfo.chunkId, dirPath); snapshotTimer.stop(meterRegistry.timer(SNAPSHOT_TIMER)); chunkInfo.setSizeInBytesOnDisk(totalBytes); diff --git a/astra/src/main/java/com/slack/astra/chunkManager/CachingChunkManager.java b/astra/src/main/java/com/slack/astra/chunkManager/CachingChunkManager.java index e454bc2825..b7e75cf048 100644 --- a/astra/src/main/java/com/slack/astra/chunkManager/CachingChunkManager.java +++ b/astra/src/main/java/com/slack/astra/chunkManager/CachingChunkManager.java @@ -2,7 +2,7 @@ import static com.slack.astra.clusterManager.CacheNodeAssignmentService.snapshotMetadataBySnapshotId; -import com.slack.astra.blobfs.ChunkStore; +import com.slack.astra.blobfs.BlobStore; import com.slack.astra.chunk.Chunk; import com.slack.astra.chunk.ReadOnlyChunkImpl; import com.slack.astra.chunk.SearchContext; @@ -39,7 +39,7 @@ public class CachingChunkManager extends ChunkManagerBase { private final MeterRegistry meterRegistry; private final AsyncCuratorFramework curatorFramework; - private final ChunkStore chunkStore; + private final BlobStore blobStore; private final SearchContext searchContext; private final String s3Bucket; private final String dataDirectoryPrefix; @@ -61,7 +61,7 @@ public class CachingChunkManager extends ChunkManagerBase { public CachingChunkManager( MeterRegistry registry, AsyncCuratorFramework curatorFramework, - ChunkStore chunkStore, + BlobStore blobStore, SearchContext searchContext, String s3Bucket, String dataDirectoryPrefix, @@ -70,7 +70,7 @@ public CachingChunkManager( long capacityBytes) { this.meterRegistry = registry; this.curatorFramework = curatorFramework; - this.chunkStore = chunkStore; + this.blobStore = blobStore; this.searchContext = searchContext; this.s3Bucket = s3Bucket; this.dataDirectoryPrefix = dataDirectoryPrefix; @@ -103,7 +103,7 @@ protected void startUp() throws Exception { new ReadOnlyChunkImpl<>( curatorFramework, meterRegistry, - chunkStore, + blobStore, searchContext, s3Bucket, dataDirectoryPrefix, @@ -153,12 +153,12 @@ public static CachingChunkManager fromConfig( AsyncCuratorFramework curatorFramework, AstraConfigs.S3Config s3Config, AstraConfigs.CacheConfig cacheConfig, - ChunkStore chunkStore) + BlobStore blobStore) throws Exception { return new CachingChunkManager<>( meterRegistry, curatorFramework, - chunkStore, + blobStore, SearchContext.fromConfig(cacheConfig.getServerConfig()), s3Config.getS3Bucket(), cacheConfig.getDataDirectory(), @@ -211,7 +211,7 @@ private void onAssignmentHandler(CacheNodeAssignment assignment) { new ReadOnlyChunkImpl<>( curatorFramework, meterRegistry, - chunkStore, + blobStore, searchContext, s3Bucket, dataDirectoryPrefix, diff --git a/astra/src/main/java/com/slack/astra/chunkManager/ChunkRolloverFactory.java b/astra/src/main/java/com/slack/astra/chunkManager/ChunkRolloverFactory.java index 08bb930db7..feecc7461c 100644 --- a/astra/src/main/java/com/slack/astra/chunkManager/ChunkRolloverFactory.java +++ b/astra/src/main/java/com/slack/astra/chunkManager/ChunkRolloverFactory.java @@ -1,6 +1,6 @@ package com.slack.astra.chunkManager; -import com.slack.astra.blobfs.ChunkStore; +import com.slack.astra.blobfs.BlobStore; import com.slack.astra.chunk.ReadWriteChunk; import com.slack.astra.chunkrollover.ChunkRollOverStrategy; import io.micrometer.core.instrument.MeterRegistry; @@ -12,18 +12,18 @@ */ public class ChunkRolloverFactory { private final ChunkRollOverStrategy chunkRollOverStrategy; - private final ChunkStore chunkStore; + private final BlobStore blobStore; private final MeterRegistry meterRegistry; public ChunkRolloverFactory( - ChunkRollOverStrategy chunkRollOverStrategy, ChunkStore chunkStore, MeterRegistry registry) { + ChunkRollOverStrategy chunkRollOverStrategy, BlobStore blobStore, MeterRegistry registry) { this.chunkRollOverStrategy = chunkRollOverStrategy; - this.chunkStore = chunkStore; + this.blobStore = blobStore; this.meterRegistry = registry; } public RollOverChunkTask getRollOverChunkTask(ReadWriteChunk chunk) { - return new RollOverChunkTask<>(chunk, meterRegistry, chunkStore); + return new RollOverChunkTask<>(chunk, meterRegistry, blobStore); } public ChunkRollOverStrategy getChunkRollOverStrategy() { diff --git a/astra/src/main/java/com/slack/astra/chunkManager/IndexingChunkManager.java b/astra/src/main/java/com/slack/astra/chunkManager/IndexingChunkManager.java index 2db1ac285d..590f76d609 100644 --- a/astra/src/main/java/com/slack/astra/chunkManager/IndexingChunkManager.java +++ b/astra/src/main/java/com/slack/astra/chunkManager/IndexingChunkManager.java @@ -11,7 +11,7 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; -import com.slack.astra.blobfs.ChunkStore; +import com.slack.astra.blobfs.BlobStore; import com.slack.astra.chunk.Chunk; import com.slack.astra.chunk.ChunkInfo; import com.slack.astra.chunk.IndexingChunkImpl; @@ -60,7 +60,7 @@ public class IndexingChunkManager extends ChunkManagerBase { private final String chunkDataPrefix; - private final ChunkStore chunkStore; + private final BlobStore blobStore; private final ChunkRollOverStrategy chunkRollOverStrategy; private final AsyncCuratorFramework curatorFramework; private final SearchContext searchContext; @@ -116,7 +116,7 @@ public IndexingChunkManager( String dataDirectory, ChunkRollOverStrategy chunkRollOverStrategy, MeterRegistry registry, - ChunkStore chunkStore, + BlobStore blobStore, ListeningExecutorService rolloverExecutorService, AsyncCuratorFramework curatorFramework, SearchContext searchContext, @@ -132,7 +132,7 @@ public IndexingChunkManager( liveMessagesIndexedGauge = registry.gauge(LIVE_MESSAGES_INDEXED, new AtomicLong(0)); liveBytesIndexedGauge = registry.gauge(LIVE_BYTES_INDEXED, new AtomicLong(0)); - this.chunkStore = chunkStore; + this.blobStore = blobStore; this.rolloverExecutorService = rolloverExecutorService; this.rolloverFuture = null; this.curatorFramework = curatorFramework; @@ -198,7 +198,7 @@ private void doRollover(ReadWriteChunk currentChunk) { currentChunk.info().setChunkLastUpdatedTimeEpochMs(Instant.now().toEpochMilli()); RollOverChunkTask rollOverChunkTask = - new RollOverChunkTask<>(currentChunk, meterRegistry, chunkStore); + new RollOverChunkTask<>(currentChunk, meterRegistry, blobStore); if ((rolloverFuture == null) || rolloverFuture.isDone()) { rolloverFuture = rolloverExecutorService.submit(rollOverChunkTask); @@ -444,7 +444,7 @@ public static IndexingChunkManager fromConfig( MeterRegistry meterRegistry, AsyncCuratorFramework curatorFramework, AstraConfigs.IndexerConfig indexerConfig, - ChunkStore chunkStore, + BlobStore blobStore, AstraConfigs.S3Config s3Config) { ChunkRollOverStrategy chunkRollOverStrategy = @@ -455,7 +455,7 @@ public static IndexingChunkManager fromConfig( indexerConfig.getDataDirectory(), chunkRollOverStrategy, meterRegistry, - chunkStore, + blobStore, makeDefaultRollOverExecutor(), curatorFramework, SearchContext.fromConfig(indexerConfig.getServerConfig()), diff --git a/astra/src/main/java/com/slack/astra/chunkManager/RecoveryChunkManager.java b/astra/src/main/java/com/slack/astra/chunkManager/RecoveryChunkManager.java index 4e845e175e..1ca98b5732 100644 --- a/astra/src/main/java/com/slack/astra/chunkManager/RecoveryChunkManager.java +++ b/astra/src/main/java/com/slack/astra/chunkManager/RecoveryChunkManager.java @@ -8,7 +8,7 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; -import com.slack.astra.blobfs.ChunkStore; +import com.slack.astra.blobfs.BlobStore; import com.slack.astra.chunk.Chunk; import com.slack.astra.chunk.ChunkFactory; import com.slack.astra.chunk.ReadWriteChunk; @@ -218,7 +218,7 @@ public static RecoveryChunkManager fromConfig( SearchMetadataStore searchMetadataStore, SnapshotMetadataStore snapshotMetadataStore, AstraConfigs.IndexerConfig indexerConfig, - ChunkStore chunkStore) + BlobStore blobStore) throws Exception { SearchContext searchContext = SearchContext.fromConfig(indexerConfig.getServerConfig()); @@ -233,7 +233,7 @@ public static RecoveryChunkManager fromConfig( searchContext); ChunkRolloverFactory chunkRolloverFactory = - new ChunkRolloverFactory(new NeverRolloverChunkStrategy(), chunkStore, meterRegistry); + new ChunkRolloverFactory(new NeverRolloverChunkStrategy(), blobStore, meterRegistry); return new RecoveryChunkManager<>(recoveryChunkFactory, chunkRolloverFactory, meterRegistry); } diff --git a/astra/src/main/java/com/slack/astra/chunkManager/RollOverChunkTask.java b/astra/src/main/java/com/slack/astra/chunkManager/RollOverChunkTask.java index ab6c0680dd..05e30bd07c 100644 --- a/astra/src/main/java/com/slack/astra/chunkManager/RollOverChunkTask.java +++ b/astra/src/main/java/com/slack/astra/chunkManager/RollOverChunkTask.java @@ -1,6 +1,6 @@ package com.slack.astra.chunkManager; -import com.slack.astra.blobfs.ChunkStore; +import com.slack.astra.blobfs.BlobStore; import com.slack.astra.chunk.ReadWriteChunk; import io.micrometer.core.instrument.Counter; import io.micrometer.core.instrument.MeterRegistry; @@ -30,12 +30,12 @@ public class RollOverChunkTask implements Callable { private final Timer rollOverTimer; private final ReadWriteChunk chunk; - private final ChunkStore chunkStore; + private final BlobStore blobStore; public RollOverChunkTask( - ReadWriteChunk chunk, MeterRegistry meterRegistry, ChunkStore chunkStore) { + ReadWriteChunk chunk, MeterRegistry meterRegistry, BlobStore blobStore) { this.chunk = chunk; - this.chunkStore = chunkStore; + this.blobStore = blobStore; rolloversInitiatedCounter = meterRegistry.counter(ROLLOVERS_INITIATED); rolloversCompletedCounter = meterRegistry.counter(ROLLOVERS_COMPLETED); rolloversFailedCounter = meterRegistry.counter(ROLLOVERS_FAILED); @@ -53,7 +53,7 @@ private Boolean doRollover() { rolloversInitiatedCounter.increment(); // Run pre-snapshot and upload chunk to blob store. chunk.preSnapshot(); - if (!chunk.snapshotToS3(chunkStore)) { + if (!chunk.snapshotToS3(blobStore)) { LOG.warn("Failed to snapshot the chunk to S3"); rolloversFailedCounter.increment(); return false; diff --git a/astra/src/main/java/com/slack/astra/clusterManager/SnapshotDeletionService.java b/astra/src/main/java/com/slack/astra/clusterManager/SnapshotDeletionService.java index d9676c7d5e..bf52731899 100644 --- a/astra/src/main/java/com/slack/astra/clusterManager/SnapshotDeletionService.java +++ b/astra/src/main/java/com/slack/astra/clusterManager/SnapshotDeletionService.java @@ -12,7 +12,7 @@ import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.RateLimiter; import com.google.common.util.concurrent.ThreadFactoryBuilder; -import com.slack.astra.blobfs.ChunkStore; +import com.slack.astra.blobfs.BlobStore; import com.slack.astra.metadata.replica.ReplicaMetadataStore; import com.slack.astra.metadata.snapshot.SnapshotMetadataStore; import com.slack.astra.proto.config.AstraConfigs; @@ -51,7 +51,7 @@ public class SnapshotDeletionService extends AbstractScheduledService { private final ReplicaMetadataStore replicaMetadataStore; private final SnapshotMetadataStore snapshotMetadataStore; private final MeterRegistry meterRegistry; - private final ChunkStore chunkStore; + private final BlobStore blobStore; @VisibleForTesting protected int futuresListTimeoutSecs; @@ -76,7 +76,7 @@ public class SnapshotDeletionService extends AbstractScheduledService { public SnapshotDeletionService( ReplicaMetadataStore replicaMetadataStore, SnapshotMetadataStore snapshotMetadataStore, - ChunkStore chunkStore, + BlobStore blobStore, AstraConfigs.ManagerConfig managerConfig, MeterRegistry meterRegistry) { @@ -89,7 +89,7 @@ public SnapshotDeletionService( this.managerConfig = managerConfig; this.replicaMetadataStore = replicaMetadataStore; this.snapshotMetadataStore = snapshotMetadataStore; - this.chunkStore = chunkStore; + this.blobStore = blobStore; this.meterRegistry = meterRegistry; // This functions as the overall "timeout" for deleteExpiredSnapshotsWithoutReplicas, and should @@ -191,7 +191,7 @@ protected int deleteExpiredSnapshotsWithoutReplicas() { // store. If for some reason the object delete fails, it will throw // an exception, leaving the metadata and try again on the next run. LOG.debug("Starting delete of snapshot {}", snapshotMetadata); - chunkStore.delete(snapshotMetadata.snapshotId); + blobStore.delete(snapshotMetadata.snapshotId); snapshotMetadataStore.deleteSync(snapshotMetadata); } catch (Exception e) { LOG.error("Exception deleting snapshot", e); diff --git a/astra/src/main/java/com/slack/astra/recovery/RecoveryService.java b/astra/src/main/java/com/slack/astra/recovery/RecoveryService.java index e3d37393f5..aa51f536f5 100644 --- a/astra/src/main/java/com/slack/astra/recovery/RecoveryService.java +++ b/astra/src/main/java/com/slack/astra/recovery/RecoveryService.java @@ -8,7 +8,7 @@ import com.google.common.util.concurrent.AbstractIdleService; import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.protobuf.TextFormat; -import com.slack.astra.blobfs.ChunkStore; +import com.slack.astra.blobfs.BlobStore; import com.slack.astra.chunk.SearchContext; import com.slack.astra.chunkManager.RecoveryChunkManager; import com.slack.astra.logstore.LogMessage; @@ -59,7 +59,7 @@ public class RecoveryService extends AbstractIdleService { private final SearchContext searchContext; private final AsyncCuratorFramework curatorFramework; private final MeterRegistry meterRegistry; - private final ChunkStore chunkStore; + private final BlobStore blobStore; private final AstraConfigs.AstraConfig AstraConfig; private final AdminClient adminClient; @@ -92,12 +92,12 @@ public RecoveryService( AstraConfigs.AstraConfig AstraConfig, AsyncCuratorFramework curatorFramework, MeterRegistry meterRegistry, - ChunkStore chunkStore) { + BlobStore blobStore) { this.curatorFramework = curatorFramework; this.searchContext = SearchContext.fromConfig(AstraConfig.getRecoveryConfig().getServerConfig()); this.meterRegistry = meterRegistry; - this.chunkStore = chunkStore; + this.blobStore = blobStore; this.AstraConfig = AstraConfig; adminClient = @@ -299,7 +299,7 @@ boolean handleRecoveryTask(RecoveryTaskMetadata recoveryTaskMetadata) { searchMetadataStore, snapshotMetadataStore, AstraConfig.getIndexerConfig(), - chunkStore); + blobStore); // Ingest data in parallel LogMessageWriterImpl logMessageWriterImpl = new LogMessageWriterImpl(chunkManager); diff --git a/astra/src/main/java/com/slack/astra/server/Astra.java b/astra/src/main/java/com/slack/astra/server/Astra.java index c2cc690690..90e6f011d1 100644 --- a/astra/src/main/java/com/slack/astra/server/Astra.java +++ b/astra/src/main/java/com/slack/astra/server/Astra.java @@ -3,7 +3,7 @@ import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.Service; import com.google.common.util.concurrent.ServiceManager; -import com.slack.astra.blobfs.ChunkStore; +import com.slack.astra.blobfs.BlobStore; import com.slack.astra.blobfs.S3AsyncUtil; import com.slack.astra.bulkIngestApi.BulkIngestApi; import com.slack.astra.bulkIngestApi.BulkIngestKafkaProducer; @@ -139,10 +139,10 @@ public void start() throws Exception { curatorFramework = CuratorBuilder.build( prometheusMeterRegistry, astraConfig.getMetadataStoreConfig().getZookeeperConfig()); - ChunkStore chunkStore = new ChunkStore(s3Client, astraConfig.getS3Config().getS3Bucket()); + BlobStore blobStore = new BlobStore(s3Client, astraConfig.getS3Config().getS3Bucket()); Set services = - getServices(curatorFramework, astraConfig, chunkStore, prometheusMeterRegistry); + getServices(curatorFramework, astraConfig, blobStore, prometheusMeterRegistry); serviceManager = new ServiceManager(services); serviceManager.addListener(getServiceManagerListener(), MoreExecutors.directExecutor()); @@ -152,7 +152,7 @@ public void start() throws Exception { private static Set getServices( AsyncCuratorFramework curatorFramework, AstraConfigs.AstraConfig astraConfig, - ChunkStore chunkStore, + BlobStore blobStore, PrometheusMeterRegistry meterRegistry) throws Exception { Set services = new HashSet<>(); @@ -165,7 +165,7 @@ private static Set getServices( meterRegistry, curatorFramework, astraConfig.getIndexerConfig(), - chunkStore, + blobStore, astraConfig.getS3Config()); services.add(chunkManager); @@ -236,7 +236,7 @@ private static Set getServices( curatorFramework, astraConfig.getS3Config(), astraConfig.getCacheConfig(), - chunkStore); + blobStore); services.add(chunkManager); HpaMetricMetadataStore hpaMetricMetadataStore = @@ -330,11 +330,7 @@ private static Set getServices( SnapshotDeletionService snapshotDeletionService = new SnapshotDeletionService( - replicaMetadataStore, - snapshotMetadataStore, - chunkStore, - managerConfig, - meterRegistry); + replicaMetadataStore, snapshotMetadataStore, blobStore, managerConfig, meterRegistry); services.add(snapshotDeletionService); CacheNodeMetadataStore cacheNodeMetadataStore = new CacheNodeMetadataStore(curatorFramework); @@ -399,7 +395,7 @@ private static Set getServices( services.add(armeriaService); RecoveryService recoveryService = - new RecoveryService(astraConfig, curatorFramework, meterRegistry, chunkStore); + new RecoveryService(astraConfig, curatorFramework, meterRegistry, blobStore); services.add(recoveryService); } diff --git a/astra/src/test/java/com/slack/astra/blobfs/BlobStoreTest.java b/astra/src/test/java/com/slack/astra/blobfs/BlobStoreTest.java new file mode 100644 index 0000000000..883c5f6ed3 --- /dev/null +++ b/astra/src/test/java/com/slack/astra/blobfs/BlobStoreTest.java @@ -0,0 +1,218 @@ +package com.slack.astra.blobfs; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import com.adobe.testing.s3mock.junit5.S3MockExtension; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.List; +import java.util.Objects; +import java.util.UUID; +import java.util.concurrent.ExecutionException; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import software.amazon.awssdk.services.s3.S3AsyncClient; +import software.amazon.awssdk.services.s3.model.ListObjectsRequest; + +class BlobStoreTest { + private static final String TEST_BUCKET = "blobStoreTest"; + + @RegisterExtension + public static final S3MockExtension S3_MOCK_EXTENSION = + S3MockExtension.builder() + .silent() + .withInitialBuckets(TEST_BUCKET) + .withSecureConnection(false) + .build(); + + private final S3AsyncClient s3Client = + S3TestUtils.createS3CrtClient(S3_MOCK_EXTENSION.getServiceEndpoint()); + + @Test + void testUploadDownload() throws IOException { + BlobStore blobStore = new BlobStore(s3Client, TEST_BUCKET); + + Path directoryUpload = Files.createTempDirectory(""); + Path foo = Files.createTempFile(directoryUpload, "", ""); + try (FileWriter fileWriter = new FileWriter(foo.toFile())) { + fileWriter.write("Example test"); + } + String chunkId = UUID.randomUUID().toString(); + blobStore.upload(chunkId, directoryUpload); + + // what goes up, must come down + Path directoryDownloaded = Files.createTempDirectory(""); + blobStore.download(chunkId, directoryDownloaded); + + File[] filesDownloaded = directoryDownloaded.toFile().listFiles(); + assertThat(Objects.requireNonNull(filesDownloaded).length).isEqualTo(1); + + // contents of the file we uploaded should match + assertThat(Files.readAllBytes(filesDownloaded[0].toPath())).isEqualTo(Files.readAllBytes(foo)); + } + + @Test + void testUploadEmptyPrefix() throws IOException { + BlobStore blobStore = new BlobStore(s3Client, TEST_BUCKET); + + Path directoryUpload = Files.createTempDirectory(""); + + assertThatThrownBy(() -> blobStore.upload("", directoryUpload)) + .isInstanceOf(AssertionError.class); + assertThatThrownBy(() -> blobStore.upload(null, directoryUpload)) + .isInstanceOf(AssertionError.class); + } + + @Test + void testUploadEmpty() throws IOException { + BlobStore blobStore = new BlobStore(s3Client, TEST_BUCKET); + + Path directoryUpload = Files.createTempDirectory(""); + String chunkId = UUID.randomUUID().toString(); + + assertThatThrownBy(() -> blobStore.upload(chunkId, directoryUpload)) + .isInstanceOf(AssertionError.class); + } + + @Test + void testUploadNotADirectory() throws IOException { + BlobStore blobStore = new BlobStore(s3Client, TEST_BUCKET); + + Path directory = Files.createTempDirectory(""); + Path fileUpload = Files.createTempFile(directory, "", ""); + String chunkId = UUID.randomUUID().toString(); + + assertThatThrownBy(() -> blobStore.upload(chunkId, fileUpload)) + .isInstanceOf(AssertionError.class); + } + + @Test + void testDownloadDoesNotExist() throws IOException { + BlobStore blobStore = new BlobStore(s3Client, TEST_BUCKET); + Path directoryDownloaded = Files.createTempDirectory(""); + blobStore.download(UUID.randomUUID().toString(), directoryDownloaded); + assertThat(Objects.requireNonNull(directoryDownloaded.toFile().listFiles()).length) + .isEqualTo(0); + } + + @Test + void testDownloadNotADirectory() throws IOException { + BlobStore blobStore = new BlobStore(s3Client, TEST_BUCKET); + + Path directory = Files.createTempDirectory(""); + Path fileLocation = Files.createTempFile(directory, "", ""); + String chunkId = UUID.randomUUID().toString(); + + assertThatThrownBy(() -> blobStore.download(chunkId, fileLocation)) + .isInstanceOf(AssertionError.class); + } + + @Test + void testDownloadEmptyPrefix() throws IOException { + BlobStore blobStore = new BlobStore(s3Client, TEST_BUCKET); + + Path directoryDownload = Files.createTempDirectory(""); + + assertThatThrownBy(() -> blobStore.download("", directoryDownload)) + .isInstanceOf(AssertionError.class); + assertThatThrownBy(() -> blobStore.download(null, directoryDownload)) + .isInstanceOf(AssertionError.class); + } + + @Test + void testDeleteMultipleFiles() throws IOException, ExecutionException, InterruptedException { + BlobStore blobStore = new BlobStore(s3Client, TEST_BUCKET); + + Path directoryUpload = Files.createTempDirectory(""); + Path foo = Files.createTempFile(directoryUpload, "", ""); + try (FileWriter fileWriter = new FileWriter(foo.toFile())) { + fileWriter.write("Example test 1"); + } + Path bar = Files.createTempFile(directoryUpload, "", ""); + try (FileWriter fileWriter = new FileWriter(bar.toFile())) { + fileWriter.write("Example test 2"); + } + String chunkId = UUID.randomUUID().toString(); + blobStore.upload(chunkId, directoryUpload); + assertThat( + s3Client + .listObjects( + ListObjectsRequest.builder().bucket(TEST_BUCKET).prefix(chunkId).build()) + .get() + .contents() + .size()) + .isEqualTo(2); + + boolean deleted = blobStore.delete(chunkId); + assertThat(deleted).isTrue(); + assertThat( + s3Client + .listObjects( + ListObjectsRequest.builder().bucket(TEST_BUCKET).prefix(chunkId).build()) + .get() + .contents() + .size()) + .isEqualTo(0); + } + + @Test + void testDeleteDoesNotExist() { + BlobStore blobStore = new BlobStore(s3Client, TEST_BUCKET); + boolean deleted = blobStore.delete(UUID.randomUUID().toString()); + assertThat(deleted).isFalse(); + } + + @Test + void testDeleteBadPrefix() { + BlobStore blobStore = new BlobStore(s3Client, TEST_BUCKET); + + assertThatThrownBy(() -> blobStore.delete("")).isInstanceOf(AssertionError.class); + assertThatThrownBy(() -> blobStore.delete(null)).isInstanceOf(AssertionError.class); + } + + @Test + void testListFiles() throws IOException { + BlobStore blobStore = new BlobStore(s3Client, TEST_BUCKET); + String chunkId = UUID.randomUUID().toString(); + + assertThat(blobStore.listFiles(chunkId).size()).isEqualTo(0); + + Path directoryUpload = Files.createTempDirectory(""); + Path foo = Files.createTempFile(directoryUpload, "", ""); + try (FileWriter fileWriter = new FileWriter(foo.toFile())) { + fileWriter.write("Example test 1"); + } + Path bar = Files.createTempFile(directoryUpload, "", ""); + try (FileWriter fileWriter = new FileWriter(bar.toFile())) { + fileWriter.write("Example test 2"); + } + blobStore.upload(chunkId, directoryUpload); + + assertThat(blobStore.listFiles(chunkId).size()).isEqualTo(2); + assertThat(blobStore.listFiles(chunkId)) + .containsExactlyInAnyOrderElementsOf( + List.of( + String.format("%s/%s", chunkId, foo.getFileName().toString()), + String.format("%s/%s", chunkId, bar.getFileName().toString()))); + } + + @Test + void testListBadPrefix() { + BlobStore blobStore = new BlobStore(s3Client, TEST_BUCKET); + + assertThatThrownBy(() -> blobStore.listFiles("")).isInstanceOf(AssertionError.class); + assertThatThrownBy(() -> blobStore.listFiles(null)).isInstanceOf(AssertionError.class); + } + + @Test + void testListFilesNonExistingPrefix() { + BlobStore blobStore = new BlobStore(s3Client, TEST_BUCKET); + String chunkId = UUID.randomUUID().toString(); + + assertThat(blobStore.listFiles(chunkId).size()).isEqualTo(0); + } +} diff --git a/astra/src/test/java/com/slack/astra/blobfs/ChunkStoreTest.java b/astra/src/test/java/com/slack/astra/blobfs/ChunkStoreTest.java deleted file mode 100644 index be05b37013..0000000000 --- a/astra/src/test/java/com/slack/astra/blobfs/ChunkStoreTest.java +++ /dev/null @@ -1,134 +0,0 @@ -package com.slack.astra.blobfs; - -import static org.assertj.core.api.Assertions.assertThat; - -import com.adobe.testing.s3mock.junit5.S3MockExtension; -import java.io.File; -import java.io.FileWriter; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.List; -import java.util.Objects; -import java.util.UUID; -import java.util.concurrent.ExecutionException; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.RegisterExtension; -import software.amazon.awssdk.services.s3.S3AsyncClient; -import software.amazon.awssdk.services.s3.model.ListObjectsRequest; - -class ChunkStoreTest { - private static final String TEST_BUCKET = "chunkStoreTest"; - - @RegisterExtension - public static final S3MockExtension S3_MOCK_EXTENSION = - S3MockExtension.builder() - .silent() - .withInitialBuckets(TEST_BUCKET) - .withSecureConnection(false) - .build(); - - private final S3AsyncClient s3Client = - S3TestUtils.createS3CrtClient(S3_MOCK_EXTENSION.getServiceEndpoint()); - - @Test - void testUploadDownload() throws IOException { - ChunkStore chunkStore = new ChunkStore(s3Client, TEST_BUCKET); - - Path directoryUpload = Files.createTempDirectory(""); - Path foo = Files.createTempFile(directoryUpload, "", ""); - try (FileWriter fileWriter = new FileWriter(foo.toFile())) { - fileWriter.write("Example test"); - } - String chunkId = UUID.randomUUID().toString(); - chunkStore.upload(chunkId, directoryUpload); - - // what goes up, must come down - Path directoryDownloaded = Files.createTempDirectory(""); - chunkStore.download(chunkId, directoryDownloaded); - - File[] filesDownloaded = directoryDownloaded.toFile().listFiles(); - assertThat(Objects.requireNonNull(filesDownloaded).length).isEqualTo(1); - - // contents of the file we uploaded should match - assertThat(Files.readAllBytes(filesDownloaded[0].toPath())).isEqualTo(Files.readAllBytes(foo)); - } - - @Test - void testDownloadDoesNotExist() throws IOException { - ChunkStore chunkStore = new ChunkStore(s3Client, TEST_BUCKET); - Path directoryDownloaded = Files.createTempDirectory(""); - chunkStore.download(UUID.randomUUID().toString(), directoryDownloaded); - } - - @Test - void testDeleteMultipleFiles() throws IOException, ExecutionException, InterruptedException { - ChunkStore chunkStore = new ChunkStore(s3Client, TEST_BUCKET); - - Path directoryUpload = Files.createTempDirectory(""); - Path foo = Files.createTempFile(directoryUpload, "", ""); - try (FileWriter fileWriter = new FileWriter(foo.toFile())) { - fileWriter.write("Example test 1"); - } - Path bar = Files.createTempFile(directoryUpload, "", ""); - try (FileWriter fileWriter = new FileWriter(bar.toFile())) { - fileWriter.write("Example test 2"); - } - String chunkId = UUID.randomUUID().toString(); - chunkStore.upload(chunkId, directoryUpload); - assertThat( - s3Client - .listObjects( - ListObjectsRequest.builder().bucket(TEST_BUCKET).prefix(chunkId).build()) - .get() - .contents() - .size()) - .isEqualTo(2); - - boolean deleted = chunkStore.delete(chunkId); - assertThat(deleted).isTrue(); - assertThat( - s3Client - .listObjects( - ListObjectsRequest.builder().bucket(TEST_BUCKET).prefix(chunkId).build()) - .get() - .contents() - .size()) - .isEqualTo(0); - } - - @Test - void testDeleteDoesNotExist() { - ChunkStore chunkStore = new ChunkStore(s3Client, TEST_BUCKET); - boolean deleted = chunkStore.delete(UUID.randomUUID().toString()); - assertThat(deleted).isFalse(); - } - - @Test - void testListFiles() throws IOException { - ChunkStore chunkStore = new ChunkStore(s3Client, TEST_BUCKET); - String chunkId = UUID.randomUUID().toString(); - - assertThat(chunkStore.listFiles(chunkId).size()).isEqualTo(0); - - Path directoryUpload = Files.createTempDirectory(""); - Path foo = Files.createTempFile(directoryUpload, "", ""); - try (FileWriter fileWriter = new FileWriter(foo.toFile())) { - fileWriter.write("Example test 1"); - } - Path bar = Files.createTempFile(directoryUpload, "", ""); - try (FileWriter fileWriter = new FileWriter(bar.toFile())) { - fileWriter.write("Example test 2"); - } - chunkStore.upload(chunkId, directoryUpload); - - assertThat(chunkStore.listFiles(chunkId).size()).isEqualTo(2); - assertThat(chunkStore.listFiles(chunkId)) - .containsExactlyInAnyOrderElementsOf( - List.of( - String.format( - "%s/%s", chunkStore.getRemotePath(chunkId), foo.getFileName().toString()), - String.format( - "%s/%s", chunkStore.getRemotePath(chunkId), bar.getFileName().toString()))); - } -} diff --git a/astra/src/test/java/com/slack/astra/chunk/IndexingChunkImplTest.java b/astra/src/test/java/com/slack/astra/chunk/IndexingChunkImplTest.java index dc254480f8..cfed9e59db 100644 --- a/astra/src/test/java/com/slack/astra/chunk/IndexingChunkImplTest.java +++ b/astra/src/test/java/com/slack/astra/chunk/IndexingChunkImplTest.java @@ -17,7 +17,7 @@ import brave.Tracing; import com.adobe.testing.s3mock.junit5.S3MockExtension; -import com.slack.astra.blobfs.ChunkStore; +import com.slack.astra.blobfs.BlobStore; import com.slack.astra.blobfs.S3TestUtils; import com.slack.astra.logstore.LogMessage; import com.slack.astra.logstore.LuceneIndexStoreImpl; @@ -632,10 +632,10 @@ public void testSnapshotToNonExistentS3BucketFails() String bucket = "invalid-bucket"; S3AsyncClient s3AsyncClient = S3TestUtils.createS3CrtClient(S3_MOCK_EXTENSION.getServiceEndpoint()); - ChunkStore chunkStore = new ChunkStore(s3AsyncClient, bucket); + BlobStore blobStore = new BlobStore(s3AsyncClient, bucket); // Snapshot to S3 without creating the s3 bucket. - assertThat(chunk.snapshotToS3(chunkStore)).isFalse(); + assertThat(chunk.snapshotToS3(blobStore)).isFalse(); // Metadata checks List afterSnapshots = @@ -692,10 +692,10 @@ public void testSnapshotToS3UsingChunkApi() throws Exception { S3AsyncClient s3AsyncClient = S3TestUtils.createS3CrtClient(S3_MOCK_EXTENSION.getServiceEndpoint()); s3AsyncClient.createBucket(CreateBucketRequest.builder().bucket(bucket).build()).get(); - ChunkStore chunkStore = new ChunkStore(s3AsyncClient, bucket); + BlobStore blobStore = new BlobStore(s3AsyncClient, bucket); // Snapshot to S3 - assertThat(chunk.snapshotToS3(chunkStore)).isTrue(); + assertThat(chunk.snapshotToS3(blobStore)).isTrue(); // depending on heap and CFS files this can be 5 or 19. assertThat(getCount(INDEX_FILES_UPLOAD, registry)).isGreaterThan(5); diff --git a/astra/src/test/java/com/slack/astra/chunk/ReadOnlyChunkImplTest.java b/astra/src/test/java/com/slack/astra/chunk/ReadOnlyChunkImplTest.java index da0f6bbc0d..bac632edff 100644 --- a/astra/src/test/java/com/slack/astra/chunk/ReadOnlyChunkImplTest.java +++ b/astra/src/test/java/com/slack/astra/chunk/ReadOnlyChunkImplTest.java @@ -15,7 +15,7 @@ import brave.Tracing; import com.adobe.testing.s3mock.junit5.S3MockExtension; -import com.slack.astra.blobfs.ChunkStore; +import com.slack.astra.blobfs.BlobStore; import com.slack.astra.blobfs.S3TestUtils; import com.slack.astra.logstore.LogMessage; import com.slack.astra.logstore.LuceneIndexStoreImpl; @@ -69,7 +69,7 @@ public class ReadOnlyChunkImplTest { private TestingServer testingServer; private MeterRegistry meterRegistry; - private ChunkStore chunkStore; + private BlobStore blobStore; @RegisterExtension public static final S3MockExtension S3_MOCK_EXTENSION = @@ -87,7 +87,7 @@ public void startup() throws Exception { S3AsyncClient s3AsyncClient = S3TestUtils.createS3CrtClient(S3_MOCK_EXTENSION.getServiceEndpoint()); - chunkStore = new ChunkStore(s3AsyncClient, TEST_S3_BUCKET); + blobStore = new BlobStore(s3AsyncClient, TEST_S3_BUCKET); } @AfterEach @@ -128,7 +128,7 @@ public void shouldHandleChunkLivecycle() throws Exception { new ReadOnlyChunkImpl<>( curatorFramework, meterRegistry, - chunkStore, + blobStore, searchContext, AstraConfig.getS3Config().getS3Bucket(), AstraConfig.getCacheConfig().getDataDirectory(), @@ -260,7 +260,7 @@ public void shouldHandleMissingS3Assets() throws Exception { new ReadOnlyChunkImpl<>( curatorFramework, meterRegistry, - chunkStore, + blobStore, SearchContext.fromConfig(AstraConfig.getCacheConfig().getServerConfig()), AstraConfig.getS3Config().getS3Bucket(), AstraConfig.getCacheConfig().getDataDirectory(), @@ -326,7 +326,7 @@ public void shouldHandleMissingZkData() throws Exception { new ReadOnlyChunkImpl<>( curatorFramework, meterRegistry, - chunkStore, + blobStore, SearchContext.fromConfig(AstraConfig.getCacheConfig().getServerConfig()), AstraConfig.getS3Config().getS3Bucket(), AstraConfig.getCacheConfig().getDataDirectory(), @@ -393,7 +393,7 @@ public void closeShouldCleanupLiveChunkCorrectly() throws Exception { new ReadOnlyChunkImpl<>( curatorFramework, meterRegistry, - chunkStore, + blobStore, SearchContext.fromConfig(AstraConfig.getCacheConfig().getServerConfig()), AstraConfig.getS3Config().getS3Bucket(), AstraConfig.getCacheConfig().getDataDirectory(), @@ -524,7 +524,7 @@ public void shouldHandleDynamicChunkSizeLifecycle() throws Exception { new ReadOnlyChunkImpl<>( curatorFramework, meterRegistry, - chunkStore, + blobStore, searchContext, AstraConfig.getS3Config().getS3Bucket(), AstraConfig.getCacheConfig().getDataDirectory(), @@ -670,7 +670,7 @@ private void initializeBlobStorageWithIndex(String snapshotId) throws Exception assertThat(dirPath.toFile().listFiles().length).isGreaterThanOrEqualTo(filesToUpload.size()); // Copy files to S3. - chunkStore.upload(snapshotId, dirPath); + blobStore.upload(snapshotId, dirPath); } private void initializeCacheNodeAssignment( diff --git a/astra/src/test/java/com/slack/astra/chunk/RecoveryChunkImplTest.java b/astra/src/test/java/com/slack/astra/chunk/RecoveryChunkImplTest.java index d9710d4350..b9bb9991f4 100644 --- a/astra/src/test/java/com/slack/astra/chunk/RecoveryChunkImplTest.java +++ b/astra/src/test/java/com/slack/astra/chunk/RecoveryChunkImplTest.java @@ -15,7 +15,7 @@ import brave.Tracing; import com.adobe.testing.s3mock.junit5.S3MockExtension; -import com.slack.astra.blobfs.ChunkStore; +import com.slack.astra.blobfs.BlobStore; import com.slack.astra.blobfs.S3TestUtils; import com.slack.astra.logstore.LogMessage; import com.slack.astra.logstore.LuceneIndexStoreImpl; @@ -616,10 +616,10 @@ public void testSnapshotToNonExistentS3BucketFails() { S3AsyncClient s3AsyncClient = S3TestUtils.createS3CrtClient(S3_MOCK_EXTENSION.getServiceEndpoint()); - ChunkStore chunkStore = new ChunkStore(s3AsyncClient, bucket); + BlobStore blobStore = new BlobStore(s3AsyncClient, bucket); // Snapshot to S3 without creating the s3 bucket. - assertThat(chunk.snapshotToS3(chunkStore)).isFalse(); + assertThat(chunk.snapshotToS3(blobStore)).isFalse(); // No live snapshot or search metadata is published since the S3 snapshot failed. assertThat(AstraMetadataTestUtils.listSyncUncached(snapshotMetadataStore)).isEmpty(); @@ -667,10 +667,10 @@ public void testSnapshotToS3UsingChunkApi() throws Exception { S3AsyncClient s3AsyncClient = S3TestUtils.createS3CrtClient(S3_MOCK_EXTENSION.getServiceEndpoint()); s3AsyncClient.createBucket(CreateBucketRequest.builder().bucket(bucket).build()).get(); - ChunkStore chunkStore = new ChunkStore(s3AsyncClient, bucket); + BlobStore blobStore = new BlobStore(s3AsyncClient, bucket); // Snapshot to S3 - assertThat(chunk.snapshotToS3(chunkStore)).isTrue(); + assertThat(chunk.snapshotToS3(blobStore)).isTrue(); // depending on heap and CFS files this can be 5 or 19. assertThat(getCount(INDEX_FILES_UPLOAD, registry)).isGreaterThan(5); @@ -687,7 +687,7 @@ public void testSnapshotToS3UsingChunkApi() throws Exception { assertThat(afterSnapshots.size()).isEqualTo(1); assertThat(afterSnapshots).contains(ChunkInfo.toSnapshotMetadata(chunk.info(), "")); - assertThat(chunkStore.listFiles(afterSnapshots.get(0).snapshotId).size()).isGreaterThan(0); + assertThat(blobStore.listFiles(afterSnapshots.get(0).snapshotId).size()).isGreaterThan(0); // Only non-live snapshots. No live snapshots. assertThat(afterSnapshots.stream().filter(SnapshotMetadata::isLive).count()).isZero(); // No search nodes are added for recovery chunk. diff --git a/astra/src/test/java/com/slack/astra/chunkManager/CachingChunkManagerTest.java b/astra/src/test/java/com/slack/astra/chunkManager/CachingChunkManagerTest.java index 67420041d8..362f3eada8 100644 --- a/astra/src/test/java/com/slack/astra/chunkManager/CachingChunkManagerTest.java +++ b/astra/src/test/java/com/slack/astra/chunkManager/CachingChunkManagerTest.java @@ -14,7 +14,7 @@ import static org.awaitility.Awaitility.await; import com.adobe.testing.s3mock.junit5.S3MockExtension; -import com.slack.astra.blobfs.ChunkStore; +import com.slack.astra.blobfs.BlobStore; import com.slack.astra.blobfs.S3TestUtils; import com.slack.astra.chunk.Chunk; import com.slack.astra.chunk.ReadOnlyChunkImpl; @@ -61,7 +61,7 @@ public class CachingChunkManagerTest { private TestingServer testingServer; private MeterRegistry meterRegistry; - private ChunkStore chunkStore; + private BlobStore blobStore; @RegisterExtension public static final S3MockExtension S3_MOCK_EXTENSION = @@ -83,7 +83,7 @@ public void startup() throws Exception { S3AsyncClient s3AsyncClient = S3TestUtils.createS3CrtClient(S3_MOCK_EXTENSION.getServiceEndpoint()); - chunkStore = new ChunkStore(s3AsyncClient, TEST_S3_BUCKET); + blobStore = new BlobStore(s3AsyncClient, TEST_S3_BUCKET); } @AfterEach @@ -143,7 +143,7 @@ private CachingChunkManager initChunkManager() throws TimeoutExcepti new CachingChunkManager<>( meterRegistry, curatorFramework, - chunkStore, + blobStore, SearchContext.fromConfig(AstraConfig.getCacheConfig().getServerConfig()), AstraConfig.getS3Config().getS3Bucket(), AstraConfig.getCacheConfig().getDataDirectory(), @@ -207,7 +207,7 @@ private void initializeBlobStorageWithIndex(String snapshotId) throws Exception assertThat(dirPath.toFile().listFiles().length).isGreaterThanOrEqualTo(filesToUpload.size()); // Copy files to S3. - chunkStore.upload(snapshotId, dirPath); + blobStore.upload(snapshotId, dirPath); } @Test @@ -254,14 +254,11 @@ public void testCreatesChunksOnAssignment() throws Exception { await() .ignoreExceptions() .until( - () -> - Objects.requireNonNull( - chunkStore - .download(snapshotId, Path.of("/tmp/test1")) - .toFile() - .listFiles()) - .length - > 0); + () -> { + Path path = Path.of("/tmp/test1"); + blobStore.download(snapshotId, path); + return Objects.requireNonNull(path.toFile().listFiles()).length > 0; + }); initAssignment(snapshotId); await() @@ -297,14 +294,12 @@ public void testBasicChunkEviction() throws Exception { await() .ignoreExceptions() .until( - () -> - Objects.requireNonNull( - chunkStore - .download(snapshotId, Path.of("/tmp/test2")) - .toFile() - .listFiles()) - .length - > 0); + () -> { + Path path = Path.of("/tmp/test2"); + blobStore.download(snapshotId, path); + return Objects.requireNonNull(path.toFile().listFiles()).length > 0; + }); + CacheNodeAssignment assignment = initAssignment(snapshotId); // assert chunks created diff --git a/astra/src/test/java/com/slack/astra/chunkManager/IndexingChunkManagerTest.java b/astra/src/test/java/com/slack/astra/chunkManager/IndexingChunkManagerTest.java index e162601d8c..872b321571 100644 --- a/astra/src/test/java/com/slack/astra/chunkManager/IndexingChunkManagerTest.java +++ b/astra/src/test/java/com/slack/astra/chunkManager/IndexingChunkManagerTest.java @@ -33,7 +33,7 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import com.google.protobuf.ByteString; -import com.slack.astra.blobfs.ChunkStore; +import com.slack.astra.blobfs.BlobStore; import com.slack.astra.blobfs.S3TestUtils; import com.slack.astra.chunk.Chunk; import com.slack.astra.chunk.ChunkInfo; @@ -113,7 +113,7 @@ public class IndexingChunkManagerTest { private S3AsyncClient s3AsyncClient; private static final String ZK_PATH_PREFIX = "testZK"; - private ChunkStore chunkStore; + private BlobStore blobStore; private TestingServer localZkServer; private AsyncCuratorFramework curatorFramework; private SnapshotMetadataStore snapshotMetadataStore; @@ -125,7 +125,7 @@ public void setUp() throws Exception { metricsRegistry = new SimpleMeterRegistry(); // create an S3 client and a bucket for test s3AsyncClient = S3TestUtils.createS3CrtClient(S3_MOCK_EXTENSION.getServiceEndpoint()); - chunkStore = new ChunkStore(s3AsyncClient, S3_TEST_BUCKET); + blobStore = new BlobStore(s3AsyncClient, S3_TEST_BUCKET); localZkServer = new TestingServer(); localZkServer.start(); @@ -158,7 +158,7 @@ public void tearDown() throws TimeoutException, IOException { private void initChunkManager( ChunkRollOverStrategy chunkRollOverStrategy, - ChunkStore chunkStore, + BlobStore blobStore, ListeningExecutorService listeningExecutorService) throws IOException, TimeoutException { SearchContext searchContext = new SearchContext(TEST_HOST, TEST_PORT); @@ -168,7 +168,7 @@ private void initChunkManager( tmpPath.toFile().getAbsolutePath(), chunkRollOverStrategy, metricsRegistry, - chunkStore, + blobStore, listeningExecutorService, curatorFramework, searchContext, @@ -179,7 +179,7 @@ private void initChunkManager( private void initChunkManager( ChunkRollOverStrategy chunkRollOverStrategy, - ChunkStore chunkStore, + BlobStore blobStore, ListeningExecutorService listeningExecutorService, AstraConfigs.IndexerConfig indexerConfig) throws IOException, TimeoutException { @@ -190,7 +190,7 @@ private void initChunkManager( tmpPath.toFile().getAbsolutePath(), chunkRollOverStrategy, metricsRegistry, - chunkStore, + blobStore, listeningExecutorService, curatorFramework, searchContext, @@ -207,7 +207,7 @@ public void testDeleteOverMaxThresholdGreaterThanZero() throws IOException, Time AstraConfigs.IndexerConfig indexerConfig = AstraConfigUtil.makeIndexerConfig(TEST_PORT, 1000, 100, 1, 1_000_000_000L); initChunkManager( - chunkRollOverStrategy, chunkStore, MoreExecutors.newDirectExecutorService(), indexerConfig); + chunkRollOverStrategy, blobStore, MoreExecutors.newDirectExecutorService(), indexerConfig); assertThat(chunkManager.getChunkList().isEmpty()).isTrue(); final Instant startTime = @@ -284,7 +284,7 @@ public void testDeleteStaleDataDoesNothingWhenGivenLimitLessThan0() AstraConfigs.IndexerConfig indexerConfig = AstraConfigUtil.makeIndexerConfig(TEST_PORT, 1000, 100, -1, 10_000); initChunkManager( - chunkRollOverStrategy, chunkStore, MoreExecutors.newDirectExecutorService(), indexerConfig); + chunkRollOverStrategy, blobStore, MoreExecutors.newDirectExecutorService(), indexerConfig); assertThat(chunkManager.getChunkList().isEmpty()).isTrue(); final Instant startTime = @@ -330,7 +330,7 @@ public void closeDuringCleanerTask() new MessageSizeOrCountBasedRolloverStrategy( metricsRegistry, 10 * 1024 * 1024 * 1024L, 1000000L); - initChunkManager(chunkRollOverStrategy, chunkStore, MoreExecutors.newDirectExecutorService()); + initChunkManager(chunkRollOverStrategy, blobStore, MoreExecutors.newDirectExecutorService()); List messages = SpanUtil.makeSpansWithTimeDifference(1, 11, 1, Instant.now()); int offset = 1; @@ -372,7 +372,7 @@ public void testAddMessage() throws Exception { metricsRegistry, 10 * 1024 * 1024 * 1024L, 1000000L); final String CHUNK_DATA_PREFIX = "testData"; - initChunkManager(chunkRollOverStrategy, chunkStore, MoreExecutors.newDirectExecutorService()); + initChunkManager(chunkRollOverStrategy, blobStore, MoreExecutors.newDirectExecutorService()); List messages = SpanUtil.makeSpansWithTimeDifference(1, 100, 1, Instant.now()); int actualChunkSize = 0; @@ -601,7 +601,7 @@ public void testAddAndSearchMessageInMultipleSlices() throws Exception { ChunkRollOverStrategy chunkRollOverStrategy = new MessageSizeOrCountBasedRolloverStrategy(metricsRegistry, 10 * 1024 * 1024 * 1024L, 10L); - initChunkManager(chunkRollOverStrategy, chunkStore, MoreExecutors.newDirectExecutorService()); + initChunkManager(chunkRollOverStrategy, blobStore, MoreExecutors.newDirectExecutorService()); List messages = SpanUtil.makeSpansWithTimeDifference(1, 15, 1, Instant.now()); int offset = 1; @@ -629,7 +629,7 @@ public void testAddAndSearchMessageInSpecificChunks() throws Exception { ChunkRollOverStrategy chunkRollOverStrategy = new MessageSizeOrCountBasedRolloverStrategy(metricsRegistry, 10 * 1024 * 1024 * 1024L, 10L); - initChunkManager(chunkRollOverStrategy, chunkStore, MoreExecutors.newDirectExecutorService()); + initChunkManager(chunkRollOverStrategy, blobStore, MoreExecutors.newDirectExecutorService()); List messages = SpanUtil.makeSpansWithTimeDifference(1, 15, 1, Instant.now()); int offset = 1; @@ -719,7 +719,7 @@ public void testAddMessageWithPropertyTypeConflicts() throws Exception { new MessageSizeOrCountBasedRolloverStrategy(metricsRegistry, 10 * 1024 * 1024 * 1024L, 10L); ListeningExecutorService rollOverExecutor = IndexingChunkManager.makeDefaultRollOverExecutor(); - initChunkManager(chunkRollOverStrategy, chunkStore, rollOverExecutor); + initChunkManager(chunkRollOverStrategy, blobStore, rollOverExecutor); // Add a message int offset = 1; @@ -781,7 +781,7 @@ public void testMessagesAddedToActiveChunks() throws Exception { ChunkRollOverStrategy chunkRollOverStrategy = new DiskOrMessageCountBasedRolloverStrategy(metricsRegistry, 10 * 1024 * 1024 * 1024L, 2L); - initChunkManager(chunkRollOverStrategy, chunkStore, MoreExecutors.newDirectExecutorService()); + initChunkManager(chunkRollOverStrategy, blobStore, MoreExecutors.newDirectExecutorService()); // Add a message List msgs = SpanUtil.makeSpansWithTimeDifference(1, 4, 1000, Instant.now()); @@ -833,7 +833,7 @@ public void testMultiThreadedChunkRollover() throws Exception { new DiskOrMessageCountBasedRolloverStrategy(metricsRegistry, 10 * 1024 * 1024 * 1024L, 10L); ListeningExecutorService rollOverExecutor = IndexingChunkManager.makeDefaultRollOverExecutor(); - initChunkManager(chunkRollOverStrategy, chunkStore, rollOverExecutor); + initChunkManager(chunkRollOverStrategy, blobStore, rollOverExecutor); List messages = SpanUtil.makeSpansWithTimeDifference(1, 25, 1, Instant.now()); // Add 11 messages to initiate first roll over. @@ -866,7 +866,7 @@ public void testAddMessagesToChunkWithRollover() throws Exception { ChunkRollOverStrategy chunkRollOverStrategy = new DiskOrMessageCountBasedRolloverStrategy(metricsRegistry, 10 * 1024 * 1024 * 1024L, 10L); - initChunkManager(chunkRollOverStrategy, chunkStore, MoreExecutors.newDirectExecutorService()); + initChunkManager(chunkRollOverStrategy, blobStore, MoreExecutors.newDirectExecutorService()); List messages = SpanUtil.makeSpansWithTimeDifference(1, 25, 1, Instant.now()); // Add 11 messages to initiate first roll over. @@ -956,7 +956,7 @@ public void testAllChunkFailures() throws Exception { ChunkRollOverStrategy chunkRollOverStrategy = new DiskOrMessageCountBasedRolloverStrategy(metricsRegistry, 10 * 1024 * 1024 * 1024L, 10L); - initChunkManager(chunkRollOverStrategy, chunkStore, MoreExecutors.newDirectExecutorService()); + initChunkManager(chunkRollOverStrategy, blobStore, MoreExecutors.newDirectExecutorService()); List messages = SpanUtil.makeSpansWithTimeDifference(1, 25, 1, Instant.now()); // Add 11 messages to initiate first roll over. @@ -1034,7 +1034,7 @@ public void testCommitInvalidChunk() throws Exception { final ChunkRollOverStrategy chunkRollOverStrategy = new DiskOrMessageCountBasedRolloverStrategy(metricsRegistry, 10 * 1024 * 1024 * 1024L, 10L); - initChunkManager(chunkRollOverStrategy, chunkStore, MoreExecutors.newDirectExecutorService()); + initChunkManager(chunkRollOverStrategy, blobStore, MoreExecutors.newDirectExecutorService()); int offset = 1; for (Trace.Span m : messages) { @@ -1070,7 +1070,7 @@ public void testMultiChunkSearch() throws Exception { final ChunkRollOverStrategy chunkRollOverStrategy = new DiskOrMessageCountBasedRolloverStrategy(metricsRegistry, 10 * 1024 * 1024 * 1024L, 10L); - initChunkManager(chunkRollOverStrategy, chunkStore, MoreExecutors.newDirectExecutorService()); + initChunkManager(chunkRollOverStrategy, blobStore, MoreExecutors.newDirectExecutorService()); int offset = 1; for (Trace.Span m : messages) { @@ -1184,7 +1184,7 @@ public void testChunkRollOverInProgressExceptionIsThrown() throws Exception { final ChunkRollOverStrategy chunkRollOverStrategy = new DiskOrMessageCountBasedRolloverStrategy(metricsRegistry, 10 * 1024 * 1024 * 1024L, 10L); initChunkManager( - chunkRollOverStrategy, chunkStore, IndexingChunkManager.makeDefaultRollOverExecutor()); + chunkRollOverStrategy, blobStore, IndexingChunkManager.makeDefaultRollOverExecutor()); assertThat(AstraMetadataTestUtils.listSyncUncached(snapshotMetadataStore)).isEmpty(); @@ -1231,7 +1231,7 @@ public void testSuccessfulRollOverFinishesOnClose() throws Exception { final ChunkRollOverStrategy chunkRollOverStrategy = new DiskOrMessageCountBasedRolloverStrategy(metricsRegistry, 10 * 1024 * 1024 * 1024L, 10L); initChunkManager( - chunkRollOverStrategy, chunkStore, IndexingChunkManager.makeDefaultRollOverExecutor()); + chunkRollOverStrategy, blobStore, IndexingChunkManager.makeDefaultRollOverExecutor()); // Adding a message and close the chunkManager right away should still finish the failed // rollover. @@ -1281,7 +1281,7 @@ public void testFailedRollOverFinishesOnClose() throws Exception { new DiskOrMessageCountBasedRolloverStrategy(metricsRegistry, 10 * 1024 * 1024 * 1024L, 10L); initChunkManager( chunkRollOverStrategy, - new ChunkStore(s3AsyncClient, S3_TEST_BUCKET + "Fail"), + new BlobStore(s3AsyncClient, S3_TEST_BUCKET + "Fail"), IndexingChunkManager.makeDefaultRollOverExecutor()); // Adding a message and close the chunkManager right away should still finish the failed @@ -1327,7 +1327,7 @@ public void testRollOverFailure() new DiskOrMessageCountBasedRolloverStrategy(metricsRegistry, 10 * 1024 * 1024 * 1024L, 10L); initChunkManager( chunkRollOverStrategy, - new ChunkStore(s3AsyncClient, S3_TEST_BUCKET + "Fail"), + new BlobStore(s3AsyncClient, S3_TEST_BUCKET + "Fail"), IndexingChunkManager.makeDefaultRollOverExecutor()); int offset = 1; @@ -1373,7 +1373,7 @@ public void testRollOverFailureWithDirectExecutor() new DiskOrMessageCountBasedRolloverStrategy(metricsRegistry, 10 * 1024 * 1024 * 1024L, 10L); initChunkManager( chunkRollOverStrategy, - new ChunkStore(s3AsyncClient, S3_TEST_BUCKET + "Fail"), + new BlobStore(s3AsyncClient, S3_TEST_BUCKET + "Fail"), MoreExecutors.newDirectExecutorService()); // Adding a messages very quickly when running a rollover in background would result in an @@ -1414,7 +1414,7 @@ public void testNewFieldAddedToSchema() throws IOException, TimeoutException { ChunkRollOverStrategy chunkRollOverStrategy = new DiskOrMessageCountBasedRolloverStrategy( metricsRegistry, 10 * 1024 * 1024 * 1024L, 100L); - initChunkManager(chunkRollOverStrategy, chunkStore, MoreExecutors.newDirectExecutorService()); + initChunkManager(chunkRollOverStrategy, blobStore, MoreExecutors.newDirectExecutorService()); List messages1 = SpanUtil.makeSpansWithTimeDifference(1, 10, 1, Instant.now()); Map schemaBefore = chunkManager.getSchema(); @@ -1463,7 +1463,7 @@ public void testMultipleByteRollOversSuccessfully() new DiskOrMessageCountBasedRolloverStrategy( metricsRegistry, maxBytesPerChunk, msgsPerChunk); initChunkManager( - chunkRollOverStrategy, chunkStore, IndexingChunkManager.makeDefaultRollOverExecutor()); + chunkRollOverStrategy, blobStore, IndexingChunkManager.makeDefaultRollOverExecutor()); List messages1 = messages.subList(0, 3); List messages2 = messages.subList(3, 6); @@ -1520,7 +1520,7 @@ public void testMultipleCountRollOversSuccessfully() new DiskOrMessageCountBasedRolloverStrategy( metricsRegistry, 10 * 1024 * 1024 * 1024L, msgsPerChunk); initChunkManager( - chunkRollOverStrategy, chunkStore, IndexingChunkManager.makeDefaultRollOverExecutor()); + chunkRollOverStrategy, blobStore, IndexingChunkManager.makeDefaultRollOverExecutor()); List messages1 = messages.subList(0, 10); List messages2 = messages.subList(10, 20); diff --git a/astra/src/test/java/com/slack/astra/chunkManager/RecoveryChunkManagerTest.java b/astra/src/test/java/com/slack/astra/chunkManager/RecoveryChunkManagerTest.java index e6db8714f4..d62a0192bb 100644 --- a/astra/src/test/java/com/slack/astra/chunkManager/RecoveryChunkManagerTest.java +++ b/astra/src/test/java/com/slack/astra/chunkManager/RecoveryChunkManagerTest.java @@ -22,7 +22,7 @@ import brave.Tracing; import com.adobe.testing.s3mock.junit5.S3MockExtension; -import com.slack.astra.blobfs.ChunkStore; +import com.slack.astra.blobfs.BlobStore; import com.slack.astra.blobfs.S3TestUtils; import com.slack.astra.chunk.ChunkInfo; import com.slack.astra.chunk.ReadWriteChunk; @@ -83,7 +83,7 @@ public class RecoveryChunkManagerTest { private S3AsyncClient s3AsyncClient; private static final String ZK_PATH_PREFIX = "testZK"; - private ChunkStore chunkStore; + private BlobStore blobStore; private TestingServer localZkServer; private AsyncCuratorFramework curatorFramework; private SearchMetadataStore searchMetadataStore; @@ -97,7 +97,7 @@ public void setUp() throws Exception { metricsRegistry = new SimpleMeterRegistry(); // create an S3 client. s3AsyncClient = S3TestUtils.createS3CrtClient(S3_MOCK_EXTENSION.getServiceEndpoint()); - chunkStore = new ChunkStore(s3AsyncClient, S3_TEST_BUCKET); + blobStore = new BlobStore(s3AsyncClient, S3_TEST_BUCKET); localZkServer = new TestingServer(); localZkServer.start(); @@ -130,7 +130,7 @@ public void tearDown() throws TimeoutException, IOException, InterruptedExceptio localZkServer.stop(); } - private void initChunkManager(ChunkStore chunkStore) throws Exception { + private void initChunkManager(BlobStore blobStore) throws Exception { AstraConfig = AstraConfigUtil.makeAstraConfig( "localhost:9090", @@ -153,7 +153,7 @@ private void initChunkManager(ChunkStore chunkStore) throws Exception { searchMetadataStore, snapshotMetadataStore, AstraConfig.getIndexerConfig(), - chunkStore); + blobStore); chunkManager.startAsync(); chunkManager.awaitRunning(DEFAULT_START_STOP_DURATION); @@ -161,7 +161,7 @@ private void initChunkManager(ChunkStore chunkStore) throws Exception { @Test public void testAddMessageAndRollover() throws Exception { - initChunkManager(chunkStore); + initChunkManager(blobStore); List messages = SpanUtil.makeSpansWithTimeDifference(1, 100, 1, Instant.now()); int actualChunkSize = 0; @@ -365,7 +365,7 @@ private void testChunkManagerSearch( @Test public void testAddMessageWithPropertyTypeConflicts() throws Exception { - initChunkManager(chunkStore); + initChunkManager(blobStore); // Add a valid message int offset = 1; @@ -413,7 +413,7 @@ public void testAddMessageWithPropertyTypeConflicts() throws Exception { @Test public void testAddMessagesWithFailedRollOverStopsIngestion() throws Exception { // Use a non-existent bucket to induce roll-over failure. - initChunkManager(new ChunkStore(s3AsyncClient, "fakebucket")); + initChunkManager(new BlobStore(s3AsyncClient, "fakebucket")); int offset = 1; diff --git a/astra/src/test/java/com/slack/astra/chunkrollover/DiskOrMessageCountBasedRolloverStrategyTest.java b/astra/src/test/java/com/slack/astra/chunkrollover/DiskOrMessageCountBasedRolloverStrategyTest.java index 9933ac9603..a8b2d1fb63 100644 --- a/astra/src/test/java/com/slack/astra/chunkrollover/DiskOrMessageCountBasedRolloverStrategyTest.java +++ b/astra/src/test/java/com/slack/astra/chunkrollover/DiskOrMessageCountBasedRolloverStrategyTest.java @@ -14,7 +14,7 @@ import com.adobe.testing.s3mock.junit5.S3MockExtension; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; -import com.slack.astra.blobfs.ChunkStore; +import com.slack.astra.blobfs.BlobStore; import com.slack.astra.blobfs.S3TestUtils; import com.slack.astra.chunk.SearchContext; import com.slack.astra.chunkManager.IndexingChunkManager; @@ -69,7 +69,7 @@ public class DiskOrMessageCountBasedRolloverStrategyTest { private SimpleMeterRegistry metricsRegistry; private S3AsyncClient s3AsyncClient; private static final String ZK_PATH_PREFIX = "testZK"; - private ChunkStore chunkStore; + private BlobStore blobStore; private TestingServer localZkServer; private AsyncCuratorFramework curatorFramework; @@ -90,7 +90,7 @@ public void setUp() throws Exception { metricsRegistry = new SimpleMeterRegistry(); s3AsyncClient = S3TestUtils.createS3CrtClient(S3_MOCK_EXTENSION.getServiceEndpoint()); - chunkStore = new ChunkStore(s3AsyncClient, S3_TEST_BUCKET); + blobStore = new BlobStore(s3AsyncClient, S3_TEST_BUCKET); localZkServer = new TestingServer(); localZkServer.start(); @@ -136,7 +136,7 @@ private void initChunkManager( tmpPath.toFile().getAbsolutePath(), chunkRollOverStrategy, metricsRegistry, - chunkStore, + blobStore, listeningExecutorService, curatorFramework, searchContext, diff --git a/astra/src/test/java/com/slack/astra/clusterManager/SnapshotDeletionServiceTest.java b/astra/src/test/java/com/slack/astra/clusterManager/SnapshotDeletionServiceTest.java index d97f661e9c..8f31b86cfe 100644 --- a/astra/src/test/java/com/slack/astra/clusterManager/SnapshotDeletionServiceTest.java +++ b/astra/src/test/java/com/slack/astra/clusterManager/SnapshotDeletionServiceTest.java @@ -17,7 +17,7 @@ import brave.Tracing; import com.adobe.testing.s3mock.junit5.S3MockExtension; -import com.slack.astra.blobfs.ChunkStore; +import com.slack.astra.blobfs.BlobStore; import com.slack.astra.blobfs.S3TestUtils; import com.slack.astra.metadata.core.CuratorBuilder; import com.slack.astra.metadata.replica.ReplicaMetadata; @@ -66,7 +66,7 @@ public class SnapshotDeletionServiceTest { private SnapshotMetadataStore snapshotMetadataStore; private ReplicaMetadataStore replicaMetadataStore; private S3AsyncClient s3AsyncClient; - private ChunkStore chunkStore; + private BlobStore blobStore; @BeforeEach public void setup() throws Exception { @@ -88,7 +88,7 @@ public void setup() throws Exception { replicaMetadataStore = spy(new ReplicaMetadataStore(curatorFramework)); s3AsyncClient = S3TestUtils.createS3CrtClient(S3_MOCK_EXTENSION.getServiceEndpoint()); - chunkStore = spy(new ChunkStore(s3AsyncClient, S3_TEST_BUCKET)); + blobStore = spy(new BlobStore(s3AsyncClient, S3_TEST_BUCKET)); } @AfterEach @@ -128,7 +128,7 @@ public void shouldThrowOnInvalidSnapshotLifespan() { new SnapshotDeletionService( replicaMetadataStore, snapshotMetadataStore, - chunkStore, + blobStore, managerConfig, meterRegistry)); } @@ -156,7 +156,7 @@ public void shouldDeleteExpiredSnapshotNoReplicas() throws Exception { Path directory = Files.createTempDirectory(""); Files.createTempFile(directory, "", ""); String chunkId = UUID.randomUUID().toString(); - chunkStore.upload(chunkId, directory); + blobStore.upload(chunkId, directory); SnapshotMetadata snapshotMetadata = new SnapshotMetadata( @@ -171,13 +171,13 @@ public void shouldDeleteExpiredSnapshotNoReplicas() throws Exception { SnapshotDeletionService snapshotDeletionService = new SnapshotDeletionService( - replicaMetadataStore, snapshotMetadataStore, chunkStore, managerConfig, meterRegistry); + replicaMetadataStore, snapshotMetadataStore, blobStore, managerConfig, meterRegistry); int deletes = snapshotDeletionService.deleteExpiredSnapshotsWithoutReplicas(); assertThat(deletes).isEqualTo(1); await().until(() -> snapshotMetadataStore.listSync().isEmpty()); - verify(chunkStore, times(1)).delete(eq(chunkId)); + verify(blobStore, times(1)).delete(eq(chunkId)); assertThat(MetricsUtil.getCount(SnapshotDeletionService.SNAPSHOT_DELETE_SUCCESS, meterRegistry)) .isEqualTo(1); @@ -211,7 +211,7 @@ public void shouldNotDeleteExpiredSnapshotWithReplicas() throws Exception { Path directory = Files.createTempDirectory(""); Files.createTempFile(directory, "", ""); String chunkId = UUID.randomUUID().toString(); - chunkStore.upload(chunkId, directory); + blobStore.upload(chunkId, directory); SnapshotMetadata snapshotMetadata = new SnapshotMetadata( @@ -238,9 +238,9 @@ public void shouldNotDeleteExpiredSnapshotWithReplicas() throws Exception { SnapshotDeletionService snapshotDeletionService = new SnapshotDeletionService( - replicaMetadataStore, snapshotMetadataStore, chunkStore, managerConfig, meterRegistry); + replicaMetadataStore, snapshotMetadataStore, blobStore, managerConfig, meterRegistry); - List s3CrtBlobFsFiles = chunkStore.listFiles(chunkId); + List s3CrtBlobFsFiles = blobStore.listFiles(chunkId); assertThat(s3CrtBlobFsFiles.size()).isNotEqualTo(0); int deletes = snapshotDeletionService.deleteExpiredSnapshotsWithoutReplicas(); @@ -248,8 +248,8 @@ public void shouldNotDeleteExpiredSnapshotWithReplicas() throws Exception { assertThat(snapshotMetadataStore.listSync()).containsExactlyInAnyOrder(snapshotMetadata); assertThat(replicaMetadataStore.listSync()).containsExactlyInAnyOrder(replicaMetadata); - verify(chunkStore, times(0)).delete(any()); - assertThat(chunkStore.listFiles(chunkId)).isEqualTo(s3CrtBlobFsFiles); + verify(blobStore, times(0)).delete(any()); + assertThat(blobStore.listFiles(chunkId)).isEqualTo(s3CrtBlobFsFiles); assertThat(MetricsUtil.getCount(SnapshotDeletionService.SNAPSHOT_DELETE_SUCCESS, meterRegistry)) .isEqualTo(0); @@ -282,14 +282,14 @@ public void shouldHandleNoReplicasNoSnapshots() throws IOException { SnapshotDeletionService snapshotDeletionService = new SnapshotDeletionService( - replicaMetadataStore, snapshotMetadataStore, chunkStore, managerConfig, meterRegistry); + replicaMetadataStore, snapshotMetadataStore, blobStore, managerConfig, meterRegistry); int deletes = snapshotDeletionService.deleteExpiredSnapshotsWithoutReplicas(); assertThat(deletes).isEqualTo(0); assertThat(snapshotMetadataStore.listSync().size()).isEqualTo(0); assertThat(replicaMetadataStore.listSync().size()).isEqualTo(0); - verify(chunkStore, times(0)).delete(any()); + verify(blobStore, times(0)).delete(any()); assertThat(MetricsUtil.getCount(SnapshotDeletionService.SNAPSHOT_DELETE_SUCCESS, meterRegistry)) .isEqualTo(0); @@ -323,7 +323,7 @@ public void shouldHandleNoReplicasUnexpiredSnapshots() throws Exception { Path directory = Files.createTempDirectory(""); Files.createTempFile(directory, "", ""); String chunkId = UUID.randomUUID().toString(); - chunkStore.upload(chunkId, directory); + blobStore.upload(chunkId, directory); SnapshotMetadata snapshotMetadata = new SnapshotMetadata( @@ -335,19 +335,19 @@ public void shouldHandleNoReplicasUnexpiredSnapshots() throws Exception { 0); snapshotMetadataStore.createAsync(snapshotMetadata); await().until(() -> snapshotMetadataStore.listSync().size() == 1); - List s3CrtBlobFsFiles = chunkStore.listFiles(chunkId); + List s3CrtBlobFsFiles = blobStore.listFiles(chunkId); assertThat(s3CrtBlobFsFiles.size()).isNotEqualTo(0); SnapshotDeletionService snapshotDeletionService = new SnapshotDeletionService( - replicaMetadataStore, snapshotMetadataStore, chunkStore, managerConfig, meterRegistry); + replicaMetadataStore, snapshotMetadataStore, blobStore, managerConfig, meterRegistry); int deletes = snapshotDeletionService.deleteExpiredSnapshotsWithoutReplicas(); assertThat(deletes).isEqualTo(0); assertThat(snapshotMetadataStore.listSync()).containsExactlyInAnyOrder(snapshotMetadata); - verify(chunkStore, times(0)).delete(any()); - assertThat(chunkStore.listFiles(chunkId)).isEqualTo(s3CrtBlobFsFiles); + verify(blobStore, times(0)).delete(any()); + assertThat(blobStore.listFiles(chunkId)).isEqualTo(s3CrtBlobFsFiles); assertThat(MetricsUtil.getCount(SnapshotDeletionService.SNAPSHOT_DELETE_SUCCESS, meterRegistry)) .isEqualTo(0); @@ -381,7 +381,7 @@ public void shouldHandleReplicasWithLongerLifespanThanSnapshots() throws Excepti Path directory = Files.createTempDirectory(""); Files.createTempFile(directory, "", ""); String chunkId = UUID.randomUUID().toString(); - chunkStore.upload(chunkId, directory); + blobStore.upload(chunkId, directory); // snapshot is expired SnapshotMetadata snapshotMetadata = @@ -407,20 +407,20 @@ public void shouldHandleReplicasWithLongerLifespanThanSnapshots() throws Excepti await().until(() -> snapshotMetadataStore.listSync().size() == 1); await().until(() -> replicaMetadataStore.listSync().size() == 1); - List s3CrtBlobFsFiles = chunkStore.listFiles(chunkId); + List s3CrtBlobFsFiles = blobStore.listFiles(chunkId); assertThat(s3CrtBlobFsFiles.size()).isNotEqualTo(0); SnapshotDeletionService snapshotDeletionService = new SnapshotDeletionService( - replicaMetadataStore, snapshotMetadataStore, chunkStore, managerConfig, meterRegistry); + replicaMetadataStore, snapshotMetadataStore, blobStore, managerConfig, meterRegistry); int deletes = snapshotDeletionService.deleteExpiredSnapshotsWithoutReplicas(); assertThat(deletes).isEqualTo(0); assertThat(snapshotMetadataStore.listSync()).containsExactlyInAnyOrder(snapshotMetadata); assertThat(replicaMetadataStore.listSync()).containsExactlyInAnyOrder(replicaMetadata); - verify(chunkStore, times(0)).delete(any()); - assertThat(chunkStore.listFiles(chunkId)).isEqualTo(s3CrtBlobFsFiles); + verify(blobStore, times(0)).delete(any()); + assertThat(blobStore.listFiles(chunkId)).isEqualTo(s3CrtBlobFsFiles); assertThat(MetricsUtil.getCount(SnapshotDeletionService.SNAPSHOT_DELETE_SUCCESS, meterRegistry)) .isEqualTo(0); @@ -454,7 +454,7 @@ public void shouldHandleExceptionalObjectStorageDelete() throws Exception { Path directory = Files.createTempDirectory(""); Files.createTempFile(directory, "", ""); String chunkId = UUID.randomUUID().toString(); - chunkStore.upload(chunkId, directory); + blobStore.upload(chunkId, directory); // snapshot is expired SnapshotMetadata snapshotMetadata = @@ -471,8 +471,8 @@ public void shouldHandleExceptionalObjectStorageDelete() throws Exception { SnapshotDeletionService snapshotDeletionService = new SnapshotDeletionService( - replicaMetadataStore, snapshotMetadataStore, chunkStore, managerConfig, meterRegistry); - doThrow(new RuntimeException()).when(chunkStore).delete(any()); + replicaMetadataStore, snapshotMetadataStore, blobStore, managerConfig, meterRegistry); + doThrow(new RuntimeException()).when(blobStore).delete(any()); int deletes = snapshotDeletionService.deleteExpiredSnapshotsWithoutReplicas(); assertThat(deletes).isEqualTo(0); @@ -511,7 +511,7 @@ public void shouldHandleFailedZkDelete() throws Exception { Path directory = Files.createTempDirectory(""); Files.createTempFile(directory, "", ""); String chunkId = UUID.randomUUID().toString(); - chunkStore.upload(chunkId, directory); + blobStore.upload(chunkId, directory); // snapshot is expired SnapshotMetadata snapshotMetadata = @@ -527,20 +527,20 @@ public void shouldHandleFailedZkDelete() throws Exception { SnapshotDeletionService snapshotDeletionService = new SnapshotDeletionService( - replicaMetadataStore, snapshotMetadataStore, chunkStore, managerConfig, meterRegistry); + replicaMetadataStore, snapshotMetadataStore, blobStore, managerConfig, meterRegistry); AsyncStage asyncStage = mock(AsyncStage.class); when(asyncStage.toCompletableFuture()) .thenReturn(CompletableFuture.failedFuture(new Exception())); doReturn(asyncStage).when(snapshotMetadataStore).deleteAsync(any(SnapshotMetadata.class)); - assertThat(chunkStore.listFiles(chunkId)).isNotEmpty(); + assertThat(blobStore.listFiles(chunkId)).isNotEmpty(); int deletes = snapshotDeletionService.deleteExpiredSnapshotsWithoutReplicas(); assertThat(deletes).isEqualTo(0); assertThat(snapshotMetadataStore.listSync()).containsExactlyInAnyOrder(snapshotMetadata); - verify(chunkStore, times(1)).delete(any()); - assertThat(chunkStore.listFiles(chunkId)).isEmpty(); + verify(blobStore, times(1)).delete(any()); + assertThat(blobStore.listFiles(chunkId)).isEmpty(); assertThat(MetricsUtil.getCount(SnapshotDeletionService.SNAPSHOT_DELETE_SUCCESS, meterRegistry)) .isEqualTo(0); @@ -574,7 +574,7 @@ public void shouldHandleFailedObjectDelete() throws Exception { Path directory = Files.createTempDirectory(""); Files.createTempFile(directory, "", ""); String chunkId = UUID.randomUUID().toString(); - chunkStore.upload(chunkId, directory); + blobStore.upload(chunkId, directory); // snapshot is expired SnapshotMetadata snapshotMetadata = @@ -590,8 +590,8 @@ public void shouldHandleFailedObjectDelete() throws Exception { SnapshotDeletionService snapshotDeletionService = new SnapshotDeletionService( - replicaMetadataStore, snapshotMetadataStore, chunkStore, managerConfig, meterRegistry); - doThrow(new RuntimeException()).when(chunkStore).delete(any()); + replicaMetadataStore, snapshotMetadataStore, blobStore, managerConfig, meterRegistry); + doThrow(new RuntimeException()).when(blobStore).delete(any()); int deletes = snapshotDeletionService.deleteExpiredSnapshotsWithoutReplicas(); assertThat(deletes).isEqualTo(0); @@ -630,7 +630,7 @@ public void shouldRetryTimedOutZkDeleteNextRun() throws Exception { Path directory = Files.createTempDirectory(""); Files.createTempFile(directory, "", ""); String chunkId = UUID.randomUUID().toString(); - chunkStore.upload(chunkId, directory); + blobStore.upload(chunkId, directory); // snapshot is expired SnapshotMetadata snapshotMetadata = @@ -646,9 +646,9 @@ public void shouldRetryTimedOutZkDeleteNextRun() throws Exception { SnapshotDeletionService snapshotDeletionService = new SnapshotDeletionService( - replicaMetadataStore, snapshotMetadataStore, chunkStore, managerConfig, meterRegistry); + replicaMetadataStore, snapshotMetadataStore, blobStore, managerConfig, meterRegistry); snapshotDeletionService.futuresListTimeoutSecs = 2; - List s3CrtBlobFsFiles = chunkStore.listFiles(chunkId); + List s3CrtBlobFsFiles = blobStore.listFiles(chunkId); assertThat(s3CrtBlobFsFiles.size()).isNotEqualTo(0); ExecutorService timeoutServiceExecutor = Executors.newSingleThreadExecutor(); @@ -671,8 +671,8 @@ public void shouldRetryTimedOutZkDeleteNextRun() throws Exception { assertThat(deletes).isEqualTo(0); assertThat(snapshotMetadataStore.listSync()).containsExactlyInAnyOrder(snapshotMetadata); - verify(chunkStore, times(1)).delete(any()); - assertThat(chunkStore.listFiles(chunkId)).isEmpty(); + verify(blobStore, times(1)).delete(any()); + assertThat(blobStore.listFiles(chunkId)).isEmpty(); assertThat(MetricsUtil.getCount(SnapshotDeletionService.SNAPSHOT_DELETE_SUCCESS, meterRegistry)) .isEqualTo(0); @@ -688,8 +688,8 @@ public void shouldRetryTimedOutZkDeleteNextRun() throws Exception { assertThat(deletesRetry).isEqualTo(1); await().until(() -> snapshotMetadataStore.listSync().isEmpty()); - verify(chunkStore, times(2)).delete(any()); - assertThat(chunkStore.listFiles(chunkId)).isEmpty(); + verify(blobStore, times(2)).delete(any()); + assertThat(blobStore.listFiles(chunkId)).isEmpty(); assertThat(MetricsUtil.getCount(SnapshotDeletionService.SNAPSHOT_DELETE_SUCCESS, meterRegistry)) .isEqualTo(1); @@ -725,7 +725,7 @@ public void shouldRetryFailedObjectStorageDeleteNextRun() throws Exception { Path directory = Files.createTempDirectory(""); Files.createTempFile(directory, "", ""); String chunkId = UUID.randomUUID().toString(); - chunkStore.upload(chunkId, directory); + blobStore.upload(chunkId, directory); // snapshot is expired SnapshotMetadata snapshotMetadata = @@ -742,14 +742,14 @@ public void shouldRetryFailedObjectStorageDeleteNextRun() throws Exception { SnapshotDeletionService snapshotDeletionService = new SnapshotDeletionService( - replicaMetadataStore, snapshotMetadataStore, chunkStore, managerConfig, meterRegistry); - doThrow(new RuntimeException()).when(chunkStore).delete(any()); + replicaMetadataStore, snapshotMetadataStore, blobStore, managerConfig, meterRegistry); + doThrow(new RuntimeException()).when(blobStore).delete(any()); int deletes = snapshotDeletionService.deleteExpiredSnapshotsWithoutReplicas(); assertThat(deletes).isEqualTo(0); assertThat(snapshotMetadataStore.listSync().size()).isEqualTo(1); - verify(chunkStore, times(1)).delete(any()); + verify(blobStore, times(1)).delete(any()); assertThat(MetricsUtil.getCount(SnapshotDeletionService.SNAPSHOT_DELETE_SUCCESS, meterRegistry)) .isEqualTo(0); @@ -759,12 +759,12 @@ public void shouldRetryFailedObjectStorageDeleteNextRun() throws Exception { MetricsUtil.getTimerCount(SnapshotDeletionService.SNAPSHOT_DELETE_TIMER, meterRegistry)) .isEqualTo(1); - doCallRealMethod().when(chunkStore).delete(any()); + doCallRealMethod().when(blobStore).delete(any()); int deleteRetry = snapshotDeletionService.deleteExpiredSnapshotsWithoutReplicas(); assertThat(deleteRetry).isEqualTo(1); await().until(() -> snapshotMetadataStore.listSync().size() == 0); - verify(chunkStore, times(2)).delete(any()); + verify(blobStore, times(2)).delete(any()); assertThat(MetricsUtil.getCount(SnapshotDeletionService.SNAPSHOT_DELETE_SUCCESS, meterRegistry)) .isEqualTo(1); @@ -798,7 +798,7 @@ public void shouldHandleSnapshotDeleteLifecycle() throws Exception { Path directory = Files.createTempDirectory(""); Files.createTempFile(directory, "", ""); String chunkId = UUID.randomUUID().toString(); - chunkStore.upload(chunkId, directory); + blobStore.upload(chunkId, directory); SnapshotMetadata snapshotMetadata = new SnapshotMetadata( @@ -813,12 +813,12 @@ public void shouldHandleSnapshotDeleteLifecycle() throws Exception { SnapshotDeletionService snapshotDeletionService = new SnapshotDeletionService( - replicaMetadataStore, snapshotMetadataStore, chunkStore, managerConfig, meterRegistry); + replicaMetadataStore, snapshotMetadataStore, blobStore, managerConfig, meterRegistry); snapshotDeletionService.startAsync(); snapshotDeletionService.awaitRunning(DEFAULT_START_STOP_DURATION); await().until(() -> snapshotMetadataStore.listSync().size() == 0); - verify(chunkStore, times(1)).delete(eq(chunkId)); + verify(blobStore, times(1)).delete(eq(chunkId)); await() .until( diff --git a/astra/src/test/java/com/slack/astra/logstore/LuceneIndexStoreImplTest.java b/astra/src/test/java/com/slack/astra/logstore/LuceneIndexStoreImplTest.java index 34bfac8a66..56c285095b 100644 --- a/astra/src/test/java/com/slack/astra/logstore/LuceneIndexStoreImplTest.java +++ b/astra/src/test/java/com/slack/astra/logstore/LuceneIndexStoreImplTest.java @@ -15,7 +15,7 @@ import brave.Tracing; import com.adobe.testing.s3mock.junit5.S3MockExtension; import com.google.protobuf.ByteString; -import com.slack.astra.blobfs.ChunkStore; +import com.slack.astra.blobfs.BlobStore; import com.slack.astra.blobfs.S3TestUtils; import com.slack.astra.logstore.LogMessage.ReservedField; import com.slack.astra.logstore.schema.SchemaAwareLogDocumentBuilderImpl; @@ -366,10 +366,10 @@ public void testS3Snapshot() throws Exception { S3TestUtils.createS3CrtClient(S3_MOCK_EXTENSION.getServiceEndpoint()); String bucket = "snapshot-test"; s3AsyncClient.createBucket(CreateBucketRequest.builder().bucket(bucket).build()).get(); - ChunkStore chunkStore = new ChunkStore(s3AsyncClient, bucket); + BlobStore blobStore = new BlobStore(s3AsyncClient, bucket); String chunkId = UUID.randomUUID().toString(); - chunkStore.upload(chunkId, dirPath); + blobStore.upload(chunkId, dirPath); for (String fileName : activeFiles) { File fileToCopy = new File(dirPath.toString(), fileName); @@ -394,7 +394,7 @@ public void testS3Snapshot() throws Exception { // then fail) FileUtils.cleanDirectory(tmpPath.toFile()); // Download files from S3 to local FS. - chunkStore.download(chunkId, tmpPath.toAbsolutePath()); + blobStore.download(chunkId, tmpPath.toAbsolutePath()); // the delta is the presence of the write.lock file, which is released but still in // the directory return Objects.requireNonNull(tmpPath.toFile().listFiles()).length diff --git a/astra/src/test/java/com/slack/astra/recovery/RecoveryServiceTest.java b/astra/src/test/java/com/slack/astra/recovery/RecoveryServiceTest.java index 60596058d8..d6c12cf603 100644 --- a/astra/src/test/java/com/slack/astra/recovery/RecoveryServiceTest.java +++ b/astra/src/test/java/com/slack/astra/recovery/RecoveryServiceTest.java @@ -24,7 +24,7 @@ import brave.Tracing; import com.adobe.testing.s3mock.junit5.S3MockExtension; import com.google.common.collect.Maps; -import com.slack.astra.blobfs.ChunkStore; +import com.slack.astra.blobfs.BlobStore; import com.slack.astra.blobfs.S3TestUtils; import com.slack.astra.metadata.core.AstraMetadataTestUtils; import com.slack.astra.metadata.core.CuratorBuilder; @@ -80,7 +80,7 @@ public class RecoveryServiceTest { private TestingServer zkServer; private MeterRegistry meterRegistry; - private ChunkStore chunkStore; + private BlobStore blobStore; private TestKafkaServer kafkaServer; private S3AsyncClient s3AsyncClient; private RecoveryService recoveryService; @@ -93,7 +93,7 @@ public void setup() throws Exception { meterRegistry = new SimpleMeterRegistry(); zkServer = new TestingServer(); s3AsyncClient = S3TestUtils.createS3CrtClient(S3_MOCK_EXTENSION.getServiceEndpoint()); - chunkStore = new ChunkStore(s3AsyncClient, TEST_S3_BUCKET); + blobStore = new BlobStore(s3AsyncClient, TEST_S3_BUCKET); } @AfterEach @@ -149,7 +149,7 @@ public void testShouldHandleRecoveryTask() throws Exception { CuratorBuilder.build(meterRegistry, astraCfg.getMetadataStoreConfig().getZookeeperConfig()); // Start recovery service - recoveryService = new RecoveryService(astraCfg, curatorFramework, meterRegistry, chunkStore); + recoveryService = new RecoveryService(astraCfg, curatorFramework, meterRegistry, blobStore); recoveryService.startAsync(); recoveryService.awaitRunning(DEFAULT_START_STOP_DURATION); @@ -166,7 +166,7 @@ public void testShouldHandleRecoveryTask() throws Exception { List snapshots = AstraMetadataTestUtils.listSyncUncached(snapshotMetadataStore); assertThat(snapshots.size()).isEqualTo(1); - assertThat(chunkStore.listFiles(snapshots.get(0).snapshotId).size()).isGreaterThan(0); + assertThat(blobStore.listFiles(snapshots.get(0).snapshotId).size()).isGreaterThan(0); assertThat(getCount(MESSAGES_RECEIVED_COUNTER, meterRegistry)).isEqualTo(31); assertThat(getCount(MESSAGES_FAILED_COUNTER, meterRegistry)).isEqualTo(0); assertThat(getCount(ROLLOVERS_INITIATED, meterRegistry)).isEqualTo(1); @@ -232,7 +232,7 @@ public void testShouldHandleRecoveryTaskWithCompletelyUnavailableOffsets() throw // Start recovery service recoveryService = - new RecoveryService(astraCfg, curatorFramework, components.meterRegistry, chunkStore); + new RecoveryService(astraCfg, curatorFramework, components.meterRegistry, blobStore); recoveryService.startAsync(); recoveryService.awaitRunning(DEFAULT_START_STOP_DURATION); long startOffset = 1; @@ -314,7 +314,7 @@ public void testShouldHandleRecoveryTaskWithPartiallyUnavailableOffsets() throws // Start recovery service recoveryService = - new RecoveryService(astraCfg, curatorFramework, components.meterRegistry, chunkStore); + new RecoveryService(astraCfg, curatorFramework, components.meterRegistry, blobStore); recoveryService.startAsync(); recoveryService.awaitRunning(DEFAULT_START_STOP_DURATION); @@ -334,7 +334,7 @@ public void testShouldHandleRecoveryTaskWithPartiallyUnavailableOffsets() throws List snapshots = AstraMetadataTestUtils.listSyncUncached(snapshotMetadataStore); assertThat(snapshots.size()).isEqualTo(1); - assertThat(chunkStore.listFiles(snapshots.get(0).snapshotId).size()).isGreaterThan(0); + assertThat(blobStore.listFiles(snapshots.get(0).snapshotId).size()).isGreaterThan(0); assertThat(getCount(MESSAGES_FAILED_COUNTER, meterRegistry)).isEqualTo(0); assertThat(getCount(ROLLOVERS_INITIATED, meterRegistry)).isEqualTo(0); assertThat(getCount(ROLLOVERS_COMPLETED, meterRegistry)).isEqualTo(0); @@ -351,7 +351,7 @@ public void testShouldHandleRecoveryTaskFailure() throws Exception { // Start recovery service recoveryService = new RecoveryService( - astraCfg, curatorFramework, meterRegistry, new ChunkStore(s3AsyncClient, fakeS3Bucket)); + astraCfg, curatorFramework, meterRegistry, new BlobStore(s3AsyncClient, fakeS3Bucket)); recoveryService.startAsync(); recoveryService.awaitRunning(DEFAULT_START_STOP_DURATION); @@ -390,7 +390,7 @@ public void testShouldHandleRecoveryTaskAssignmentSuccess() throws Exception { CuratorBuilder.build(meterRegistry, astraCfg.getMetadataStoreConfig().getZookeeperConfig()); // Start recovery service - recoveryService = new RecoveryService(astraCfg, curatorFramework, meterRegistry, chunkStore); + recoveryService = new RecoveryService(astraCfg, curatorFramework, meterRegistry, blobStore); recoveryService.startAsync(); recoveryService.awaitRunning(DEFAULT_START_STOP_DURATION); @@ -455,7 +455,7 @@ public void testShouldHandleRecoveryTaskAssignmentSuccess() throws Exception { List snapshots = AstraMetadataTestUtils.listSyncUncached(snapshotMetadataStore); assertThat(AstraMetadataTestUtils.listSyncUncached(snapshotMetadataStore).size()).isEqualTo(1); - assertThat(chunkStore.listFiles(snapshots.get(0).snapshotId).size()).isGreaterThan(0); + assertThat(blobStore.listFiles(snapshots.get(0).snapshotId).size()).isGreaterThan(0); assertThat(getCount(MESSAGES_RECEIVED_COUNTER, meterRegistry)).isEqualTo(31); assertThat(getCount(MESSAGES_FAILED_COUNTER, meterRegistry)).isEqualTo(0); assertThat(getCount(ROLLOVERS_INITIATED, meterRegistry)).isEqualTo(1); @@ -473,7 +473,7 @@ public void testShouldHandleRecoveryTaskAssignmentFailure() throws Exception { // Start recovery service recoveryService = new RecoveryService( - astraCfg, curatorFramework, meterRegistry, new ChunkStore(s3AsyncClient, fakeS3Bucket)); + astraCfg, curatorFramework, meterRegistry, new BlobStore(s3AsyncClient, fakeS3Bucket)); recoveryService.startAsync(); recoveryService.awaitRunning(DEFAULT_START_STOP_DURATION); @@ -648,7 +648,7 @@ public void shouldHandleInvalidRecoveryTasks() throws Exception { CuratorBuilder.build(meterRegistry, astraCfg.getMetadataStoreConfig().getZookeeperConfig()); // Start recovery service - recoveryService = new RecoveryService(astraCfg, curatorFramework, meterRegistry, chunkStore); + recoveryService = new RecoveryService(astraCfg, curatorFramework, meterRegistry, blobStore); recoveryService.startAsync(); recoveryService.awaitRunning(DEFAULT_START_STOP_DURATION); diff --git a/astra/src/test/java/com/slack/astra/testlib/ChunkManagerUtil.java b/astra/src/test/java/com/slack/astra/testlib/ChunkManagerUtil.java index d4ae4b3a4f..97e1ad8130 100644 --- a/astra/src/test/java/com/slack/astra/testlib/ChunkManagerUtil.java +++ b/astra/src/test/java/com/slack/astra/testlib/ChunkManagerUtil.java @@ -5,7 +5,7 @@ import com.adobe.testing.s3mock.junit5.S3MockExtension; import com.google.common.io.Files; import com.google.common.util.concurrent.MoreExecutors; -import com.slack.astra.blobfs.ChunkStore; +import com.slack.astra.blobfs.BlobStore; import com.slack.astra.blobfs.S3TestUtils; import com.slack.astra.chunk.SearchContext; import com.slack.astra.chunkManager.IndexingChunkManager; @@ -89,7 +89,7 @@ public ChunkManagerUtil( tempFolder = Files.createTempDir(); // TODO: don't use beta func. s3AsyncClient = S3TestUtils.createS3CrtClient(s3MockExtension.getServiceEndpoint()); - ChunkStore chunkStore = new ChunkStore(s3AsyncClient, s3Bucket); + BlobStore blobStore = new BlobStore(s3AsyncClient, s3Bucket); this.zkServer = zkServer; // noop if zk has already been started by the caller @@ -107,7 +107,7 @@ public ChunkManagerUtil( tempFolder.getAbsolutePath(), chunkRollOverStrategy, meterRegistry, - chunkStore, + blobStore, MoreExecutors.newDirectExecutorService(), curatorFramework, searchContext,