-
Notifications
You must be signed in to change notification settings - Fork 231
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
perf(core): batch persistent meta when delete large scale segment (#1670
) * perf(core): reduce persistent meta when delete large scale segment * perf(core): batch persistent meta when delete large scale segment * perf(core): batch persistent meta when delete large scale segment
- Loading branch information
1 parent
4596ba6
commit 393b0d3
Showing
2 changed files
with
175 additions
and
1 deletion.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
117 changes: 117 additions & 0 deletions
117
core/src/test/java/kafka/log/streamaspect/ElasticLogSegmentManagerTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,117 @@ | ||
/* | ||
* Copyright 2024, AutoMQ HK Limited. | ||
* | ||
* Use of this software is governed by the Business Source License | ||
* included in the file BSL.md | ||
* | ||
* As of the Change Date specified in that file, in accordance with | ||
* the Business Source License, use of this software will be governed | ||
* by the Apache License, Version 2.0 | ||
*/ | ||
|
||
package kafka.log.streamaspect; | ||
|
||
import org.junit.jupiter.api.Tag; | ||
import org.junit.jupiter.api.Test; | ||
|
||
import java.util.HashSet; | ||
import java.util.Set; | ||
import java.util.concurrent.CompletableFuture; | ||
import java.util.concurrent.CountDownLatch; | ||
import java.util.concurrent.TimeUnit; | ||
|
||
import static org.junit.jupiter.api.Assertions.assertTrue; | ||
import static org.mockito.ArgumentMatchers.any; | ||
import static org.mockito.ArgumentMatchers.anyLong; | ||
import static org.mockito.Mockito.atLeastOnce; | ||
import static org.mockito.Mockito.atMost; | ||
import static org.mockito.Mockito.mock; | ||
import static org.mockito.Mockito.spy; | ||
import static org.mockito.Mockito.times; | ||
import static org.mockito.Mockito.verify; | ||
import static org.mockito.Mockito.when; | ||
|
||
@Tag("S3Unit") | ||
public class ElasticLogSegmentManagerTest { | ||
@Test | ||
public void testSegmentDelete() { | ||
ElasticLogMeta logMeta = mock(ElasticLogMeta.class); | ||
ElasticLogSegment logSegment = mock(ElasticLogSegment.class); | ||
MetaStream metaStream = mock(MetaStream.class); | ||
|
||
when(metaStream.append(any(MetaKeyValue.class))).thenReturn(CompletableFuture.completedFuture(null)); | ||
|
||
ElasticLogStreamManager elasticLogStreamManager = mock(ElasticLogStreamManager.class); | ||
|
||
ElasticLogSegmentManager manager = spy(new ElasticLogSegmentManager(metaStream, elasticLogStreamManager, "testLargeScaleSegmentDelete")); | ||
|
||
when(manager.remove(anyLong())).thenReturn(logSegment); | ||
|
||
when(manager.asyncPersistLogMeta()).thenReturn(CompletableFuture.completedFuture(logMeta)); | ||
|
||
ElasticLogSegmentManager.EventListener listener = manager.new EventListener(); | ||
listener.onEvent(1, ElasticLogSegmentEvent.SEGMENT_DELETE); | ||
|
||
verify(manager, atLeastOnce()).asyncPersistLogMeta(); | ||
verify(manager, atMost(2)).asyncPersistLogMeta(); | ||
} | ||
|
||
@Test | ||
public void testLargeScaleSegmentDelete() throws InterruptedException { | ||
ElasticLogMeta logMeta = mock(ElasticLogMeta.class); | ||
ElasticLogSegment logSegment = mock(ElasticLogSegment.class); | ||
MetaStream metaStream = mock(MetaStream.class); | ||
|
||
when(metaStream.append(any(MetaKeyValue.class))).thenReturn(CompletableFuture.completedFuture(null)); | ||
|
||
ElasticLogStreamManager elasticLogStreamManager = mock(ElasticLogStreamManager.class); | ||
|
||
ElasticLogSegmentManager manager = spy(new ElasticLogSegmentManager(metaStream, elasticLogStreamManager, "testLargeScaleSegmentDelete")); | ||
|
||
Set<Long> removedSegmentId = new HashSet<>(); | ||
|
||
when(manager.remove(anyLong())).thenAnswer(invocation -> { | ||
long id = invocation.getArgument(0); | ||
removedSegmentId.add(id); | ||
return logSegment; | ||
}); | ||
|
||
CountDownLatch latch = new CountDownLatch(2); | ||
|
||
when(manager.asyncPersistLogMeta()) | ||
.thenAnswer(invocation -> { | ||
CompletableFuture<Object> cf = new CompletableFuture<>() | ||
.completeOnTimeout(logMeta, 100, TimeUnit.MILLISECONDS); | ||
|
||
cf.whenComplete((res, e) -> { | ||
latch.countDown(); | ||
}); | ||
|
||
return cf; | ||
}); | ||
|
||
ElasticLogSegmentManager.EventListener listener = spy(manager.new EventListener()); | ||
|
||
for (long i = 0L; i < 10L; i++) { | ||
listener.onEvent(i, ElasticLogSegmentEvent.SEGMENT_DELETE); | ||
} | ||
|
||
// expect the first and the tail should call the persist method. | ||
verify(manager, times(2)).asyncPersistLogMeta(); | ||
|
||
// check all segmentId removed. | ||
for (long i = 0; i < 10L; i++) { | ||
assertTrue(removedSegmentId.contains(i)); | ||
} | ||
|
||
latch.await(); | ||
|
||
// the request can be finished. | ||
CompletableFuture<ElasticLogMeta> pendingPersistentMetaCf = listener.getPendingPersistentMetaCf(); | ||
pendingPersistentMetaCf.join(); | ||
|
||
// all the queue can be removed. | ||
assertTrue(listener.getPendingDeleteSegmentQueue().isEmpty()); | ||
|
||
} | ||
} |