Skip to content
Open
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
Original file line number Diff line number Diff line change
Expand Up @@ -20,10 +20,8 @@ import java.io.File
import java.net.InetSocketAddress
import java.nio.file.Files
import java.nio.file.Paths
import java.util.OptionalInt
import java.util.{Optional, OptionalInt, Collection => JCollection, Map => JMap}
import java.util.concurrent.CompletableFuture
import java.util.{Map => JMap}
import java.util.{Collection => JCollection}
import kafka.server.KafkaConfig
import kafka.utils.CoreUtils
import kafka.utils.Logging
Expand All @@ -40,7 +38,7 @@ import org.apache.kafka.common.requests.RequestHeader
import org.apache.kafka.common.security.JaasContext
import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.utils.{LogContext, Time, Utils}
import org.apache.kafka.raft.{Endpoints, ExternalKRaftMetrics, FileQuorumStateStore, KafkaNetworkChannel, KafkaRaftClient, KafkaRaftClientDriver, LeaderAndEpoch, MetadataLogConfig, QuorumConfig, RaftClient, ReplicatedLog, TimingWheelExpirationService}
import org.apache.kafka.raft.{Endpoints, ExternalKRaftMetrics, FileQuorumStateStore, KafkaNetworkChannel, KafkaRaftClient, KafkaRaftClientDriver, MetadataLogConfig, QuorumConfig, RaftManager, ReplicatedLog, TimingWheelExpirationService}
import org.apache.kafka.server.ProcessRole
import org.apache.kafka.server.common.Feature
import org.apache.kafka.server.common.serialization.RecordSerde
Expand All @@ -50,7 +48,6 @@ import org.apache.kafka.server.util.timer.SystemTimer
import org.apache.kafka.storage.internals.log.{LogManager, UnifiedLog}

import scala.jdk.CollectionConverters._
import scala.jdk.OptionConverters._

