Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
18 commits
Select commit Hold shift + click to select a range
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
22 changes: 20 additions & 2 deletions core/src/main/scala/kafka/server/BrokerServer.scala
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,7 @@ import org.apache.kafka.metadata.{BrokerState, ListenerInfo}
import org.apache.kafka.metadata.publisher.AclPublisher
import org.apache.kafka.security.CredentialProvider
import org.apache.kafka.server.authorizer.Authorizer
import org.apache.kafka.server.common.{ApiMessageAndVersion, DirectoryEventHandler, NodeToControllerChannelManager, TopicIdPartition}
import org.apache.kafka.server.common.{ApiMessageAndVersion, BrokerReadyCallback, DirectoryEventHandler, NodeToControllerChannelManager, TopicIdPartition}
import org.apache.kafka.server.config.{ConfigType, DelegationTokenManagerConfigs}
import org.apache.kafka.server.log.remote.storage.{RemoteLogManager, RemoteLogManagerConfig}
import org.apache.kafka.server.metrics.{ClientMetricsReceiverPlugin, KafkaYammerMetrics}
Expand All @@ -64,7 +64,7 @@ import java.util
import java.util.Optional
import java.util.concurrent.locks.{Condition, ReentrantLock}
import java.util.concurrent.{CompletableFuture, ExecutionException, TimeUnit, TimeoutException}
import scala.collection.Map
import scala.collection.{Map, mutable}
import scala.jdk.CollectionConverters._
import scala.jdk.OptionConverters.RichOption

Expand Down Expand Up @@ -162,6 +162,8 @@ class BrokerServer(

var persister: Persister = _

private val brokerReadyCallbacks = new mutable.ListBuffer[BrokerReadyCallback]()

private def maybeChangeStatus(from: ProcessStatus, to: ProcessStatus): Boolean = {
lock.lock()
try {
Expand Down Expand Up @@ -283,6 +285,9 @@ class BrokerServer(

remoteLogManagerOpt = createRemoteLogManager(listenerInfo)

remoteLogManagerOpt.foreach(rlm =>
registerBrokerReadyCallback(rlm.remoteLogMetadataManager()))

alterPartitionManager = AlterPartitionManager(
config,
scheduler = kafkaScheduler,
Expand Down Expand Up @@ -589,6 +594,15 @@ class BrokerServer(
"all of the SocketServer Acceptors to be started",
enableRequestProcessingFuture, startupDeadline, time)

brokerReadyCallbacks.foreach { callback =>
try {
callback.onBrokerReady()
} catch {
case e: Exception =>
error(s"Error executing broker ready callback: ${callback.getClass.getSimpleName}", e)
}
}

maybeChangeStatus(STARTING, STARTED)
} catch {
case e: Throwable =>
Expand All @@ -599,6 +613,10 @@ class BrokerServer(
}
}

def registerBrokerReadyCallback(callback: BrokerReadyCallback): Unit = {
brokerReadyCallbacks += callback
}

private def createGroupCoordinator(): GroupCoordinator = {
// Create group coordinator, but don't start it until we've started replica manager.
// Hardcode Time.SYSTEM for now as some Streams tests fail otherwise, it would be good
Expand Down
Binary file not shown.
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

import org.apache.kafka.common.Configurable;
import org.apache.kafka.common.TopicIdPartition;
import org.apache.kafka.server.common.BrokerReadyCallback;

import java.io.Closeable;
import java.util.Iterator;
Expand Down Expand Up @@ -52,7 +53,7 @@
* The following tags are automatically added to all metrics registered: <code>config</code> set to
* <code>remote.log.metadata.manager.class.name</code>, and <code>class</code> set to the RemoteLogMetadataManager class name.
*/
public interface RemoteLogMetadataManager extends Configurable, Closeable {
public interface RemoteLogMetadataManager extends BrokerReadyCallback, Configurable, Closeable {

/**
* This method is used to add {@link RemoteLogSegmentMetadata} asynchronously with the containing {@link RemoteLogSegmentId} into {@link RemoteLogMetadataManager}.
Expand Down Expand Up @@ -242,4 +243,6 @@ default Optional<RemoteLogSegmentMetadata> nextSegmentWithTxnIndex(TopicIdPartit
default boolean isReady(TopicIdPartition topicIdPartition) {
return true;
}

default void onBrokerReady() {}
}
Original file line number Diff line number Diff line change
Expand Up @@ -296,7 +296,6 @@ public void configure(Map<String, ?> configs) {
// successful.
initializationThread = KafkaThread.nonDaemon(
"RLMMInitializationThread", () -> initializeResources(rlmmConfig));
initializationThread.start();
log.info("Successfully configured topic-based RLMM with config: {}", rlmmConfig);
} else {
log.info("Skipping configure as it is already configured.");
Expand Down Expand Up @@ -370,6 +369,11 @@ private void initializeResources(TopicBasedRemoteLogMetadataManagerConfig rlmmCo
initializationFailed = true;
}
}
@Override
public void onBrokerReady() {
log.info("Broker is ready for requests, now initializing RLMM resources");
initializationThread.start();
}

boolean doesTopicExist(Admin admin, String topic) throws ExecutionException, InterruptedException {
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -69,8 +69,8 @@ public final class TopicBasedRemoteLogMetadataManagerConfig {
"retrying RemoteLogMetadataManager resources initialization again.";

public static final String REMOTE_LOG_METADATA_INITIALIZATION_RETRY_MAX_TIMEOUT_MS_DOC = "The maximum amount of time in milliseconds " +
"for retrying RemoteLogMetadataManager resources initialization. When total retry intervals reach this timeout, initialization " +
"is considered as failed and broker starts shutting down.";
"for retrying RemoteLogMetadataManager resources initialization. The timer for this timeout begins after the broker is ready to handle requests. " +
"When total retry intervals reach this timeout, initialization is considered as failed and broker starts shutting down.";

public static final String REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX = "remote.log.metadata.common.client.";
public static final String REMOTE_LOG_METADATA_PRODUCER_PREFIX = "remote.log.metadata.producer.";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -423,7 +423,7 @@ private Plugin<RemoteLogMetadataManager> configAndWrapRlmmPlugin(RemoteLogMetada
return Plugin.wrapInstance(rlmm, metrics, RemoteLogManagerConfig.REMOTE_LOG_METADATA_MANAGER_CLASS_NAME_PROP);
}

RemoteLogMetadataManager remoteLogMetadataManager() {
public RemoteLogMetadataManager remoteLogMetadataManager() {
return remoteLogMetadataManagerPlugin.get();
}

Expand Down