chia7712 commented on code in PR #19972:
URL: https://github.com/apache/kafka/pull/19972#discussion_r2160412721


##########
storage/src/test/java/org/apache/kafka/storage/internals/log/LogSegmentTest.java:
##########
@@ -856,6 +865,48 @@ public void 
testNonMonotonicTimestampForMultipleBatchesInMemoryRecords() throws
         assertEquals(new TimestampOffset(2, 2), segment.timeIndex().entry(1));
     }
 
+    @Test
+    @Timeout(30)
+    public void testConcurrentAccessToMaxTimestampSoFar() throws Exception {
+        int numThreads = 16;
+        ExecutorService executor = Executors.newFixedThreadPool(numThreads);
+        TimeIndex mockTimeIndex = mock(TimeIndex.class);
+        when(mockTimeIndex.lastEntry()).thenReturn(new 
TimestampOffset(RecordBatch.NO_TIMESTAMP, 0L));
+
+        try {
+            // to reproduce race, we iterate test for certain duration
+            long remainingDurationNanos = Duration.ofSeconds(1).toNanos();
+            while (remainingDurationNanos > 0) {
+                long t0 = System.nanoTime();
+                clearInvocations(mockTimeIndex);
+                try (LogSegment seg = spy(LogTestUtils.createSegment(0, 
logDir, 10, Time.SYSTEM))) {
+                    when(seg.timeIndex()).thenReturn(mockTimeIndex);
+                    List<Future<?>> futures = new ArrayList<>();
+                    for (int i = 0; i < numThreads; i++) {
+                        futures.add(executor.submit(() -> {

Review Comment:
   ```java
   futures.add(executor.submit(() -> 
assertDoesNotThrow(seg::maxTimestampSoFar)));
   ```



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/LogSegment.java:
##########
@@ -192,8 +192,13 @@ public void sanityCheck(boolean timeIndexFileNewlyCreated) 
throws IOException {
      * the time index).
      */
     public TimestampOffset readMaxTimestampAndOffsetSoFar() throws IOException 
{
-        if (maxTimestampAndOffsetSoFar == TimestampOffset.UNKNOWN)
-            maxTimestampAndOffsetSoFar = timeIndex().lastEntry();
+        if (maxTimestampAndOffsetSoFar == TimestampOffset.UNKNOWN) {
+            synchronized (this) {

Review Comment:
   The monitor lock of `LogSegment` may be used by external code. Perhaps it 
would be safer to use the monitor lock of an internal `Object` instead?
   ```java
   private final Object maxTimestampAndOffsetLock = new Object();
   
   synchronized (maxTimestampAndOffsetLock) {
   }
   
   ```



##########
storage/src/test/java/org/apache/kafka/storage/internals/log/LogSegmentTest.java:
##########
@@ -856,6 +865,48 @@ public void 
testNonMonotonicTimestampForMultipleBatchesInMemoryRecords() throws
         assertEquals(new TimestampOffset(2, 2), segment.timeIndex().entry(1));
     }
 
+    @Test
+    @Timeout(30)
+    public void testConcurrentAccessToMaxTimestampSoFar() throws Exception {
+        int numThreads = 16;
+        ExecutorService executor = Executors.newFixedThreadPool(numThreads);
+        TimeIndex mockTimeIndex = mock(TimeIndex.class);
+        when(mockTimeIndex.lastEntry()).thenReturn(new 
TimestampOffset(RecordBatch.NO_TIMESTAMP, 0L));
+
+        try {
+            // to reproduce race, we iterate test for certain duration
+            long remainingDurationNanos = Duration.ofSeconds(1).toNanos();
+            while (remainingDurationNanos > 0) {
+                long t0 = System.nanoTime();
+                clearInvocations(mockTimeIndex);
+                try (LogSegment seg = spy(LogTestUtils.createSegment(0, 
logDir, 10, Time.SYSTEM))) {
+                    when(seg.timeIndex()).thenReturn(mockTimeIndex);
+                    List<Future<?>> futures = new ArrayList<>();
+                    for (int i = 0; i < numThreads; i++) {
+                        futures.add(executor.submit(() -> {

Review Comment:
   or we can use `CompletableFuture` to streamline the code.
   ```java
           long remainingDurationNanos = Duration.ofSeconds(1).toNanos();
           while (remainingDurationNanos > 0) {
               long t0 = System.nanoTime();
               clearInvocations(mockTimeIndex);
               try (LogSegment seg = spy(LogTestUtils.createSegment(0, logDir, 
10, Time.SYSTEM))) {
                   when(seg.timeIndex()).thenReturn(mockTimeIndex);
                   var futures = IntStream.range(0, 
numThreads).mapToObj(ignored -> CompletableFuture.runAsync(() -> 
assertDoesNotThrow(seg::maxTimestampSoFar))).toList();
                   futures.forEach(CompletableFuture::join);
                   // timeIndex.lastEntry should be called once if no race
                   verify(mockTimeIndex, times(1)).lastEntry();
   
                   long elapsedNanos = System.nanoTime() - t0;
                   remainingDurationNanos -= elapsedNanos;
               }
           }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to