Skip to content

Commit

Permalink
[aiven] workaround to randomize remote fetch partition
Browse files Browse the repository at this point in the history
At the moment, it's always the first partition with remote fetch information that gets fetched at a time.
This PR is a workaround until better distribution is implemented (either parallel fetching or randomized) by adding a randomizer on the partitions with remote info available.
  • Loading branch information
jeqo committed May 16, 2024
1 parent fcbcb70 commit a12c6c7
Show file tree
Hide file tree
Showing 2 changed files with 71 additions and 4 deletions.
30 changes: 26 additions & 4 deletions core/src/main/scala/kafka/server/ReplicaManager.scala
Original file line number Diff line number Diff line change
Expand Up @@ -68,9 +68,11 @@ import java.util.concurrent.atomic.AtomicBoolean
import java.util.concurrent.locks.Lock
import java.util.concurrent.{CompletableFuture, Future, RejectedExecutionException, TimeUnit}
import java.util.{Optional, OptionalInt, OptionalLong}
import scala.collection.mutable.ArrayBuffer
import scala.collection.{Map, Seq, Set, mutable}
import scala.compat.java8.OptionConverters._
import scala.jdk.CollectionConverters._
import scala.util.Random

/*
* Result metadata of a log append operation on the log
Expand Down Expand Up @@ -1646,7 +1648,9 @@ class ReplicaManager(val config: KafkaConfig,
var errorReadingData = false

// The 1st topic-partition that has to be read from remote storage
var remoteFetchInfo: Optional[RemoteStorageFetchInfo] = Optional.empty()
// WORKAROUND: randomize partition fetched
// var remoteFetchInfo: Optional[RemoteStorageFetchInfo] = Optional.empty()
var remoteFetchInfos: Seq[Optional[RemoteStorageFetchInfo]] = Seq.empty

var hasDivergingEpoch = false
var hasPreferredReadReplica = false
Expand All @@ -1657,8 +1661,12 @@ class ReplicaManager(val config: KafkaConfig,
brokerTopicStats.allTopicsStats.totalFetchRequestRate.mark()
if (logReadResult.error != Errors.NONE)
errorReadingData = true
if (!remoteFetchInfo.isPresent && logReadResult.info.delayedRemoteStorageFetch.isPresent) {
remoteFetchInfo = logReadResult.info.delayedRemoteStorageFetch
// WORKAROUND: randomize partition fetched
//if (!remoteFetchInfo.isPresent && logReadResult.info.delayedRemoteStorageFetch.isPresent) {
// remoteFetchInfo = logReadResult.info.delayedRemoteStorageFetch
//}
if (logReadResult.info.delayedRemoteStorageFetch.isPresent) {
remoteFetchInfos = remoteFetchInfos ++ Seq(logReadResult.info.delayedRemoteStorageFetch)
}
if (logReadResult.divergingEpoch.nonEmpty)
hasDivergingEpoch = true
Expand All @@ -1668,6 +1676,8 @@ class ReplicaManager(val config: KafkaConfig,
logReadResultMap.put(topicIdPartition, logReadResult)
}

val remoteFetchInfo: Optional[RemoteStorageFetchInfo] = maybeRemoteFetchInfo(remoteFetchInfos)

// Respond immediately if no remote fetches are required and any of the below conditions is true
// 1) fetch request does not want to wait
// 2) fetch request does not require any data
Expand All @@ -1684,7 +1694,7 @@ class ReplicaManager(val config: KafkaConfig,
responseCallback(fetchPartitionData)
} else {
// construct the fetch results from the read results
val fetchPartitionStatus = new mutable.ArrayBuffer[(TopicIdPartition, FetchPartitionStatus)]
val fetchPartitionStatus = new ArrayBuffer[(TopicIdPartition, FetchPartitionStatus)]
fetchInfos.foreach { case (topicIdPartition, partitionData) =>
logReadResultMap.get(topicIdPartition).foreach(logReadResult => {
val logOffsetMetadata = logReadResult.info.fetchOffsetMetadata
Expand Down Expand Up @@ -1723,6 +1733,18 @@ class ReplicaManager(val config: KafkaConfig,
}
}

// WORKAROUND: randomize partition fetched
private val rnd = new Random()
def maybeRemoteFetchInfo(remoteFetchInfos: Seq[Optional[RemoteStorageFetchInfo]]): Optional[RemoteStorageFetchInfo] = {
val remoteFetchInfo: Optional[RemoteStorageFetchInfo] = if (remoteFetchInfos.isEmpty) {
Optional.empty()
} else {
val i = rnd.nextInt(remoteFetchInfos.size)
remoteFetchInfos(i)
}
remoteFetchInfo
}

/**
* Read from multiple topic partitions at the given offset up to maxSize bytes
*/
Expand Down
45 changes: 45 additions & 0 deletions core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
Original file line number Diff line number Diff line change
Expand Up @@ -6558,6 +6558,51 @@ class ReplicaManagerTest {
))
}
}

@Test
def testRandomizedRemoteFetchInfo(): Unit = {
// Given
val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect, logDirCount = 2)
val config = KafkaConfig.fromProps(props)
val logDirFiles = config.logDirs.map(new File(_))
val logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size)
val logManager = TestUtils.createLogManager(logDirFiles, defaultConfig = new LogConfig(new Properties()), time = time)
val mockZkClient = mock(classOf[KafkaZkClient])
val replicaManager = new ReplicaManager(
metrics = metrics,
config = config,
time = time,
scheduler = time.scheduler,
logManager = logManager,
quotaManagers = quotaManager,
metadataCache = MetadataCache.zkMetadataCache(config.brokerId, config.interBrokerProtocolVersion),
logDirFailureChannel = logDirFailureChannel,
alterPartitionManager = alterPartitionManager,
threadNamePrefix = Option(this.getClass.getName),
zkClient = Option(mockZkClient),
)

var infos: Seq[Optional[RemoteStorageFetchInfo]] = Seq()

// When
val emptyRemoteFetchInfo = replicaManager.maybeRemoteFetchInfo(infos)

// Then
assertFalse(emptyRemoteFetchInfo.isPresent)

// Given
val info0 = new RemoteStorageFetchInfo(1000, false, new TopicPartition(topic, 0), null, FetchIsolation.HIGH_WATERMARK, false)
val info1 = new RemoteStorageFetchInfo(1000, false, new TopicPartition(topic, 1), null, FetchIsolation.HIGH_WATERMARK, false)
val info2 = new RemoteStorageFetchInfo(1000, false, new TopicPartition(topic, 2), null, FetchIsolation.HIGH_WATERMARK, false)
infos = infos ++ Seq(Optional.of(info0), Optional.of(info1), Optional.of(info2))

// When
val someRemoteFetchInfo = replicaManager.maybeRemoteFetchInfo(infos)

// Then
assertTrue(someRemoteFetchInfo.isPresent)
println(someRemoteFetchInfo.get())
}
}

class MockReplicaSelector extends ReplicaSelector {
Expand Down

0 comments on commit a12c6c7

Please sign in to comment.