Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

KAFKA-14485: Move LogCleaner exceptions to storage module #18534

Merged
merged 1 commit into from
Jan 16, 2025
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
3 changes: 1 addition & 2 deletions core/src/main/scala/kafka/log/LogCleaner.scala
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@ import java.io.{File, IOException}
import java.nio._
import java.util.Date
import java.util.concurrent.TimeUnit
import kafka.common._
import kafka.log.LogCleaner.{CleanerRecopyPercentMetricName, DeadThreadCountMetricName, MaxBufferUtilizationPercentMetricName, MaxCleanTimeMetricName, MaxCompactionDelayMetricsName}
import kafka.server.{BrokerReconfigurable, KafkaConfig}
import kafka.utils.{Logging, Pool}
Expand All @@ -35,7 +34,7 @@ import org.apache.kafka.common.utils.{BufferSupplier, Time}
import org.apache.kafka.server.config.ServerConfigs
import org.apache.kafka.server.metrics.KafkaMetricsGroup
import org.apache.kafka.server.util.ShutdownableThread
import org.apache.kafka.storage.internals.log.{AbortedTxn, CleanerConfig, LastRecord, LogDirFailureChannel, LogSegment, LogSegmentOffsetOverflowException, OffsetMap, SkimpyOffsetMap, TransactionIndex}
import org.apache.kafka.storage.internals.log.{AbortedTxn, CleanerConfig, LastRecord, LogCleaningAbortedException, LogDirFailureChannel, LogSegment, LogSegmentOffsetOverflowException, OffsetMap, SkimpyOffsetMap, ThreadShutdownException, TransactionIndex}
import org.apache.kafka.storage.internals.utils.Throttler

import scala.jdk.CollectionConverters._
Expand Down
3 changes: 1 addition & 2 deletions core/src/main/scala/kafka/log/LogCleanerManager.scala
Original file line number Diff line number Diff line change
Expand Up @@ -21,14 +21,13 @@ import java.lang.{Long => JLong}
import java.io.File
import java.util.concurrent.TimeUnit
import java.util.concurrent.locks.ReentrantLock
import kafka.common.LogCleaningAbortedException
import kafka.utils.CoreUtils._
import kafka.utils.{Logging, Pool}
import org.apache.kafka.common.{KafkaException, TopicPartition}
import org.apache.kafka.common.errors.KafkaStorageException
import org.apache.kafka.common.utils.Time
import org.apache.kafka.storage.internals.checkpoint.OffsetCheckpointFile
import org.apache.kafka.storage.internals.log.LogDirFailureChannel
import org.apache.kafka.storage.internals.log.{LogCleaningAbortedException, LogDirFailureChannel}
import org.apache.kafka.server.metrics.KafkaMetricsGroup

import java.util.Comparator
Expand Down
13 changes: 6 additions & 7 deletions core/src/test/scala/unit/kafka/log/LogCleanerTest.scala
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@

package kafka.log

import kafka.common._
import kafka.server.KafkaConfig
import kafka.utils.{CoreUtils, Logging, Pool, TestUtils}
import org.apache.kafka.common.TopicPartition
Expand All @@ -29,7 +28,7 @@ import org.apache.kafka.common.utils.Utils
import org.apache.kafka.coordinator.transaction.TransactionLogConfig
import org.apache.kafka.server.metrics.{KafkaMetricsGroup, KafkaYammerMetrics}
import org.apache.kafka.server.util.MockTime
import org.apache.kafka.storage.internals.log.{AbortedTxn, AppendOrigin, CleanerConfig, LocalLog, LogAppendInfo, LogConfig, LogDirFailureChannel, LogFileUtils, LogLoader, LogSegment, LogSegments, LogStartOffsetIncrementReason, OffsetMap, ProducerStateManager, ProducerStateManagerConfig}
import org.apache.kafka.storage.internals.log.{AbortedTxn, AppendOrigin, CleanerConfig, LocalLog, LogAppendInfo, LogCleaningAbortedException, LogConfig, LogDirFailureChannel, LogFileUtils, LogLoader, LogSegment, LogSegments, LogStartOffsetIncrementReason, OffsetMap, ProducerStateManager, ProducerStateManagerConfig}
import org.apache.kafka.storage.internals.utils.Throttler
import org.apache.kafka.storage.log.metrics.BrokerTopicStats
import org.junit.jupiter.api.Assertions._
Expand Down Expand Up @@ -1218,18 +1217,18 @@ class LogCleanerTest extends Logging {

def distinctValuesBySegment = log.logSegments.asScala.map(s => s.log.records.asScala.map(record => TestUtils.readString(record.value)).toSet.size).toSeq

val disctinctValuesBySegmentBeforeClean = distinctValuesBySegment
val distinctValuesBySegmentBeforeClean = distinctValuesBySegment
assertTrue(distinctValuesBySegment.reverse.tail.forall(_ > N),
"Test is not effective unless each segment contains duplicates. Increase segment size or decrease number of keys.")

cleaner.clean(LogToClean(new TopicPartition("test", 0), log, 0, firstUncleanableOffset))

val distinctValuesBySegmentAfterClean = distinctValuesBySegment

assertTrue(disctinctValuesBySegmentBeforeClean.zip(distinctValuesBySegmentAfterClean)
assertTrue(distinctValuesBySegmentBeforeClean.zip(distinctValuesBySegmentAfterClean)
.take(numCleanableSegments).forall { case (before, after) => after < before },
"The cleanable segments should have fewer number of values after cleaning")
assertTrue(disctinctValuesBySegmentBeforeClean.zip(distinctValuesBySegmentAfterClean)
assertTrue(distinctValuesBySegmentBeforeClean.zip(distinctValuesBySegmentAfterClean)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nice catch!

.slice(numCleanableSegments, numTotalSegments).forall { x => x._1 == x._2 }, "The uncleanable segments should have the same number of values after cleaning")
}

Expand All @@ -1241,9 +1240,9 @@ class LogCleanerTest extends Logging {
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))

// create 6 segments with only one message in each segment
def createRecorcs = TestUtils.singletonRecords(value = Array.fill[Byte](25)(0), key = 1.toString.getBytes)
def createRecords = TestUtils.singletonRecords(value = Array.fill[Byte](25)(0), key = 1.toString.getBytes)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nice catch!

for (_ <- 0 until 6)
log.appendAsLeader(createRecorcs, leaderEpoch = 0)
log.appendAsLeader(createRecords, leaderEpoch = 0)

val logToClean = LogToClean(new TopicPartition("test", 0), log, log.activeSegment.baseOffset, log.activeSegment.baseOffset)

Expand Down
Original file line number Diff line number Diff line change
@@ -1,10 +1,10 @@
/**
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* 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
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
Expand All @@ -14,11 +14,10 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package kafka.common
package org.apache.kafka.storage.internals.log;

/**
* Thrown when a log cleaning task is requested to be aborted.
*/
class LogCleaningAbortedException extends RuntimeException() {
public class LogCleaningAbortedException extends RuntimeException {
}
Original file line number Diff line number Diff line change
@@ -1,10 +1,10 @@
/**
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* 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
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
Expand All @@ -14,11 +14,10 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package kafka.common
package org.apache.kafka.storage.internals.log;

/**
* An exception that indicates a thread is being shut down normally.
*/
class ThreadShutdownException extends RuntimeException {
public class ThreadShutdownException extends RuntimeException {
}
Loading