object KafkaRaftManager {
private def createLogDirectory(logDir: File, logDirName: String): File = {
Expand Down Expand Up @@ -85,29 +82,6 @@ object KafkaRaftManager {
}
}

trait RaftManager[T] {
def handleRequest(
context: RequestContext,
header: RequestHeader,
request: ApiMessage,
createdTimeMs: Long
): CompletableFuture[ApiMessage]

def register(
listener: RaftClient.Listener[T]
): Unit

def leaderAndEpoch: LeaderAndEpoch

def client: RaftClient[T]

def replicatedLog: ReplicatedLog

def voterNode(id: Int, listener: ListenerName): Option[Node]

def recordSerde: RecordSerde[T]
}

class KafkaRaftManager[T](
clusterId: String,
config: KafkaConfig,
Expand Down Expand Up @@ -178,12 +152,6 @@ class KafkaRaftManager[T](
CoreUtils.swallow(dataDirLock.foreach(_.destroy()), this)
}

override def register(
listener: RaftClient.Listener[T]
): Unit = {
client.register(listener)
}

override def handleRequest(
context: RequestContext,
header: RequestHeader,
Expand Down Expand Up @@ -292,12 +260,8 @@ class KafkaRaftManager[T](
(controllerListenerName, networkClient)
}

override def leaderAndEpoch: LeaderAndEpoch = {
client.leaderAndEpoch
}

override def voterNode(id: Int, listener: ListenerName): Option[Node] = {
client.voterNode(id, listener).toScala
override def voterNode(id: Int, listener: ListenerName): Optional[Node] = {
client.voterNode(id, listener)
Copy link
Member

Choose a reason for hiding this comment

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

If the delegation is only a single line and the client is an interface, perhaps we don't need to wrap it with RaftManager

}

override def recordSerde: RecordSerde[T] = serde
Expand Down
4 changes: 2 additions & 2 deletions core/src/main/scala/kafka/server/ControllerApis.scala
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@ import java.util.Map.Entry
import java.util.concurrent.CompletableFuture
import java.util.function.Consumer
import kafka.network.RequestChannel
import kafka.raft.RaftManager
import kafka.server.QuotaFactory.QuotaManagers
import kafka.server.logger.RuntimeLoggerManager
import kafka.server.metadata.KRaftMetadataCache
Expand Down Expand Up @@ -55,6 +54,7 @@ import org.apache.kafka.image.publisher.ControllerRegistrationsPublisher
import org.apache.kafka.metadata.{BrokerHeartbeatReply, BrokerRegistrationReply}
import org.apache.kafka.common.security.auth.KafkaPrincipal
import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.raft.RaftManager
import org.apache.kafka.server.{ApiVersionManager, DelegationTokenManager, ProcessRole}
import org.apache.kafka.server.authorizer.Authorizer
import org.apache.kafka.server.common.{ApiMessageAndVersion, RequestLocal}
Expand Down Expand Up @@ -1070,7 +1070,7 @@ class ControllerApis(
EndpointType.CONTROLLER,
clusterId,
() => registrationsPublisher.describeClusterControllers(request.context.listenerName()),
() => raftManager.leaderAndEpoch.leaderId().orElse(-1)
() => raftManager.client.leaderAndEpoch.leaderId().orElse(-1)
)
requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
new DescribeClusterResponse(response.setThrottleTimeMs(requestThrottleMs)))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@

package kafka.server

import kafka.raft.RaftManager
import kafka.utils.Logging
import org.apache.kafka.clients._
import org.apache.kafka.common.metrics.Metrics
Expand All @@ -28,6 +27,7 @@ import org.apache.kafka.common.security.JaasContext
import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.utils.{LogContext, Time}
import org.apache.kafka.common.{Node, Reconfigurable}
import org.apache.kafka.raft.RaftManager
import org.apache.kafka.server.common.{ApiMessageAndVersion, ControllerRequestCompletionHandler, NodeToControllerChannelManager}
import org.apache.kafka.server.util.{InterBrokerSendThread, RequestAndCompletionHandler}

Expand All @@ -37,7 +37,7 @@ import java.util.concurrent.LinkedBlockingDeque
import java.util.concurrent.atomic.AtomicReference
import scala.collection.Seq
import scala.jdk.CollectionConverters._
import scala.jdk.OptionConverters.{RichOption, RichOptionalInt}
import scala.jdk.OptionConverters.{RichOption, RichOptional, RichOptionalInt}

case class ControllerInformation(
node: Option[Node],
Expand Down Expand Up @@ -79,10 +79,10 @@ class RaftControllerNodeProvider(
val saslMechanism: String
) extends ControllerNodeProvider with Logging {

private def idToNode(id: Int): Option[Node] = raftManager.voterNode(id, listenerName)
private def idToNode(id: Int): Option[Node] = raftManager.voterNode(id, listenerName).toScala

override def getControllerInfo(): ControllerInformation =
ControllerInformation(raftManager.leaderAndEpoch.leaderId.toScala.flatMap(idToNode),
ControllerInformation(raftManager.client.leaderAndEpoch.leaderId.toScala.flatMap(idToNode),
listenerName, securityProtocol, saslMechanism)
}

Expand Down
2 changes: 1 addition & 1 deletion core/src/main/scala/kafka/server/SharedServer.scala
Original file line number Diff line number Diff line change
Expand Up @@ -342,7 +342,7 @@ class SharedServer(
throw new RuntimeException("Unable to install metadata publishers.", t)
}
}
_raftManager.register(loader)
_raftManager.client.register(loader)
debug("Completed SharedServer startup.")
started = true
} catch {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,14 +18,14 @@
package kafka.tools

import kafka.network.RequestChannel
import kafka.raft.RaftManager
import kafka.server.ApiRequestHandler
import kafka.utils.Logging
import org.apache.kafka.common.internals.FatalExitError
import org.apache.kafka.common.message.{BeginQuorumEpochResponseData, EndQuorumEpochResponseData, FetchResponseData, FetchSnapshotResponseData, VoteResponseData}
import org.apache.kafka.common.protocol.{ApiKeys, ApiMessage}
import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, BeginQuorumEpochResponse, EndQuorumEpochResponse, FetchResponse, FetchSnapshotResponse, VoteResponse}
import org.apache.kafka.common.utils.Time
import org.apache.kafka.raft.RaftManager
import org.apache.kafka.server.ApiVersionManager
import org.apache.kafka.server.common.RequestLocal

Expand Down
6 changes: 3 additions & 3 deletions core/src/main/scala/kafka/tools/TestRaftServer.scala
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import java.util.concurrent.atomic.{AtomicInteger, AtomicLong}
import java.util.concurrent.{CompletableFuture, CountDownLatch, LinkedBlockingDeque, TimeUnit}
import joptsimple.{OptionException, OptionSpec}
import kafka.network.SocketServer
import kafka.raft.{DefaultExternalKRaftMetrics, KafkaRaftManager, RaftManager}
import kafka.raft.{DefaultExternalKRaftMetrics, KafkaRaftManager}
import kafka.server.{KafkaConfig, KafkaRequestHandlerPool}
import kafka.utils.{CoreUtils, Logging}
import org.apache.kafka.common.errors.InvalidConfigurationException
Expand All @@ -37,7 +37,7 @@ import org.apache.kafka.common.security.token.delegation.internals.DelegationTok
import org.apache.kafka.common.utils.{Exit, Time, Utils}
import org.apache.kafka.common.{TopicPartition, Uuid, protocol}
import org.apache.kafka.raft.errors.NotLeaderException
import org.apache.kafka.raft.{Batch, BatchReader, Endpoints, LeaderAndEpoch, QuorumConfig, RaftClient}
import org.apache.kafka.raft.{Batch, BatchReader, Endpoints, LeaderAndEpoch, QuorumConfig, RaftClient, RaftManager}
import org.apache.kafka.security.CredentialProvider
import org.apache.kafka.server.SimpleApiVersionManager
import org.apache.kafka.server.common.{FinalizedFeatures, MetadataVersion}
Expand Down Expand Up @@ -181,7 +181,7 @@ class TestRaftServer(

private var claimedEpoch: Option[Int] = None

raftManager.register(this)
raftManager.client.register(this)

override def handleLeaderChange(newLeaderAndEpoch: LeaderAndEpoch): Unit = {
if (newLeaderAndEpoch.isLeader(config.nodeId)) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ class AllocateProducerIdsRequestTest(cluster: ClusterInstance) {
def testAllocateProducersIdSentToController(): Unit = {
val sourceBroker = cluster.brokers.values().stream().findFirst().get().asInstanceOf[BrokerServer]

val controllerId = sourceBroker.raftManager.leaderAndEpoch.leaderId().getAsInt
val controllerId = sourceBroker.raftManager.client.leaderAndEpoch.leaderId().getAsInt
val controllerServer = cluster.controllers.values().stream()
.filter(_.config.nodeId == controllerId)
.findFirst()
Expand All @@ -50,7 +50,7 @@ class AllocateProducerIdsRequestTest(cluster: ClusterInstance) {
def testAllocateProducersIdSentToNonController(): Unit = {
val sourceBroker = cluster.brokers.values().stream().findFirst().get().asInstanceOf[BrokerServer]

val controllerId = sourceBroker.raftManager.leaderAndEpoch.leaderId().getAsInt
val controllerId = sourceBroker.raftManager.client.leaderAndEpoch.leaderId().getAsInt
val controllerServer = cluster.controllers().values().stream()
.filter(_.config.nodeId != controllerId)
.findFirst()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
package kafka.server

import kafka.network.RequestChannel
import kafka.raft.RaftManager
import kafka.server.QuotaFactory.QuotaManagers
import kafka.server.metadata.KRaftMetadataCache
import org.apache.kafka.clients.admin.AlterConfigOp
Expand Down Expand Up @@ -56,7 +55,7 @@ import org.apache.kafka.image.publisher.ControllerRegistrationsPublisher
import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.network.metrics.RequestChannelMetrics
import org.apache.kafka.network.Session
import org.apache.kafka.raft.QuorumConfig
import org.apache.kafka.raft.{QuorumConfig, RaftManager}
import org.apache.kafka.server.SimpleApiVersionManager
import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult, Authorizer}
import org.apache.kafka.server.common.{ApiMessageAndVersion, FinalizedFeatures, KRaftVersion, MetadataVersion, ProducerIdsBlock, RequestLocal}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -293,7 +293,7 @@ class LeaderEpochIntegrationTest extends QuorumTestHarness with Logging {
}

private def waitUntilQuorumLeaderElected(controllerServer: ControllerServer, timeout: Long = JTestUtils.DEFAULT_MAX_WAIT_MS): Int = {
val (leaderAndEpoch, _) = computeUntilTrue(controllerServer.raftManager.leaderAndEpoch, waitTime = timeout)(_.leaderId().isPresent)
val (leaderAndEpoch, _) = computeUntilTrue(controllerServer.raftManager.client.leaderAndEpoch, waitTime = timeout)(_.leaderId().isPresent)
leaderAndEpoch.leaderId().orElseThrow(() => new AssertionError(s"Quorum Controller leader not elected after $timeout ms"))
}

Expand Down
46 changes: 46 additions & 0 deletions raft/src/main/java/org/apache/kafka/raft/RaftManager.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,46 @@
/*
* 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.raft;

import org.apache.kafka.common.Node;
import org.apache.kafka.common.network.ListenerName;
import org.apache.kafka.common.protocol.ApiMessage;
import org.apache.kafka.common.requests.RequestContext;
import org.apache.kafka.common.requests.RequestHeader;
import org.apache.kafka.server.common.serialization.RecordSerde;

import java.util.Optional;
import java.util.concurrent.CompletableFuture;

public interface RaftManager<T> {

CompletableFuture<ApiMessage> handleRequest(
RequestContext context,
RequestHeader header,
ApiMessage request,
long createdTimeMs
);

RaftClient<T> client();

ReplicatedLog replicatedLog();

Optional<Node> voterNode(int id, ListenerName listener);

RecordSerde<T> recordSerde();
}