Skip to content
Draft
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
2 changes: 2 additions & 0 deletions core/src/main/scala/kafka/server/AbstractFetcherThread.scala
Original file line number Diff line number Diff line change
Expand Up @@ -101,6 +101,8 @@ abstract class AbstractFetcherThread(name: String,

protected def endOffsetForEpoch(topicPartition: TopicPartition, epoch: Int): Optional[OffsetAndEpoch]

protected def shouldFetchFromLastTieredOffset(topicPartition: TopicPartition, leaderEndOffset: Long, replicaEndOffset: Long): Boolean

override def shutdown(): Unit = {
initiateShutdown()
inLock(partitionMapLock) {
Expand Down
29 changes: 29 additions & 0 deletions core/src/main/scala/kafka/server/LocalLeaderEndPoint.scala
Original file line number Diff line number Diff line change
Expand Up @@ -135,6 +135,35 @@ class LocalLeaderEndPoint(sourceBroker: BrokerEndPoint,
new OffsetAndEpoch(localLogStartOffset, epoch.orElse(0))
}

override def fetchEarliestPendingUploadOffset(topicPartition: TopicPartition, currentLeaderEpoch: Int): OffsetAndEpoch = {
val partition = replicaManager.getPartitionOrException(topicPartition)
val log = partition.localLogOrException

if (!log.remoteLogEnabled())
return new OffsetAndEpoch(-1L, -1)

val highestRemoteOffset = log.highestOffsetInRemoteStorage()

if (highestRemoteOffset == -1L) {
val localLogStartOffset = fetchEarliestLocalOffset(topicPartition, currentLeaderEpoch)
val logStartOffset = fetchEarliestOffset(topicPartition, currentLeaderEpoch)

if (localLogStartOffset.offset() == logStartOffset.offset()) {
// No segments have been uploaded yet
return logStartOffset;
Copy link
Contributor

Choose a reason for hiding this comment

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

can we refactor the code to avoid using multiple return statements?

https://tpolecat.github.io/2014/05/09/return.html

} else {
// Leader currently does not know about the already uploaded segments
return new OffsetAndEpoch(-1L, -1);
}
}

val logStartOffset = fetchEarliestOffset(topicPartition, currentLeaderEpoch)
val earliestPendingUploadOffset = Math.max(highestRemoteOffset + 1, logStartOffset.offset())
val epoch = log.leaderEpochCache.epochForOffset(earliestPendingUploadOffset)

new OffsetAndEpoch(earliestPendingUploadOffset, epoch.orElse(0))
}

override def fetchEpochEndOffsets(partitions: util.Map[TopicPartition, OffsetForLeaderEpochRequestData.OffsetForLeaderPartition]): util.Map[TopicPartition, EpochEndOffset] = {
partitions.asScala.map { case (tp, epochData) =>
try {
Expand Down
4 changes: 4 additions & 0 deletions core/src/main/scala/kafka/server/RemoteLeaderEndPoint.scala
Original file line number Diff line number Diff line change
Expand Up @@ -105,6 +105,10 @@ class RemoteLeaderEndPoint(logPrefix: String,
fetchOffset(topicPartition, currentLeaderEpoch, ListOffsetsRequest.EARLIEST_LOCAL_TIMESTAMP)
}

override def fetchEarliestPendingUploadOffset(topicPartition: TopicPartition, currentLeaderEpoch: Int): OffsetAndEpoch = {
fetchOffset(topicPartition, currentLeaderEpoch, ListOffsetsRequest.EARLIEST_PENDING_UPLOAD_TIMESTAMP)
}

private def fetchOffset(topicPartition: TopicPartition, currentLeaderEpoch: Int, timestamp: Long): OffsetAndEpoch = {
val topic = new ListOffsetsTopic()
.setName(topicPartition.topic)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,8 @@ class ReplicaAlterLogDirsThread(name: String,
replicaMgr.futureLocalLogOrException(topicPartition).endOffsetForEpoch(epoch)
}

override protected def shouldFetchFromLastTieredOffset(topicPartition: TopicPartition, leaderEndOffset: Long, replicaEndOffset: Long): Boolean = false

// process fetched data
override def processPartitionData(
topicPartition: TopicPartition,
Expand Down
11 changes: 11 additions & 0 deletions core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,17 @@ class ReplicaFetcherThread(name: String,
replicaMgr.localLogOrException(topicPartition).endOffsetForEpoch(epoch)
}

override protected[server] def shouldFetchFromLastTieredOffset(topicPartition: TopicPartition, leaderEndOffset: Long, replicaEndOffset: Long): Boolean = {
val isCompactTopic = replicaMgr.localLog(topicPartition).exists(_.config.compact)
val remoteStorageEnabled = replicaMgr.localLog(topicPartition).exists(_.remoteLogEnabled())

brokerConfig.followerFetchLastTieredOffsetEnable &&
remoteStorageEnabled &&
!isCompactTopic &&
replicaEndOffset == 0 &&
leaderEndOffset != 0
}

override def initiateShutdown(): Boolean = {
val justShutdown = super.initiateShutdown()
if (justShutdown) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,88 @@
/*
* 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.server

import kafka.log.LogManager
import kafka.utils.TestUtils
import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.server.common.OffsetAndEpoch
import org.apache.kafka.storage.internals.log.LogConfig
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Test

import java.io.File
import java.util.Properties
import scala.collection.Map
import scala.jdk.CollectionConverters._

class LocalLeaderEndPointRemoteTest extends LocalLeaderEndPointTestBase {
Copy link
Contributor

Choose a reason for hiding this comment

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

Why do we have to split the test? Can they be part of the existing LocalLeaderEndPointTest itself?


override def createLogManager(config: KafkaConfig): LogManager = {
val logProps = new Properties()
logProps.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
// Keep cleanup.policy=delete (default), not compact, so remote storage is allowed
val defaultLogConfig = LogConfig.fromProps(Map.empty[String, Object].asJava, logProps)

TestUtils.createLogManager(
config.logDirs.asScala.map(new File(_)),
defaultConfig = defaultLogConfig,
remoteStorageSystemEnable = true
)
}

@Test
def testEarliestPendingUploadOffsetWhenNoSegmentsUploaded(): Unit = {
// Append some records; no remote upload happened yet
appendRecords(replicaManager, topicIdPartition, records)
.onFire(response => assertEquals(Errors.NONE, response.error))

val expected = endPoint.fetchEarliestOffset(topicPartition, 0)
val result = endPoint.fetchEarliestPendingUploadOffset(topicPartition, 0)
assertEquals(expected, result)
}

@Test
def testEarliestPendingUploadOffsetWhenLocalStartGreaterThanStart(): Unit = {
appendRecords(replicaManager, topicIdPartition, records)
.onFire(response => assertEquals(Errors.NONE, response.error))

// Bump epoch and advance local log start offset without changing log start offset
bumpLeaderEpoch()
replicaManager.logManager.getLog(topicPartition).foreach(_.updateLocalLogStartOffset(3))

val result = endPoint.fetchEarliestPendingUploadOffset(topicPartition, 1)
assertEquals(new OffsetAndEpoch(-1L, -1), result)
}

@Test
def testEarliestPendingUploadOffsetWhenHighestRemoteOffsetKnown(): Unit = {
appendRecords(replicaManager, topicIdPartition, records)
.onFire(response => assertEquals(Errors.NONE, response.error))

// Highest remote is 1 => earliest pending should be max(1+1, logStart)
val log = replicaManager.getPartitionOrException(topicPartition).localLogOrException
log.updateHighestOffsetInRemoteStorage(1)

val expectedOffset = Math.max(2L, log.logStartOffset())
val epoch = log.leaderEpochCache().epochForOffset(expectedOffset).orElse(0)

val result = endPoint.fetchEarliestPendingUploadOffset(topicPartition, 0)
assertEquals(new OffsetAndEpoch(expectedOffset, epoch), result)
}
}
154 changes: 3 additions & 151 deletions core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala
Original file line number Diff line number Diff line change
Expand Up @@ -17,104 +17,18 @@

package kafka.server

import kafka.server.QuotaFactory.QuotaManagers
import kafka.utils.{CoreUtils, Logging, TestUtils}
import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.{TopicIdPartition, Uuid}
import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition
import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset
import org.apache.kafka.common.metadata.{FeatureLevelRecord, PartitionChangeRecord, PartitionRecord, TopicRecord}
import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.record.{MemoryRecords, SimpleRecord}
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
import org.apache.kafka.image.{MetadataDelta, MetadataImage, MetadataProvenance}
import org.apache.kafka.metadata.KRaftMetadataCache
import org.apache.kafka.server.common.{KRaftVersion, MetadataVersion, OffsetAndEpoch}
import org.apache.kafka.server.network.BrokerEndPoint
import org.apache.kafka.server.LeaderEndPoint
import org.apache.kafka.server.util.{MockScheduler, MockTime}
import org.apache.kafka.storage.internals.log.{AppendOrigin, LogDirFailureChannel}
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
import org.apache.kafka.server.common.OffsetAndEpoch
import org.junit.jupiter.api.Test
import org.junit.jupiter.api.Assertions._
import org.mockito.Mockito.mock

import java.io.File
import java.util.{Map => JMap}
import scala.collection.Map
import scala.jdk.CollectionConverters._

class LocalLeaderEndPointTest extends Logging {

val time = new MockTime
val topicId = Uuid.randomUuid()
val topic = "test"
val partition = 5
val topicIdPartition = new TopicIdPartition(topicId, partition, topic)
val topicPartition = topicIdPartition.topicPartition()
val sourceBroker: BrokerEndPoint = new BrokerEndPoint(0, "localhost", 9092)
var replicaManager: ReplicaManager = _
var endPoint: LeaderEndPoint = _
var quotaManager: QuotaManagers = _
var image: MetadataImage = _

@BeforeEach
def setUp(): Unit = {
val props = TestUtils.createBrokerConfig(sourceBroker.id, port = sourceBroker.port)
val config = KafkaConfig.fromProps(props)
val mockLogMgr = TestUtils.createLogManager(config.logDirs.asScala.map(new File(_)))
val alterPartitionManager = mock(classOf[AlterPartitionManager])
val metrics = new Metrics
quotaManager = QuotaFactory.instantiate(config, metrics, time, "", "")
replicaManager = new ReplicaManager(
metrics = metrics,
config = config,
time = time,
scheduler = new MockScheduler(time),
logManager = mockLogMgr,
quotaManagers = quotaManager,
metadataCache = new KRaftMetadataCache(config.brokerId, () => KRaftVersion.KRAFT_VERSION_0),
logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size),
alterPartitionManager = alterPartitionManager
)

val delta = new MetadataDelta(MetadataImage.EMPTY)
delta.replay(new FeatureLevelRecord()
.setName(MetadataVersion.FEATURE_NAME)
.setFeatureLevel(MetadataVersion.MINIMUM_VERSION.featureLevel())
)
delta.replay(new TopicRecord()
.setName(topic)
.setTopicId(topicId)
)
delta.replay(new PartitionRecord()
.setPartitionId(partition)
.setTopicId(topicId)
.setReplicas(java.util.List.of[Integer](sourceBroker.id))
.setIsr(java.util.List.of[Integer](sourceBroker.id))
.setLeader(sourceBroker.id)
.setLeaderEpoch(0)
.setPartitionEpoch(0)
)

image = delta.apply(MetadataProvenance.EMPTY)
replicaManager.applyDelta(delta.topicsDelta(), image)

replicaManager.getPartitionOrException(topicPartition)
.localLogOrException
endPoint = new LocalLeaderEndPoint(
sourceBroker,
config,
replicaManager,
QuotaFactory.UNBOUNDED_QUOTA
)
}

@AfterEach
def tearDown(): Unit = {
CoreUtils.swallow(replicaManager.shutdown(checkpointHW = false), this)
CoreUtils.swallow(quotaManager.shutdown(), this)
}
class LocalLeaderEndPointTest extends LocalLeaderEndPointTestBase {

@Test
def testFetchLatestOffset(): Unit = {
Expand Down Expand Up @@ -233,66 +147,4 @@ class LocalLeaderEndPointTest extends Logging {
assertEquals(expected, result)
}

private class CallbackResult[T] {
private var value: Option[T] = None
private var fun: Option[T => Unit] = None

private def hasFired: Boolean = {
value.isDefined
}

def fire(value: T): Unit = {
this.value = Some(value)
fun.foreach(f => f(value))
}

def onFire(fun: T => Unit): CallbackResult[T] = {
this.fun = Some(fun)
if (this.hasFired) fire(value.get)
this
}
}

private def bumpLeaderEpoch(): Unit = {
val delta = new MetadataDelta(image)
delta.replay(new PartitionChangeRecord()
.setTopicId(topicId)
.setPartitionId(partition)
.setLeader(sourceBroker.id)
)

image = delta.apply(MetadataProvenance.EMPTY)
replicaManager.applyDelta(delta.topicsDelta, image)
}

private def appendRecords(replicaManager: ReplicaManager,
partition: TopicIdPartition,
records: MemoryRecords,
origin: AppendOrigin = AppendOrigin.CLIENT,
requiredAcks: Short = -1): CallbackResult[PartitionResponse] = {
val result = new CallbackResult[PartitionResponse]()
def appendCallback(responses: scala.collection.Map[TopicIdPartition, PartitionResponse]): Unit = {
val response = responses.get(partition)
assertTrue(response.isDefined)
result.fire(response.get)
}

replicaManager.appendRecords(
timeout = 1000,
requiredAcks = requiredAcks,
internalTopicsAllowed = false,
origin = origin,
entriesPerPartition = Map(partition -> records),
responseCallback = appendCallback)

result
}

private def records: MemoryRecords = {
MemoryRecords.withRecords(Compression.NONE,
new SimpleRecord("first message".getBytes()),
new SimpleRecord("second message".getBytes()),
new SimpleRecord("third message".getBytes()),
)
}
}
Loading