chia7712 commented on code in PR #20089: URL: https://github.com/apache/kafka/pull/20089#discussion_r2207516464
########## coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorLoaderImplTest.java: ########## @@ -0,0 +1,706 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.coordinator.common.runtime; + +import kafka.server.ReplicaManager; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.compress.Compression; +import org.apache.kafka.common.errors.NotLeaderOrFollowerException; +import org.apache.kafka.common.record.ControlRecordType; +import org.apache.kafka.common.record.EndTransactionMarker; +import org.apache.kafka.common.record.FileRecords; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.record.RecordBatch; +import org.apache.kafka.common.record.SimpleRecord; +import org.apache.kafka.common.requests.TransactionResult; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.server.storage.log.FetchIsolation; +import org.apache.kafka.storage.internals.log.FetchDataInfo; +import org.apache.kafka.storage.internals.log.LogOffsetMetadata; +import org.apache.kafka.storage.internals.log.UnifiedLog; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.mockito.ArgumentCaptor; +import org.mockito.ArgumentMatchers; +import org.mockito.invocation.InvocationOnMock; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.function.Function; + +import scala.Option; + +import static org.apache.kafka.test.TestUtils.assertFutureThrows; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static scala.jdk.javaapi.OptionConverters.toJava; +import static scala.jdk.javaapi.OptionConverters.toScala; + +@SuppressWarnings("unchecked") +@Timeout(60) +class CoordinatorLoaderImplTest { + + private record Tuple<K, V>(K key, V value) { + } + + private static class StringKeyValueDeserializer implements Deserializer<Tuple<String, String>> { + + @Override + public Tuple<String, String> deserialize(ByteBuffer key, ByteBuffer value) throws RuntimeException { + return new Tuple<>( + StandardCharsets.UTF_8.decode(key).toString(), + StandardCharsets.UTF_8.decode(value).toString() + ); + } + } + + @Test + void testNonexistentPartition() throws Exception { + TopicPartition tp = new TopicPartition("foo", 0); + ReplicaManager replicaManager = mock(ReplicaManager.class); + Function<TopicPartition, Optional<UnifiedLog>> partitionLogSupplier = partition -> toJava(replicaManager.getLog(partition)); + Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier = partition -> Optional.ofNullable((Long) replicaManager.getLogEndOffset(partition).get()); + Deserializer<Tuple<String, String>> serde = mock(Deserializer.class); + CoordinatorPlayback<Tuple<String, String>> coordinator = mock(CoordinatorPlayback.class); + + try (CoordinatorLoaderImpl<Tuple<String, String>> loader = new CoordinatorLoaderImpl<>( + Time.SYSTEM, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 + )) { + when(replicaManager.getLog(tp)).thenReturn(Option.empty()); + + assertFutureThrows(NotLeaderOrFollowerException.class, loader.load(tp, coordinator)); + } + } + + @Test + void testLoadingIsRejectedWhenClosed() throws Exception { + TopicPartition tp = new TopicPartition("foo", 0); + ReplicaManager replicaManager = mock(ReplicaManager.class); + Function<TopicPartition, Optional<UnifiedLog>> partitionLogSupplier = partition -> toJava(replicaManager.getLog(partition)); + Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier = partition -> Optional.ofNullable((Long) replicaManager.getLogEndOffset(partition).get()); + Deserializer<Tuple<String, String>> serde = mock(Deserializer.class); + CoordinatorPlayback<Tuple<String, String>> coordinator = mock(CoordinatorPlayback.class); + + try (CoordinatorLoaderImpl<Tuple<String, String>> loader = new CoordinatorLoaderImpl<>( + Time.SYSTEM, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 + )) { + loader.close(); + assertFutureThrows(RuntimeException.class, loader.load(tp, coordinator)); + } + } + + @Test + void testLoading() throws Exception { + TopicPartition tp = new TopicPartition("foo", 0); + ReplicaManager replicaManager = mock(ReplicaManager.class); + Function<TopicPartition, Optional<UnifiedLog>> partitionLogSupplier = partition -> toJava(replicaManager.getLog(partition)); + Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier = partition -> Optional.ofNullable((Long) replicaManager.getLogEndOffset(partition).get()); + Deserializer<Tuple<String, String>> serde = new StringKeyValueDeserializer(); + UnifiedLog log = mock(UnifiedLog.class); + CoordinatorPlayback<Tuple<String, String>> coordinator = mock(CoordinatorPlayback.class); + + try (CoordinatorLoaderImpl<Tuple<String, String>> loader = new CoordinatorLoaderImpl<>( + Time.SYSTEM, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 + )) { + when(replicaManager.getLog(tp)).thenReturn(toScala(Optional.of(log))); + when(log.logStartOffset()).thenReturn(0L); + when(replicaManager.getLogEndOffset(tp)).thenReturn(toScala(Optional.of(9L))); + when(log.highWatermark()).thenReturn(0L); + + FetchDataInfo readResult1 = logReadResult(0, Arrays.asList( + new SimpleRecord("k1".getBytes(), "v1".getBytes()), + new SimpleRecord("k2".getBytes(), "v2".getBytes()) + )); + + when(log.read(0L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult1); + + FetchDataInfo readResult2 = logReadResult(2, Arrays.asList( + new SimpleRecord("k3".getBytes(), "v3".getBytes()), + new SimpleRecord("k4".getBytes(), "v4".getBytes()), + new SimpleRecord("k5".getBytes(), "v5".getBytes()) + )); + + when(log.read(2L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult2); + + FetchDataInfo readResult3 = logReadResult(5, 100L, (short) 5, Arrays.asList( + new SimpleRecord("k6".getBytes(), "v6".getBytes()), + new SimpleRecord("k7".getBytes(), "v7".getBytes()) + )); + + when(log.read(5L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult3); + + FetchDataInfo readResult4 = logReadResult( + 7, + 100L, + (short) 5, + ControlRecordType.COMMIT + ); + + when(log.read(7L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult4); + + FetchDataInfo readResult5 = logReadResult( + 8, + 500L, + (short) 10, + ControlRecordType.ABORT + ); + + when(log.read(8L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult5); + + assertNotNull(loader.load(tp, coordinator).get(10, TimeUnit.SECONDS)); + + verify(coordinator).replay(0L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k1", "v1")); + verify(coordinator).replay(1L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k2", "v2")); + verify(coordinator).replay(2L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k3", "v3")); + verify(coordinator).replay(3L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k4", "v4")); + verify(coordinator).replay(4L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k5", "v5")); + verify(coordinator).replay(5L, 100L, (short) 5, new Tuple<>("k6", "v6")); + verify(coordinator).replay(6L, 100L, (short) 5, new Tuple<>("k7", "v7")); + verify(coordinator).replayEndTransactionMarker(100L, (short) 5, TransactionResult.COMMIT); + verify(coordinator).replayEndTransactionMarker(500L, (short) 10, TransactionResult.ABORT); + verify(coordinator).updateLastWrittenOffset(2L); + verify(coordinator).updateLastWrittenOffset(5L); + verify(coordinator).updateLastWrittenOffset(7L); + verify(coordinator).updateLastWrittenOffset(8L); + verify(coordinator).updateLastCommittedOffset(0L); + } + } + + @Test + void testLoadingStoppedWhenClosed() throws Exception { + TopicPartition tp = new TopicPartition("foo", 0); + ReplicaManager replicaManager = mock(ReplicaManager.class); + Function<TopicPartition, Optional<UnifiedLog>> partitionLogSupplier = partition -> toJava(replicaManager.getLog(partition)); + Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier = partition -> Optional.ofNullable((Long) replicaManager.getLogEndOffset(partition).get()); + Deserializer<Tuple<String, String>> serde = new StringKeyValueDeserializer(); + UnifiedLog log = mock(UnifiedLog.class); + CoordinatorPlayback<Tuple<String, String>> coordinator = mock(CoordinatorPlayback.class); + + try (CoordinatorLoaderImpl<Tuple<String, String>> loader = new CoordinatorLoaderImpl<>( + Time.SYSTEM, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 + )) { + when(replicaManager.getLog(tp)).thenReturn(toScala(Optional.of(log))); + when(log.logStartOffset()).thenReturn(0L); + when(replicaManager.getLogEndOffset(tp)).thenReturn(toScala(Optional.of(100L))); + + FetchDataInfo readResult = logReadResult(0, Arrays.asList( + new SimpleRecord("k1".getBytes(), "v1".getBytes()), + new SimpleRecord("k2".getBytes(), "v2".getBytes()) + )); + + CountDownLatch latch = new CountDownLatch(1); + when(log.read( + anyLong(), + eq(1000), + eq(FetchIsolation.LOG_END), + eq(true) + )).thenAnswer((InvocationOnMock invocation) -> { + latch.countDown(); + return readResult; + }); + + CompletableFuture<CoordinatorLoader.LoadSummary> result = loader.load(tp, coordinator); + boolean completed = latch.await(10, TimeUnit.SECONDS); + assertTrue(completed, "Log read timeout: Latch did not count down in time."); + loader.close(); + + RuntimeException ex = assertFutureThrows(RuntimeException.class, result); + assertNotNull(ex); + assertEquals("Coordinator loader is closed.", ex.getMessage()); + } + } + + @Test + void testUnknownRecordTypeAreIgnored() throws Exception { + TopicPartition tp = new TopicPartition("foo", 0); + ReplicaManager replicaManager = mock(ReplicaManager.class); + Function<TopicPartition, Optional<UnifiedLog>> partitionLogSupplier = partition -> toJava(replicaManager.getLog(partition)); + Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier = partition -> Optional.ofNullable((Long) replicaManager.getLogEndOffset(partition).get()); + StringKeyValueDeserializer serde = mock(StringKeyValueDeserializer.class); + UnifiedLog log = mock(UnifiedLog.class); + CoordinatorPlayback<Tuple<String, String>> coordinator = mock(CoordinatorPlayback.class); + + try (CoordinatorLoaderImpl<Tuple<String, String>> loader = new CoordinatorLoaderImpl<>( + Time.SYSTEM, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 + )) { + when(replicaManager.getLog(tp)).thenReturn(toScala(Optional.of(log))); + when(log.logStartOffset()).thenReturn(0L); + when(replicaManager.getLogEndOffset(tp)).thenReturn(toScala(Optional.of(2L))); + + FetchDataInfo readResult = logReadResult(0, Arrays.asList( + new SimpleRecord("k1".getBytes(), "v1".getBytes()), + new SimpleRecord("k2".getBytes(), "v2".getBytes()) + )); + + when(log.read(0L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult); + + when(serde.deserialize(any(ByteBuffer.class), any(ByteBuffer.class))) + .thenThrow(new Deserializer.UnknownRecordTypeException((short) 1)) + .thenReturn(new Tuple<>("k2", "v2")); + + loader.load(tp, coordinator).get(10, TimeUnit.SECONDS); + + verify(coordinator).replay(1L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k2", "v2")); + } + } + + @Test + void testDeserializationErrorFailsTheLoading() throws Exception { + TopicPartition tp = new TopicPartition("foo", 0); + ReplicaManager replicaManager = mock(ReplicaManager.class); + Function<TopicPartition, Optional<UnifiedLog>> partitionLogSupplier = partition -> toJava(replicaManager.getLog(partition)); + Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier = partition -> Optional.ofNullable((Long) replicaManager.getLogEndOffset(partition).get()); + StringKeyValueDeserializer serde = mock(StringKeyValueDeserializer.class); + UnifiedLog log = mock(UnifiedLog.class); + CoordinatorPlayback<Tuple<String, String>> coordinator = mock(CoordinatorPlayback.class); + + try (CoordinatorLoaderImpl<Tuple<String, String>> loader = new CoordinatorLoaderImpl<>( + Time.SYSTEM, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 + )) { + when(replicaManager.getLog(tp)).thenReturn(toScala(Optional.of(log))); + when(log.logStartOffset()).thenReturn(0L); + when(replicaManager.getLogEndOffset(tp)).thenReturn(toScala(Optional.of(2L))); + + FetchDataInfo readResult = logReadResult(0, Arrays.asList( + new SimpleRecord("k1".getBytes(), "v1".getBytes()), + new SimpleRecord("k2".getBytes(), "v2".getBytes()) + )); + + when(log.read(0L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult); + + when(serde.deserialize(any(ByteBuffer.class), any(ByteBuffer.class))) + .thenThrow(new RuntimeException("Error!")); + + RuntimeException ex = assertFutureThrows(RuntimeException.class, loader.load(tp, coordinator)); + + assertNotNull(ex); + assertEquals(String.format("Deserializing record DefaultRecord(offset=0, timestamp=-1, key=2 bytes, value=2 bytes) from %s failed due to: Error!", tp), ex.getMessage()); + } + } + + @Test + void testLoadGroupAndOffsetsWithCorruptedLog() throws Exception { + // Simulate a case where startOffset < endOffset but log is empty. This could theoretically happen + // when all the records are expired and the active segment is truncated or when the partition + // is accidentally corrupted. + TopicPartition tp = new TopicPartition("foo", 0); + ReplicaManager replicaManager = mock(ReplicaManager.class); + Function<TopicPartition, Optional<UnifiedLog>> partitionLogSupplier = partition -> toJava(replicaManager.getLog(partition)); + Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier = partition -> Optional.ofNullable((Long) replicaManager.getLogEndOffset(partition).get()); + StringKeyValueDeserializer serde = mock(StringKeyValueDeserializer.class); + UnifiedLog log = mock(UnifiedLog.class); + CoordinatorPlayback<Tuple<String, String>> coordinator = mock(CoordinatorPlayback.class); + + try (CoordinatorLoaderImpl<Tuple<String, String>> loader = new CoordinatorLoaderImpl<>( + Time.SYSTEM, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 + )) { + when(replicaManager.getLog(tp)).thenReturn(toScala(Optional.of(log))); + when(log.logStartOffset()).thenReturn(0L); + when(replicaManager.getLogEndOffset(tp)).thenReturn(toScala(Optional.of(10L))); + + FetchDataInfo readResult = logReadResult(0, List.of()); + + when(log.read(0L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult); + + assertNotNull(loader.load(tp, coordinator).get(10, TimeUnit.SECONDS)); + } + } + + @Test + void testLoadSummary() throws Exception { + TopicPartition tp = new TopicPartition("foo", 0); + ReplicaManager replicaManager = mock(ReplicaManager.class); + Function<TopicPartition, Optional<UnifiedLog>> partitionLogSupplier = partition -> toJava(replicaManager.getLog(partition)); + Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier = partition -> Optional.ofNullable((Long) replicaManager.getLogEndOffset(partition).get()); + StringKeyValueDeserializer serde = new StringKeyValueDeserializer(); + UnifiedLog log = mock(UnifiedLog.class); + CoordinatorPlayback<Tuple<String, String>> coordinator = mock(CoordinatorPlayback.class); + MockTime time = new MockTime(); + + try (CoordinatorLoaderImpl<Tuple<String, String>> loader = new CoordinatorLoaderImpl<>( + time, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 + )) { + long startTimeMs = time.milliseconds(); + when(replicaManager.getLog(tp)).thenReturn(toScala(Optional.of(log))); + when(log.logStartOffset()).thenReturn(0L); + when(replicaManager.getLogEndOffset(tp)).thenReturn(toScala(Optional.of(5L))); + + FetchDataInfo readResult1 = logReadResult(0, Arrays.asList( + new SimpleRecord("k1".getBytes(), "v1".getBytes()), + new SimpleRecord("k2".getBytes(), "v2".getBytes()) + )); + + when(log.read(0L, 1000, FetchIsolation.LOG_END, true)) + .thenAnswer((InvocationOnMock invocation) -> { + time.sleep(1000); + return readResult1; + }); + + FetchDataInfo readResult2 = logReadResult(2, Arrays.asList( + new SimpleRecord("k3".getBytes(), "v3".getBytes()), + new SimpleRecord("k4".getBytes(), "v4".getBytes()), + new SimpleRecord("k5".getBytes(), "v5".getBytes()) + )); + + when(log.read(2L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult2); + + CoordinatorLoader.LoadSummary summary = loader.load(tp, coordinator).get(10, TimeUnit.SECONDS); + assertEquals(startTimeMs, summary.startTimeMs()); + assertEquals(startTimeMs + 1000, summary.endTimeMs()); + assertEquals(5, summary.numRecords()); + assertEquals(readResult1.records.sizeInBytes() + readResult2.records.sizeInBytes(), summary.numBytes()); + } + } + + @Test + void testUpdateLastWrittenOffsetOnBatchLoaded() throws Exception { + TopicPartition tp = new TopicPartition("foo", 0); + ReplicaManager replicaManager = mock(ReplicaManager.class); + Function<TopicPartition, Optional<UnifiedLog>> partitionLogSupplier = partition -> toJava(replicaManager.getLog(partition)); + Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier = partition -> Optional.ofNullable((Long) replicaManager.getLogEndOffset(partition).get()); + StringKeyValueDeserializer serde = new StringKeyValueDeserializer(); + UnifiedLog log = mock(UnifiedLog.class); + CoordinatorPlayback<Tuple<String, String>> coordinator = mock(CoordinatorPlayback.class); + + try (CoordinatorLoaderImpl<Tuple<String, String>> loader = new CoordinatorLoaderImpl<>( + Time.SYSTEM, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 + )) { + when(replicaManager.getLog(tp)).thenReturn(toScala(Optional.of(log))); + when(log.logStartOffset()).thenReturn(0L); + when(log.highWatermark()).thenReturn(0L, 0L, 2L); + when(replicaManager.getLogEndOffset(tp)).thenReturn(toScala(Optional.of(7L))); + + FetchDataInfo readResult1 = logReadResult(0, Arrays.asList( + new SimpleRecord("k1".getBytes(), "v1".getBytes()), + new SimpleRecord("k2".getBytes(), "v2".getBytes()) + )); + + when(log.read(0L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult1); + + FetchDataInfo readResult2 = logReadResult(2, Arrays.asList( + new SimpleRecord("k3".getBytes(), "v3".getBytes()), + new SimpleRecord("k4".getBytes(), "v4".getBytes()), + new SimpleRecord("k5".getBytes(), "v5".getBytes()) + )); + + when(log.read(2L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult2); + + FetchDataInfo readResult3 = logReadResult(5, Arrays.asList( + new SimpleRecord("k6".getBytes(), "v6".getBytes()), + new SimpleRecord("k7".getBytes(), "v7".getBytes()) + )); + + when(log.read(5L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult3); + + assertNotNull(loader.load(tp, coordinator).get(10, TimeUnit.SECONDS)); + + verify(coordinator).replay(0L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k1", "v1")); + verify(coordinator).replay(1L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k2", "v2")); + verify(coordinator).replay(2L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k3", "v3")); + verify(coordinator).replay(3L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k4", "v4")); + verify(coordinator).replay(4L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k5", "v5")); + verify(coordinator).replay(5L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k6", "v6")); + verify(coordinator).replay(6L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k7", "v7")); + verify(coordinator, times(0)).updateLastWrittenOffset(0L); + verify(coordinator, times(1)).updateLastWrittenOffset(2L); + verify(coordinator, times(1)).updateLastWrittenOffset(5L); + verify(coordinator, times(1)).updateLastWrittenOffset(7L); + verify(coordinator, times(1)).updateLastCommittedOffset(0L); + verify(coordinator, times(1)).updateLastCommittedOffset(2L); + verify(coordinator, times(0)).updateLastCommittedOffset(5L); + } + } + + @Test + void testUpdateLastWrittenOffsetAndUpdateLastCommittedOffsetNoRecordsRead() throws Exception { + TopicPartition tp = new TopicPartition("foo", 0); + ReplicaManager replicaManager = mock(ReplicaManager.class); + Function<TopicPartition, Optional<UnifiedLog>> partitionLogSupplier = partition -> toJava(replicaManager.getLog(partition)); + Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier = partition -> Optional.ofNullable((Long) replicaManager.getLogEndOffset(partition).get()); + StringKeyValueDeserializer serde = new StringKeyValueDeserializer(); + UnifiedLog log = mock(UnifiedLog.class); + CoordinatorPlayback<Tuple<String, String>> coordinator = mock(CoordinatorPlayback.class); + + try (CoordinatorLoaderImpl<Tuple<String, String>> loader = new CoordinatorLoaderImpl<>( + Time.SYSTEM, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 + )) { + when(replicaManager.getLog(tp)).thenReturn(toScala(Optional.of(log))); + when(log.logStartOffset()).thenReturn(0L); + when(log.highWatermark()).thenReturn(0L); + when(replicaManager.getLogEndOffset(tp)).thenReturn(toScala(Optional.of(0L))); + + assertNotNull(loader.load(tp, coordinator).get(10, TimeUnit.SECONDS)); + + verify(coordinator, times(0)).updateLastWrittenOffset(anyLong()); + verify(coordinator, times(0)).updateLastCommittedOffset(anyLong()); + } + } + + @Test + void testUpdateLastWrittenOffsetOnBatchLoadedWhileHighWatermarkAhead() throws Exception { + TopicPartition tp = new TopicPartition("foo", 0); + ReplicaManager replicaManager = mock(ReplicaManager.class); + Function<TopicPartition, Optional<UnifiedLog>> partitionLogSupplier = partition -> toJava(replicaManager.getLog(partition)); + Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier = partition -> Optional.ofNullable((Long) replicaManager.getLogEndOffset(partition).get()); + StringKeyValueDeserializer serde = new StringKeyValueDeserializer(); + UnifiedLog log = mock(UnifiedLog.class); + CoordinatorPlayback<Tuple<String, String>> coordinator = mock(CoordinatorPlayback.class); + + try (CoordinatorLoaderImpl<Tuple<String, String>> loader = new CoordinatorLoaderImpl<>( + Time.SYSTEM, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 + )) { + when(replicaManager.getLog(tp)).thenReturn(toScala(Optional.of(log))); + when(log.logStartOffset()).thenReturn(0L); + when(log.highWatermark()).thenReturn(5L, 7L, 7L); + when(replicaManager.getLogEndOffset(tp)).thenReturn(toScala(Optional.of(7L))); + + FetchDataInfo readResult1 = logReadResult(0, Arrays.asList( + new SimpleRecord("k1".getBytes(), "v1".getBytes()), + new SimpleRecord("k2".getBytes(), "v2".getBytes()) + )); + + when(log.read(0L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult1); + + FetchDataInfo readResult2 = logReadResult(2, Arrays.asList( + new SimpleRecord("k3".getBytes(), "v3".getBytes()), + new SimpleRecord("k4".getBytes(), "v4".getBytes()), + new SimpleRecord("k5".getBytes(), "v5".getBytes()) + )); + + when(log.read(2L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult2); + + FetchDataInfo readResult3 = logReadResult(5, Arrays.asList( + new SimpleRecord("k6".getBytes(), "v6".getBytes()), + new SimpleRecord("k7".getBytes(), "v7".getBytes()) + )); + + when(log.read(5L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult3); + + assertNotNull(loader.load(tp, coordinator).get(10, TimeUnit.SECONDS)); + + verify(coordinator).replay(0L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k1", "v1")); + verify(coordinator).replay(1L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k2", "v2")); + verify(coordinator).replay(2L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k3", "v3")); + verify(coordinator).replay(3L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k4", "v4")); + verify(coordinator).replay(4L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k5", "v5")); + verify(coordinator).replay(5L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k6", "v6")); + verify(coordinator).replay(6L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k7", "v7")); + verify(coordinator, times(0)).updateLastWrittenOffset(0L); + verify(coordinator, times(0)).updateLastWrittenOffset(2L); + verify(coordinator, times(0)).updateLastWrittenOffset(5L); + verify(coordinator, times(1)).updateLastWrittenOffset(7L); + verify(coordinator, times(0)).updateLastCommittedOffset(0L); + verify(coordinator, times(0)).updateLastCommittedOffset(2L); + verify(coordinator, times(0)).updateLastCommittedOffset(5L); + verify(coordinator, times(1)).updateLastCommittedOffset(7L); + } + } + + @Test + void testPartitionGoesOfflineDuringLoad() throws Exception { + TopicPartition tp = new TopicPartition("foo", 0); + ReplicaManager replicaManager = mock(ReplicaManager.class); + Function<TopicPartition, Optional<UnifiedLog>> partitionLogSupplier = partition -> toJava(replicaManager.getLog(partition)); + Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier = partition -> Optional.ofNullable((Long) replicaManager.getLogEndOffset(partition).get()); + StringKeyValueDeserializer serde = new StringKeyValueDeserializer(); + UnifiedLog log = mock(UnifiedLog.class); + CoordinatorPlayback<Tuple<String, String>> coordinator = mock(CoordinatorPlayback.class); + + try (CoordinatorLoaderImpl<Tuple<String, String>> loader = new CoordinatorLoaderImpl<>( + Time.SYSTEM, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 + )) { + when(replicaManager.getLog(tp)).thenReturn(toScala(Optional.of(log))); + when(log.logStartOffset()).thenReturn(0L); + when(log.highWatermark()).thenReturn(0L); + when(replicaManager.getLogEndOffset(tp)).thenReturn(toScala(Optional.of(5L))).thenReturn(toScala(Optional.of(-1L))); + + FetchDataInfo readResult1 = logReadResult(0, Arrays.asList( + new SimpleRecord("k1".getBytes(), "v1".getBytes()), + new SimpleRecord("k2".getBytes(), "v2".getBytes()) + )); + + when(log.read(0L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult1); + + FetchDataInfo readResult2 = logReadResult(2, Arrays.asList( + new SimpleRecord("k3".getBytes(), "v3".getBytes()), + new SimpleRecord("k4".getBytes(), "v4".getBytes()), + new SimpleRecord("k5".getBytes(), "v5".getBytes()) + )); + + when(log.read(2L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult2); + + assertFutureThrows(NotLeaderOrFollowerException.class, loader.load(tp, coordinator)); + } + } + + private FetchDataInfo logReadResult(long startOffset, List<SimpleRecord> records) throws IOException { + return logReadResult(startOffset, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, records); + } + + private FetchDataInfo logReadResult( + long startOffset, + long producerId, + short producerEpoch, + List<SimpleRecord> records + ) throws IOException { + FileRecords fileRecords = mock(FileRecords.class); + MemoryRecords memoryRecords; + if (producerId == RecordBatch.NO_PRODUCER_ID) { + memoryRecords = MemoryRecords.withRecords( + startOffset, + Compression.NONE, + records.toArray(new SimpleRecord[0]) + ); + } else { + memoryRecords = MemoryRecords.withTransactionalRecords( + startOffset, + Compression.NONE, + producerId, + producerEpoch, + 0, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + records.toArray(new SimpleRecord[0]) + ); + } + + when(fileRecords.sizeInBytes()).thenReturn(memoryRecords.sizeInBytes()); + + ArgumentCaptor<ByteBuffer> bufferCapture = ArgumentCaptor.forClass(ByteBuffer.class); Review Comment: ```java doAnswer(invocation -> { ByteBuffer buffer = invocation.getArgument(0); buffer.put(memoryRecords.buffer().duplicate()); buffer.flip(); return null; }).when(fileRecords).readInto(any(ByteBuffer.class), ArgumentMatchers.anyInt()); ``` ########## coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorLoaderImpl.java: ########## @@ -0,0 +1,338 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.coordinator.common.runtime; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.NotLeaderOrFollowerException; +import org.apache.kafka.common.record.ControlRecordType; +import org.apache.kafka.common.record.FileRecords; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.record.MutableRecordBatch; +import org.apache.kafka.common.record.Record; +import org.apache.kafka.common.record.Records; +import org.apache.kafka.common.requests.TransactionResult; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.server.storage.log.FetchIsolation; +import org.apache.kafka.server.util.KafkaScheduler; +import org.apache.kafka.storage.internals.log.FetchDataInfo; +import org.apache.kafka.storage.internals.log.UnifiedLog; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Function; + +/** + * Coordinator loader which reads records from a partition and replays them + * to a group coordinator. + * + * @param <T> The record type. + */ +public class CoordinatorLoaderImpl<T> implements CoordinatorLoader<T> { + + private static final Logger LOG = LoggerFactory.getLogger(CoordinatorLoaderImpl.class); + + private final Time time; + private final Function<TopicPartition, Optional<UnifiedLog>> partitionLogSupplier; + private final Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier; + private final Deserializer<T> deserializer; + private final int loadBufferSize; + + private final AtomicBoolean isRunning = new AtomicBoolean(true); + private final KafkaScheduler scheduler = new KafkaScheduler(1); + + public CoordinatorLoaderImpl( + Time time, + Function<TopicPartition, Optional<UnifiedLog>> partitionLogSupplier, + Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier, + Deserializer<T> deserializer, + int loadBufferSize + ) { + this.time = time; + this.partitionLogSupplier = partitionLogSupplier; + this.partitionLogEndOffsetSupplier = partitionLogEndOffsetSupplier; + this.deserializer = deserializer; + this.loadBufferSize = loadBufferSize; + this.scheduler.startup(); + } + + /** + * Loads the coordinator by reading all the records from the TopicPartition + * and applying them to the Replayable object. + * + * @param tp The TopicPartition to read from. + * @param coordinator The object to apply records to. + */ + @Override + public CompletableFuture<LoadSummary> load(TopicPartition tp, CoordinatorPlayback<T> coordinator) { + final CompletableFuture<LoadSummary> future = new CompletableFuture<>(); + long startTimeMs = time.milliseconds(); + try { + ScheduledFuture<?> result = scheduler.scheduleOnce(String.format("Load coordinator from %s", tp), + () -> doLoad(tp, coordinator, future, startTimeMs)); + if (result.isCancelled()) { + future.completeExceptionally(new RuntimeException("Coordinator loader is closed.")); + } + } catch (Exception e) { + future.completeExceptionally(e); + } + return future; + } + + private void doLoad( + TopicPartition tp, + CoordinatorPlayback<T> coordinator, + CompletableFuture<LoadSummary> future, + long startTimeMs + ) { + long schedulerQueueTimeMs = time.milliseconds() - startTimeMs; + try { + Optional<UnifiedLog> logOpt = partitionLogSupplier.apply(tp); + if (logOpt.isEmpty()) { + future.completeExceptionally(new NotLeaderOrFollowerException( + "Could not load records from " + tp + " because the log does not exist.")); + return; + } + + UnifiedLog log = logOpt.get(); + + // Buffer may not be needed if records are read from memory. + ByteBuffer buffer = ByteBuffer.allocate(0); + long currentOffset = log.logStartOffset(); + LoadStats stats = new LoadStats(); + + long previousHighWatermark = -1L; + while (shouldFetchNextBatch(currentOffset, logEndOffset(tp), stats.readAtLeastOneRecord)) { + FetchDataInfo fetchDataInfo = log.read(currentOffset, loadBufferSize, FetchIsolation.LOG_END, true); + + stats.readAtLeastOneRecord = fetchDataInfo.records.sizeInBytes() > 0; + + MemoryRecords memoryRecords = toReadableMemoryRecords(tp, fetchDataInfo.records, buffer); Review Comment: the current approach LGTM. could you please add comments? ########## coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorLoaderImplTest.java: ########## @@ -0,0 +1,706 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.coordinator.common.runtime; + +import kafka.server.ReplicaManager; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.compress.Compression; +import org.apache.kafka.common.errors.NotLeaderOrFollowerException; +import org.apache.kafka.common.record.ControlRecordType; +import org.apache.kafka.common.record.EndTransactionMarker; +import org.apache.kafka.common.record.FileRecords; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.record.RecordBatch; +import org.apache.kafka.common.record.SimpleRecord; +import org.apache.kafka.common.requests.TransactionResult; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.server.storage.log.FetchIsolation; +import org.apache.kafka.storage.internals.log.FetchDataInfo; +import org.apache.kafka.storage.internals.log.LogOffsetMetadata; +import org.apache.kafka.storage.internals.log.UnifiedLog; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.mockito.ArgumentCaptor; +import org.mockito.ArgumentMatchers; +import org.mockito.invocation.InvocationOnMock; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.function.Function; + +import scala.Option; + +import static org.apache.kafka.test.TestUtils.assertFutureThrows; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static scala.jdk.javaapi.OptionConverters.toJava; +import static scala.jdk.javaapi.OptionConverters.toScala; + +@SuppressWarnings("unchecked") +@Timeout(60) +class CoordinatorLoaderImplTest { + + private record Tuple<K, V>(K key, V value) { + } + + private static class StringKeyValueDeserializer implements Deserializer<Tuple<String, String>> { + + @Override + public Tuple<String, String> deserialize(ByteBuffer key, ByteBuffer value) throws RuntimeException { + return new Tuple<>( + StandardCharsets.UTF_8.decode(key).toString(), + StandardCharsets.UTF_8.decode(value).toString() + ); + } + } + + @Test + void testNonexistentPartition() throws Exception { + TopicPartition tp = new TopicPartition("foo", 0); + ReplicaManager replicaManager = mock(ReplicaManager.class); + Function<TopicPartition, Optional<UnifiedLog>> partitionLogSupplier = partition -> toJava(replicaManager.getLog(partition)); + Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier = partition -> Optional.ofNullable((Long) replicaManager.getLogEndOffset(partition).get()); + Deserializer<Tuple<String, String>> serde = mock(Deserializer.class); + CoordinatorPlayback<Tuple<String, String>> coordinator = mock(CoordinatorPlayback.class); + + try (CoordinatorLoaderImpl<Tuple<String, String>> loader = new CoordinatorLoaderImpl<>( + Time.SYSTEM, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 + )) { + when(replicaManager.getLog(tp)).thenReturn(Option.empty()); + + assertFutureThrows(NotLeaderOrFollowerException.class, loader.load(tp, coordinator)); + } + } + + @Test + void testLoadingIsRejectedWhenClosed() throws Exception { + TopicPartition tp = new TopicPartition("foo", 0); + ReplicaManager replicaManager = mock(ReplicaManager.class); + Function<TopicPartition, Optional<UnifiedLog>> partitionLogSupplier = partition -> toJava(replicaManager.getLog(partition)); + Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier = partition -> Optional.ofNullable((Long) replicaManager.getLogEndOffset(partition).get()); + Deserializer<Tuple<String, String>> serde = mock(Deserializer.class); + CoordinatorPlayback<Tuple<String, String>> coordinator = mock(CoordinatorPlayback.class); + + try (CoordinatorLoaderImpl<Tuple<String, String>> loader = new CoordinatorLoaderImpl<>( + Time.SYSTEM, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 + )) { + loader.close(); + assertFutureThrows(RuntimeException.class, loader.load(tp, coordinator)); + } + } + + @Test + void testLoading() throws Exception { + TopicPartition tp = new TopicPartition("foo", 0); + ReplicaManager replicaManager = mock(ReplicaManager.class); + Function<TopicPartition, Optional<UnifiedLog>> partitionLogSupplier = partition -> toJava(replicaManager.getLog(partition)); + Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier = partition -> Optional.ofNullable((Long) replicaManager.getLogEndOffset(partition).get()); + Deserializer<Tuple<String, String>> serde = new StringKeyValueDeserializer(); + UnifiedLog log = mock(UnifiedLog.class); + CoordinatorPlayback<Tuple<String, String>> coordinator = mock(CoordinatorPlayback.class); + + try (CoordinatorLoaderImpl<Tuple<String, String>> loader = new CoordinatorLoaderImpl<>( + Time.SYSTEM, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 + )) { + when(replicaManager.getLog(tp)).thenReturn(toScala(Optional.of(log))); + when(log.logStartOffset()).thenReturn(0L); + when(replicaManager.getLogEndOffset(tp)).thenReturn(toScala(Optional.of(9L))); + when(log.highWatermark()).thenReturn(0L); + + FetchDataInfo readResult1 = logReadResult(0, Arrays.asList( + new SimpleRecord("k1".getBytes(), "v1".getBytes()), + new SimpleRecord("k2".getBytes(), "v2".getBytes()) + )); + + when(log.read(0L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult1); + + FetchDataInfo readResult2 = logReadResult(2, Arrays.asList( + new SimpleRecord("k3".getBytes(), "v3".getBytes()), + new SimpleRecord("k4".getBytes(), "v4".getBytes()), + new SimpleRecord("k5".getBytes(), "v5".getBytes()) + )); + + when(log.read(2L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult2); + + FetchDataInfo readResult3 = logReadResult(5, 100L, (short) 5, Arrays.asList( + new SimpleRecord("k6".getBytes(), "v6".getBytes()), + new SimpleRecord("k7".getBytes(), "v7".getBytes()) + )); + + when(log.read(5L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult3); + + FetchDataInfo readResult4 = logReadResult( + 7, + 100L, + (short) 5, + ControlRecordType.COMMIT + ); + + when(log.read(7L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult4); + + FetchDataInfo readResult5 = logReadResult( + 8, + 500L, + (short) 10, + ControlRecordType.ABORT + ); + + when(log.read(8L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult5); + + assertNotNull(loader.load(tp, coordinator).get(10, TimeUnit.SECONDS)); + + verify(coordinator).replay(0L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k1", "v1")); + verify(coordinator).replay(1L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k2", "v2")); + verify(coordinator).replay(2L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k3", "v3")); + verify(coordinator).replay(3L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k4", "v4")); + verify(coordinator).replay(4L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k5", "v5")); + verify(coordinator).replay(5L, 100L, (short) 5, new Tuple<>("k6", "v6")); + verify(coordinator).replay(6L, 100L, (short) 5, new Tuple<>("k7", "v7")); + verify(coordinator).replayEndTransactionMarker(100L, (short) 5, TransactionResult.COMMIT); + verify(coordinator).replayEndTransactionMarker(500L, (short) 10, TransactionResult.ABORT); + verify(coordinator).updateLastWrittenOffset(2L); + verify(coordinator).updateLastWrittenOffset(5L); + verify(coordinator).updateLastWrittenOffset(7L); + verify(coordinator).updateLastWrittenOffset(8L); + verify(coordinator).updateLastCommittedOffset(0L); + } + } + + @Test + void testLoadingStoppedWhenClosed() throws Exception { + TopicPartition tp = new TopicPartition("foo", 0); + ReplicaManager replicaManager = mock(ReplicaManager.class); + Function<TopicPartition, Optional<UnifiedLog>> partitionLogSupplier = partition -> toJava(replicaManager.getLog(partition)); + Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier = partition -> Optional.ofNullable((Long) replicaManager.getLogEndOffset(partition).get()); + Deserializer<Tuple<String, String>> serde = new StringKeyValueDeserializer(); + UnifiedLog log = mock(UnifiedLog.class); + CoordinatorPlayback<Tuple<String, String>> coordinator = mock(CoordinatorPlayback.class); + + try (CoordinatorLoaderImpl<Tuple<String, String>> loader = new CoordinatorLoaderImpl<>( + Time.SYSTEM, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 + )) { + when(replicaManager.getLog(tp)).thenReturn(toScala(Optional.of(log))); + when(log.logStartOffset()).thenReturn(0L); + when(replicaManager.getLogEndOffset(tp)).thenReturn(toScala(Optional.of(100L))); + + FetchDataInfo readResult = logReadResult(0, Arrays.asList( + new SimpleRecord("k1".getBytes(), "v1".getBytes()), + new SimpleRecord("k2".getBytes(), "v2".getBytes()) + )); + + CountDownLatch latch = new CountDownLatch(1); + when(log.read( + anyLong(), + eq(1000), + eq(FetchIsolation.LOG_END), + eq(true) + )).thenAnswer((InvocationOnMock invocation) -> { + latch.countDown(); + return readResult; + }); + + CompletableFuture<CoordinatorLoader.LoadSummary> result = loader.load(tp, coordinator); + boolean completed = latch.await(10, TimeUnit.SECONDS); + assertTrue(completed, "Log read timeout: Latch did not count down in time."); + loader.close(); + + RuntimeException ex = assertFutureThrows(RuntimeException.class, result); + assertNotNull(ex); + assertEquals("Coordinator loader is closed.", ex.getMessage()); + } + } + + @Test + void testUnknownRecordTypeAreIgnored() throws Exception { + TopicPartition tp = new TopicPartition("foo", 0); + ReplicaManager replicaManager = mock(ReplicaManager.class); + Function<TopicPartition, Optional<UnifiedLog>> partitionLogSupplier = partition -> toJava(replicaManager.getLog(partition)); + Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier = partition -> Optional.ofNullable((Long) replicaManager.getLogEndOffset(partition).get()); + StringKeyValueDeserializer serde = mock(StringKeyValueDeserializer.class); + UnifiedLog log = mock(UnifiedLog.class); + CoordinatorPlayback<Tuple<String, String>> coordinator = mock(CoordinatorPlayback.class); + + try (CoordinatorLoaderImpl<Tuple<String, String>> loader = new CoordinatorLoaderImpl<>( + Time.SYSTEM, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 + )) { + when(replicaManager.getLog(tp)).thenReturn(toScala(Optional.of(log))); + when(log.logStartOffset()).thenReturn(0L); + when(replicaManager.getLogEndOffset(tp)).thenReturn(toScala(Optional.of(2L))); + + FetchDataInfo readResult = logReadResult(0, Arrays.asList( + new SimpleRecord("k1".getBytes(), "v1".getBytes()), + new SimpleRecord("k2".getBytes(), "v2".getBytes()) + )); + + when(log.read(0L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult); + + when(serde.deserialize(any(ByteBuffer.class), any(ByteBuffer.class))) + .thenThrow(new Deserializer.UnknownRecordTypeException((short) 1)) + .thenReturn(new Tuple<>("k2", "v2")); + + loader.load(tp, coordinator).get(10, TimeUnit.SECONDS); + + verify(coordinator).replay(1L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k2", "v2")); + } + } + + @Test + void testDeserializationErrorFailsTheLoading() throws Exception { + TopicPartition tp = new TopicPartition("foo", 0); + ReplicaManager replicaManager = mock(ReplicaManager.class); + Function<TopicPartition, Optional<UnifiedLog>> partitionLogSupplier = partition -> toJava(replicaManager.getLog(partition)); + Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier = partition -> Optional.ofNullable((Long) replicaManager.getLogEndOffset(partition).get()); + StringKeyValueDeserializer serde = mock(StringKeyValueDeserializer.class); + UnifiedLog log = mock(UnifiedLog.class); + CoordinatorPlayback<Tuple<String, String>> coordinator = mock(CoordinatorPlayback.class); + + try (CoordinatorLoaderImpl<Tuple<String, String>> loader = new CoordinatorLoaderImpl<>( + Time.SYSTEM, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 + )) { + when(replicaManager.getLog(tp)).thenReturn(toScala(Optional.of(log))); + when(log.logStartOffset()).thenReturn(0L); + when(replicaManager.getLogEndOffset(tp)).thenReturn(toScala(Optional.of(2L))); + + FetchDataInfo readResult = logReadResult(0, Arrays.asList( + new SimpleRecord("k1".getBytes(), "v1".getBytes()), + new SimpleRecord("k2".getBytes(), "v2".getBytes()) + )); + + when(log.read(0L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult); + + when(serde.deserialize(any(ByteBuffer.class), any(ByteBuffer.class))) + .thenThrow(new RuntimeException("Error!")); + + RuntimeException ex = assertFutureThrows(RuntimeException.class, loader.load(tp, coordinator)); + + assertNotNull(ex); + assertEquals(String.format("Deserializing record DefaultRecord(offset=0, timestamp=-1, key=2 bytes, value=2 bytes) from %s failed due to: Error!", tp), ex.getMessage()); + } + } + + @Test + void testLoadGroupAndOffsetsWithCorruptedLog() throws Exception { + // Simulate a case where startOffset < endOffset but log is empty. This could theoretically happen + // when all the records are expired and the active segment is truncated or when the partition + // is accidentally corrupted. + TopicPartition tp = new TopicPartition("foo", 0); + ReplicaManager replicaManager = mock(ReplicaManager.class); + Function<TopicPartition, Optional<UnifiedLog>> partitionLogSupplier = partition -> toJava(replicaManager.getLog(partition)); + Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier = partition -> Optional.ofNullable((Long) replicaManager.getLogEndOffset(partition).get()); + StringKeyValueDeserializer serde = mock(StringKeyValueDeserializer.class); + UnifiedLog log = mock(UnifiedLog.class); + CoordinatorPlayback<Tuple<String, String>> coordinator = mock(CoordinatorPlayback.class); + + try (CoordinatorLoaderImpl<Tuple<String, String>> loader = new CoordinatorLoaderImpl<>( + Time.SYSTEM, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 + )) { + when(replicaManager.getLog(tp)).thenReturn(toScala(Optional.of(log))); + when(log.logStartOffset()).thenReturn(0L); + when(replicaManager.getLogEndOffset(tp)).thenReturn(toScala(Optional.of(10L))); + + FetchDataInfo readResult = logReadResult(0, List.of()); + + when(log.read(0L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult); + + assertNotNull(loader.load(tp, coordinator).get(10, TimeUnit.SECONDS)); + } + } + + @Test + void testLoadSummary() throws Exception { + TopicPartition tp = new TopicPartition("foo", 0); + ReplicaManager replicaManager = mock(ReplicaManager.class); + Function<TopicPartition, Optional<UnifiedLog>> partitionLogSupplier = partition -> toJava(replicaManager.getLog(partition)); + Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier = partition -> Optional.ofNullable((Long) replicaManager.getLogEndOffset(partition).get()); + StringKeyValueDeserializer serde = new StringKeyValueDeserializer(); + UnifiedLog log = mock(UnifiedLog.class); + CoordinatorPlayback<Tuple<String, String>> coordinator = mock(CoordinatorPlayback.class); + MockTime time = new MockTime(); + + try (CoordinatorLoaderImpl<Tuple<String, String>> loader = new CoordinatorLoaderImpl<>( + time, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 + )) { + long startTimeMs = time.milliseconds(); + when(replicaManager.getLog(tp)).thenReturn(toScala(Optional.of(log))); + when(log.logStartOffset()).thenReturn(0L); + when(replicaManager.getLogEndOffset(tp)).thenReturn(toScala(Optional.of(5L))); + + FetchDataInfo readResult1 = logReadResult(0, Arrays.asList( + new SimpleRecord("k1".getBytes(), "v1".getBytes()), + new SimpleRecord("k2".getBytes(), "v2".getBytes()) + )); + + when(log.read(0L, 1000, FetchIsolation.LOG_END, true)) + .thenAnswer((InvocationOnMock invocation) -> { + time.sleep(1000); + return readResult1; + }); + + FetchDataInfo readResult2 = logReadResult(2, Arrays.asList( + new SimpleRecord("k3".getBytes(), "v3".getBytes()), + new SimpleRecord("k4".getBytes(), "v4".getBytes()), + new SimpleRecord("k5".getBytes(), "v5".getBytes()) + )); + + when(log.read(2L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult2); + + CoordinatorLoader.LoadSummary summary = loader.load(tp, coordinator).get(10, TimeUnit.SECONDS); + assertEquals(startTimeMs, summary.startTimeMs()); + assertEquals(startTimeMs + 1000, summary.endTimeMs()); + assertEquals(5, summary.numRecords()); + assertEquals(readResult1.records.sizeInBytes() + readResult2.records.sizeInBytes(), summary.numBytes()); + } + } + + @Test + void testUpdateLastWrittenOffsetOnBatchLoaded() throws Exception { + TopicPartition tp = new TopicPartition("foo", 0); + ReplicaManager replicaManager = mock(ReplicaManager.class); + Function<TopicPartition, Optional<UnifiedLog>> partitionLogSupplier = partition -> toJava(replicaManager.getLog(partition)); + Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier = partition -> Optional.ofNullable((Long) replicaManager.getLogEndOffset(partition).get()); + StringKeyValueDeserializer serde = new StringKeyValueDeserializer(); + UnifiedLog log = mock(UnifiedLog.class); + CoordinatorPlayback<Tuple<String, String>> coordinator = mock(CoordinatorPlayback.class); + + try (CoordinatorLoaderImpl<Tuple<String, String>> loader = new CoordinatorLoaderImpl<>( + Time.SYSTEM, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 + )) { + when(replicaManager.getLog(tp)).thenReturn(toScala(Optional.of(log))); + when(log.logStartOffset()).thenReturn(0L); + when(log.highWatermark()).thenReturn(0L, 0L, 2L); + when(replicaManager.getLogEndOffset(tp)).thenReturn(toScala(Optional.of(7L))); + + FetchDataInfo readResult1 = logReadResult(0, Arrays.asList( + new SimpleRecord("k1".getBytes(), "v1".getBytes()), + new SimpleRecord("k2".getBytes(), "v2".getBytes()) + )); + + when(log.read(0L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult1); + + FetchDataInfo readResult2 = logReadResult(2, Arrays.asList( + new SimpleRecord("k3".getBytes(), "v3".getBytes()), + new SimpleRecord("k4".getBytes(), "v4".getBytes()), + new SimpleRecord("k5".getBytes(), "v5".getBytes()) + )); + + when(log.read(2L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult2); + + FetchDataInfo readResult3 = logReadResult(5, Arrays.asList( + new SimpleRecord("k6".getBytes(), "v6".getBytes()), + new SimpleRecord("k7".getBytes(), "v7".getBytes()) + )); + + when(log.read(5L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult3); + + assertNotNull(loader.load(tp, coordinator).get(10, TimeUnit.SECONDS)); + + verify(coordinator).replay(0L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k1", "v1")); + verify(coordinator).replay(1L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k2", "v2")); + verify(coordinator).replay(2L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k3", "v3")); + verify(coordinator).replay(3L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k4", "v4")); + verify(coordinator).replay(4L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k5", "v5")); + verify(coordinator).replay(5L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k6", "v6")); + verify(coordinator).replay(6L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k7", "v7")); + verify(coordinator, times(0)).updateLastWrittenOffset(0L); + verify(coordinator, times(1)).updateLastWrittenOffset(2L); + verify(coordinator, times(1)).updateLastWrittenOffset(5L); + verify(coordinator, times(1)).updateLastWrittenOffset(7L); + verify(coordinator, times(1)).updateLastCommittedOffset(0L); + verify(coordinator, times(1)).updateLastCommittedOffset(2L); + verify(coordinator, times(0)).updateLastCommittedOffset(5L); + } + } + + @Test + void testUpdateLastWrittenOffsetAndUpdateLastCommittedOffsetNoRecordsRead() throws Exception { + TopicPartition tp = new TopicPartition("foo", 0); + ReplicaManager replicaManager = mock(ReplicaManager.class); + Function<TopicPartition, Optional<UnifiedLog>> partitionLogSupplier = partition -> toJava(replicaManager.getLog(partition)); + Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier = partition -> Optional.ofNullable((Long) replicaManager.getLogEndOffset(partition).get()); + StringKeyValueDeserializer serde = new StringKeyValueDeserializer(); + UnifiedLog log = mock(UnifiedLog.class); + CoordinatorPlayback<Tuple<String, String>> coordinator = mock(CoordinatorPlayback.class); + + try (CoordinatorLoaderImpl<Tuple<String, String>> loader = new CoordinatorLoaderImpl<>( + Time.SYSTEM, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 + )) { + when(replicaManager.getLog(tp)).thenReturn(toScala(Optional.of(log))); + when(log.logStartOffset()).thenReturn(0L); + when(log.highWatermark()).thenReturn(0L); + when(replicaManager.getLogEndOffset(tp)).thenReturn(toScala(Optional.of(0L))); + + assertNotNull(loader.load(tp, coordinator).get(10, TimeUnit.SECONDS)); + + verify(coordinator, times(0)).updateLastWrittenOffset(anyLong()); + verify(coordinator, times(0)).updateLastCommittedOffset(anyLong()); + } + } + + @Test + void testUpdateLastWrittenOffsetOnBatchLoadedWhileHighWatermarkAhead() throws Exception { + TopicPartition tp = new TopicPartition("foo", 0); + ReplicaManager replicaManager = mock(ReplicaManager.class); + Function<TopicPartition, Optional<UnifiedLog>> partitionLogSupplier = partition -> toJava(replicaManager.getLog(partition)); + Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier = partition -> Optional.ofNullable((Long) replicaManager.getLogEndOffset(partition).get()); + StringKeyValueDeserializer serde = new StringKeyValueDeserializer(); + UnifiedLog log = mock(UnifiedLog.class); + CoordinatorPlayback<Tuple<String, String>> coordinator = mock(CoordinatorPlayback.class); + + try (CoordinatorLoaderImpl<Tuple<String, String>> loader = new CoordinatorLoaderImpl<>( + Time.SYSTEM, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 + )) { + when(replicaManager.getLog(tp)).thenReturn(toScala(Optional.of(log))); + when(log.logStartOffset()).thenReturn(0L); + when(log.highWatermark()).thenReturn(5L, 7L, 7L); + when(replicaManager.getLogEndOffset(tp)).thenReturn(toScala(Optional.of(7L))); + + FetchDataInfo readResult1 = logReadResult(0, Arrays.asList( + new SimpleRecord("k1".getBytes(), "v1".getBytes()), + new SimpleRecord("k2".getBytes(), "v2".getBytes()) + )); + + when(log.read(0L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult1); + + FetchDataInfo readResult2 = logReadResult(2, Arrays.asList( + new SimpleRecord("k3".getBytes(), "v3".getBytes()), + new SimpleRecord("k4".getBytes(), "v4".getBytes()), + new SimpleRecord("k5".getBytes(), "v5".getBytes()) + )); + + when(log.read(2L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult2); + + FetchDataInfo readResult3 = logReadResult(5, Arrays.asList( + new SimpleRecord("k6".getBytes(), "v6".getBytes()), + new SimpleRecord("k7".getBytes(), "v7".getBytes()) + )); + + when(log.read(5L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult3); + + assertNotNull(loader.load(tp, coordinator).get(10, TimeUnit.SECONDS)); + + verify(coordinator).replay(0L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k1", "v1")); + verify(coordinator).replay(1L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k2", "v2")); + verify(coordinator).replay(2L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k3", "v3")); + verify(coordinator).replay(3L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k4", "v4")); + verify(coordinator).replay(4L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k5", "v5")); + verify(coordinator).replay(5L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k6", "v6")); + verify(coordinator).replay(6L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, new Tuple<>("k7", "v7")); + verify(coordinator, times(0)).updateLastWrittenOffset(0L); + verify(coordinator, times(0)).updateLastWrittenOffset(2L); + verify(coordinator, times(0)).updateLastWrittenOffset(5L); + verify(coordinator, times(1)).updateLastWrittenOffset(7L); + verify(coordinator, times(0)).updateLastCommittedOffset(0L); + verify(coordinator, times(0)).updateLastCommittedOffset(2L); + verify(coordinator, times(0)).updateLastCommittedOffset(5L); + verify(coordinator, times(1)).updateLastCommittedOffset(7L); + } + } + + @Test + void testPartitionGoesOfflineDuringLoad() throws Exception { + TopicPartition tp = new TopicPartition("foo", 0); + ReplicaManager replicaManager = mock(ReplicaManager.class); + Function<TopicPartition, Optional<UnifiedLog>> partitionLogSupplier = partition -> toJava(replicaManager.getLog(partition)); + Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier = partition -> Optional.ofNullable((Long) replicaManager.getLogEndOffset(partition).get()); + StringKeyValueDeserializer serde = new StringKeyValueDeserializer(); + UnifiedLog log = mock(UnifiedLog.class); + CoordinatorPlayback<Tuple<String, String>> coordinator = mock(CoordinatorPlayback.class); + + try (CoordinatorLoaderImpl<Tuple<String, String>> loader = new CoordinatorLoaderImpl<>( + Time.SYSTEM, + partitionLogSupplier, + partitionLogEndOffsetSupplier, + serde, + 1000 + )) { + when(replicaManager.getLog(tp)).thenReturn(toScala(Optional.of(log))); + when(log.logStartOffset()).thenReturn(0L); + when(log.highWatermark()).thenReturn(0L); + when(replicaManager.getLogEndOffset(tp)).thenReturn(toScala(Optional.of(5L))).thenReturn(toScala(Optional.of(-1L))); + + FetchDataInfo readResult1 = logReadResult(0, Arrays.asList( + new SimpleRecord("k1".getBytes(), "v1".getBytes()), + new SimpleRecord("k2".getBytes(), "v2".getBytes()) + )); + + when(log.read(0L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult1); + + FetchDataInfo readResult2 = logReadResult(2, Arrays.asList( + new SimpleRecord("k3".getBytes(), "v3".getBytes()), + new SimpleRecord("k4".getBytes(), "v4".getBytes()), + new SimpleRecord("k5".getBytes(), "v5".getBytes()) + )); + + when(log.read(2L, 1000, FetchIsolation.LOG_END, true)) + .thenReturn(readResult2); + + assertFutureThrows(NotLeaderOrFollowerException.class, loader.load(tp, coordinator)); + } + } + + private FetchDataInfo logReadResult(long startOffset, List<SimpleRecord> records) throws IOException { + return logReadResult(startOffset, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, records); + } + + private FetchDataInfo logReadResult( + long startOffset, + long producerId, + short producerEpoch, + List<SimpleRecord> records + ) throws IOException { + FileRecords fileRecords = mock(FileRecords.class); + MemoryRecords memoryRecords; + if (producerId == RecordBatch.NO_PRODUCER_ID) { + memoryRecords = MemoryRecords.withRecords( + startOffset, + Compression.NONE, + records.toArray(new SimpleRecord[0]) + ); + } else { + memoryRecords = MemoryRecords.withTransactionalRecords( + startOffset, + Compression.NONE, + producerId, + producerEpoch, + 0, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + records.toArray(new SimpleRecord[0]) + ); + } + + when(fileRecords.sizeInBytes()).thenReturn(memoryRecords.sizeInBytes()); + + ArgumentCaptor<ByteBuffer> bufferCapture = ArgumentCaptor.forClass(ByteBuffer.class); + doAnswer(invocation -> { + ByteBuffer buffer = bufferCapture.getValue(); + buffer.put(memoryRecords.buffer().duplicate()); + buffer.flip(); + return null; + }).when(fileRecords).readInto(bufferCapture.capture(), ArgumentMatchers.anyInt()); + + return new FetchDataInfo(new LogOffsetMetadata(startOffset), fileRecords); + } + + private FetchDataInfo logReadResult( + long startOffset, + long producerId, + short producerEpoch, + ControlRecordType controlRecordType + ) throws IOException { + FileRecords fileRecords = mock(FileRecords.class); + MemoryRecords memoryRecords = MemoryRecords.withEndTransactionMarker( + startOffset, + 0L, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + producerId, + producerEpoch, + new EndTransactionMarker(controlRecordType, 0) + ); + + when(fileRecords.sizeInBytes()).thenReturn(memoryRecords.sizeInBytes()); + + ArgumentCaptor<ByteBuffer> bufferCapture = ArgumentCaptor.forClass(ByteBuffer.class); Review Comment: ditto ########## coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorLoaderImpl.java: ########## @@ -0,0 +1,330 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.coordinator.common.runtime; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.NotLeaderOrFollowerException; +import org.apache.kafka.common.record.ControlRecordType; +import org.apache.kafka.common.record.FileRecords; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.record.MutableRecordBatch; +import org.apache.kafka.common.record.Record; +import org.apache.kafka.common.record.Records; +import org.apache.kafka.common.requests.TransactionResult; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.server.storage.log.FetchIsolation; +import org.apache.kafka.server.util.KafkaScheduler; +import org.apache.kafka.storage.internals.log.FetchDataInfo; +import org.apache.kafka.storage.internals.log.UnifiedLog; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Function; + +/** + * Coordinator loader which reads records from a partition and replays them + * to a group coordinator. + * + * @param <T> The record type. + */ +public class CoordinatorLoaderImpl<T> implements CoordinatorLoader<T> { + + private static final Logger LOG = LoggerFactory.getLogger(CoordinatorLoaderImpl.class); + + private final Time time; + private final Function<TopicPartition, Optional<UnifiedLog>> partitionLogSupplier; + private final Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier; + private final Deserializer<T> deserializer; + private final int loadBufferSize; + + private final AtomicBoolean isRunning = new AtomicBoolean(true); + private final KafkaScheduler scheduler = new KafkaScheduler(1); + + public CoordinatorLoaderImpl( + Time time, + Function<TopicPartition, Optional<UnifiedLog>> partitionLogSupplier, + Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier, + Deserializer<T> deserializer, + int loadBufferSize + ) { + this.time = time; + this.partitionLogSupplier = partitionLogSupplier; + this.partitionLogEndOffsetSupplier = partitionLogEndOffsetSupplier; + this.deserializer = deserializer; + this.loadBufferSize = loadBufferSize; + this.scheduler.startup(); + } + + /** + * Loads the coordinator by reading all the records from the TopicPartition + * and applying them to the Replayable object. + * + * @param tp The TopicPartition to read from. + * @param coordinator The object to apply records to. + */ + @Override + public CompletableFuture<LoadSummary> load(TopicPartition tp, CoordinatorPlayback<T> coordinator) { + final CompletableFuture<LoadSummary> future = new CompletableFuture<>(); + long startTimeMs = time.milliseconds(); + try { + ScheduledFuture<?> result = scheduler.scheduleOnce(String.format("Load coordinator from %s", tp), + () -> doLoad(tp, coordinator, future, startTimeMs)); + if (result.isCancelled()) { + future.completeExceptionally(new RuntimeException("Coordinator loader is closed.")); + } + } catch (Exception e) { + future.completeExceptionally(e); + } + return future; + } + + private void doLoad( + TopicPartition tp, + CoordinatorPlayback<T> coordinator, + CompletableFuture<LoadSummary> future, + long startTimeMs + ) { + long schedulerQueueTimeMs = time.milliseconds() - startTimeMs; + try { + Optional<UnifiedLog> logOpt = partitionLogSupplier.apply(tp); + if (logOpt.isEmpty()) { + future.completeExceptionally(new NotLeaderOrFollowerException( + "Could not load records from " + tp + " because the log does not exist.")); + return; + } + + UnifiedLog log = logOpt.get(); + + // Buffer may not be needed if records are read from memory. + ByteBuffer buffer = ByteBuffer.allocate(0); + long currentOffset = log.logStartOffset(); + LoadStats stats = new LoadStats(); + + long previousHighWatermark = -1L; + while (shouldFetchNextBatch(currentOffset, logEndOffset(tp), stats.readAtLeastOneRecord)) { + FetchDataInfo fetchDataInfo = log.read(currentOffset, loadBufferSize, FetchIsolation.LOG_END, true); + + stats.readAtLeastOneRecord = fetchDataInfo.records.sizeInBytes() > 0; + + MemoryRecords memoryRecords = toReadableMemoryRecords(tp, fetchDataInfo.records, buffer); + if (fetchDataInfo.records instanceof FileRecords) { + buffer = memoryRecords.buffer(); + } + + ReplayResult replayResult = processMemoryRecords(tp, log, memoryRecords, coordinator, stats, currentOffset, previousHighWatermark); + currentOffset = replayResult.nextOffset; + previousHighWatermark = replayResult.highWatermark; + } + + long endTimeMs = time.milliseconds(); + + if (logEndOffset(tp) == -1L) { + future.completeExceptionally(new NotLeaderOrFollowerException( + String.format("Stopped loading records from %s because the partition is not online or is no longer the leader.", tp) + )); + } else if (isRunning.get()) { + future.complete(new LoadSummary(startTimeMs, endTimeMs, schedulerQueueTimeMs, stats.numRecords, stats.numBytes)); + } else { + future.completeExceptionally(new RuntimeException("Coordinator loader is closed.")); + } + } catch (Throwable ex) { + future.completeExceptionally(ex); + } + } + + private long logEndOffset(TopicPartition tp) { + return partitionLogEndOffsetSupplier.apply(tp).orElse(-1L); + } + + /** + * Returns true if it's still valid to fetch the next batch of records. + * <p> + * This method ensures fetching continues only under safe and meaningful conditions: + * <ul> + * <li>The current offset is less than the log end offset.</li> + * <li>At least one record was read in the previous fetch. This ensures that fetching stops even if the + * current offset remains smaller than the log end offset but the log is empty. This could happen with compacted topics.</li> + * <li>The log end offset is not -1L, which ensures the partition is online and is still the leader.</li> + * <li>The loader is still running.</li> + * </ul> + */ + private boolean shouldFetchNextBatch(long currentOffset, long logEndOffset, boolean readAtLeastOneRecord) { + return currentOffset < logEndOffset && readAtLeastOneRecord && isRunning.get(); + } + + private MemoryRecords toReadableMemoryRecords(TopicPartition tp, Records records, ByteBuffer buffer) throws IOException { + if (records instanceof MemoryRecords memoryRecords) { + return memoryRecords; + } else if (records instanceof FileRecords fileRecords) { + int sizeInBytes = fileRecords.sizeInBytes(); + int bytesNeeded = Math.max(loadBufferSize, sizeInBytes); + + // "minOneMessage = true in the above log.read() means that the buffer may need to + // be grown to ensure progress can be made. + if (buffer.capacity() < bytesNeeded) { + if (loadBufferSize < bytesNeeded) { + LOG.warn("Loaded metadata from {} with buffer larger ({} bytes) than" + + " configured buffer size ({} bytes).", tp, bytesNeeded, loadBufferSize); + } + + buffer = ByteBuffer.allocate(bytesNeeded); + } else { + buffer.clear(); + } + + fileRecords.readInto(buffer, 0); + return MemoryRecords.readableRecords(buffer); + } else { + throw new IllegalArgumentException("Unsupported record type: " + records.getClass()); + } + } + + private ReplayResult processMemoryRecords( + TopicPartition tp, + UnifiedLog log, + MemoryRecords memoryRecords, + CoordinatorPlayback<T> coordinator, + LoadStats loadStats, + long currentOffset, + long previousHighWatermark + ) { + + for (MutableRecordBatch batch : memoryRecords.batches()) { + if (batch.isControlBatch()) { + for (Record record : batch) { + ControlRecordType controlRecord = ControlRecordType.parse(record.key()); + if (controlRecord == ControlRecordType.COMMIT) { + if (LOG.isTraceEnabled()) { + LOG.trace("Replaying end transaction marker from {} at offset {} to commit" + + " transaction with producer id {} and producer epoch {}.", + tp, record.offset(), batch.producerId(), batch.producerEpoch()); + } + coordinator.replayEndTransactionMarker( + batch.producerId(), + batch.producerEpoch(), + TransactionResult.COMMIT + ); + } else if (controlRecord == ControlRecordType.ABORT) { + if (LOG.isTraceEnabled()) { + LOG.trace("Replaying end transaction marker from {} at offset {} to abort" + + " transaction with producer id {} and producer epoch {}.", + tp, record.offset(), batch.producerId(), batch.producerEpoch()); + } + coordinator.replayEndTransactionMarker( + batch.producerId(), + batch.producerEpoch(), + TransactionResult.ABORT + ); + } + } + } else { + for (Record record : batch) { + loadStats.numRecords++; + + Optional<T> coordinatorRecordOpt = Optional.empty(); + try { + coordinatorRecordOpt = Optional.ofNullable(deserializer.deserialize(record.key(), record.value())); + } catch (Deserializer.UnknownRecordTypeException ex) { + LOG.warn("Unknown record type {} while loading offsets and group metadata from {}." + + " Ignoring it. It could be a left over from an aborted upgrade.", ex.unknownType(), tp); + } catch (RuntimeException ex) { + String msg = String.format("Deserializing record %s from %s failed due to: %s", record, tp, ex.getMessage()); + LOG.error(msg); + throw new RuntimeException(msg, ex); Review Comment: `LOG.error` could accept exception directly. Perhaps we could replace `due to: %s` by `LOG.error(msg, ex)`? ########## coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorLoaderImpl.java: ########## @@ -0,0 +1,330 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.coordinator.common.runtime; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.NotLeaderOrFollowerException; +import org.apache.kafka.common.record.ControlRecordType; +import org.apache.kafka.common.record.FileRecords; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.record.MutableRecordBatch; +import org.apache.kafka.common.record.Record; +import org.apache.kafka.common.record.Records; +import org.apache.kafka.common.requests.TransactionResult; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.server.storage.log.FetchIsolation; +import org.apache.kafka.server.util.KafkaScheduler; +import org.apache.kafka.storage.internals.log.FetchDataInfo; +import org.apache.kafka.storage.internals.log.UnifiedLog; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Function; + +/** + * Coordinator loader which reads records from a partition and replays them + * to a group coordinator. + * + * @param <T> The record type. + */ +public class CoordinatorLoaderImpl<T> implements CoordinatorLoader<T> { + + private static final Logger LOG = LoggerFactory.getLogger(CoordinatorLoaderImpl.class); + + private final Time time; + private final Function<TopicPartition, Optional<UnifiedLog>> partitionLogSupplier; + private final Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier; + private final Deserializer<T> deserializer; + private final int loadBufferSize; + + private final AtomicBoolean isRunning = new AtomicBoolean(true); + private final KafkaScheduler scheduler = new KafkaScheduler(1); + + public CoordinatorLoaderImpl( + Time time, + Function<TopicPartition, Optional<UnifiedLog>> partitionLogSupplier, + Function<TopicPartition, Optional<Long>> partitionLogEndOffsetSupplier, + Deserializer<T> deserializer, + int loadBufferSize + ) { + this.time = time; + this.partitionLogSupplier = partitionLogSupplier; + this.partitionLogEndOffsetSupplier = partitionLogEndOffsetSupplier; + this.deserializer = deserializer; + this.loadBufferSize = loadBufferSize; + this.scheduler.startup(); + } + + /** + * Loads the coordinator by reading all the records from the TopicPartition + * and applying them to the Replayable object. + * + * @param tp The TopicPartition to read from. + * @param coordinator The object to apply records to. + */ + @Override + public CompletableFuture<LoadSummary> load(TopicPartition tp, CoordinatorPlayback<T> coordinator) { + final CompletableFuture<LoadSummary> future = new CompletableFuture<>(); + long startTimeMs = time.milliseconds(); + try { + ScheduledFuture<?> result = scheduler.scheduleOnce(String.format("Load coordinator from %s", tp), + () -> doLoad(tp, coordinator, future, startTimeMs)); + if (result.isCancelled()) { + future.completeExceptionally(new RuntimeException("Coordinator loader is closed.")); + } + } catch (Exception e) { + future.completeExceptionally(e); + } + return future; + } + + private void doLoad( + TopicPartition tp, + CoordinatorPlayback<T> coordinator, + CompletableFuture<LoadSummary> future, + long startTimeMs + ) { + long schedulerQueueTimeMs = time.milliseconds() - startTimeMs; + try { + Optional<UnifiedLog> logOpt = partitionLogSupplier.apply(tp); + if (logOpt.isEmpty()) { + future.completeExceptionally(new NotLeaderOrFollowerException( + "Could not load records from " + tp + " because the log does not exist.")); + return; + } + + UnifiedLog log = logOpt.get(); + + // Buffer may not be needed if records are read from memory. + ByteBuffer buffer = ByteBuffer.allocate(0); + long currentOffset = log.logStartOffset(); + LoadStats stats = new LoadStats(); + + long previousHighWatermark = -1L; + while (shouldFetchNextBatch(currentOffset, logEndOffset(tp), stats.readAtLeastOneRecord)) { + FetchDataInfo fetchDataInfo = log.read(currentOffset, loadBufferSize, FetchIsolation.LOG_END, true); + + stats.readAtLeastOneRecord = fetchDataInfo.records.sizeInBytes() > 0; + + MemoryRecords memoryRecords = toReadableMemoryRecords(tp, fetchDataInfo.records, buffer); + if (fetchDataInfo.records instanceof FileRecords) { + buffer = memoryRecords.buffer(); + } + + ReplayResult replayResult = processMemoryRecords(tp, log, memoryRecords, coordinator, stats, currentOffset, previousHighWatermark); + currentOffset = replayResult.nextOffset; + previousHighWatermark = replayResult.highWatermark; + } + + long endTimeMs = time.milliseconds(); + + if (logEndOffset(tp) == -1L) { + future.completeExceptionally(new NotLeaderOrFollowerException( + String.format("Stopped loading records from %s because the partition is not online or is no longer the leader.", tp) + )); + } else if (isRunning.get()) { + future.complete(new LoadSummary(startTimeMs, endTimeMs, schedulerQueueTimeMs, stats.numRecords, stats.numBytes)); + } else { + future.completeExceptionally(new RuntimeException("Coordinator loader is closed.")); + } + } catch (Throwable ex) { + future.completeExceptionally(ex); + } + } + + private long logEndOffset(TopicPartition tp) { + return partitionLogEndOffsetSupplier.apply(tp).orElse(-1L); + } + + /** + * Returns true if it's still valid to fetch the next batch of records. + * <p> + * This method ensures fetching continues only under safe and meaningful conditions: + * <ul> + * <li>The current offset is less than the log end offset.</li> + * <li>At least one record was read in the previous fetch. This ensures that fetching stops even if the + * current offset remains smaller than the log end offset but the log is empty. This could happen with compacted topics.</li> + * <li>The log end offset is not -1L, which ensures the partition is online and is still the leader.</li> + * <li>The loader is still running.</li> + * </ul> + */ + private boolean shouldFetchNextBatch(long currentOffset, long logEndOffset, boolean readAtLeastOneRecord) { + return currentOffset < logEndOffset && readAtLeastOneRecord && isRunning.get(); + } + + private MemoryRecords toReadableMemoryRecords(TopicPartition tp, Records records, ByteBuffer buffer) throws IOException { + if (records instanceof MemoryRecords memoryRecords) { + return memoryRecords; + } else if (records instanceof FileRecords fileRecords) { + int sizeInBytes = fileRecords.sizeInBytes(); + int bytesNeeded = Math.max(loadBufferSize, sizeInBytes); + + // "minOneMessage = true in the above log.read() means that the buffer may need to + // be grown to ensure progress can be made. + if (buffer.capacity() < bytesNeeded) { + if (loadBufferSize < bytesNeeded) { + LOG.warn("Loaded metadata from {} with buffer larger ({} bytes) than" + + " configured buffer size ({} bytes).", tp, bytesNeeded, loadBufferSize); + } + + buffer = ByteBuffer.allocate(bytesNeeded); + } else { + buffer.clear(); + } + + fileRecords.readInto(buffer, 0); + return MemoryRecords.readableRecords(buffer); + } else { + throw new IllegalArgumentException("Unsupported record type: " + records.getClass()); + } + } + + private ReplayResult processMemoryRecords( + TopicPartition tp, + UnifiedLog log, + MemoryRecords memoryRecords, + CoordinatorPlayback<T> coordinator, + LoadStats loadStats, + long currentOffset, + long previousHighWatermark + ) { + + for (MutableRecordBatch batch : memoryRecords.batches()) { + if (batch.isControlBatch()) { + for (Record record : batch) { + ControlRecordType controlRecord = ControlRecordType.parse(record.key()); + if (controlRecord == ControlRecordType.COMMIT) { + if (LOG.isTraceEnabled()) { + LOG.trace("Replaying end transaction marker from {} at offset {} to commit" + + " transaction with producer id {} and producer epoch {}.", + tp, record.offset(), batch.producerId(), batch.producerEpoch()); + } + coordinator.replayEndTransactionMarker( + batch.producerId(), + batch.producerEpoch(), + TransactionResult.COMMIT + ); + } else if (controlRecord == ControlRecordType.ABORT) { + if (LOG.isTraceEnabled()) { + LOG.trace("Replaying end transaction marker from {} at offset {} to abort" + + " transaction with producer id {} and producer epoch {}.", + tp, record.offset(), batch.producerId(), batch.producerEpoch()); + } + coordinator.replayEndTransactionMarker( + batch.producerId(), + batch.producerEpoch(), + TransactionResult.ABORT + ); + } + } + } else { + for (Record record : batch) { + loadStats.numRecords++; + + Optional<T> coordinatorRecordOpt = Optional.empty(); + try { + coordinatorRecordOpt = Optional.ofNullable(deserializer.deserialize(record.key(), record.value())); + } catch (Deserializer.UnknownRecordTypeException ex) { + LOG.warn("Unknown record type {} while loading offsets and group metadata from {}." + + " Ignoring it. It could be a left over from an aborted upgrade.", ex.unknownType(), tp); + } catch (RuntimeException ex) { + String msg = String.format("Deserializing record %s from %s failed due to: %s", record, tp, ex.getMessage()); + LOG.error(msg); + throw new RuntimeException(msg, ex); + } + + coordinatorRecordOpt.ifPresent(coordinatorRecord -> { + try { + if (LOG.isTraceEnabled()) { + LOG.trace("Replaying record {} from {} at offset {} with producer id {}" + + " and producer epoch {}.", coordinatorRecord, tp, record.offset(), batch.producerId(), batch.producerEpoch()); + } + coordinator.replay( + record.offset(), + batch.producerId(), + batch.producerEpoch(), + coordinatorRecord + ); + } catch (RuntimeException ex) { + String msg = String.format("Replaying record %s from %s at offset %d with producer id %d and" + + " producer epoch %d failed due to: %s.", coordinatorRecord, tp, record.offset(), + batch.producerId(), batch.producerEpoch(), ex.getMessage()); + LOG.error(msg); Review Comment: ditto -- 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