Skip to content
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
35 changes: 14 additions & 21 deletions core/src/main/java/kafka/log/remote/RemoteLogManager.java
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@
import kafka.log.remote.quota.RLMQuotaManager;
import kafka.log.remote.quota.RLMQuotaManagerConfig;
import kafka.server.BrokerTopicStats;
import kafka.server.KafkaConfig;
import kafka.server.QuotaType;
import kafka.server.StopPartition;
import org.apache.kafka.common.KafkaException;
Expand Down Expand Up @@ -152,7 +151,7 @@ public class RemoteLogManager implements Closeable {

private static final Logger LOGGER = LoggerFactory.getLogger(RemoteLogManager.class);
private static final String REMOTE_LOG_READER_THREAD_NAME_PREFIX = "remote-log-reader";
private final KafkaConfig config;
private final RemoteLogManagerConfig rlmConfig;
private final int brokerId;
private final String logDir;
private final Time time;
Expand Down Expand Up @@ -193,7 +192,7 @@ public class RemoteLogManager implements Closeable {
/**
* Creates RemoteLogManager instance with the given arguments.
*
* @param config Configuration required for remote logging subsystem(tiered storage) at the broker level.
* @param rlmConfig Configuration required for remote logging subsystem(tiered storage) at the broker level.
* @param brokerId id of the current broker.
* @param logDir directory of Kafka log segments.
* @param time Time instance.
Expand All @@ -203,7 +202,7 @@ public class RemoteLogManager implements Closeable {
* @param brokerTopicStats BrokerTopicStats instance to update the respective metrics.
* @param metrics Metrics instance
*/
public RemoteLogManager(KafkaConfig config,
public RemoteLogManager(RemoteLogManagerConfig rlmConfig,
int brokerId,
String logDir,
String clusterId,
Expand All @@ -212,7 +211,7 @@ public RemoteLogManager(KafkaConfig config,
BiConsumer<TopicPartition, Long> updateRemoteLogStartOffset,
BrokerTopicStats brokerTopicStats,
Metrics metrics) throws IOException {
this.config = config;
this.rlmConfig = rlmConfig;
this.brokerId = brokerId;
this.logDir = logDir;
this.clusterId = clusterId;
Expand All @@ -227,8 +226,7 @@ public RemoteLogManager(KafkaConfig config,
rlmCopyQuotaManager = createRLMCopyQuotaManager();
rlmFetchQuotaManager = createRLMFetchQuotaManager();

RemoteLogManagerConfig rlmConfig = config.remoteLogManagerConfig();
indexCache = new RemoteIndexCache(config.remoteLogIndexFileCacheTotalSizeBytes(), remoteLogStorageManager, logDir);
indexCache = new RemoteIndexCache(rlmConfig.remoteLogIndexFileCacheTotalSizeBytes(), remoteLogStorageManager, logDir);
delayInMs = rlmConfig.remoteLogManagerTaskIntervalMs();
rlmScheduledThreadPool = new RLMScheduledThreadPool(rlmConfig.remoteLogManagerThreadPoolSize());

Expand Down Expand Up @@ -276,29 +274,27 @@ Duration quotaTimeout() {
}

RLMQuotaManager createRLMCopyQuotaManager() {
return new RLMQuotaManager(copyQuotaManagerConfig(config), metrics, QuotaType.RLMCopy$.MODULE$,
return new RLMQuotaManager(copyQuotaManagerConfig(rlmConfig), metrics, QuotaType.RLMCopy$.MODULE$,
"Tracking copy byte-rate for Remote Log Manager", time);
}

RLMQuotaManager createRLMFetchQuotaManager() {
return new RLMQuotaManager(fetchQuotaManagerConfig(config), metrics, QuotaType.RLMFetch$.MODULE$,
return new RLMQuotaManager(fetchQuotaManagerConfig(rlmConfig), metrics, QuotaType.RLMFetch$.MODULE$,
"Tracking fetch byte-rate for Remote Log Manager", time);
}

public boolean isRemoteLogFetchQuotaExceeded() {
return rlmFetchQuotaManager.isQuotaExceeded();
}

static RLMQuotaManagerConfig copyQuotaManagerConfig(KafkaConfig config) {
RemoteLogManagerConfig rlmConfig = config.remoteLogManagerConfig();
return new RLMQuotaManagerConfig(config.remoteLogManagerCopyMaxBytesPerSecond(),
static RLMQuotaManagerConfig copyQuotaManagerConfig(RemoteLogManagerConfig rlmConfig) {
return new RLMQuotaManagerConfig(rlmConfig.remoteLogManagerCopyMaxBytesPerSecond(),
rlmConfig.remoteLogManagerCopyNumQuotaSamples(),
rlmConfig.remoteLogManagerCopyQuotaWindowSizeSeconds());
}

static RLMQuotaManagerConfig fetchQuotaManagerConfig(KafkaConfig config) {
RemoteLogManagerConfig rlmConfig = config.remoteLogManagerConfig();
return new RLMQuotaManagerConfig(config.remoteLogManagerFetchMaxBytesPerSecond(),
static RLMQuotaManagerConfig fetchQuotaManagerConfig(RemoteLogManagerConfig rlmConfig) {
return new RLMQuotaManagerConfig(rlmConfig.remoteLogManagerFetchMaxBytesPerSecond(),
rlmConfig.remoteLogManagerFetchNumQuotaSamples(),
rlmConfig.remoteLogManagerFetchQuotaWindowSizeSeconds());
}
Expand All @@ -315,7 +311,6 @@ private <T> T createDelegate(ClassLoader classLoader, String className) {

@SuppressWarnings("removal")
RemoteStorageManager createRemoteStorageManager() {
RemoteLogManagerConfig rlmConfig = config.remoteLogManagerConfig();
return java.security.AccessController.doPrivileged(new PrivilegedAction<RemoteStorageManager>() {
private final String classPath = rlmConfig.remoteStorageManagerClassPath();

Expand All @@ -332,14 +327,13 @@ public RemoteStorageManager run() {
}

private void configureRSM() {
final Map<String, Object> rsmProps = new HashMap<>(config.remoteLogManagerConfig().remoteStorageManagerProps());
final Map<String, Object> rsmProps = new HashMap<>(rlmConfig.remoteStorageManagerProps());
rsmProps.put(ServerConfigs.BROKER_ID_CONFIG, brokerId);
remoteLogStorageManager.configure(rsmProps);
}

@SuppressWarnings("removal")
RemoteLogMetadataManager createRemoteLogMetadataManager() {
RemoteLogManagerConfig rlmConfig = config.remoteLogManagerConfig();
return java.security.AccessController.doPrivileged(new PrivilegedAction<RemoteLogMetadataManager>() {
private final String classPath = rlmConfig.remoteLogMetadataManagerClassPath();

Expand All @@ -366,7 +360,7 @@ private void configureRLMM() {
rlmmProps.put(REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX + "security.protocol", e.securityProtocol().name);
});
// update the remoteLogMetadataProps here to override endpoint config if any
rlmmProps.putAll(config.remoteLogManagerConfig().remoteLogMetadataManagerProps());
rlmmProps.putAll(rlmConfig.remoteLogMetadataManagerProps());

rlmmProps.put(ServerConfigs.BROKER_ID_CONFIG, brokerId);
rlmmProps.put(LOG_DIR_CONFIG, logDir);
Expand Down Expand Up @@ -418,7 +412,7 @@ public void onLeadershipChange(Set<Partition> partitionsBecomeLeader,
Map<String, Uuid> topicIds) {
LOGGER.debug("Received leadership changes for leaders: {} and followers: {}", partitionsBecomeLeader, partitionsBecomeFollower);

if (config.remoteLogManagerConfig().isRemoteStorageSystemEnabled() && !isRemoteLogManagerConfigured()) {
if (rlmConfig.isRemoteStorageSystemEnabled() && !isRemoteLogManagerConfigured()) {
throw new KafkaException("RemoteLogManager is not configured when remote storage system is enabled");
}

Expand Down Expand Up @@ -1748,7 +1742,6 @@ public Future<Void> asyncRead(RemoteStorageFetchInfo fetchInfo, Consumer<RemoteL

void doHandleLeaderOrFollowerPartitions(TopicIdPartition topicPartition,
Consumer<RLMTask> convertToLeaderOrFollower) {
RemoteLogManagerConfig rlmConfig = config.remoteLogManagerConfig();
RLMTaskWithFuture rlmTaskWithFuture = leaderOrFollowerTasks.computeIfAbsent(topicPartition,
topicIdPartition -> {
RLMTask task = new RLMTask(topicIdPartition, rlmConfig.remoteLogMetadataCustomMetadataMaxBytes());
Expand Down
2 changes: 1 addition & 1 deletion core/src/main/scala/kafka/server/BrokerServer.scala
Original file line number Diff line number Diff line change
Expand Up @@ -621,7 +621,7 @@ class BrokerServer(

protected def createRemoteLogManager(): Option[RemoteLogManager] = {
if (config.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) {
Some(new RemoteLogManager(config, config.brokerId, config.logDirs.head, clusterId, time,
Some(new RemoteLogManager(config.remoteLogManagerConfig, config.brokerId, config.logDirs.head, clusterId, time,
(tp: TopicPartition) => logManager.getLog(tp).asJava,
(tp: TopicPartition, remoteLogStartOffset: java.lang.Long) => {
logManager.getLog(tp).foreach { log =>
Expand Down
10 changes: 1 addition & 9 deletions core/src/main/scala/kafka/server/KafkaConfig.scala
Original file line number Diff line number Diff line change
Expand Up @@ -231,7 +231,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
val zkEnableSecureAcls: Boolean = getBoolean(ZkConfigs.ZK_ENABLE_SECURE_ACLS_CONFIG)
val zkMaxInFlightRequests: Int = getInt(ZkConfigs.ZK_MAX_IN_FLIGHT_REQUESTS_CONFIG)

private val _remoteLogManagerConfig = new RemoteLogManagerConfig(props)
private val _remoteLogManagerConfig = new RemoteLogManagerConfig(this)
def remoteLogManagerConfig = _remoteLogManagerConfig

private def zkBooleanConfigOrSystemPropertyWithDefaultValue(propKey: String): Boolean = {
Expand Down Expand Up @@ -869,14 +869,6 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami

def logLocalRetentionMs: java.lang.Long = getLong(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP)

def remoteFetchMaxWaitMs = getInt(RemoteLogManagerConfig.REMOTE_FETCH_MAX_WAIT_MS_PROP)

def remoteLogIndexFileCacheTotalSizeBytes: Long = getLong(RemoteLogManagerConfig.REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES_PROP)

def remoteLogManagerCopyMaxBytesPerSecond: Long = getLong(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP)

def remoteLogManagerFetchMaxBytesPerSecond: Long = getLong(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_PROP)

validateValues()

@nowarn("cat=deprecation")
Expand Down
2 changes: 1 addition & 1 deletion core/src/main/scala/kafka/server/KafkaServer.scala
Original file line number Diff line number Diff line change
Expand Up @@ -691,7 +691,7 @@ class KafkaServer(

protected def createRemoteLogManager(): Option[RemoteLogManager] = {
if (config.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) {
Some(new RemoteLogManager(config, config.brokerId, config.logDirs.head, clusterId, time,
Some(new RemoteLogManager(config.remoteLogManagerConfig, config.brokerId, config.logDirs.head, clusterId, time,
(tp: TopicPartition) => logManager.getLog(tp).asJava,
(tp: TopicPartition, remoteLogStartOffset: java.lang.Long) => {
logManager.getLog(tp).foreach { log =>
Expand Down
2 changes: 1 addition & 1 deletion core/src/main/scala/kafka/server/ReplicaManager.scala
Original file line number Diff line number Diff line change
Expand Up @@ -1479,7 +1479,7 @@ class ReplicaManager(val config: KafkaConfig,
return Some(createLogReadResult(e))
}

val remoteFetchMaxWaitMs = config.remoteFetchMaxWaitMs.toLong
val remoteFetchMaxWaitMs = config.remoteLogManagerConfig.remoteFetchMaxWaitMs().toLong
val remoteFetch = new DelayedRemoteFetch(remoteFetchTask, remoteFetchResult, remoteFetchInfo, remoteFetchMaxWaitMs,
fetchPartitionStatus, params, logReadResults, this, responseCallback)
delayedRemoteFetchPurgatory.tryCompleteElseWatch(remoteFetch, Seq(key))
Expand Down
Loading