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


##########
storage/src/test/java/org/apache/kafka/storage/internals/log/LogCleanerTest.java:
##########
@@ -0,0 +1,2712 @@
+/*
+ * 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.storage.internals.log;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.compress.Compression;
+import org.apache.kafka.common.config.AbstractConfig;
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.message.AbortedTxn;
+import org.apache.kafka.common.record.TimestampType;
+import org.apache.kafka.common.record.internal.CompressionType;
+import org.apache.kafka.common.record.internal.ControlRecordType;
+import org.apache.kafka.common.record.internal.EndTransactionMarker;
+import org.apache.kafka.common.record.internal.FileRecords;
+import org.apache.kafka.common.record.internal.LegacyRecord;
+import org.apache.kafka.common.record.internal.MemoryRecords;
+import org.apache.kafka.common.record.internal.MemoryRecordsBuilder;
+import org.apache.kafka.common.record.internal.Record;
+import org.apache.kafka.common.record.internal.RecordBatch;
+import org.apache.kafka.common.record.internal.Records;
+import org.apache.kafka.common.record.internal.SimpleRecord;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.coordinator.transaction.TransactionLogConfig;
+import org.apache.kafka.server.common.RequestLocal;
+import org.apache.kafka.server.common.TransactionVersion;
+import org.apache.kafka.server.config.ServerConfigs;
+import org.apache.kafka.server.config.ServerLogConfigs;
+import org.apache.kafka.server.metrics.KafkaMetricsGroup;
+import org.apache.kafka.server.metrics.KafkaYammerMetrics;
+import org.apache.kafka.server.util.MockTime;
+import org.apache.kafka.storage.internals.utils.Throttler;
+import org.apache.kafka.storage.log.metrics.BrokerTopicStats;
+import org.apache.kafka.test.TestUtils;
+
+import com.yammer.metrics.core.MetricName;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.MockedConstruction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.file.Paths;
+import java.security.DigestException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.stream.IntStream;
+import java.util.stream.LongStream;
+import java.util.stream.StreamSupport;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.mockConstruction;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+
+/**
+ * Unit tests for the log cleaning logic
+ */
+public class LogCleanerTest {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(LogCleanerTest.class);
+
+    private final File tmpdir = TestUtils.tempDirectory();
+    private final File dir = TestUtils.randomPartitionLogDir(tmpdir);
+    private final LogConfig logConfig;
+    private final MockTime time = new MockTime();
+    private final Throttler throttler = new Throttler(Double.MAX_VALUE, 
Long.MAX_VALUE, "throttler", "entries", time);
+    private final int tombstoneRetentionMs = 86400000;
+    private final long largeTimestamp = Long.MAX_VALUE - tombstoneRetentionMs 
- 1;
+    private final ProducerStateManagerConfig producerStateManagerConfig = new 
ProducerStateManagerConfig(TransactionLogConfig.PRODUCER_ID_EXPIRATION_MS_DEFAULT,
 false);
+
+    public LogCleanerTest() {
+        Properties logProps = new Properties();
+        logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024);
+        logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, 1024);
+        logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, 
TopicConfig.CLEANUP_POLICY_COMPACT);
+        logConfig = new LogConfig(logProps);
+    }
+
+    @AfterEach
+    public void teardown() throws IOException {
+        Utils.swallow(LOG, time.scheduler::shutdown);
+        Utils.delete(tmpdir);
+    }
+
+    @Test
+    public void testRemoveMetricsOnClose() {
+        try (MockedConstruction<KafkaMetricsGroup> mockMetricsGroupCtor = 
mockConstruction(KafkaMetricsGroup.class)) {
+            LogCleaner logCleaner = new LogCleaner(new CleanerConfig(true),
+                List.of(TestUtils.tempDirectory(), TestUtils.tempDirectory()),
+                new ConcurrentHashMap<>(),
+                new LogDirFailureChannel(1),
+                time);
+            Map<String, List<Map<String, String>>> metricsToVerify = new 
HashMap<>();
+            
logCleaner.cleanerManager().gaugeMetricNameWithTag().forEach((metricName, 
tagList) -> {
+                List<Map<String, String>> tags = List.copyOf(tagList);
+                metricsToVerify.put(metricName, tags);
+            });
+            // shutdown logCleaner so that metrics are removed
+            logCleaner.shutdown();
+
+            var mockMetricsGroup = mockMetricsGroupCtor.constructed().get(0);
+            int numMetricsRegistered = LogCleaner.METRIC_NAMES.size();
+            verify(mockMetricsGroup, 
times(numMetricsRegistered)).newGauge(anyString(), any());
+
+            // verify that each metric in `LogCleaner` is removed
+            LogCleaner.METRIC_NAMES.forEach(name -> 
verify(mockMetricsGroup).removeMetric(name));
+
+            // verify that each metric in `LogCleanerManager` is removed
+            var mockLogCleanerManagerMetricsGroup = 
mockMetricsGroupCtor.constructed().get(1);
+            LogCleanerManager.GAUGE_METRIC_NAME_NO_TAG.forEach(metricName ->
+                
verify(mockLogCleanerManagerMetricsGroup).newGauge(eq(metricName), any()));
+            metricsToVerify.forEach((metricName, tagList) ->
+                tagList.forEach(tags ->
+                    
verify(mockLogCleanerManagerMetricsGroup).newGauge(eq(metricName), any(), 
eq(tags))));
+
+            LogCleanerManager.GAUGE_METRIC_NAME_NO_TAG.forEach(name ->
+                verify(mockLogCleanerManagerMetricsGroup).removeMetric(name));
+            metricsToVerify.forEach((metricName, tagList) ->
+                tagList.forEach(tags ->
+                    
verify(mockLogCleanerManagerMetricsGroup).removeMetric(eq(metricName), 
eq(tags))));
+
+            // assert that we have verified all invocations on
+            verifyNoMoreInteractions(mockMetricsGroup);
+            verifyNoMoreInteractions(mockLogCleanerManagerMetricsGroup);
+        }
+    }
+
+    @Test
+    public void testMetricsActiveAfterReconfiguration() {
+        LogCleaner logCleaner = new LogCleaner(new CleanerConfig(true),
+            List.of(TestUtils.tempDirectory()),
+            new ConcurrentHashMap<>(),
+            new LogDirFailureChannel(1),
+            time);
+
+        try {
+            logCleaner.startup();
+            List<String> registeredMetrics = 
KafkaYammerMetrics.defaultRegistry()
+                
.allMetrics().keySet().stream().map(MetricName::getName).toList();
+            List<String> nonexistent = LogCleaner.METRIC_NAMES.stream()
+                .filter(metric -> 
!registeredMetrics.contains(metric)).toList();
+            assertEquals(0, nonexistent.size(), nonexistent + " should be 
existent");
+
+            logCleaner.reconfigure(makeReconfigureConfig(Map.of()), 
makeReconfigureConfig(Map.of()));
+
+            List<String> registeredMetrics2 = 
KafkaYammerMetrics.defaultRegistry()
+                
.allMetrics().keySet().stream().map(MetricName::getName).toList();
+            List<String> nonexistent2 = LogCleaner.METRIC_NAMES.stream()
+                .filter(n -> !registeredMetrics2.contains(n)).toList();
+            assertEquals(0, nonexistent2.size(), nonexistent2 + " should be 
existent");
+        } finally {
+            logCleaner.shutdown();
+        }
+    }
+
+    /**
+     * Test simple log cleaning
+     */
+    @Test
+    public void testCleanSegments() throws IOException {
+        Cleaner cleaner = makeCleaner(Integer.MAX_VALUE);
+        Properties logProps = new Properties();
+        logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024);
+
+        UnifiedLog log = makeLog(LogConfig.fromProps(logConfig.originals(), 
logProps));
+
+        // append messages to the log until we have four segments
+        while (log.numberOfSegments() < 4) {
+            log.appendAsLeader(record((int) log.logEndOffset(), (int) 
log.logEndOffset()), 0);
+        }
+        List<Long> keysFound = LogTestUtils.keysInLog(log);
+        assertEquals(LongStream.range(0L, 
log.logEndOffset()).boxed().toList(), keysFound);
+
+        // pretend we have the following keys
+        List<Long> keys = List.of(1L, 3L, 5L, 7L, 9L);
+        var map = new LogTestUtils.FakeOffsetMap(Integer.MAX_VALUE);
+        keys.forEach(k -> map.put(key(k), Long.MAX_VALUE));
+
+        // clean the log
+        List<LogSegment> segments = 
log.logSegments().stream().limit(3).toList();
+        CleanerStats stats = new CleanerStats(Time.SYSTEM);
+        long expectedBytesRead = 
segments.stream().mapToLong(LogSegment::size).sum();
+        List<Long> shouldRemain = 
LogTestUtils.keysInLog(log).stream().filter(key -> 
!keys.contains(key)).toList();
+        log.updateHighWatermark(segments.get(segments.size() - 
1).readNextOffset());
+        cleaner.cleanSegments(log, segments, map, 0L, stats, new 
CleanedTransactionMetadata(), -1);
+        assertEquals(shouldRemain, LogTestUtils.keysInLog(log));
+        assertEquals(expectedBytesRead, stats.bytesRead());
+    }
+
+    @Test
+    public void testCleanSegmentsWithConcurrentSegmentDeletion() throws 
IOException, DigestException {
+        CountDownLatch deleteStartLatch = new CountDownLatch(1);
+        CountDownLatch deleteCompleteLatch = new CountDownLatch(1);
+
+        // Construct a log instance. The replaceSegments() method of the log 
instance is overridden so that
+        // it waits for another thread to execute deleteOldSegments()
+        Properties logProps = new Properties();
+        logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 1024);
+        logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, 
TopicConfig.CLEANUP_POLICY_COMPACT + "," + TopicConfig.CLEANUP_POLICY_DELETE);
+        LogConfig config = LogConfig.fromProps(logConfig.originals(), 
logProps);
+        TopicPartition topicPartition = 
UnifiedLog.parseTopicPartitionName(dir);
+        var logDirFailureChannel = new LogDirFailureChannel(10);
+        int maxTransactionTimeoutMs = 5 * 60 * 1000;
+        int producerIdExpirationCheckIntervalMs = 
TransactionLogConfig.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_DEFAULT;
+        var logSegments = new LogSegments(topicPartition);
+        var leaderEpochCache = UnifiedLog.createLeaderEpochCache(
+            dir, topicPartition, logDirFailureChannel, Optional.empty(), 
time.scheduler);
+        var producerStateManager = new ProducerStateManager(topicPartition, 
dir,
+            maxTransactionTimeoutMs, producerStateManagerConfig, time);
+        var offsets = new LogLoader(
+            dir,
+            topicPartition,
+            config,
+            time.scheduler,
+            time,
+            logDirFailureChannel,
+            true,
+            logSegments,
+            0L,
+            0L,
+            leaderEpochCache,
+            producerStateManager,
+            new ConcurrentHashMap<>(),
+            false
+        ).load();
+        var localLog = new LocalLog(dir, config, logSegments, 
offsets.recoveryPoint(),
+            offsets.nextOffsetMetadata(), time.scheduler, time, 
topicPartition, logDirFailureChannel);
+        var log = new UnifiedLog(offsets.logStartOffset(),
+            localLog,
+            new BrokerTopicStats(),
+            producerIdExpirationCheckIntervalMs,
+            leaderEpochCache,
+            producerStateManager,
+            Optional.empty(),
+            false,
+            LogOffsetsListener.NO_OP_OFFSETS_LISTENER) {
+            @Override
+            public void replaceSegments(List<LogSegment> newSegments, 
List<LogSegment> oldSegments) throws IOException {
+                deleteStartLatch.countDown();
+                try {
+                    if (!deleteCompleteLatch.await(5000, 
TimeUnit.MILLISECONDS)) {
+                        throw new IllegalStateException("Log segment deletion 
timed out");
+                    }
+                } catch (InterruptedException e) {
+                    throw new IllegalStateException(e);
+                }
+                super.replaceSegments(newSegments, oldSegments);
+            }
+        };
+
+        // Start an async task that executes log.deleteOldSegments() right 
before replaceSegments() is executed.
+        CompletableFuture.runAsync(() -> {
+            try {
+                deleteStartLatch.await(5000, TimeUnit.MILLISECONDS);
+                log.updateHighWatermark(log.activeSegment().baseOffset());
+                
log.maybeIncrementLogStartOffset(log.activeSegment().baseOffset(), 
LogStartOffsetIncrementReason.LeaderOffsetIncremented);
+                log.updateHighWatermark(log.activeSegment().baseOffset());
+                log.deleteOldSegments();
+                deleteCompleteLatch.countDown();
+            } catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+        });
+
+        // Append records so that segment number increase to 3
+        while (log.numberOfSegments() < 3) {
+            log.appendAsLeader(record(0, (int) log.logEndOffset()), 0);
+            log.roll(Optional.empty());
+        }
+        assertEquals(3, log.numberOfSegments());
+
+        // Remember reference to the first log and determine its file name 
expected for async deletion
+        FileRecords firstLogFile = log.logSegments().get(0).log();
+        String expectedFileName = 
Utils.replaceSuffix(firstLogFile.file().getPath(), "", 
LogFileUtils.DELETED_FILE_SUFFIX);
+
+        // Clean the log. This should trigger replaceSegments() and 
deleteOldSegments();
+        var offsetMap = new LogTestUtils.FakeOffsetMap(Integer.MAX_VALUE);
+        Cleaner cleaner = makeCleaner(Integer.MAX_VALUE);
+        List<LogSegment> segments = List.copyOf(log.logSegments(0, 
log.activeSegment().baseOffset()));
+        CleanerStats stats = new CleanerStats(Time.SYSTEM);
+        cleaner.buildOffsetMap(log, 0, log.activeSegment().baseOffset(), 
offsetMap, stats);
+        log.updateHighWatermark(segments.get(segments.size() - 
1).readNextOffset());
+        cleaner.cleanSegments(log, segments, offsetMap, 0L, stats, new 
CleanedTransactionMetadata(), -1);
+
+        // Validate based on the file name that log segment file is renamed 
exactly once for async deletion
+        assertEquals(expectedFileName, firstLogFile.file().getPath());
+        assertEquals(2, log.numberOfSegments());
+    }
+
+    @Test
+    public void testSizeTrimmedForPreallocatedAndCompactedTopic() throws 
IOException, DigestException {
+        int originalMaxFileSize = 1024;
+        Cleaner cleaner = makeCleaner(2);
+        Properties logProps = new Properties();
+        logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 
originalMaxFileSize);
+        logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, "compact");
+        logProps.put(TopicConfig.PREALLOCATE_CONFIG, "true");
+        UnifiedLog log = makeLog(LogConfig.fromProps(logConfig.originals(), 
logProps));
+
+        log.appendAsLeader(record(0, 0), 0); // offset 0
+        log.appendAsLeader(record(1, 1), 0); // offset 1
+        log.appendAsLeader(record(0, 0), 0); // offset 2
+        log.appendAsLeader(record(1, 1), 0); // offset 3
+        log.appendAsLeader(record(0, 0), 0); // offset 4
+        // roll the segment, so we can clean the messages already appended
+        log.roll();
+
+        // clean the log with only one message removed
+        cleaner.clean(new LogToClean(log, 2, log.activeSegment().baseOffset(), 
false));
+
+        assertTrue(log.logSegments().iterator().next().log().channel().size() 
< originalMaxFileSize,
+            "Cleaned segment file should be trimmed to its real size.");
+    }
+
+    @Test
+    public void testDuplicateCheckAfterCleaning() throws IOException, 
DigestException {
+        Cleaner cleaner = makeCleaner(Integer.MAX_VALUE);
+        Properties logProps = new Properties();
+        logProps.put(LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG, 2048);
+        UnifiedLog log = makeLog(LogConfig.fromProps(logConfig.originals(), 
logProps));
+
+        short producerEpoch = 0;
+        long pid1 = 1;
+        long pid2 = 2;
+        long pid3 = 3;
+        long pid4 = 4;
+
+        appendIdempotentAsLeader(log, pid1, producerEpoch).append(List.of(1, 
2, 3));

Review Comment:
   The functional interface `ProducerAppender` seems a bit redundant. Why not 
have `appendIdempotentAsLeader` return `LogAppendInfo` directly?



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to