mirror of https://github.com/apache/kafka.git
KAFKA-15265: Reapply dynamic remote configs after broker restart (#16353)
The below remote log configs can be configured dynamically: 1. remote.log.manager.copy.max.bytes.per.second 2. remote.log.manager.fetch.max.bytes.per.second and 3. remote.log.index.file.cache.total.size.bytes If those values are configured dynamically, then during the broker restart, it ensures the dynamic values are loaded instead of the static values from the config. Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Satish Duggana <satishd@apache.org>, Luke Chen <showuon@gmail.com>
This commit is contained in:
parent
ceab1fe658
commit
8abeaf3cb4
|
@ -24,6 +24,7 @@ import kafka.log.UnifiedLog;
|
||||||
import kafka.log.remote.quota.RLMQuotaManager;
|
import kafka.log.remote.quota.RLMQuotaManager;
|
||||||
import kafka.log.remote.quota.RLMQuotaManagerConfig;
|
import kafka.log.remote.quota.RLMQuotaManagerConfig;
|
||||||
import kafka.server.BrokerTopicStats;
|
import kafka.server.BrokerTopicStats;
|
||||||
|
import kafka.server.KafkaConfig;
|
||||||
import kafka.server.QuotaType;
|
import kafka.server.QuotaType;
|
||||||
import kafka.server.StopPartition;
|
import kafka.server.StopPartition;
|
||||||
import org.apache.kafka.common.KafkaException;
|
import org.apache.kafka.common.KafkaException;
|
||||||
|
@ -151,7 +152,7 @@ public class RemoteLogManager implements Closeable {
|
||||||
|
|
||||||
private static final Logger LOGGER = LoggerFactory.getLogger(RemoteLogManager.class);
|
private static final Logger LOGGER = LoggerFactory.getLogger(RemoteLogManager.class);
|
||||||
private static final String REMOTE_LOG_READER_THREAD_NAME_PREFIX = "remote-log-reader";
|
private static final String REMOTE_LOG_READER_THREAD_NAME_PREFIX = "remote-log-reader";
|
||||||
private final RemoteLogManagerConfig rlmConfig;
|
private final KafkaConfig config;
|
||||||
private final int brokerId;
|
private final int brokerId;
|
||||||
private final String logDir;
|
private final String logDir;
|
||||||
private final Time time;
|
private final Time time;
|
||||||
|
@ -192,7 +193,7 @@ public class RemoteLogManager implements Closeable {
|
||||||
/**
|
/**
|
||||||
* Creates RemoteLogManager instance with the given arguments.
|
* Creates RemoteLogManager instance with the given arguments.
|
||||||
*
|
*
|
||||||
* @param rlmConfig Configuration required for remote logging subsystem(tiered storage) at the broker level.
|
* @param config Configuration required for remote logging subsystem(tiered storage) at the broker level.
|
||||||
* @param brokerId id of the current broker.
|
* @param brokerId id of the current broker.
|
||||||
* @param logDir directory of Kafka log segments.
|
* @param logDir directory of Kafka log segments.
|
||||||
* @param time Time instance.
|
* @param time Time instance.
|
||||||
|
@ -202,7 +203,7 @@ public class RemoteLogManager implements Closeable {
|
||||||
* @param brokerTopicStats BrokerTopicStats instance to update the respective metrics.
|
* @param brokerTopicStats BrokerTopicStats instance to update the respective metrics.
|
||||||
* @param metrics Metrics instance
|
* @param metrics Metrics instance
|
||||||
*/
|
*/
|
||||||
public RemoteLogManager(RemoteLogManagerConfig rlmConfig,
|
public RemoteLogManager(KafkaConfig config,
|
||||||
int brokerId,
|
int brokerId,
|
||||||
String logDir,
|
String logDir,
|
||||||
String clusterId,
|
String clusterId,
|
||||||
|
@ -211,7 +212,7 @@ public class RemoteLogManager implements Closeable {
|
||||||
BiConsumer<TopicPartition, Long> updateRemoteLogStartOffset,
|
BiConsumer<TopicPartition, Long> updateRemoteLogStartOffset,
|
||||||
BrokerTopicStats brokerTopicStats,
|
BrokerTopicStats brokerTopicStats,
|
||||||
Metrics metrics) throws IOException {
|
Metrics metrics) throws IOException {
|
||||||
this.rlmConfig = rlmConfig;
|
this.config = config;
|
||||||
this.brokerId = brokerId;
|
this.brokerId = brokerId;
|
||||||
this.logDir = logDir;
|
this.logDir = logDir;
|
||||||
this.clusterId = clusterId;
|
this.clusterId = clusterId;
|
||||||
|
@ -226,7 +227,8 @@ public class RemoteLogManager implements Closeable {
|
||||||
rlmCopyQuotaManager = createRLMCopyQuotaManager();
|
rlmCopyQuotaManager = createRLMCopyQuotaManager();
|
||||||
rlmFetchQuotaManager = createRLMFetchQuotaManager();
|
rlmFetchQuotaManager = createRLMFetchQuotaManager();
|
||||||
|
|
||||||
indexCache = new RemoteIndexCache(rlmConfig.remoteLogIndexFileCacheTotalSizeBytes(), remoteLogStorageManager, logDir);
|
RemoteLogManagerConfig rlmConfig = config.remoteLogManagerConfig();
|
||||||
|
indexCache = new RemoteIndexCache(config.remoteLogIndexFileCacheTotalSizeBytes(), remoteLogStorageManager, logDir);
|
||||||
delayInMs = rlmConfig.remoteLogManagerTaskIntervalMs();
|
delayInMs = rlmConfig.remoteLogManagerTaskIntervalMs();
|
||||||
rlmScheduledThreadPool = new RLMScheduledThreadPool(rlmConfig.remoteLogManagerThreadPoolSize());
|
rlmScheduledThreadPool = new RLMScheduledThreadPool(rlmConfig.remoteLogManagerThreadPoolSize());
|
||||||
|
|
||||||
|
@ -274,12 +276,12 @@ public class RemoteLogManager implements Closeable {
|
||||||
}
|
}
|
||||||
|
|
||||||
RLMQuotaManager createRLMCopyQuotaManager() {
|
RLMQuotaManager createRLMCopyQuotaManager() {
|
||||||
return new RLMQuotaManager(copyQuotaManagerConfig(rlmConfig), metrics, QuotaType.RLMCopy$.MODULE$,
|
return new RLMQuotaManager(copyQuotaManagerConfig(config), metrics, QuotaType.RLMCopy$.MODULE$,
|
||||||
"Tracking copy byte-rate for Remote Log Manager", time);
|
"Tracking copy byte-rate for Remote Log Manager", time);
|
||||||
}
|
}
|
||||||
|
|
||||||
RLMQuotaManager createRLMFetchQuotaManager() {
|
RLMQuotaManager createRLMFetchQuotaManager() {
|
||||||
return new RLMQuotaManager(fetchQuotaManagerConfig(rlmConfig), metrics, QuotaType.RLMFetch$.MODULE$,
|
return new RLMQuotaManager(fetchQuotaManagerConfig(config), metrics, QuotaType.RLMFetch$.MODULE$,
|
||||||
"Tracking fetch byte-rate for Remote Log Manager", time);
|
"Tracking fetch byte-rate for Remote Log Manager", time);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -287,14 +289,16 @@ public class RemoteLogManager implements Closeable {
|
||||||
return rlmFetchQuotaManager.isQuotaExceeded();
|
return rlmFetchQuotaManager.isQuotaExceeded();
|
||||||
}
|
}
|
||||||
|
|
||||||
static RLMQuotaManagerConfig copyQuotaManagerConfig(RemoteLogManagerConfig rlmConfig) {
|
static RLMQuotaManagerConfig copyQuotaManagerConfig(KafkaConfig config) {
|
||||||
return new RLMQuotaManagerConfig(rlmConfig.remoteLogManagerCopyMaxBytesPerSecond(),
|
RemoteLogManagerConfig rlmConfig = config.remoteLogManagerConfig();
|
||||||
|
return new RLMQuotaManagerConfig(config.remoteLogManagerCopyMaxBytesPerSecond(),
|
||||||
rlmConfig.remoteLogManagerCopyNumQuotaSamples(),
|
rlmConfig.remoteLogManagerCopyNumQuotaSamples(),
|
||||||
rlmConfig.remoteLogManagerCopyQuotaWindowSizeSeconds());
|
rlmConfig.remoteLogManagerCopyQuotaWindowSizeSeconds());
|
||||||
}
|
}
|
||||||
|
|
||||||
static RLMQuotaManagerConfig fetchQuotaManagerConfig(RemoteLogManagerConfig rlmConfig) {
|
static RLMQuotaManagerConfig fetchQuotaManagerConfig(KafkaConfig config) {
|
||||||
return new RLMQuotaManagerConfig(rlmConfig.remoteLogManagerFetchMaxBytesPerSecond(),
|
RemoteLogManagerConfig rlmConfig = config.remoteLogManagerConfig();
|
||||||
|
return new RLMQuotaManagerConfig(config.remoteLogManagerFetchMaxBytesPerSecond(),
|
||||||
rlmConfig.remoteLogManagerFetchNumQuotaSamples(),
|
rlmConfig.remoteLogManagerFetchNumQuotaSamples(),
|
||||||
rlmConfig.remoteLogManagerFetchQuotaWindowSizeSeconds());
|
rlmConfig.remoteLogManagerFetchQuotaWindowSizeSeconds());
|
||||||
}
|
}
|
||||||
|
@ -311,6 +315,7 @@ public class RemoteLogManager implements Closeable {
|
||||||
|
|
||||||
@SuppressWarnings("removal")
|
@SuppressWarnings("removal")
|
||||||
RemoteStorageManager createRemoteStorageManager() {
|
RemoteStorageManager createRemoteStorageManager() {
|
||||||
|
RemoteLogManagerConfig rlmConfig = config.remoteLogManagerConfig();
|
||||||
return java.security.AccessController.doPrivileged(new PrivilegedAction<RemoteStorageManager>() {
|
return java.security.AccessController.doPrivileged(new PrivilegedAction<RemoteStorageManager>() {
|
||||||
private final String classPath = rlmConfig.remoteStorageManagerClassPath();
|
private final String classPath = rlmConfig.remoteStorageManagerClassPath();
|
||||||
|
|
||||||
|
@ -327,13 +332,14 @@ public class RemoteLogManager implements Closeable {
|
||||||
}
|
}
|
||||||
|
|
||||||
private void configureRSM() {
|
private void configureRSM() {
|
||||||
final Map<String, Object> rsmProps = new HashMap<>(rlmConfig.remoteStorageManagerProps());
|
final Map<String, Object> rsmProps = new HashMap<>(config.remoteLogManagerConfig().remoteStorageManagerProps());
|
||||||
rsmProps.put(ServerConfigs.BROKER_ID_CONFIG, brokerId);
|
rsmProps.put(ServerConfigs.BROKER_ID_CONFIG, brokerId);
|
||||||
remoteLogStorageManager.configure(rsmProps);
|
remoteLogStorageManager.configure(rsmProps);
|
||||||
}
|
}
|
||||||
|
|
||||||
@SuppressWarnings("removal")
|
@SuppressWarnings("removal")
|
||||||
RemoteLogMetadataManager createRemoteLogMetadataManager() {
|
RemoteLogMetadataManager createRemoteLogMetadataManager() {
|
||||||
|
RemoteLogManagerConfig rlmConfig = config.remoteLogManagerConfig();
|
||||||
return java.security.AccessController.doPrivileged(new PrivilegedAction<RemoteLogMetadataManager>() {
|
return java.security.AccessController.doPrivileged(new PrivilegedAction<RemoteLogMetadataManager>() {
|
||||||
private final String classPath = rlmConfig.remoteLogMetadataManagerClassPath();
|
private final String classPath = rlmConfig.remoteLogMetadataManagerClassPath();
|
||||||
|
|
||||||
|
@ -360,7 +366,7 @@ public class RemoteLogManager implements Closeable {
|
||||||
rlmmProps.put(REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX + "security.protocol", e.securityProtocol().name);
|
rlmmProps.put(REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX + "security.protocol", e.securityProtocol().name);
|
||||||
});
|
});
|
||||||
// update the remoteLogMetadataProps here to override endpoint config if any
|
// update the remoteLogMetadataProps here to override endpoint config if any
|
||||||
rlmmProps.putAll(rlmConfig.remoteLogMetadataManagerProps());
|
rlmmProps.putAll(config.remoteLogManagerConfig().remoteLogMetadataManagerProps());
|
||||||
|
|
||||||
rlmmProps.put(ServerConfigs.BROKER_ID_CONFIG, brokerId);
|
rlmmProps.put(ServerConfigs.BROKER_ID_CONFIG, brokerId);
|
||||||
rlmmProps.put(LOG_DIR_CONFIG, logDir);
|
rlmmProps.put(LOG_DIR_CONFIG, logDir);
|
||||||
|
@ -412,7 +418,7 @@ public class RemoteLogManager implements Closeable {
|
||||||
Map<String, Uuid> topicIds) {
|
Map<String, Uuid> topicIds) {
|
||||||
LOGGER.debug("Received leadership changes for leaders: {} and followers: {}", partitionsBecomeLeader, partitionsBecomeFollower);
|
LOGGER.debug("Received leadership changes for leaders: {} and followers: {}", partitionsBecomeLeader, partitionsBecomeFollower);
|
||||||
|
|
||||||
if (this.rlmConfig.isRemoteStorageSystemEnabled() && !isRemoteLogManagerConfigured()) {
|
if (config.remoteLogManagerConfig().isRemoteStorageSystemEnabled() && !isRemoteLogManagerConfigured()) {
|
||||||
throw new KafkaException("RemoteLogManager is not configured when remote storage system is enabled");
|
throw new KafkaException("RemoteLogManager is not configured when remote storage system is enabled");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1742,9 +1748,10 @@ public class RemoteLogManager implements Closeable {
|
||||||
|
|
||||||
void doHandleLeaderOrFollowerPartitions(TopicIdPartition topicPartition,
|
void doHandleLeaderOrFollowerPartitions(TopicIdPartition topicPartition,
|
||||||
Consumer<RLMTask> convertToLeaderOrFollower) {
|
Consumer<RLMTask> convertToLeaderOrFollower) {
|
||||||
|
RemoteLogManagerConfig rlmConfig = config.remoteLogManagerConfig();
|
||||||
RLMTaskWithFuture rlmTaskWithFuture = leaderOrFollowerTasks.computeIfAbsent(topicPartition,
|
RLMTaskWithFuture rlmTaskWithFuture = leaderOrFollowerTasks.computeIfAbsent(topicPartition,
|
||||||
topicIdPartition -> {
|
topicIdPartition -> {
|
||||||
RLMTask task = new RLMTask(topicIdPartition, this.rlmConfig.remoteLogMetadataCustomMetadataMaxBytes());
|
RLMTask task = new RLMTask(topicIdPartition, rlmConfig.remoteLogMetadataCustomMetadataMaxBytes());
|
||||||
// set this upfront when it is getting initialized instead of doing it after scheduling.
|
// set this upfront when it is getting initialized instead of doing it after scheduling.
|
||||||
convertToLeaderOrFollower.accept(task);
|
convertToLeaderOrFollower.accept(task);
|
||||||
LOGGER.info("Created a new task: {} and getting scheduled", task);
|
LOGGER.info("Created a new task: {} and getting scheduled", task);
|
||||||
|
|
|
@ -621,7 +621,7 @@ class BrokerServer(
|
||||||
|
|
||||||
protected def createRemoteLogManager(): Option[RemoteLogManager] = {
|
protected def createRemoteLogManager(): Option[RemoteLogManager] = {
|
||||||
if (config.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) {
|
if (config.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) {
|
||||||
Some(new RemoteLogManager(config.remoteLogManagerConfig, config.brokerId, config.logDirs.head, clusterId, time,
|
Some(new RemoteLogManager(config, config.brokerId, config.logDirs.head, clusterId, time,
|
||||||
(tp: TopicPartition) => logManager.getLog(tp).asJava,
|
(tp: TopicPartition) => logManager.getLog(tp).asJava,
|
||||||
(tp: TopicPartition, remoteLogStartOffset: java.lang.Long) => {
|
(tp: TopicPartition, remoteLogStartOffset: java.lang.Long) => {
|
||||||
logManager.getLog(tp).foreach { log =>
|
logManager.getLog(tp).foreach { log =>
|
||||||
|
|
|
@ -871,6 +871,12 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
|
||||||
|
|
||||||
def remoteFetchMaxWaitMs = getInt(RemoteLogManagerConfig.REMOTE_FETCH_MAX_WAIT_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()
|
validateValues()
|
||||||
|
|
||||||
@nowarn("cat=deprecation")
|
@nowarn("cat=deprecation")
|
||||||
|
|
|
@ -691,7 +691,7 @@ class KafkaServer(
|
||||||
|
|
||||||
protected def createRemoteLogManager(): Option[RemoteLogManager] = {
|
protected def createRemoteLogManager(): Option[RemoteLogManager] = {
|
||||||
if (config.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) {
|
if (config.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) {
|
||||||
Some(new RemoteLogManager(config.remoteLogManagerConfig, config.brokerId, config.logDirs.head, clusterId, time,
|
Some(new RemoteLogManager(config, config.brokerId, config.logDirs.head, clusterId, time,
|
||||||
(tp: TopicPartition) => logManager.getLog(tp).asJava,
|
(tp: TopicPartition) => logManager.getLog(tp).asJava,
|
||||||
(tp: TopicPartition, remoteLogStartOffset: java.lang.Long) => {
|
(tp: TopicPartition, remoteLogStartOffset: java.lang.Long) => {
|
||||||
logManager.getLog(tp).foreach { log =>
|
logManager.getLog(tp).foreach { log =>
|
||||||
|
|
|
@ -193,7 +193,7 @@ public class RemoteLogManagerTest {
|
||||||
private final RemoteStorageManager remoteStorageManager = mock(RemoteStorageManager.class);
|
private final RemoteStorageManager remoteStorageManager = mock(RemoteStorageManager.class);
|
||||||
private final RemoteLogMetadataManager remoteLogMetadataManager = mock(RemoteLogMetadataManager.class);
|
private final RemoteLogMetadataManager remoteLogMetadataManager = mock(RemoteLogMetadataManager.class);
|
||||||
private final RLMQuotaManager rlmCopyQuotaManager = mock(RLMQuotaManager.class);
|
private final RLMQuotaManager rlmCopyQuotaManager = mock(RLMQuotaManager.class);
|
||||||
private RemoteLogManagerConfig remoteLogManagerConfig = null;
|
private KafkaConfig config;
|
||||||
|
|
||||||
private BrokerTopicStats brokerTopicStats = null;
|
private BrokerTopicStats brokerTopicStats = null;
|
||||||
private final Metrics metrics = new Metrics(time);
|
private final Metrics metrics = new Metrics(time);
|
||||||
|
@ -223,10 +223,11 @@ public class RemoteLogManagerTest {
|
||||||
Properties props = kafka.utils.TestUtils.createDummyBrokerConfig();
|
Properties props = kafka.utils.TestUtils.createDummyBrokerConfig();
|
||||||
props.setProperty(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, "true");
|
props.setProperty(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, "true");
|
||||||
props.setProperty(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_TASK_INTERVAL_MS_PROP, "100");
|
props.setProperty(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_TASK_INTERVAL_MS_PROP, "100");
|
||||||
remoteLogManagerConfig = createRLMConfig(props);
|
createRLMConfig(props);
|
||||||
|
config = KafkaConfig.fromProps(props);
|
||||||
brokerTopicStats = new BrokerTopicStats(KafkaConfig.fromProps(props).remoteLogManagerConfig().isRemoteStorageSystemEnabled());
|
brokerTopicStats = new BrokerTopicStats(KafkaConfig.fromProps(props).remoteLogManagerConfig().isRemoteStorageSystemEnabled());
|
||||||
|
|
||||||
remoteLogManager = new RemoteLogManager(remoteLogManagerConfig, brokerId, logDir, clusterId, time,
|
remoteLogManager = new RemoteLogManager(config, brokerId, logDir, clusterId, time,
|
||||||
tp -> Optional.of(mockLog),
|
tp -> Optional.of(mockLog),
|
||||||
(topicPartition, offset) -> currentLogStartOffset.set(offset),
|
(topicPartition, offset) -> currentLogStartOffset.set(offset),
|
||||||
brokerTopicStats, metrics) {
|
brokerTopicStats, metrics) {
|
||||||
|
@ -378,10 +379,13 @@ public class RemoteLogManagerTest {
|
||||||
@Test
|
@Test
|
||||||
void testRemoteLogMetadataManagerWithEndpointConfigOverridden() throws IOException {
|
void testRemoteLogMetadataManagerWithEndpointConfigOverridden() throws IOException {
|
||||||
Properties props = new Properties();
|
Properties props = new Properties();
|
||||||
|
props.put("zookeeper.connect", kafka.utils.TestUtils.MockZkConnect());
|
||||||
// override common security.protocol by adding "RLMM prefix" and "remote log metadata common client prefix"
|
// override common security.protocol by adding "RLMM prefix" and "remote log metadata common client prefix"
|
||||||
props.put(DEFAULT_REMOTE_LOG_METADATA_MANAGER_CONFIG_PREFIX + REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX + "security.protocol", "SSL");
|
props.put(DEFAULT_REMOTE_LOG_METADATA_MANAGER_CONFIG_PREFIX + REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX + "security.protocol", "SSL");
|
||||||
|
createRLMConfig(props);
|
||||||
|
KafkaConfig config = KafkaConfig.fromProps(props);
|
||||||
try (RemoteLogManager remoteLogManager = new RemoteLogManager(
|
try (RemoteLogManager remoteLogManager = new RemoteLogManager(
|
||||||
createRLMConfig(props),
|
config,
|
||||||
brokerId,
|
brokerId,
|
||||||
logDir,
|
logDir,
|
||||||
clusterId,
|
clusterId,
|
||||||
|
@ -1282,7 +1286,7 @@ public class RemoteLogManagerTest {
|
||||||
void testGetClassLoaderAwareRemoteStorageManager() throws Exception {
|
void testGetClassLoaderAwareRemoteStorageManager() throws Exception {
|
||||||
ClassLoaderAwareRemoteStorageManager rsmManager = mock(ClassLoaderAwareRemoteStorageManager.class);
|
ClassLoaderAwareRemoteStorageManager rsmManager = mock(ClassLoaderAwareRemoteStorageManager.class);
|
||||||
try (RemoteLogManager remoteLogManager =
|
try (RemoteLogManager remoteLogManager =
|
||||||
new RemoteLogManager(remoteLogManagerConfig, brokerId, logDir, clusterId, time,
|
new RemoteLogManager(config, brokerId, logDir, clusterId, time,
|
||||||
t -> Optional.empty(),
|
t -> Optional.empty(),
|
||||||
(topicPartition, offset) -> { },
|
(topicPartition, offset) -> { },
|
||||||
brokerTopicStats, metrics) {
|
brokerTopicStats, metrics) {
|
||||||
|
@ -1544,7 +1548,7 @@ public class RemoteLogManagerTest {
|
||||||
public void testRemoveMetricsOnClose() throws IOException {
|
public void testRemoveMetricsOnClose() throws IOException {
|
||||||
MockedConstruction<KafkaMetricsGroup> mockMetricsGroupCtor = mockConstruction(KafkaMetricsGroup.class);
|
MockedConstruction<KafkaMetricsGroup> mockMetricsGroupCtor = mockConstruction(KafkaMetricsGroup.class);
|
||||||
try {
|
try {
|
||||||
RemoteLogManager remoteLogManager = new RemoteLogManager(remoteLogManagerConfig, brokerId, logDir, clusterId,
|
RemoteLogManager remoteLogManager = new RemoteLogManager(config, brokerId, logDir, clusterId,
|
||||||
time, tp -> Optional.of(mockLog), (topicPartition, offset) -> { }, brokerTopicStats, metrics) {
|
time, tp -> Optional.of(mockLog), (topicPartition, offset) -> { }, brokerTopicStats, metrics) {
|
||||||
public RemoteStorageManager createRemoteStorageManager() {
|
public RemoteStorageManager createRemoteStorageManager() {
|
||||||
return remoteStorageManager;
|
return remoteStorageManager;
|
||||||
|
@ -1939,7 +1943,7 @@ public class RemoteLogManagerTest {
|
||||||
else
|
else
|
||||||
return Collections.emptyIterator();
|
return Collections.emptyIterator();
|
||||||
});
|
});
|
||||||
try (RemoteLogManager remoteLogManager = new RemoteLogManager(remoteLogManagerConfig, brokerId, logDir, clusterId, time,
|
try (RemoteLogManager remoteLogManager = new RemoteLogManager(config, brokerId, logDir, clusterId, time,
|
||||||
tp -> Optional.of(mockLog),
|
tp -> Optional.of(mockLog),
|
||||||
(topicPartition, offset) -> { },
|
(topicPartition, offset) -> { },
|
||||||
brokerTopicStats, metrics) {
|
brokerTopicStats, metrics) {
|
||||||
|
@ -1964,7 +1968,7 @@ public class RemoteLogManagerTest {
|
||||||
when(remoteLogMetadataManager.listRemoteLogSegments(eq(leaderTopicIdPartition), anyInt()))
|
when(remoteLogMetadataManager.listRemoteLogSegments(eq(leaderTopicIdPartition), anyInt()))
|
||||||
.thenReturn(Collections.emptyIterator());
|
.thenReturn(Collections.emptyIterator());
|
||||||
|
|
||||||
try (RemoteLogManager remoteLogManager = new RemoteLogManager(remoteLogManagerConfig, brokerId, logDir, clusterId, time,
|
try (RemoteLogManager remoteLogManager = new RemoteLogManager(config, brokerId, logDir, clusterId, time,
|
||||||
tp -> Optional.of(mockLog),
|
tp -> Optional.of(mockLog),
|
||||||
(topicPartition, offset) -> { },
|
(topicPartition, offset) -> { },
|
||||||
brokerTopicStats, metrics) {
|
brokerTopicStats, metrics) {
|
||||||
|
@ -1998,7 +2002,7 @@ public class RemoteLogManagerTest {
|
||||||
});
|
});
|
||||||
|
|
||||||
AtomicLong logStartOffset = new AtomicLong(0);
|
AtomicLong logStartOffset = new AtomicLong(0);
|
||||||
try (RemoteLogManager remoteLogManager = new RemoteLogManager(remoteLogManagerConfig, brokerId, logDir, clusterId, time,
|
try (RemoteLogManager remoteLogManager = new RemoteLogManager(config, brokerId, logDir, clusterId, time,
|
||||||
tp -> Optional.of(mockLog),
|
tp -> Optional.of(mockLog),
|
||||||
(topicPartition, offset) -> logStartOffset.set(offset),
|
(topicPartition, offset) -> logStartOffset.set(offset),
|
||||||
brokerTopicStats, metrics) {
|
brokerTopicStats, metrics) {
|
||||||
|
@ -2430,7 +2434,7 @@ public class RemoteLogManagerTest {
|
||||||
@Test
|
@Test
|
||||||
public void testDeleteRetentionMsOnExpiredSegment() throws RemoteStorageException, IOException {
|
public void testDeleteRetentionMsOnExpiredSegment() throws RemoteStorageException, IOException {
|
||||||
AtomicLong logStartOffset = new AtomicLong(0);
|
AtomicLong logStartOffset = new AtomicLong(0);
|
||||||
try (RemoteLogManager remoteLogManager = new RemoteLogManager(remoteLogManagerConfig, brokerId, logDir, clusterId, time,
|
try (RemoteLogManager remoteLogManager = new RemoteLogManager(config, brokerId, logDir, clusterId, time,
|
||||||
tp -> Optional.of(mockLog),
|
tp -> Optional.of(mockLog),
|
||||||
(topicPartition, offset) -> logStartOffset.set(offset),
|
(topicPartition, offset) -> logStartOffset.set(offset),
|
||||||
brokerTopicStats, metrics) {
|
brokerTopicStats, metrics) {
|
||||||
|
@ -2572,7 +2576,7 @@ public class RemoteLogManagerTest {
|
||||||
);
|
);
|
||||||
|
|
||||||
try (RemoteLogManager remoteLogManager = new RemoteLogManager(
|
try (RemoteLogManager remoteLogManager = new RemoteLogManager(
|
||||||
remoteLogManagerConfig,
|
config,
|
||||||
brokerId,
|
brokerId,
|
||||||
logDir,
|
logDir,
|
||||||
clusterId,
|
clusterId,
|
||||||
|
@ -2645,7 +2649,7 @@ public class RemoteLogManagerTest {
|
||||||
);
|
);
|
||||||
|
|
||||||
try (RemoteLogManager remoteLogManager = new RemoteLogManager(
|
try (RemoteLogManager remoteLogManager = new RemoteLogManager(
|
||||||
remoteLogManagerConfig,
|
config,
|
||||||
brokerId,
|
brokerId,
|
||||||
logDir,
|
logDir,
|
||||||
clusterId,
|
clusterId,
|
||||||
|
@ -2730,7 +2734,7 @@ public class RemoteLogManagerTest {
|
||||||
|
|
||||||
|
|
||||||
try (RemoteLogManager remoteLogManager = new RemoteLogManager(
|
try (RemoteLogManager remoteLogManager = new RemoteLogManager(
|
||||||
remoteLogManagerConfig,
|
config,
|
||||||
brokerId,
|
brokerId,
|
||||||
logDir,
|
logDir,
|
||||||
clusterId,
|
clusterId,
|
||||||
|
@ -2775,18 +2779,23 @@ public class RemoteLogManagerTest {
|
||||||
@Test
|
@Test
|
||||||
public void testCopyQuotaManagerConfig() {
|
public void testCopyQuotaManagerConfig() {
|
||||||
Properties defaultProps = new Properties();
|
Properties defaultProps = new Properties();
|
||||||
RemoteLogManagerConfig defaultRlmConfig = createRLMConfig(defaultProps);
|
defaultProps.put("zookeeper.connect", kafka.utils.TestUtils.MockZkConnect());
|
||||||
|
createRLMConfig(defaultProps);
|
||||||
|
KafkaConfig defaultRlmConfig = KafkaConfig.fromProps(defaultProps);
|
||||||
RLMQuotaManagerConfig defaultConfig = RemoteLogManager.copyQuotaManagerConfig(defaultRlmConfig);
|
RLMQuotaManagerConfig defaultConfig = RemoteLogManager.copyQuotaManagerConfig(defaultRlmConfig);
|
||||||
assertEquals(DEFAULT_REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND, defaultConfig.quotaBytesPerSecond());
|
assertEquals(DEFAULT_REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND, defaultConfig.quotaBytesPerSecond());
|
||||||
assertEquals(DEFAULT_REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_NUM, defaultConfig.numQuotaSamples());
|
assertEquals(DEFAULT_REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_NUM, defaultConfig.numQuotaSamples());
|
||||||
assertEquals(DEFAULT_REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_SIZE_SECONDS, defaultConfig.quotaWindowSizeSeconds());
|
assertEquals(DEFAULT_REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_SIZE_SECONDS, defaultConfig.quotaWindowSizeSeconds());
|
||||||
|
|
||||||
Properties customProps = new Properties();
|
Properties customProps = new Properties();
|
||||||
|
customProps.put("zookeeper.connect", kafka.utils.TestUtils.MockZkConnect());
|
||||||
customProps.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP, 100);
|
customProps.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP, 100);
|
||||||
customProps.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_NUM_PROP, 31);
|
customProps.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_NUM_PROP, 31);
|
||||||
customProps.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_SIZE_SECONDS_PROP, 1);
|
customProps.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_SIZE_SECONDS_PROP, 1);
|
||||||
RemoteLogManagerConfig rlmConfig = createRLMConfig(customProps);
|
createRLMConfig(customProps);
|
||||||
RLMQuotaManagerConfig rlmCopyQuotaManagerConfig = RemoteLogManager.copyQuotaManagerConfig(rlmConfig);
|
KafkaConfig config = KafkaConfig.fromProps(customProps);
|
||||||
|
|
||||||
|
RLMQuotaManagerConfig rlmCopyQuotaManagerConfig = RemoteLogManager.copyQuotaManagerConfig(config);
|
||||||
assertEquals(100L, rlmCopyQuotaManagerConfig.quotaBytesPerSecond());
|
assertEquals(100L, rlmCopyQuotaManagerConfig.quotaBytesPerSecond());
|
||||||
assertEquals(31, rlmCopyQuotaManagerConfig.numQuotaSamples());
|
assertEquals(31, rlmCopyQuotaManagerConfig.numQuotaSamples());
|
||||||
assertEquals(1, rlmCopyQuotaManagerConfig.quotaWindowSizeSeconds());
|
assertEquals(1, rlmCopyQuotaManagerConfig.quotaWindowSizeSeconds());
|
||||||
|
@ -2795,17 +2804,22 @@ public class RemoteLogManagerTest {
|
||||||
@Test
|
@Test
|
||||||
public void testFetchQuotaManagerConfig() {
|
public void testFetchQuotaManagerConfig() {
|
||||||
Properties defaultProps = new Properties();
|
Properties defaultProps = new Properties();
|
||||||
RemoteLogManagerConfig defaultRlmConfig = createRLMConfig(defaultProps);
|
defaultProps.put("zookeeper.connect", kafka.utils.TestUtils.MockZkConnect());
|
||||||
|
createRLMConfig(defaultProps);
|
||||||
|
KafkaConfig defaultRlmConfig = KafkaConfig.fromProps(defaultProps);
|
||||||
|
|
||||||
RLMQuotaManagerConfig defaultConfig = RemoteLogManager.fetchQuotaManagerConfig(defaultRlmConfig);
|
RLMQuotaManagerConfig defaultConfig = RemoteLogManager.fetchQuotaManagerConfig(defaultRlmConfig);
|
||||||
assertEquals(DEFAULT_REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND, defaultConfig.quotaBytesPerSecond());
|
assertEquals(DEFAULT_REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND, defaultConfig.quotaBytesPerSecond());
|
||||||
assertEquals(DEFAULT_REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_NUM, defaultConfig.numQuotaSamples());
|
assertEquals(DEFAULT_REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_NUM, defaultConfig.numQuotaSamples());
|
||||||
assertEquals(DEFAULT_REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_SIZE_SECONDS, defaultConfig.quotaWindowSizeSeconds());
|
assertEquals(DEFAULT_REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_SIZE_SECONDS, defaultConfig.quotaWindowSizeSeconds());
|
||||||
|
|
||||||
Properties customProps = new Properties();
|
Properties customProps = new Properties();
|
||||||
|
customProps.put("zookeeper.connect", kafka.utils.TestUtils.MockZkConnect());
|
||||||
customProps.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_PROP, 100);
|
customProps.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_PROP, 100);
|
||||||
customProps.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_NUM_PROP, 31);
|
customProps.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_NUM_PROP, 31);
|
||||||
customProps.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_SIZE_SECONDS_PROP, 1);
|
customProps.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_SIZE_SECONDS_PROP, 1);
|
||||||
RemoteLogManagerConfig rlmConfig = createRLMConfig(customProps);
|
createRLMConfig(customProps);
|
||||||
|
KafkaConfig rlmConfig = KafkaConfig.fromProps(customProps);
|
||||||
RLMQuotaManagerConfig rlmFetchQuotaManagerConfig = RemoteLogManager.fetchQuotaManagerConfig(rlmConfig);
|
RLMQuotaManagerConfig rlmFetchQuotaManagerConfig = RemoteLogManager.fetchQuotaManagerConfig(rlmConfig);
|
||||||
assertEquals(100L, rlmFetchQuotaManagerConfig.quotaBytesPerSecond());
|
assertEquals(100L, rlmFetchQuotaManagerConfig.quotaBytesPerSecond());
|
||||||
assertEquals(31, rlmFetchQuotaManagerConfig.numQuotaSamples());
|
assertEquals(31, rlmFetchQuotaManagerConfig.numQuotaSamples());
|
||||||
|
|
|
@ -47,7 +47,7 @@ import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.api.Test
|
import org.junit.jupiter.api.Test
|
||||||
import org.mockito.ArgumentMatchers.anyString
|
import org.mockito.ArgumentMatchers.anyString
|
||||||
import org.mockito.{ArgumentCaptor, ArgumentMatchers, Mockito}
|
import org.mockito.{ArgumentCaptor, ArgumentMatchers, Mockito}
|
||||||
import org.mockito.Mockito.{mock, when}
|
import org.mockito.Mockito.{mock, verify, verifyNoMoreInteractions, when}
|
||||||
|
|
||||||
import scala.annotation.nowarn
|
import scala.annotation.nowarn
|
||||||
import scala.jdk.CollectionConverters._
|
import scala.jdk.CollectionConverters._
|
||||||
|
@ -852,49 +852,64 @@ class DynamicBrokerConfigTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
def testRemoteLogManagerCopyQuotaUpdates(): Unit = {
|
def testRemoteLogManagerCopyQuotaUpdates(): Unit = {
|
||||||
testRemoteLogManagerQuotaUpdates(
|
|
||||||
RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP,
|
|
||||||
RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND,
|
|
||||||
(remoteLogManager, quota) => Mockito.verify(remoteLogManager).updateCopyQuota(quota)
|
|
||||||
)
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
def testRemoteLogManagerFetchQuotaUpdates(): Unit = {
|
|
||||||
testRemoteLogManagerQuotaUpdates(
|
|
||||||
RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_PROP,
|
|
||||||
RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND,
|
|
||||||
(remoteLogManager, quota) => Mockito.verify(remoteLogManager).updateFetchQuota(quota)
|
|
||||||
)
|
|
||||||
}
|
|
||||||
|
|
||||||
def testRemoteLogManagerQuotaUpdates(quotaProp: String, defaultQuota: Long, verifyMethod: (RemoteLogManager, Long) => Unit): Unit = {
|
|
||||||
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 9092)
|
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 9092)
|
||||||
val config = KafkaConfig.fromProps(props)
|
val config = KafkaConfig.fromProps(props)
|
||||||
val serverMock: KafkaServer = mock(classOf[KafkaServer])
|
val serverMock: KafkaServer = mock(classOf[KafkaServer])
|
||||||
val remoteLogManagerMockOpt = Option(mock(classOf[RemoteLogManager]))
|
val remoteLogManager = mock(classOf[RemoteLogManager])
|
||||||
|
|
||||||
Mockito.when(serverMock.config).thenReturn(config)
|
Mockito.when(serverMock.config).thenReturn(config)
|
||||||
Mockito.when(serverMock.remoteLogManagerOpt).thenReturn(remoteLogManagerMockOpt)
|
Mockito.when(serverMock.remoteLogManagerOpt).thenReturn(Some(remoteLogManager))
|
||||||
|
|
||||||
config.dynamicConfig.initialize(None, None)
|
config.dynamicConfig.initialize(None, None)
|
||||||
config.dynamicConfig.addBrokerReconfigurable(new DynamicRemoteLogConfig(serverMock))
|
config.dynamicConfig.addBrokerReconfigurable(new DynamicRemoteLogConfig(serverMock))
|
||||||
|
|
||||||
assertEquals(defaultQuota, config.getLong(quotaProp))
|
assertEquals(RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND,
|
||||||
|
config.remoteLogManagerCopyMaxBytesPerSecond)
|
||||||
|
|
||||||
// Update default config
|
// Update default config
|
||||||
props.put(quotaProp, "100")
|
props.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP, "100")
|
||||||
config.dynamicConfig.updateDefaultConfig(props)
|
config.dynamicConfig.updateDefaultConfig(props)
|
||||||
assertEquals(100, config.getLong(quotaProp))
|
assertEquals(100, config.remoteLogManagerCopyMaxBytesPerSecond)
|
||||||
verifyMethod(remoteLogManagerMockOpt.get, 100)
|
verify(remoteLogManager).updateCopyQuota(100)
|
||||||
|
|
||||||
// Update per broker config
|
// Update per broker config
|
||||||
props.put(quotaProp, "200")
|
props.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP, "200")
|
||||||
config.dynamicConfig.updateBrokerConfig(0, props)
|
config.dynamicConfig.updateBrokerConfig(0, props)
|
||||||
assertEquals(200, config.getLong(quotaProp))
|
assertEquals(200, config.remoteLogManagerCopyMaxBytesPerSecond)
|
||||||
verifyMethod(remoteLogManagerMockOpt.get, 200)
|
verify(remoteLogManager).updateCopyQuota(200)
|
||||||
|
|
||||||
Mockito.verifyNoMoreInteractions(remoteLogManagerMockOpt.get)
|
verifyNoMoreInteractions(remoteLogManager)
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
def testRemoteLogManagerFetchQuotaUpdates(): Unit = {
|
||||||
|
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 9092)
|
||||||
|
val config = KafkaConfig.fromProps(props)
|
||||||
|
val serverMock: KafkaServer = mock(classOf[KafkaServer])
|
||||||
|
val remoteLogManager = mock(classOf[RemoteLogManager])
|
||||||
|
|
||||||
|
Mockito.when(serverMock.config).thenReturn(config)
|
||||||
|
Mockito.when(serverMock.remoteLogManagerOpt).thenReturn(Some(remoteLogManager))
|
||||||
|
|
||||||
|
config.dynamicConfig.initialize(None, None)
|
||||||
|
config.dynamicConfig.addBrokerReconfigurable(new DynamicRemoteLogConfig(serverMock))
|
||||||
|
|
||||||
|
assertEquals(RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND,
|
||||||
|
config.remoteLogManagerFetchMaxBytesPerSecond)
|
||||||
|
|
||||||
|
// Update default config
|
||||||
|
props.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_PROP, "100")
|
||||||
|
config.dynamicConfig.updateDefaultConfig(props)
|
||||||
|
assertEquals(100, config.remoteLogManagerFetchMaxBytesPerSecond)
|
||||||
|
verify(remoteLogManager).updateFetchQuota(100)
|
||||||
|
|
||||||
|
// Update per broker config
|
||||||
|
props.put(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_PROP, "200")
|
||||||
|
config.dynamicConfig.updateBrokerConfig(0, props)
|
||||||
|
assertEquals(200, config.remoteLogManagerFetchMaxBytesPerSecond)
|
||||||
|
verify(remoteLogManager).updateFetchQuota(200)
|
||||||
|
|
||||||
|
verifyNoMoreInteractions(remoteLogManager)
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -906,44 +921,44 @@ class DynamicBrokerConfigTest {
|
||||||
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 9092)
|
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 9092)
|
||||||
val config = KafkaConfig.fromProps(props)
|
val config = KafkaConfig.fromProps(props)
|
||||||
val serverMock: KafkaServer = mock(classOf[KafkaServer])
|
val serverMock: KafkaServer = mock(classOf[KafkaServer])
|
||||||
val remoteLogManagerMockOpt = Option(Mockito.mock(classOf[RemoteLogManager]))
|
val remoteLogManager = Mockito.mock(classOf[RemoteLogManager])
|
||||||
|
|
||||||
Mockito.when(serverMock.config).thenReturn(config)
|
Mockito.when(serverMock.config).thenReturn(config)
|
||||||
Mockito.when(serverMock.remoteLogManagerOpt).thenReturn(remoteLogManagerMockOpt)
|
Mockito.when(serverMock.remoteLogManagerOpt).thenReturn(Some(remoteLogManager))
|
||||||
|
|
||||||
config.dynamicConfig.initialize(None, None)
|
config.dynamicConfig.initialize(None, None)
|
||||||
config.dynamicConfig.addBrokerReconfigurable(new DynamicRemoteLogConfig(serverMock))
|
config.dynamicConfig.addBrokerReconfigurable(new DynamicRemoteLogConfig(serverMock))
|
||||||
|
|
||||||
// Default values
|
// Default values
|
||||||
assertEquals(RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES, config.getLong(indexFileCacheSizeProp))
|
assertEquals(RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES, config.remoteLogIndexFileCacheTotalSizeBytes)
|
||||||
assertEquals(RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND, config.getLong(copyQuotaProp))
|
assertEquals(RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND, config.remoteLogManagerCopyMaxBytesPerSecond)
|
||||||
assertEquals(RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND, config.getLong(fetchQuotaProp))
|
assertEquals(RemoteLogManagerConfig.DEFAULT_REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND, config.remoteLogManagerFetchMaxBytesPerSecond)
|
||||||
|
|
||||||
// Update default config
|
// Update default config
|
||||||
props.put(indexFileCacheSizeProp, "4")
|
props.put(indexFileCacheSizeProp, "4")
|
||||||
props.put(copyQuotaProp, "100")
|
props.put(copyQuotaProp, "100")
|
||||||
props.put(fetchQuotaProp, "200")
|
props.put(fetchQuotaProp, "200")
|
||||||
config.dynamicConfig.updateDefaultConfig(props)
|
config.dynamicConfig.updateDefaultConfig(props)
|
||||||
assertEquals(4, config.getLong(indexFileCacheSizeProp))
|
assertEquals(4, config.remoteLogIndexFileCacheTotalSizeBytes)
|
||||||
assertEquals(100, config.getLong(copyQuotaProp))
|
assertEquals(100, config.remoteLogManagerCopyMaxBytesPerSecond)
|
||||||
assertEquals(200, config.getLong(fetchQuotaProp))
|
assertEquals(200, config.remoteLogManagerFetchMaxBytesPerSecond)
|
||||||
Mockito.verify(remoteLogManagerMockOpt.get).resizeCacheSize(4)
|
verify(remoteLogManager).resizeCacheSize(4)
|
||||||
Mockito.verify(remoteLogManagerMockOpt.get).updateCopyQuota(100)
|
verify(remoteLogManager).updateCopyQuota(100)
|
||||||
Mockito.verify(remoteLogManagerMockOpt.get).updateFetchQuota(200)
|
verify(remoteLogManager).updateFetchQuota(200)
|
||||||
|
|
||||||
// Update per broker config
|
// Update per broker config
|
||||||
props.put(indexFileCacheSizeProp, "8")
|
props.put(indexFileCacheSizeProp, "8")
|
||||||
props.put(copyQuotaProp, "200")
|
props.put(copyQuotaProp, "200")
|
||||||
props.put(fetchQuotaProp, "400")
|
props.put(fetchQuotaProp, "400")
|
||||||
config.dynamicConfig.updateBrokerConfig(0, props)
|
config.dynamicConfig.updateBrokerConfig(0, props)
|
||||||
assertEquals(8, config.getLong(indexFileCacheSizeProp))
|
assertEquals(8, config.remoteLogIndexFileCacheTotalSizeBytes)
|
||||||
assertEquals(200, config.getLong(copyQuotaProp))
|
assertEquals(200, config.remoteLogManagerCopyMaxBytesPerSecond)
|
||||||
assertEquals(400, config.getLong(fetchQuotaProp))
|
assertEquals(400, config.remoteLogManagerFetchMaxBytesPerSecond)
|
||||||
Mockito.verify(remoteLogManagerMockOpt.get).resizeCacheSize(8)
|
verify(remoteLogManager).resizeCacheSize(8)
|
||||||
Mockito.verify(remoteLogManagerMockOpt.get).updateCopyQuota(200)
|
verify(remoteLogManager).updateCopyQuota(200)
|
||||||
Mockito.verify(remoteLogManagerMockOpt.get).updateFetchQuota(400)
|
verify(remoteLogManager).updateFetchQuota(400)
|
||||||
|
|
||||||
Mockito.verifyNoMoreInteractions(remoteLogManagerMockOpt.get)
|
verifyNoMoreInteractions(remoteLogManager)
|
||||||
}
|
}
|
||||||
|
|
||||||
def verifyIncorrectLogLocalRetentionProps(logLocalRetentionMs: Long,
|
def verifyIncorrectLogLocalRetentionProps(logLocalRetentionMs: Long,
|
||||||
|
|
|
@ -4093,11 +4093,11 @@ class ReplicaManagerTest {
|
||||||
props.put(RemoteLogManagerConfig.REMOTE_LOG_METADATA_MANAGER_CLASS_NAME_PROP, classOf[NoOpRemoteLogMetadataManager].getName)
|
props.put(RemoteLogManagerConfig.REMOTE_LOG_METADATA_MANAGER_CLASS_NAME_PROP, classOf[NoOpRemoteLogMetadataManager].getName)
|
||||||
// set log reader threads number to 2
|
// set log reader threads number to 2
|
||||||
props.put(RemoteLogManagerConfig.REMOTE_LOG_READER_THREADS_PROP, 2.toString)
|
props.put(RemoteLogManagerConfig.REMOTE_LOG_READER_THREADS_PROP, 2.toString)
|
||||||
val remoteLogManagerConfig = new RemoteLogManagerConfig(props)
|
val config = KafkaConfig.fromProps(props)
|
||||||
val mockLog = mock(classOf[UnifiedLog])
|
val mockLog = mock(classOf[UnifiedLog])
|
||||||
val brokerTopicStats = new BrokerTopicStats(KafkaConfig.fromProps(props).remoteLogManagerConfig.isRemoteStorageSystemEnabled())
|
val brokerTopicStats = new BrokerTopicStats(config.remoteLogManagerConfig.isRemoteStorageSystemEnabled())
|
||||||
val remoteLogManager = new RemoteLogManager(
|
val remoteLogManager = new RemoteLogManager(
|
||||||
remoteLogManagerConfig,
|
config,
|
||||||
0,
|
0,
|
||||||
TestUtils.tempRelativeDir("data").getAbsolutePath,
|
TestUtils.tempRelativeDir("data").getAbsolutePath,
|
||||||
"clusterId",
|
"clusterId",
|
||||||
|
@ -4199,11 +4199,11 @@ class ReplicaManagerTest {
|
||||||
props.put(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, true.toString)
|
props.put(RemoteLogManagerConfig.REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP, true.toString)
|
||||||
props.put(RemoteLogManagerConfig.REMOTE_STORAGE_MANAGER_CLASS_NAME_PROP, classOf[NoOpRemoteStorageManager].getName)
|
props.put(RemoteLogManagerConfig.REMOTE_STORAGE_MANAGER_CLASS_NAME_PROP, classOf[NoOpRemoteStorageManager].getName)
|
||||||
props.put(RemoteLogManagerConfig.REMOTE_LOG_METADATA_MANAGER_CLASS_NAME_PROP, classOf[NoOpRemoteLogMetadataManager].getName)
|
props.put(RemoteLogManagerConfig.REMOTE_LOG_METADATA_MANAGER_CLASS_NAME_PROP, classOf[NoOpRemoteLogMetadataManager].getName)
|
||||||
val remoteLogManagerConfig = new RemoteLogManagerConfig(props)
|
val config = KafkaConfig.fromProps(props)
|
||||||
val dummyLog = mock(classOf[UnifiedLog])
|
val dummyLog = mock(classOf[UnifiedLog])
|
||||||
val brokerTopicStats = new BrokerTopicStats(KafkaConfig.fromProps(props).remoteLogManagerConfig.isRemoteStorageSystemEnabled())
|
val brokerTopicStats = new BrokerTopicStats(config.remoteLogManagerConfig.isRemoteStorageSystemEnabled())
|
||||||
val remoteLogManager = new RemoteLogManager(
|
val remoteLogManager = new RemoteLogManager(
|
||||||
remoteLogManagerConfig,
|
config,
|
||||||
0,
|
0,
|
||||||
TestUtils.tempRelativeDir("data").getAbsolutePath,
|
TestUtils.tempRelativeDir("data").getAbsolutePath,
|
||||||
"clusterId",
|
"clusterId",
|
||||||
|
|
|
@ -378,10 +378,6 @@ public final class RemoteLogManagerConfig extends AbstractConfig {
|
||||||
return getString(REMOTE_LOG_METADATA_MANAGER_CLASS_PATH_PROP);
|
return getString(REMOTE_LOG_METADATA_MANAGER_CLASS_PATH_PROP);
|
||||||
}
|
}
|
||||||
|
|
||||||
public long remoteLogIndexFileCacheTotalSizeBytes() {
|
|
||||||
return getLong(REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES_PROP);
|
|
||||||
}
|
|
||||||
|
|
||||||
public int remoteLogManagerThreadPoolSize() {
|
public int remoteLogManagerThreadPoolSize() {
|
||||||
return getInt(REMOTE_LOG_MANAGER_THREAD_POOL_SIZE_PROP);
|
return getInt(REMOTE_LOG_MANAGER_THREAD_POOL_SIZE_PROP);
|
||||||
}
|
}
|
||||||
|
@ -447,10 +443,6 @@ public final class RemoteLogManagerConfig extends AbstractConfig {
|
||||||
return prefixProp == null ? Collections.emptyMap() : Collections.unmodifiableMap(originalsWithPrefix(prefixProp));
|
return prefixProp == null ? Collections.emptyMap() : Collections.unmodifiableMap(originalsWithPrefix(prefixProp));
|
||||||
}
|
}
|
||||||
|
|
||||||
public long remoteLogManagerCopyMaxBytesPerSecond() {
|
|
||||||
return getLong(REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP);
|
|
||||||
}
|
|
||||||
|
|
||||||
public int remoteLogManagerCopyNumQuotaSamples() {
|
public int remoteLogManagerCopyNumQuotaSamples() {
|
||||||
return getInt(REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_NUM_PROP);
|
return getInt(REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_NUM_PROP);
|
||||||
}
|
}
|
||||||
|
@ -459,10 +451,6 @@ public final class RemoteLogManagerConfig extends AbstractConfig {
|
||||||
return getInt(REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_SIZE_SECONDS_PROP);
|
return getInt(REMOTE_LOG_MANAGER_COPY_QUOTA_WINDOW_SIZE_SECONDS_PROP);
|
||||||
}
|
}
|
||||||
|
|
||||||
public long remoteLogManagerFetchMaxBytesPerSecond() {
|
|
||||||
return getLong(REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_PROP);
|
|
||||||
}
|
|
||||||
|
|
||||||
public int remoteLogManagerFetchNumQuotaSamples() {
|
public int remoteLogManagerFetchNumQuotaSamples() {
|
||||||
return getInt(REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_NUM_PROP);
|
return getInt(REMOTE_LOG_MANAGER_FETCH_QUOTA_WINDOW_NUM_PROP);
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue