|
60 | 60 |
|
61 | 61 | import io.netty.buffer.ByteBuf;
|
62 | 62 | import software.amazon.awssdk.core.async.AsyncRequestBody;
|
| 63 | +import software.amazon.awssdk.http.HttpStatusCode; |
63 | 64 | import software.amazon.awssdk.services.s3.S3AsyncClient;
|
| 65 | +import software.amazon.awssdk.services.s3.model.CreateMultipartUploadRequest; |
| 66 | +import software.amazon.awssdk.services.s3.model.CreateMultipartUploadResponse; |
64 | 67 | import software.amazon.awssdk.services.s3.model.PutObjectRequest;
|
| 68 | +import software.amazon.awssdk.services.s3.model.S3Exception; |
| 69 | +import software.amazon.awssdk.services.s3.model.UploadPartRequest; |
65 | 70 |
|
66 | 71 | import static org.junit.jupiter.api.Assertions.assertEquals;
|
67 | 72 | import static org.junit.jupiter.api.Assertions.assertTrue;
|
@@ -618,6 +623,70 @@ public void testCompactNoneExistObjects2() {
|
618 | 623 | }
|
619 | 624 | }
|
620 | 625 |
|
| 626 | + @Test |
| 627 | + public void testCompactWithUploadException() { |
| 628 | + when(config.streamSetObjectCompactionStreamSplitSize()).thenReturn(100 * 1024 * 1024L); |
| 629 | + when(config.streamSetObjectCompactionCacheSize()).thenReturn(1024 * 1024 * 1024L); |
| 630 | + when(config.objectPartSize()).thenReturn(100 * 1024 * 1024); |
| 631 | + Map<Long, List<StreamDataBlock>> streamDataBlockMap = getStreamDataBlockMapLarge(); |
| 632 | + S3ObjectMetadata objectMetadata0 = new S3ObjectMetadata(OBJECT_0, 0, S3ObjectType.STREAM_SET); |
| 633 | + S3ObjectMetadata objectMetadata1 = new S3ObjectMetadata(OBJECT_1, 0, S3ObjectType.STREAM_SET); |
| 634 | + S3ObjectMetadata objectMetadata2 = new S3ObjectMetadata(OBJECT_2, 0, S3ObjectType.STREAM_SET); |
| 635 | + List<S3ObjectMetadata> s3ObjectMetadata = List.of(objectMetadata0, objectMetadata1, objectMetadata2); |
| 636 | + this.compactionAnalyzer = new CompactionAnalyzer(config.streamSetObjectCompactionCacheSize(), config.streamSetObjectCompactionStreamSplitSize(), |
| 637 | + config.maxStreamNumPerStreamSetObject(), config.maxStreamObjectNumPerCommit()); |
| 638 | + List<CompactionPlan> compactionPlans = this.compactionAnalyzer.analyze(streamDataBlockMap, new HashSet<>()); |
| 639 | + CommitStreamSetObjectRequest request = new CommitStreamSetObjectRequest(); |
| 640 | + |
| 641 | + S3AsyncClient s3AsyncClient = Mockito.mock(S3AsyncClient.class); |
| 642 | + doAnswer(invocation -> CompletableFuture.failedFuture(S3Exception.builder().statusCode(HttpStatusCode.NOT_FOUND).build())).when(s3AsyncClient).putObject(any(PutObjectRequest.class), any(AsyncRequestBody.class)); |
| 643 | + doAnswer(invocation -> CompletableFuture.completedFuture(CreateMultipartUploadResponse.builder().uploadId("123").build())).when(s3AsyncClient).createMultipartUpload(any(CreateMultipartUploadRequest.class)); |
| 644 | + doAnswer(invocation -> CompletableFuture.failedFuture(S3Exception.builder().statusCode(HttpStatusCode.NOT_FOUND).build())).when(s3AsyncClient).uploadPart(any(UploadPartRequest.class), any(AsyncRequestBody.class)); |
| 645 | + |
| 646 | + AwsObjectStorage objectStorage = Mockito.spy(new AwsObjectStorage(s3AsyncClient, "")); |
| 647 | + doAnswer(invocation -> CompletableFuture.completedFuture(TestUtils.randomPooled(65 * 1024 * 1024))).when(objectStorage).rangeRead(any(), eq(objectMetadata0.key()), anyLong(), anyLong()); |
| 648 | + doAnswer(invocation -> CompletableFuture.completedFuture(TestUtils.randomPooled(80 * 1024 * 1024))).when(objectStorage).rangeRead(any(), eq(objectMetadata1.key()), anyLong(), anyLong()); |
| 649 | + doAnswer(invocation -> CompletableFuture.completedFuture(TestUtils.randomPooled(50 * 1024 * 1024))).when(objectStorage).rangeRead(any(), eq(objectMetadata2.key()), anyLong(), anyLong()); |
| 650 | + |
| 651 | + CompactionManager compactionManager = new CompactionManager(config, objectManager, streamManager, objectStorage); |
| 652 | + Assertions.assertThrowsExactly(CompletionException.class, |
| 653 | + () -> compactionManager.executeCompactionPlans(request, compactionPlans, s3ObjectMetadata)); |
| 654 | + for (CompactionPlan plan : compactionPlans) { |
| 655 | + plan.streamDataBlocksMap().forEach((streamId, blocks) -> blocks.forEach(block -> { |
| 656 | + if (block.getObjectId() != OBJECT_1) { |
| 657 | + block.getDataCf().thenAccept(data -> { |
| 658 | + Assertions.assertEquals(0, data.refCnt()); |
| 659 | + }).join(); |
| 660 | + } |
| 661 | + })); |
| 662 | + } |
| 663 | + } |
| 664 | + |
| 665 | + private static Map<Long, List<StreamDataBlock>> getStreamDataBlockMapLarge() { |
| 666 | + StreamDataBlock block1 = new StreamDataBlock(OBJECT_0, new DataBlockIndex(0, 0, 15, 15, 0, 15 * 1024 * 1024)); |
| 667 | + StreamDataBlock block2 = new StreamDataBlock(OBJECT_0, new DataBlockIndex(1, 0, 20, 20, 15, 50 * 1024 * 1024)); |
| 668 | + |
| 669 | + StreamDataBlock block3 = new StreamDataBlock(OBJECT_1, new DataBlockIndex(0, 15, 12, 12, 0, 20 * 1024 * 1024)); |
| 670 | + StreamDataBlock block4 = new StreamDataBlock(OBJECT_1, new DataBlockIndex(1, 20, 25, 25, 20, 60 * 1024 * 1024)); |
| 671 | + |
| 672 | + StreamDataBlock block5 = new StreamDataBlock(OBJECT_2, new DataBlockIndex(0, 27, 13, 20, 0, 20 * 1024 * 1024)); |
| 673 | + StreamDataBlock block6 = new StreamDataBlock(OBJECT_2, new DataBlockIndex(3, 0, 30, 30, 20, 30 * 1024 * 1024)); |
| 674 | + return Map.of( |
| 675 | + OBJECT_0, List.of( |
| 676 | + block1, |
| 677 | + block2 |
| 678 | + ), |
| 679 | + OBJECT_1, List.of( |
| 680 | + block3, |
| 681 | + block4 |
| 682 | + ), |
| 683 | + OBJECT_2, List.of( |
| 684 | + block5, |
| 685 | + block6 |
| 686 | + ) |
| 687 | + ); |
| 688 | + } |
| 689 | + |
621 | 690 | @Test
|
622 | 691 | public void testCompactWithLimit() {
|
623 | 692 | when(config.streamSetObjectCompactionStreamSplitSize()).thenReturn(70L);
|
|
0 commit comments