kowshik commented on a change in pull request #10280: URL: https://github.com/apache/kafka/pull/10280#discussion_r643465477
########## File path: core/src/test/scala/unit/kafka/log/LocalLogTest.scala ########## @@ -0,0 +1,734 @@ +/** + * 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 kafka.log + +import java.io.File +import java.nio.channels.ClosedChannelException +import java.nio.charset.StandardCharsets +import java.util.regex.Pattern +import java.util.{Collections, Properties} + +import kafka.server.{FetchDataInfo, KafkaConfig, LogDirFailureChannel, LogOffsetMetadata} +import kafka.utils.{MockTime, Scheduler, TestUtils} +import org.apache.kafka.common.{KafkaException, TopicPartition} +import org.apache.kafka.common.errors.KafkaStorageException +import org.apache.kafka.common.record.{CompressionType, MemoryRecords, Record, SimpleRecord} +import org.apache.kafka.common.utils.{Time, Utils} +import org.junit.jupiter.api.Assertions.{assertFalse, _} +import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} + +import scala.jdk.CollectionConverters._ + +class LocalLogTest { + + import kafka.log.LocalLogTest._ + + var config: KafkaConfig = null + val tmpDir: File = TestUtils.tempDir() + val logDir: File = TestUtils.randomPartitionLogDir(tmpDir) + val topicPartition = new TopicPartition("test_topic", 1) + val logDirFailureChannel = new LogDirFailureChannel(10) + val mockTime = new MockTime() + val log: LocalLog = createLocalLogWithActiveSegment(config = createLogConfig()) + + @BeforeEach + def setUp(): Unit = { + val props = TestUtils.createBrokerConfig(0, "127.0.0.1:1", port = -1) + config = KafkaConfig.fromProps(props) + } + + @AfterEach + def tearDown(): Unit = { + if (!log.isMemoryMappedBufferClosed) { + log.close() + } + Utils.delete(tmpDir) + } + + case class KeyValue(key: String, value: String) { + def toRecord(timestamp: => Long = mockTime.milliseconds): SimpleRecord = { + new SimpleRecord(timestamp, key.getBytes, value.getBytes) + } + } + + object KeyValue { + def fromRecord(record: Record): KeyValue = { + val key = + if (record.hasKey) + StandardCharsets.UTF_8.decode(record.key()).toString + else + "" + val value = + if (record.hasValue) + StandardCharsets.UTF_8.decode(record.value()).toString + else + "" + KeyValue(key, value) + } + } + + private def kvsToRecords(keyValues: Iterable[KeyValue]): Iterable[SimpleRecord] = { + keyValues.map(kv => kv.toRecord()) + } + + private def recordsToKvs(records: Iterable[Record]): Iterable[KeyValue] = { + records.map(r => KeyValue.fromRecord(r)) + } + + private def appendRecords(records: Iterable[SimpleRecord], + log: LocalLog = log, + initialOffset: Long = 0L): Unit = { + log.append(lastOffset = initialOffset + records.size - 1, + largestTimestamp = records.head.timestamp, + shallowOffsetOfMaxTimestamp = initialOffset, + records = MemoryRecords.withRecords(initialOffset, CompressionType.NONE, 0, records.toList : _*)) + } + + private def readRecords(log: LocalLog = log, + startOffset: Long = 0L, + maxLength: => Int = log.segments.activeSegment.size, + minOneMessage: Boolean = false, + maxOffsetMetadata: => LogOffsetMetadata = log.logEndOffsetMetadata, + includeAbortedTxns: Boolean = false): FetchDataInfo = { + log.read(startOffset, + maxLength, + minOneMessage = minOneMessage, + maxOffsetMetadata, + includeAbortedTxns = includeAbortedTxns) + } + + @Test + def testLogDeleteSuccess(): Unit = { + val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes) + appendRecords(List(record)) + log.roll() + assertEquals(2, log.segments.numberOfSegments) + assertFalse(logDir.listFiles.isEmpty) + val segmentsBeforeDelete = List[LogSegment]() ++ log.segments.values + val deletedSegments = log.delete() + assertTrue(log.segments.isEmpty) + assertEquals(segmentsBeforeDelete, deletedSegments) + assertThrows(classOf[KafkaStorageException], () => log.checkIfMemoryMappedBufferClosed()) + assertFalse(logDir.exists) + } + + @Test + def testLogDeleteFailureAfterCloseHandlers(): Unit = { + log.closeHandlers() + assertEquals(1, log.segments.numberOfSegments) + val segmentsBeforeDelete = log.segments.values + assertThrows(classOf[KafkaStorageException], () => log.delete()) + assertEquals(1, log.segments.numberOfSegments) + assertEquals(segmentsBeforeDelete, log.segments.values) + assertTrue(logDir.exists) + } + + @Test + def testUpdateConfig(): Unit = { + val oldConfig = log.config + assertEquals(oldConfig, log.config) + + val newConfig = createLogConfig() + log.updateConfig(newConfig) + assertEquals(newConfig, log.config) + } + + @Test + def testLogDirRenameToNewDir(): Unit = { + val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes) + appendRecords(List(record)) + log.roll() + assertEquals(2, log.segments.numberOfSegments) + val newLogDir = TestUtils.randomPartitionLogDir(tmpDir) + assertTrue(log.renameDir(newLogDir.getName)) + assertFalse(logDir.exists()) + assertTrue(newLogDir.exists()) + assertEquals(newLogDir, log.dir) + assertEquals(newLogDir.getParent, log.parentDir) + assertEquals(newLogDir.getParent, log.dir.getParent) + log.segments.values.foreach(segment => assertEquals(newLogDir.getPath, segment.log.file().getParentFile.getPath)) + assertEquals(2, log.segments.numberOfSegments) + } + + @Test + def testLogDirRenameToExistingDir(): Unit = { + assertFalse(log.renameDir(log.dir.getName)) + } + + @Test + def testLogFlush(): Unit = { + assertEquals(0L, log.recoveryPoint) + assertEquals(mockTime.milliseconds, log.lastFlushTime) + + val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes) + appendRecords(List(record)) + mockTime.sleep(1) + val newSegment = log.roll() + log.flush(newSegment.baseOffset) + log.markFlushed(newSegment.baseOffset) + assertEquals(1L, log.recoveryPoint) + assertEquals(mockTime.milliseconds, log.lastFlushTime) + } + + @Test + def testLogAppend(): Unit = { + val fetchDataInfoBeforeAppend = readRecords(maxLength = 1) + assertTrue(fetchDataInfoBeforeAppend.records.records.asScala.isEmpty) + + mockTime.sleep(1) + val keyValues = Seq(KeyValue("abc", "ABC"), KeyValue("de", "DE")) + appendRecords(kvsToRecords(keyValues)) + assertEquals(2L, log.logEndOffset) + assertEquals(0L, log.recoveryPoint) + val fetchDataInfo = readRecords() + assertEquals(2L, fetchDataInfo.records.records.asScala.size) + assertEquals(keyValues, recordsToKvs(fetchDataInfo.records.records.asScala)) + } + + @Test + def testLogCloseSuccess(): Unit = { + val keyValues = Seq(KeyValue("abc", "ABC"), KeyValue("de", "DE")) + appendRecords(kvsToRecords(keyValues)) + log.close() + assertThrows(classOf[ClosedChannelException], () => appendRecords(kvsToRecords(keyValues), initialOffset = 2L)) + } + + @Test + def testLogCloseIdempotent(): Unit = { + log.close() + // Check that LocalLog.close() is idempotent + log.close() + } + + @Test + def testLogCloseFailureWhenInMemoryBufferClosed(): Unit = { + val keyValues = Seq(KeyValue("abc", "ABC"), KeyValue("de", "DE")) + appendRecords(kvsToRecords(keyValues)) + log.closeHandlers() + assertThrows(classOf[KafkaStorageException], () => log.close()) + } + + @Test + def testLogCloseHandlers(): Unit = { + val keyValues = Seq(KeyValue("abc", "ABC"), KeyValue("de", "DE")) + appendRecords(kvsToRecords(keyValues)) + log.closeHandlers() + assertThrows(classOf[ClosedChannelException], + () => appendRecords(kvsToRecords(keyValues), initialOffset = 2L)) + } + + @Test + def testLogCloseHandlersIdempotent(): Unit = { + log.closeHandlers() + // Check that LocalLog.closeHandlers() is idempotent + log.closeHandlers() + } + + private def testRemoveAndDeleteSegments(asyncDelete: Boolean): Unit = { + for (offset <- 0 to 8) { + val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes) + appendRecords(List(record), initialOffset = offset) + log.roll() + } + + assertEquals(10L, log.segments.numberOfSegments) + + class TestDeletionReason extends SegmentDeletionReason { + private var _deletedSegments: Iterable[LogSegment] = List[LogSegment]() + + override def logReason(toDelete: List[LogSegment]): Unit = { + _deletedSegments = List[LogSegment]() ++ toDelete + } + + def deletedSegments: Iterable[LogSegment] = _deletedSegments + } + val reason = new TestDeletionReason() + val toDelete = List[LogSegment]() ++ log.segments.values + log.removeAndDeleteSegments(toDelete, asyncDelete = asyncDelete, reason) + if (asyncDelete) { + mockTime.sleep(log.config.fileDeleteDelayMs + 1) + } + assertTrue(log.segments.isEmpty) + assertEquals(toDelete, reason.deletedSegments) + toDelete.foreach(segment => assertTrue(segment.deleted())) + } + + @Test + def testRemoveAndDeleteSegmentsSync(): Unit = { + testRemoveAndDeleteSegments(asyncDelete = false) + } + + @Test + def testRemoveAndDeleteSegmentsAsync(): Unit = { + testRemoveAndDeleteSegments(asyncDelete = true) + } + + private def testDeleteSegmentFiles(asyncDelete: Boolean): Unit = { + for (offset <- 0 to 8) { + val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes) + appendRecords(List(record), initialOffset = offset) + log.roll() + } + + assertEquals(10L, log.segments.numberOfSegments) + + val toDelete = List[LogSegment]() ++ log.segments.values + LocalLog.deleteSegmentFiles(toDelete, asyncDelete = asyncDelete, log.dir, log.topicPartition, log.config, log.scheduler, log.logDirFailureChannel, "") + if (asyncDelete) { + toDelete.foreach { + segment => + assertFalse(segment.deleted()) + assertTrue(segment.hasSuffix(LocalLog.DeletedFileSuffix)) + } + mockTime.sleep(log.config.fileDeleteDelayMs + 1) + } + toDelete.foreach(segment => assertTrue(segment.deleted())) + } + + @Test + def testDeleteSegmentFilesSync(): Unit = { + testDeleteSegmentFiles(asyncDelete = false) + } + + @Test + def testDeleteSegmentFilesAsync(): Unit = { + testDeleteSegmentFiles(asyncDelete = true) + } + + @Test + def testDeletableSegmentsFilter(): Unit = { + for (offset <- 0 to 8) { + val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes) + appendRecords(List(record), initialOffset = offset) + log.roll() + } + + assertEquals(10L, log.segments.numberOfSegments) + + { + val deletable = log.deletableSegments( + (segment: LogSegment, _: Option[LogSegment], _: Long) => segment.baseOffset <= 5) + val expected = log.segments.nonActiveLogSegmentsFrom(0L).filter(segment => segment.baseOffset <= 5) + assertEquals(expected, deletable.toList) + } + + { + val deletable = log.deletableSegments((_: LogSegment, _: Option[LogSegment], _: Long) => true) + val expected = log.segments.nonActiveLogSegmentsFrom(0L).toList + assertEquals(expected, deletable.toList) + } + + { + val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes) + appendRecords(List(record), initialOffset = 9L) + val deletable = log.deletableSegments((_: LogSegment, _: Option[LogSegment], _: Long) => true) + val expected = log.segments.values.toList + assertEquals(expected, deletable.toList) + } + } + + @Test + def testDeletableSegmentsIteration(): Unit = { + for (offset <- 0 to 8) { + val record = new SimpleRecord(mockTime.milliseconds, "a".getBytes) + appendRecords(List(record), initialOffset = offset) + log.roll() + } + + assertEquals(10L, log.segments.numberOfSegments) + + var offset = 0 + log.deletableSegments( + (segment: LogSegment, nextSegmentOpt: Option[LogSegment], logEndOffset: Long) => { + assertEquals(offset, segment.baseOffset) + val floorSegmentOpt = log.segments.floorSegment(offset) + assertTrue(floorSegmentOpt.isDefined) + assertEquals(floorSegmentOpt.get, segment) + if (offset == log.logEndOffset) { + assertFalse(nextSegmentOpt.isDefined) + } else { + assertTrue(nextSegmentOpt.isDefined) + val higherSegmentOpt = log.segments.higherSegment(segment.baseOffset) + assertTrue(higherSegmentOpt.isDefined) + assertEquals(segment.baseOffset + 1, higherSegmentOpt.get.baseOffset) + assertEquals(higherSegmentOpt.get, nextSegmentOpt.get) + } + assertEquals(log.logEndOffset, logEndOffset) + offset += 1 + true + }) Review comment: Good question. We could assert that the returned value matches our expectations. Also we could assert that `log.deletableSegments()` does not make any mutations to the `LocalLog` object. -- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org