mirror of https://github.com/apache/kafka.git
MINOR: Make ReplicaManager, LogManager, KafkaApis easier to construct (#11320)
The ReplicaManager, LogManager, and KafkaApis class all have many constructor parameters. It is often difficult to add or remove a parameter, since there are so many locations that need to be updated. In order to address this problem, we should use named parameters when constructing these objects from Scala code. This will make it easy to add new optional parameters without modifying many test cases. It will also make it easier to read git diffs and PRs, since the parameters will have names next to them. Since Java does not support named paramters, this PR adds several Builder classes which can be used to achieve the same effect. ReplicaManager also had a secondary constructor, which this PR removes. The function of the secondary constructor was just to provide some default parameters for the main constructor. However, it is simpler just to actually use default parameters. Reviewers: David Arthur <mumrah@gmail.com>
This commit is contained in:
parent
1b0294dfc4
commit
074a3dacca
|
@ -72,6 +72,13 @@
|
|||
<allow pkg="org.apache.kafka.clients" />
|
||||
</subpackage>
|
||||
|
||||
<subpackage name="server">
|
||||
<subpackage name="builders">
|
||||
<allow pkg="kafka" />
|
||||
<allow pkg="org.apache.kafka" />
|
||||
</subpackage>
|
||||
</subpackage>
|
||||
|
||||
<subpackage name="test">
|
||||
<allow pkg="org.apache.kafka.controller"/>
|
||||
<allow pkg="org.apache.kafka.metadata"/>
|
||||
|
|
|
@ -23,7 +23,7 @@
|
|||
<!-- core -->
|
||||
<suppress checks="(NPathComplexity|ClassFanOutComplexity|CyclomaticComplexity|ClassDataAbstractionCoupling|FinalLocalVariable|LocalVariableName|MemberName|ParameterName|MethodLength|JavaNCSS|AvoidStarImport)"
|
||||
files="core[\\/]src[\\/](generated|generated-test)[\\/].+.java$"/>
|
||||
<suppress checks="NPathComplexity" files="ClusterTestExtensions.java"/>
|
||||
<suppress checks="NPathComplexity" files="(ClusterTestExtensions|KafkaApisBuilder).java"/>
|
||||
|
||||
<!-- Clients -->
|
||||
<suppress id="dontUseSystemExit"
|
||||
|
|
|
@ -0,0 +1,197 @@
|
|||
/*
|
||||
* 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 kafka.server.builders;
|
||||
|
||||
import kafka.coordinator.group.GroupCoordinator;
|
||||
import kafka.coordinator.transaction.TransactionCoordinator;
|
||||
import kafka.network.RequestChannel;
|
||||
import kafka.server.ApiVersionManager;
|
||||
import kafka.server.AutoTopicCreationManager;
|
||||
import kafka.server.BrokerTopicStats;
|
||||
import kafka.server.DelegationTokenManager;
|
||||
import kafka.server.FetchManager;
|
||||
import kafka.server.KafkaApis;
|
||||
import kafka.server.KafkaConfig;
|
||||
import kafka.server.MetadataCache;
|
||||
import kafka.server.MetadataSupport;
|
||||
import kafka.server.QuotaFactory.QuotaManagers;
|
||||
import kafka.server.ReplicaManager;
|
||||
import kafka.server.metadata.ConfigRepository;
|
||||
import org.apache.kafka.common.metrics.Metrics;
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
import org.apache.kafka.server.authorizer.Authorizer;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.Optional;
|
||||
import scala.compat.java8.OptionConverters;
|
||||
|
||||
|
||||
public class KafkaApisBuilder {
|
||||
private RequestChannel requestChannel = null;
|
||||
private MetadataSupport metadataSupport = null;
|
||||
private ReplicaManager replicaManager = null;
|
||||
private GroupCoordinator groupCoordinator = null;
|
||||
private TransactionCoordinator txnCoordinator = null;
|
||||
private AutoTopicCreationManager autoTopicCreationManager = null;
|
||||
private int brokerId = 0;
|
||||
private KafkaConfig config = null;
|
||||
private ConfigRepository configRepository = null;
|
||||
private MetadataCache metadataCache = null;
|
||||
private Metrics metrics = null;
|
||||
private Optional<Authorizer> authorizer = Optional.empty();
|
||||
private QuotaManagers quotas = null;
|
||||
private FetchManager fetchManager = null;
|
||||
private BrokerTopicStats brokerTopicStats = null;
|
||||
private String clusterId = "clusterId";
|
||||
private Time time = Time.SYSTEM;
|
||||
private DelegationTokenManager tokenManager = null;
|
||||
private ApiVersionManager apiVersionManager = null;
|
||||
|
||||
public KafkaApisBuilder setRequestChannel(RequestChannel requestChannel) {
|
||||
this.requestChannel = requestChannel;
|
||||
return this;
|
||||
}
|
||||
|
||||
public KafkaApisBuilder setMetadataSupport(MetadataSupport metadataSupport) {
|
||||
this.metadataSupport = metadataSupport;
|
||||
return this;
|
||||
}
|
||||
|
||||
public KafkaApisBuilder setReplicaManager(ReplicaManager replicaManager) {
|
||||
this.replicaManager = replicaManager;
|
||||
return this;
|
||||
}
|
||||
|
||||
public KafkaApisBuilder setGroupCoordinator(GroupCoordinator groupCoordinator) {
|
||||
this.groupCoordinator = groupCoordinator;
|
||||
return this;
|
||||
}
|
||||
|
||||
public KafkaApisBuilder setTxnCoordinator(TransactionCoordinator txnCoordinator) {
|
||||
this.txnCoordinator = txnCoordinator;
|
||||
return this;
|
||||
}
|
||||
|
||||
public KafkaApisBuilder setAutoTopicCreationManager(AutoTopicCreationManager autoTopicCreationManager) {
|
||||
this.autoTopicCreationManager = autoTopicCreationManager;
|
||||
return this;
|
||||
}
|
||||
|
||||
public KafkaApisBuilder setBrokerId(int brokerId) {
|
||||
this.brokerId = brokerId;
|
||||
return this;
|
||||
}
|
||||
|
||||
public KafkaApisBuilder setConfig(KafkaConfig config) {
|
||||
this.config = config;
|
||||
return this;
|
||||
}
|
||||
|
||||
public KafkaApisBuilder setConfigRepository(ConfigRepository configRepository) {
|
||||
this.configRepository = configRepository;
|
||||
return this;
|
||||
}
|
||||
|
||||
public KafkaApisBuilder setMetadataCache(MetadataCache metadataCache) {
|
||||
this.metadataCache = metadataCache;
|
||||
return this;
|
||||
}
|
||||
|
||||
public KafkaApisBuilder setMetrics(Metrics metrics) {
|
||||
this.metrics = metrics;
|
||||
return this;
|
||||
}
|
||||
|
||||
public KafkaApisBuilder setAuthorizer(Optional<Authorizer> authorizer) {
|
||||
this.authorizer = authorizer;
|
||||
return this;
|
||||
}
|
||||
|
||||
public KafkaApisBuilder setQuotas(QuotaManagers quotas) {
|
||||
this.quotas = quotas;
|
||||
return this;
|
||||
}
|
||||
|
||||
public KafkaApisBuilder setFetchManager(FetchManager fetchManager) {
|
||||
this.fetchManager = fetchManager;
|
||||
return this;
|
||||
}
|
||||
|
||||
public KafkaApisBuilder setBrokerTopicStats(BrokerTopicStats brokerTopicStats) {
|
||||
this.brokerTopicStats = brokerTopicStats;
|
||||
return this;
|
||||
}
|
||||
|
||||
public KafkaApisBuilder setClusterId(String clusterId) {
|
||||
this.clusterId = clusterId;
|
||||
return this;
|
||||
}
|
||||
|
||||
public KafkaApisBuilder setTime(Time time) {
|
||||
this.time = time;
|
||||
return this;
|
||||
}
|
||||
|
||||
public KafkaApisBuilder setTokenManager(DelegationTokenManager tokenManager) {
|
||||
this.tokenManager = tokenManager;
|
||||
return this;
|
||||
}
|
||||
|
||||
public KafkaApisBuilder setApiVersionManager(ApiVersionManager apiVersionManager) {
|
||||
this.apiVersionManager = apiVersionManager;
|
||||
return this;
|
||||
}
|
||||
|
||||
public KafkaApis build() {
|
||||
if (requestChannel == null) throw new RuntimeException("you must set requestChannel");
|
||||
if (metadataSupport == null) throw new RuntimeException("you must set metadataSupport");
|
||||
if (replicaManager == null) throw new RuntimeException("You must set replicaManager");
|
||||
if (groupCoordinator == null) throw new RuntimeException("You must set groupCoordinator");
|
||||
if (txnCoordinator == null) throw new RuntimeException("You must set txnCoordinator");
|
||||
if (autoTopicCreationManager == null)
|
||||
throw new RuntimeException("You must set autoTopicCreationManager");
|
||||
if (config == null) config = new KafkaConfig(Collections.emptyMap());
|
||||
if (configRepository == null) throw new RuntimeException("You must set configRepository");
|
||||
if (metadataCache == null) throw new RuntimeException("You must set metadataCache");
|
||||
if (metrics == null) throw new RuntimeException("You must set metrics");
|
||||
if (quotas == null) throw new RuntimeException("You must set quotas");
|
||||
if (fetchManager == null) throw new RuntimeException("You must set fetchManager");
|
||||
if (brokerTopicStats == null) brokerTopicStats = new BrokerTopicStats();
|
||||
if (apiVersionManager == null) throw new RuntimeException("You must set apiVersionManager");
|
||||
|
||||
return new KafkaApis(requestChannel,
|
||||
metadataSupport,
|
||||
replicaManager,
|
||||
groupCoordinator,
|
||||
txnCoordinator,
|
||||
autoTopicCreationManager,
|
||||
brokerId,
|
||||
config,
|
||||
configRepository,
|
||||
metadataCache,
|
||||
metrics,
|
||||
OptionConverters.toScala(authorizer),
|
||||
quotas,
|
||||
fetchManager,
|
||||
brokerTopicStats,
|
||||
clusterId,
|
||||
time,
|
||||
tokenManager,
|
||||
apiVersionManager);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,167 @@
|
|||
/*
|
||||
* 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 kafka.server.builders;
|
||||
|
||||
import kafka.api.ApiVersion;
|
||||
import kafka.log.CleanerConfig;
|
||||
import kafka.log.LogConfig;
|
||||
import kafka.log.LogManager;
|
||||
import kafka.server.BrokerTopicStats;
|
||||
import kafka.server.LogDirFailureChannel;
|
||||
import kafka.server.metadata.ConfigRepository;
|
||||
import kafka.utils.Scheduler;
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import scala.collection.JavaConverters;
|
||||
|
||||
|
||||
public class LogManagerBuilder {
|
||||
private List<File> logDirs = null;
|
||||
private List<File> initialOfflineDirs = Collections.emptyList();
|
||||
private ConfigRepository configRepository = null;
|
||||
private LogConfig initialDefaultConfig = null;
|
||||
private CleanerConfig cleanerConfig = null;
|
||||
private int recoveryThreadsPerDataDir = 1;
|
||||
private long flushCheckMs = 1000L;
|
||||
private long flushRecoveryOffsetCheckpointMs = 10000L;
|
||||
private long flushStartOffsetCheckpointMs = 10000L;
|
||||
private long retentionCheckMs = 1000L;
|
||||
private int maxPidExpirationMs = 60000;
|
||||
private ApiVersion interBrokerProtocolVersion = ApiVersion.latestVersion();
|
||||
private Scheduler scheduler = null;
|
||||
private BrokerTopicStats brokerTopicStats = null;
|
||||
private LogDirFailureChannel logDirFailureChannel = null;
|
||||
private Time time = Time.SYSTEM;
|
||||
private boolean keepPartitionMetadataFile = true;
|
||||
|
||||
public LogManagerBuilder setLogDirs(List<File> logDirs) {
|
||||
this.logDirs = logDirs;
|
||||
return this;
|
||||
}
|
||||
|
||||
public LogManagerBuilder setInitialOfflineDirs(List<File> initialOfflineDirs) {
|
||||
this.initialOfflineDirs = initialOfflineDirs;
|
||||
return this;
|
||||
}
|
||||
|
||||
public LogManagerBuilder setConfigRepository(ConfigRepository configRepository) {
|
||||
this.configRepository = configRepository;
|
||||
return this;
|
||||
}
|
||||
|
||||
public LogManagerBuilder setInitialDefaultConfig(LogConfig initialDefaultConfig) {
|
||||
this.initialDefaultConfig = initialDefaultConfig;
|
||||
return this;
|
||||
}
|
||||
|
||||
public LogManagerBuilder setCleanerConfig(CleanerConfig cleanerConfig) {
|
||||
this.cleanerConfig = cleanerConfig;
|
||||
return this;
|
||||
}
|
||||
|
||||
public LogManagerBuilder setRecoveryThreadsPerDataDir(int recoveryThreadsPerDataDir) {
|
||||
this.recoveryThreadsPerDataDir = recoveryThreadsPerDataDir;
|
||||
return this;
|
||||
}
|
||||
|
||||
public LogManagerBuilder setFlushCheckMs(long flushCheckMs) {
|
||||
this.flushCheckMs = flushCheckMs;
|
||||
return this;
|
||||
}
|
||||
|
||||
public LogManagerBuilder setFlushRecoveryOffsetCheckpointMs(long flushRecoveryOffsetCheckpointMs) {
|
||||
this.flushRecoveryOffsetCheckpointMs = flushRecoveryOffsetCheckpointMs;
|
||||
return this;
|
||||
}
|
||||
|
||||
public LogManagerBuilder setFlushStartOffsetCheckpointMs(long flushStartOffsetCheckpointMs) {
|
||||
this.flushStartOffsetCheckpointMs = flushStartOffsetCheckpointMs;
|
||||
return this;
|
||||
}
|
||||
|
||||
public LogManagerBuilder setRetentionCheckMs(long retentionCheckMs) {
|
||||
this.retentionCheckMs = retentionCheckMs;
|
||||
return this;
|
||||
}
|
||||
|
||||
public LogManagerBuilder setMaxPidExpirationMs(int maxPidExpirationMs) {
|
||||
this.maxPidExpirationMs = maxPidExpirationMs;
|
||||
return this;
|
||||
}
|
||||
|
||||
public LogManagerBuilder setInterBrokerProtocolVersion(ApiVersion interBrokerProtocolVersion) {
|
||||
this.interBrokerProtocolVersion = interBrokerProtocolVersion;
|
||||
return this;
|
||||
}
|
||||
|
||||
public LogManagerBuilder setScheduler(Scheduler scheduler) {
|
||||
this.scheduler = scheduler;
|
||||
return this;
|
||||
}
|
||||
|
||||
public LogManagerBuilder setBrokerTopicStats(BrokerTopicStats brokerTopicStats) {
|
||||
this.brokerTopicStats = brokerTopicStats;
|
||||
return this;
|
||||
}
|
||||
|
||||
public LogManagerBuilder setLogDirFailureChannel(LogDirFailureChannel logDirFailureChannel) {
|
||||
this.logDirFailureChannel = logDirFailureChannel;
|
||||
return this;
|
||||
}
|
||||
|
||||
public LogManagerBuilder setTime(Time time) {
|
||||
this.time = time;
|
||||
return this;
|
||||
}
|
||||
|
||||
public LogManagerBuilder setKeepPartitionMetadataFile(boolean keepPartitionMetadataFile) {
|
||||
this.keepPartitionMetadataFile = keepPartitionMetadataFile;
|
||||
return this;
|
||||
}
|
||||
|
||||
public LogManager build() {
|
||||
if (logDirs == null) throw new RuntimeException("you must set logDirs");
|
||||
if (configRepository == null) throw new RuntimeException("you must set configRepository");
|
||||
if (initialDefaultConfig == null) throw new RuntimeException("you must set initialDefaultConfig");
|
||||
if (cleanerConfig == null) throw new RuntimeException("you must set cleanerConfig");
|
||||
if (scheduler == null) throw new RuntimeException("you must set scheduler");
|
||||
if (brokerTopicStats == null) throw new RuntimeException("you must set brokerTopicStats");
|
||||
if (logDirFailureChannel == null) throw new RuntimeException("you must set logDirFailureChannel");
|
||||
|
||||
return new LogManager(JavaConverters.asScalaIteratorConverter(logDirs.iterator()).asScala().toSeq(),
|
||||
JavaConverters.asScalaIteratorConverter(initialOfflineDirs.iterator()).asScala().toSeq(),
|
||||
configRepository,
|
||||
initialDefaultConfig,
|
||||
cleanerConfig,
|
||||
recoveryThreadsPerDataDir,
|
||||
flushCheckMs,
|
||||
flushRecoveryOffsetCheckpointMs,
|
||||
flushStartOffsetCheckpointMs,
|
||||
retentionCheckMs,
|
||||
maxPidExpirationMs,
|
||||
interBrokerProtocolVersion,
|
||||
scheduler,
|
||||
brokerTopicStats,
|
||||
logDirFailureChannel,
|
||||
time,
|
||||
keepPartitionMetadataFile);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,173 @@
|
|||
/*
|
||||
* 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 kafka.server.builders;
|
||||
|
||||
import kafka.log.LogManager;
|
||||
import kafka.server.AlterIsrManager;
|
||||
import kafka.server.BrokerTopicStats;
|
||||
import kafka.server.DelayedDeleteRecords;
|
||||
import kafka.server.DelayedElectLeader;
|
||||
import kafka.server.DelayedFetch;
|
||||
import kafka.server.DelayedOperationPurgatory;
|
||||
import kafka.server.DelayedProduce;
|
||||
import kafka.server.KafkaConfig;
|
||||
import kafka.server.LogDirFailureChannel;
|
||||
import kafka.server.MetadataCache;
|
||||
import kafka.server.QuotaFactory.QuotaManagers;
|
||||
import kafka.server.ReplicaManager;
|
||||
import kafka.utils.Scheduler;
|
||||
import kafka.zk.KafkaZkClient;
|
||||
import org.apache.kafka.common.metrics.Metrics;
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
import scala.compat.java8.OptionConverters;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
|
||||
public class ReplicaManagerBuilder {
|
||||
private KafkaConfig config = null;
|
||||
private Metrics metrics = null;
|
||||
private Time time = Time.SYSTEM;
|
||||
private Scheduler scheduler = null;
|
||||
private LogManager logManager = null;
|
||||
private QuotaManagers quotaManagers = null;
|
||||
private MetadataCache metadataCache = null;
|
||||
private LogDirFailureChannel logDirFailureChannel = null;
|
||||
private AlterIsrManager alterIsrManager = null;
|
||||
private BrokerTopicStats brokerTopicStats = new BrokerTopicStats();
|
||||
private AtomicBoolean isShuttingDown = new AtomicBoolean(false);
|
||||
private Optional<KafkaZkClient> zkClient = Optional.empty();
|
||||
private Optional<DelayedOperationPurgatory<DelayedProduce>> delayedProducePurgatory = Optional.empty();
|
||||
private Optional<DelayedOperationPurgatory<DelayedFetch>> delayedFetchPurgatory = Optional.empty();
|
||||
private Optional<DelayedOperationPurgatory<DelayedDeleteRecords>> delayedDeleteRecordsPurgatory = Optional.empty();
|
||||
private Optional<DelayedOperationPurgatory<DelayedElectLeader>> delayedElectLeaderPurgatory = Optional.empty();
|
||||
private Optional<String> threadNamePrefix = Optional.empty();
|
||||
|
||||
public ReplicaManagerBuilder setConfig(KafkaConfig config) {
|
||||
this.config = config;
|
||||
return this;
|
||||
}
|
||||
|
||||
public ReplicaManagerBuilder setMetrics(Metrics metrics) {
|
||||
this.metrics = metrics;
|
||||
return this;
|
||||
}
|
||||
|
||||
public ReplicaManagerBuilder setTime(Time time) {
|
||||
this.time = time;
|
||||
return this;
|
||||
}
|
||||
|
||||
public ReplicaManagerBuilder setScheduler(Scheduler scheduler) {
|
||||
this.scheduler = scheduler;
|
||||
return this;
|
||||
}
|
||||
|
||||
public ReplicaManagerBuilder setLogManager(LogManager logManager) {
|
||||
this.logManager = logManager;
|
||||
return this;
|
||||
}
|
||||
|
||||
public ReplicaManagerBuilder setQuotaManagers(QuotaManagers quotaManagers) {
|
||||
this.quotaManagers = quotaManagers;
|
||||
return this;
|
||||
}
|
||||
|
||||
public ReplicaManagerBuilder setMetadataCache(MetadataCache metadataCache) {
|
||||
this.metadataCache = metadataCache;
|
||||
return this;
|
||||
}
|
||||
|
||||
public ReplicaManagerBuilder setLogDirFailureChannel(LogDirFailureChannel logDirFailureChannel) {
|
||||
this.logDirFailureChannel = logDirFailureChannel;
|
||||
return this;
|
||||
}
|
||||
|
||||
public ReplicaManagerBuilder setAlterIsrManager(AlterIsrManager alterIsrManager) {
|
||||
this.alterIsrManager = alterIsrManager;
|
||||
return this;
|
||||
}
|
||||
|
||||
public ReplicaManagerBuilder setBrokerTopicStats(BrokerTopicStats brokerTopicStats) {
|
||||
this.brokerTopicStats = brokerTopicStats;
|
||||
return this;
|
||||
}
|
||||
|
||||
public ReplicaManagerBuilder setIsShuttingDown(AtomicBoolean isShuttingDown) {
|
||||
this.isShuttingDown = isShuttingDown;
|
||||
return this;
|
||||
}
|
||||
|
||||
public ReplicaManagerBuilder setZkClient(KafkaZkClient zkClient) {
|
||||
this.zkClient = Optional.of(zkClient);
|
||||
return this;
|
||||
}
|
||||
|
||||
public ReplicaManagerBuilder setDelayedProducePurgatory(DelayedOperationPurgatory<DelayedProduce> delayedProducePurgatory) {
|
||||
this.delayedProducePurgatory = Optional.of(delayedProducePurgatory);
|
||||
return this;
|
||||
}
|
||||
|
||||
public ReplicaManagerBuilder setDelayedFetchPurgatory(DelayedOperationPurgatory<DelayedFetch> delayedFetchPurgatory) {
|
||||
this.delayedFetchPurgatory = Optional.of(delayedFetchPurgatory);
|
||||
return this;
|
||||
}
|
||||
|
||||
public ReplicaManagerBuilder setDelayedDeleteRecordsPurgatory(DelayedOperationPurgatory<DelayedDeleteRecords> delayedDeleteRecordsPurgatory) {
|
||||
this.delayedDeleteRecordsPurgatory = Optional.of(delayedDeleteRecordsPurgatory);
|
||||
return this;
|
||||
}
|
||||
|
||||
public ReplicaManagerBuilder setDelayedElectLeaderPurgatoryParam(DelayedOperationPurgatory<DelayedElectLeader> delayedElectLeaderPurgatory) {
|
||||
this.delayedElectLeaderPurgatory = Optional.of(delayedElectLeaderPurgatory);
|
||||
return this;
|
||||
}
|
||||
|
||||
public ReplicaManagerBuilder setThreadNamePrefix(String threadNamePrefix) {
|
||||
this.threadNamePrefix = Optional.of(threadNamePrefix);
|
||||
return this;
|
||||
}
|
||||
|
||||
public ReplicaManager build() {
|
||||
if (config == null) config = new KafkaConfig(Collections.emptyMap());
|
||||
if (metrics == null) metrics = new Metrics();
|
||||
if (logManager == null) throw new RuntimeException("You must set logManager");
|
||||
if (metadataCache == null) throw new RuntimeException("You must set metadataCache");
|
||||
if (logDirFailureChannel == null) throw new RuntimeException("You must set logDirFailureChannel");
|
||||
if (alterIsrManager == null) throw new RuntimeException("You must set alterIsrManager");
|
||||
return new ReplicaManager(config,
|
||||
metrics,
|
||||
time,
|
||||
scheduler,
|
||||
logManager,
|
||||
quotaManagers,
|
||||
metadataCache,
|
||||
logDirFailureChannel,
|
||||
alterIsrManager,
|
||||
brokerTopicStats,
|
||||
isShuttingDown,
|
||||
OptionConverters.toScala(zkClient),
|
||||
OptionConverters.toScala(delayedProducePurgatory),
|
||||
OptionConverters.toScala(delayedFetchPurgatory),
|
||||
OptionConverters.toScala(delayedDeleteRecordsPurgatory),
|
||||
OptionConverters.toScala(delayedElectLeaderPurgatory),
|
||||
OptionConverters.toScala(threadNamePrefix));
|
||||
}
|
||||
}
|
|
@ -257,10 +257,20 @@ class BrokerServer(
|
|||
)
|
||||
alterIsrManager.start()
|
||||
|
||||
this._replicaManager = new ReplicaManager(config, metrics, time, None,
|
||||
kafkaScheduler, logManager, isShuttingDown, quotaManagers,
|
||||
brokerTopicStats, metadataCache, logDirFailureChannel, alterIsrManager,
|
||||
threadNamePrefix)
|
||||
this._replicaManager = new ReplicaManager(
|
||||
config = config,
|
||||
metrics = metrics,
|
||||
time = time,
|
||||
scheduler = kafkaScheduler,
|
||||
logManager = logManager,
|
||||
quotaManagers = quotaManagers,
|
||||
metadataCache = metadataCache,
|
||||
logDirFailureChannel = logDirFailureChannel,
|
||||
alterIsrManager = alterIsrManager,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
isShuttingDown = isShuttingDown,
|
||||
zkClient = None,
|
||||
threadNamePrefix = threadNamePrefix)
|
||||
|
||||
/* start token manager */
|
||||
if (config.tokenAuthEnabled) {
|
||||
|
@ -368,10 +378,26 @@ class BrokerServer(
|
|||
|
||||
// Create the request processor objects.
|
||||
val raftSupport = RaftSupport(forwardingManager, metadataCache)
|
||||
dataPlaneRequestProcessor = new KafkaApis(socketServer.dataPlaneRequestChannel, raftSupport,
|
||||
replicaManager, groupCoordinator, transactionCoordinator, autoTopicCreationManager,
|
||||
config.nodeId, config, metadataCache, metadataCache, metrics, authorizer, quotaManagers,
|
||||
fetchManager, brokerTopicStats, clusterId, time, tokenManager, apiVersionManager)
|
||||
dataPlaneRequestProcessor = new KafkaApis(
|
||||
requestChannel = socketServer.dataPlaneRequestChannel,
|
||||
metadataSupport = raftSupport,
|
||||
replicaManager = replicaManager,
|
||||
groupCoordinator = groupCoordinator,
|
||||
txnCoordinator = transactionCoordinator,
|
||||
autoTopicCreationManager = autoTopicCreationManager,
|
||||
brokerId = config.nodeId,
|
||||
config = config,
|
||||
configRepository = metadataCache,
|
||||
metadataCache = metadataCache,
|
||||
metrics = metrics,
|
||||
authorizer = authorizer,
|
||||
quotas = quotaManagers,
|
||||
fetchManager = fetchManager,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
clusterId = clusterId,
|
||||
time = time,
|
||||
tokenManager = tokenManager,
|
||||
apiVersionManager = apiVersionManager)
|
||||
|
||||
dataPlaneRequestHandlerPool = new KafkaRequestHandlerPool(config.nodeId,
|
||||
socketServer.dataPlaneRequestChannel, dataPlaneRequestProcessor, time,
|
||||
|
|
|
@ -21,6 +21,7 @@ import java.io.{File, IOException}
|
|||
import java.net.{InetAddress, SocketTimeoutException}
|
||||
import java.util.concurrent._
|
||||
import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger}
|
||||
|
||||
import kafka.api.{KAFKA_0_9_0, KAFKA_2_2_IV0, KAFKA_2_4_IV1}
|
||||
import kafka.cluster.{Broker, EndPoint}
|
||||
import kafka.common.{GenerateBrokerIdException, InconsistentBrokerIdException, InconsistentClusterIdException}
|
||||
|
@ -29,7 +30,7 @@ import kafka.coordinator.group.GroupCoordinator
|
|||
import kafka.coordinator.transaction.{ProducerIdManager, TransactionCoordinator}
|
||||
import kafka.log.LogManager
|
||||
import kafka.metrics.{KafkaMetricsReporter, KafkaYammerMetrics}
|
||||
import kafka.network.SocketServer
|
||||
import kafka.network.{RequestChannel, SocketServer}
|
||||
import kafka.security.CredentialProvider
|
||||
import kafka.server.metadata.ZkConfigRepository
|
||||
import kafka.utils._
|
||||
|
@ -386,18 +387,35 @@ class KafkaServer(
|
|||
|
||||
/* start processing requests */
|
||||
val zkSupport = ZkSupport(adminManager, kafkaController, zkClient, forwardingManager, metadataCache)
|
||||
dataPlaneRequestProcessor = new KafkaApis(socketServer.dataPlaneRequestChannel, zkSupport, replicaManager, groupCoordinator, transactionCoordinator,
|
||||
autoTopicCreationManager, config.brokerId, config, configRepository, metadataCache, metrics, authorizer, quotaManagers,
|
||||
fetchManager, brokerTopicStats, clusterId, time, tokenManager, apiVersionManager)
|
||||
|
||||
def createKafkaApis(requestChannel: RequestChannel): KafkaApis = new KafkaApis(
|
||||
requestChannel = requestChannel,
|
||||
metadataSupport = zkSupport,
|
||||
replicaManager = replicaManager,
|
||||
groupCoordinator = groupCoordinator,
|
||||
txnCoordinator = transactionCoordinator,
|
||||
autoTopicCreationManager = autoTopicCreationManager,
|
||||
brokerId = config.brokerId,
|
||||
config = config,
|
||||
configRepository = configRepository,
|
||||
metadataCache = metadataCache,
|
||||
metrics = metrics,
|
||||
authorizer = authorizer,
|
||||
quotas = quotaManagers,
|
||||
fetchManager = fetchManager,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
clusterId = clusterId,
|
||||
time = time,
|
||||
tokenManager = tokenManager,
|
||||
apiVersionManager = apiVersionManager)
|
||||
|
||||
dataPlaneRequestProcessor = createKafkaApis(socketServer.dataPlaneRequestChannel)
|
||||
|
||||
dataPlaneRequestHandlerPool = new KafkaRequestHandlerPool(config.brokerId, socketServer.dataPlaneRequestChannel, dataPlaneRequestProcessor, time,
|
||||
config.numIoThreads, s"${SocketServer.DataPlaneMetricPrefix}RequestHandlerAvgIdlePercent", SocketServer.DataPlaneThreadPrefix)
|
||||
|
||||
socketServer.controlPlaneRequestChannelOpt.foreach { controlPlaneRequestChannel =>
|
||||
controlPlaneRequestProcessor = new KafkaApis(controlPlaneRequestChannel, zkSupport, replicaManager, groupCoordinator, transactionCoordinator,
|
||||
autoTopicCreationManager, config.brokerId, config, configRepository, metadataCache, metrics, authorizer, quotaManagers,
|
||||
fetchManager, brokerTopicStats, clusterId, time, tokenManager, apiVersionManager)
|
||||
|
||||
controlPlaneRequestProcessor = createKafkaApis(controlPlaneRequestChannel)
|
||||
controlPlaneRequestHandlerPool = new KafkaRequestHandlerPool(config.brokerId, socketServer.controlPlaneRequestChannelOpt.get, controlPlaneRequestProcessor, time,
|
||||
1, s"${SocketServer.ControlPlaneMetricPrefix}RequestHandlerAvgIdlePercent", SocketServer.ControlPlaneThreadPrefix)
|
||||
}
|
||||
|
@ -438,8 +456,20 @@ class KafkaServer(
|
|||
}
|
||||
|
||||
protected def createReplicaManager(isShuttingDown: AtomicBoolean): ReplicaManager = {
|
||||
new ReplicaManager(config, metrics, time, Some(zkClient), kafkaScheduler, logManager, isShuttingDown, quotaManagers,
|
||||
brokerTopicStats, metadataCache, logDirFailureChannel, alterIsrManager)
|
||||
new ReplicaManager(
|
||||
metrics = metrics,
|
||||
config = config,
|
||||
time = time,
|
||||
scheduler = kafkaScheduler,
|
||||
logManager = logManager,
|
||||
quotaManagers = quotaManagers,
|
||||
metadataCache = metadataCache,
|
||||
logDirFailureChannel = logDirFailureChannel,
|
||||
alterIsrManager = alterIsrManager,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
isShuttingDown = isShuttingDown,
|
||||
zkClient = Some(zkClient),
|
||||
threadNamePrefix = threadNamePrefix)
|
||||
}
|
||||
|
||||
private def initZkClient(time: Time): Unit = {
|
||||
|
|
|
@ -186,49 +186,37 @@ object ReplicaManager {
|
|||
class ReplicaManager(val config: KafkaConfig,
|
||||
metrics: Metrics,
|
||||
time: Time,
|
||||
val zkClient: Option[KafkaZkClient],
|
||||
scheduler: Scheduler,
|
||||
val logManager: LogManager,
|
||||
val isShuttingDown: AtomicBoolean,
|
||||
quotaManagers: QuotaManagers,
|
||||
val brokerTopicStats: BrokerTopicStats,
|
||||
val metadataCache: MetadataCache,
|
||||
logDirFailureChannel: LogDirFailureChannel,
|
||||
val delayedProducePurgatory: DelayedOperationPurgatory[DelayedProduce],
|
||||
val delayedFetchPurgatory: DelayedOperationPurgatory[DelayedFetch],
|
||||
val delayedDeleteRecordsPurgatory: DelayedOperationPurgatory[DelayedDeleteRecords],
|
||||
val delayedElectLeaderPurgatory: DelayedOperationPurgatory[DelayedElectLeader],
|
||||
threadNamePrefix: Option[String],
|
||||
val alterIsrManager: AlterIsrManager) extends Logging with KafkaMetricsGroup {
|
||||
val alterIsrManager: AlterIsrManager,
|
||||
val brokerTopicStats: BrokerTopicStats = new BrokerTopicStats(),
|
||||
val isShuttingDown: AtomicBoolean = new AtomicBoolean(false),
|
||||
val zkClient: Option[KafkaZkClient] = None,
|
||||
delayedProducePurgatoryParam: Option[DelayedOperationPurgatory[DelayedProduce]] = None,
|
||||
delayedFetchPurgatoryParam: Option[DelayedOperationPurgatory[DelayedFetch]] = None,
|
||||
delayedDeleteRecordsPurgatoryParam: Option[DelayedOperationPurgatory[DelayedDeleteRecords]] = None,
|
||||
delayedElectLeaderPurgatoryParam: Option[DelayedOperationPurgatory[DelayedElectLeader]] = None,
|
||||
threadNamePrefix: Option[String] = None,
|
||||
) extends Logging with KafkaMetricsGroup {
|
||||
|
||||
def this(config: KafkaConfig,
|
||||
metrics: Metrics,
|
||||
time: Time,
|
||||
zkClient: Option[KafkaZkClient],
|
||||
scheduler: Scheduler,
|
||||
logManager: LogManager,
|
||||
isShuttingDown: AtomicBoolean,
|
||||
quotaManagers: QuotaManagers,
|
||||
brokerTopicStats: BrokerTopicStats,
|
||||
metadataCache: MetadataCache,
|
||||
logDirFailureChannel: LogDirFailureChannel,
|
||||
alterIsrManager: AlterIsrManager,
|
||||
threadNamePrefix: Option[String] = None) = {
|
||||
this(config, metrics, time, zkClient, scheduler, logManager, isShuttingDown,
|
||||
quotaManagers, brokerTopicStats, metadataCache, logDirFailureChannel,
|
||||
DelayedOperationPurgatory[DelayedProduce](
|
||||
purgatoryName = "Produce", brokerId = config.brokerId,
|
||||
purgeInterval = config.producerPurgatoryPurgeIntervalRequests),
|
||||
DelayedOperationPurgatory[DelayedFetch](
|
||||
purgatoryName = "Fetch", brokerId = config.brokerId,
|
||||
purgeInterval = config.fetchPurgatoryPurgeIntervalRequests),
|
||||
DelayedOperationPurgatory[DelayedDeleteRecords](
|
||||
purgatoryName = "DeleteRecords", brokerId = config.brokerId,
|
||||
purgeInterval = config.deleteRecordsPurgatoryPurgeIntervalRequests),
|
||||
DelayedOperationPurgatory[DelayedElectLeader](
|
||||
purgatoryName = "ElectLeader", brokerId = config.brokerId),
|
||||
threadNamePrefix, alterIsrManager)
|
||||
}
|
||||
val delayedProducePurgatory = delayedProducePurgatoryParam.getOrElse(
|
||||
DelayedOperationPurgatory[DelayedProduce](
|
||||
purgatoryName = "Produce", brokerId = config.brokerId,
|
||||
purgeInterval = config.producerPurgatoryPurgeIntervalRequests))
|
||||
val delayedFetchPurgatory = delayedFetchPurgatoryParam.getOrElse(
|
||||
DelayedOperationPurgatory[DelayedFetch](
|
||||
purgatoryName = "Fetch", brokerId = config.brokerId,
|
||||
purgeInterval = config.fetchPurgatoryPurgeIntervalRequests))
|
||||
val delayedDeleteRecordsPurgatory = delayedDeleteRecordsPurgatoryParam.getOrElse(
|
||||
DelayedOperationPurgatory[DelayedDeleteRecords](
|
||||
purgatoryName = "DeleteRecords", brokerId = config.brokerId,
|
||||
purgeInterval = config.deleteRecordsPurgatoryPurgeIntervalRequests))
|
||||
val delayedElectLeaderPurgatory = delayedElectLeaderPurgatoryParam.getOrElse(
|
||||
DelayedOperationPurgatory[DelayedElectLeader](
|
||||
purgatoryName = "ElectLeader", brokerId = config.brokerId))
|
||||
|
||||
/* epoch of the controller that last changed the leader */
|
||||
@volatile private[server] var controllerEpoch: Int = KafkaController.InitialControllerEpoch
|
||||
|
|
|
@ -40,7 +40,7 @@ import scala.jdk.CollectionConverters._
|
|||
|
||||
class LogLoaderTest {
|
||||
var config: KafkaConfig = null
|
||||
val brokerTopicStats = new BrokerTopicStats
|
||||
val brokerTopicStats = new BrokerTopicStats()
|
||||
val maxProducerIdExpirationMs: Int = 60 * 60 * 1000
|
||||
val tmpDir = TestUtils.tempDir()
|
||||
val logDir = TestUtils.randomPartitionLogDir(tmpDir)
|
||||
|
@ -75,13 +75,25 @@ class LogLoaderTest {
|
|||
|
||||
// Create a LogManager with some overridden methods to facilitate interception of clean shutdown
|
||||
// flag and to inject a runtime error
|
||||
def interceptedLogManager(logConfig: LogConfig, logDirs: Seq[File], simulateError: SimulateError): LogManager = {
|
||||
new LogManager(logDirs = logDirs.map(_.getAbsoluteFile), initialOfflineDirs = Array.empty[File], new MockConfigRepository(),
|
||||
initialDefaultConfig = logConfig, cleanerConfig = CleanerConfig(enableCleaner = false), recoveryThreadsPerDataDir = 4,
|
||||
flushCheckMs = 1000L, flushRecoveryOffsetCheckpointMs = 10000L, flushStartOffsetCheckpointMs = 10000L,
|
||||
retentionCheckMs = 1000L, maxPidExpirationMs = 60 * 60 * 1000, scheduler = time.scheduler, time = time,
|
||||
brokerTopicStats = new BrokerTopicStats, logDirFailureChannel = new LogDirFailureChannel(logDirs.size),
|
||||
keepPartitionMetadataFile = config.usesTopicId, interBrokerProtocolVersion = config.interBrokerProtocolVersion) {
|
||||
def interceptedLogManager(logConfig: LogConfig, logDirs: Seq[File], simulateError: SimulateError): LogManager =
|
||||
new LogManager(
|
||||
logDirs = logDirs.map(_.getAbsoluteFile),
|
||||
initialOfflineDirs = Array.empty[File],
|
||||
configRepository = new MockConfigRepository(),
|
||||
initialDefaultConfig = logConfig,
|
||||
cleanerConfig = CleanerConfig(enableCleaner = false),
|
||||
recoveryThreadsPerDataDir = 4,
|
||||
flushCheckMs = 1000L,
|
||||
flushRecoveryOffsetCheckpointMs = 10000L,
|
||||
flushStartOffsetCheckpointMs = 10000L,
|
||||
retentionCheckMs = 1000L,
|
||||
maxPidExpirationMs = 60 * 60 * 1000,
|
||||
interBrokerProtocolVersion = config.interBrokerProtocolVersion,
|
||||
scheduler = time.scheduler,
|
||||
brokerTopicStats = new BrokerTopicStats(),
|
||||
logDirFailureChannel = new LogDirFailureChannel(logDirs.size),
|
||||
time = time,
|
||||
keepPartitionMetadataFile = config.usesTopicId) {
|
||||
|
||||
override def loadLog(logDir: File, hadCleanShutdown: Boolean, recoveryPoints: Map[TopicPartition, Long],
|
||||
logStartOffsets: Map[TopicPartition, Long], defaultConfig: LogConfig,
|
||||
|
@ -111,7 +123,6 @@ class LogLoaderTest {
|
|||
producerStateManager, None, true)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
val cleanShutdownFile = new File(logDir, LogLoader.CleanShutdownFile)
|
||||
locally {
|
||||
|
|
|
@ -24,7 +24,6 @@ import org.apache.kafka.common.utils.Utils
|
|||
import org.junit.jupiter.api._
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import kafka.utils.{KafkaScheduler, MockTime, TestUtils}
|
||||
import java.util.concurrent.atomic.AtomicBoolean
|
||||
|
||||
import kafka.cluster.Partition
|
||||
import kafka.server.metadata.MockConfigRepository
|
||||
|
@ -63,9 +62,16 @@ class HighwatermarkPersistenceTest {
|
|||
val time = new MockTime
|
||||
val quotaManager = QuotaFactory.instantiate(configs.head, metrics, time, "")
|
||||
// create replica manager
|
||||
val replicaManager = new ReplicaManager(configs.head, metrics, time, None, scheduler,
|
||||
logManagers.head, new AtomicBoolean(false), quotaManager,
|
||||
new BrokerTopicStats, MetadataCache.zkMetadataCache(configs.head.brokerId), logDirFailureChannels.head, alterIsrManager)
|
||||
val replicaManager = new ReplicaManager(
|
||||
metrics = metrics,
|
||||
config = configs.head,
|
||||
time = time,
|
||||
scheduler = scheduler,
|
||||
logManager = logManagers.head,
|
||||
quotaManagers = quotaManager,
|
||||
metadataCache = MetadataCache.zkMetadataCache(configs.head.brokerId),
|
||||
logDirFailureChannel = logDirFailureChannels.head,
|
||||
alterIsrManager = alterIsrManager)
|
||||
replicaManager.startup()
|
||||
try {
|
||||
replicaManager.checkpointHighWatermarks()
|
||||
|
@ -112,9 +118,16 @@ class HighwatermarkPersistenceTest {
|
|||
val time = new MockTime
|
||||
val quotaManager = QuotaFactory.instantiate(configs.head, metrics, time, "")
|
||||
// create replica manager
|
||||
val replicaManager = new ReplicaManager(configs.head, metrics, time, None,
|
||||
scheduler, logManagers.head, new AtomicBoolean(false), quotaManager,
|
||||
new BrokerTopicStats, MetadataCache.zkMetadataCache(configs.head.brokerId), logDirFailureChannels.head, alterIsrManager)
|
||||
val replicaManager = new ReplicaManager(
|
||||
metrics = metrics,
|
||||
config = configs.head,
|
||||
time = time,
|
||||
scheduler = scheduler,
|
||||
logManager = logManagers.head,
|
||||
quotaManagers = quotaManager,
|
||||
metadataCache = MetadataCache.zkMetadataCache(configs.head.brokerId),
|
||||
logDirFailureChannel = logDirFailureChannels.head,
|
||||
alterIsrManager = alterIsrManager)
|
||||
replicaManager.startup()
|
||||
try {
|
||||
replicaManager.checkpointHighWatermarks()
|
||||
|
|
|
@ -18,7 +18,6 @@ package kafka.server
|
|||
|
||||
import java.io.File
|
||||
import java.util.Properties
|
||||
import java.util.concurrent.atomic.AtomicBoolean
|
||||
|
||||
import kafka.cluster.Partition
|
||||
import kafka.log.{UnifiedLog, LogManager}
|
||||
|
@ -65,9 +64,16 @@ class IsrExpirationTest {
|
|||
|
||||
alterIsrManager = TestUtils.createAlterIsrManager()
|
||||
quotaManager = QuotaFactory.instantiate(configs.head, metrics, time, "")
|
||||
replicaManager = new ReplicaManager(configs.head, metrics, time, None, null, logManager, new AtomicBoolean(false),
|
||||
quotaManager, new BrokerTopicStats, MetadataCache.zkMetadataCache(configs.head.brokerId),
|
||||
new LogDirFailureChannel(configs.head.logDirs.size), alterIsrManager)
|
||||
replicaManager = new ReplicaManager(
|
||||
metrics = metrics,
|
||||
config = configs.head,
|
||||
time = time,
|
||||
scheduler = null,
|
||||
logManager = logManager,
|
||||
quotaManagers = quotaManager,
|
||||
metadataCache = MetadataCache.zkMetadataCache(configs.head.brokerId),
|
||||
logDirFailureChannel = new LogDirFailureChannel(configs.head.logDirs.size),
|
||||
alterIsrManager = alterIsrManager)
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
|
|
|
@ -171,25 +171,26 @@ class KafkaApisTest {
|
|||
}
|
||||
val apiVersionManager = new SimpleApiVersionManager(listenerType, enabledApis)
|
||||
|
||||
new KafkaApis(requestChannel,
|
||||
metadataSupport,
|
||||
replicaManager,
|
||||
groupCoordinator,
|
||||
txnCoordinator,
|
||||
autoTopicCreationManager,
|
||||
brokerId,
|
||||
config,
|
||||
configRepository,
|
||||
metadataCache,
|
||||
metrics,
|
||||
authorizer,
|
||||
quotas,
|
||||
fetchManager,
|
||||
brokerTopicStats,
|
||||
clusterId,
|
||||
time,
|
||||
null,
|
||||
apiVersionManager)
|
||||
new KafkaApis(
|
||||
metadataSupport = metadataSupport,
|
||||
requestChannel = requestChannel,
|
||||
replicaManager = replicaManager,
|
||||
groupCoordinator = groupCoordinator,
|
||||
txnCoordinator = txnCoordinator,
|
||||
autoTopicCreationManager = autoTopicCreationManager,
|
||||
brokerId = brokerId,
|
||||
config = config,
|
||||
configRepository = configRepository,
|
||||
metadataCache = metadataCache,
|
||||
metrics = metrics,
|
||||
authorizer = authorizer,
|
||||
quotas = quotas,
|
||||
fetchManager = fetchManager,
|
||||
brokerTopicStats = brokerTopicStats,
|
||||
clusterId = clusterId,
|
||||
time = time,
|
||||
tokenManager = null,
|
||||
apiVersionManager = apiVersionManager)
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -18,7 +18,6 @@ package kafka.server
|
|||
|
||||
import java.io.File
|
||||
import java.util.{Collections, Optional, Properties}
|
||||
import java.util.concurrent.atomic.AtomicBoolean
|
||||
|
||||
import kafka.cluster.Partition
|
||||
import kafka.log.{UnifiedLog, LogManager, LogOffsetSnapshot}
|
||||
|
@ -249,9 +248,16 @@ class ReplicaManagerQuotasTest {
|
|||
|
||||
val leaderBrokerId = configs.head.brokerId
|
||||
quotaManager = QuotaFactory.instantiate(configs.head, metrics, time, "")
|
||||
replicaManager = new ReplicaManager(configs.head, metrics, time, None, scheduler, logManager,
|
||||
new AtomicBoolean(false), quotaManager,
|
||||
new BrokerTopicStats, MetadataCache.zkMetadataCache(leaderBrokerId), new LogDirFailureChannel(configs.head.logDirs.size), alterIsrManager)
|
||||
replicaManager = new ReplicaManager(
|
||||
metrics = metrics,
|
||||
config = configs.head,
|
||||
time = time,
|
||||
scheduler = scheduler,
|
||||
logManager = logManager,
|
||||
quotaManagers = quotaManager,
|
||||
metadataCache = MetadataCache.zkMetadataCache(leaderBrokerId),
|
||||
logDirFailureChannel = new LogDirFailureChannel(configs.head.logDirs.size),
|
||||
alterIsrManager = alterIsrManager)
|
||||
|
||||
//create the two replicas
|
||||
for ((p, _) <- fetchInfo) {
|
||||
|
|
|
@ -21,7 +21,7 @@ import java.io.File
|
|||
import java.net.InetAddress
|
||||
import java.nio.file.Files
|
||||
import java.util
|
||||
import java.util.concurrent.atomic.{AtomicBoolean, AtomicReference}
|
||||
import java.util.concurrent.atomic.AtomicReference
|
||||
import java.util.concurrent.{CountDownLatch, TimeUnit}
|
||||
import java.util.stream.IntStream
|
||||
import java.util.{Collections, Optional, Properties}
|
||||
|
@ -101,9 +101,16 @@ class ReplicaManagerTest {
|
|||
@Test
|
||||
def testHighWaterMarkDirectoryMapping(): Unit = {
|
||||
val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)))
|
||||
val rm = new ReplicaManager(config, metrics, time, None, new MockScheduler(time), mockLogMgr,
|
||||
new AtomicBoolean(false), quotaManager, new BrokerTopicStats,
|
||||
MetadataCache.zkMetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), alterIsrManager)
|
||||
val rm = new ReplicaManager(
|
||||
metrics = metrics,
|
||||
config = config,
|
||||
time = time,
|
||||
scheduler = new MockScheduler(time),
|
||||
logManager = mockLogMgr,
|
||||
quotaManagers = quotaManager,
|
||||
metadataCache = MetadataCache.zkMetadataCache(config.brokerId),
|
||||
logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size),
|
||||
alterIsrManager = alterIsrManager)
|
||||
try {
|
||||
val partition = rm.createPartition(new TopicPartition(topic, 1))
|
||||
partition.createLogIfNotExists(isNew = false, isFutureReplica = false,
|
||||
|
@ -121,9 +128,16 @@ class ReplicaManagerTest {
|
|||
props.put("log.dir", TestUtils.tempRelativeDir("data").getAbsolutePath)
|
||||
val config = KafkaConfig.fromProps(props)
|
||||
val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)))
|
||||
val rm = new ReplicaManager(config, metrics, time, None, new MockScheduler(time), mockLogMgr,
|
||||
new AtomicBoolean(false), quotaManager, new BrokerTopicStats,
|
||||
MetadataCache.zkMetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), alterIsrManager)
|
||||
val rm = new ReplicaManager(
|
||||
metrics = metrics,
|
||||
config = config,
|
||||
time = time,
|
||||
scheduler = new MockScheduler(time),
|
||||
logManager = mockLogMgr,
|
||||
quotaManagers = quotaManager,
|
||||
metadataCache = MetadataCache.zkMetadataCache(config.brokerId),
|
||||
logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size),
|
||||
alterIsrManager = alterIsrManager)
|
||||
try {
|
||||
val partition = rm.createPartition(new TopicPartition(topic, 1))
|
||||
partition.createLogIfNotExists(isNew = false, isFutureReplica = false,
|
||||
|
@ -138,9 +152,17 @@ class ReplicaManagerTest {
|
|||
@Test
|
||||
def testIllegalRequiredAcks(): Unit = {
|
||||
val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)))
|
||||
val rm = new ReplicaManager(config, metrics, time, None, new MockScheduler(time), mockLogMgr,
|
||||
new AtomicBoolean(false), quotaManager, new BrokerTopicStats,
|
||||
MetadataCache.zkMetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), alterIsrManager, Option(this.getClass.getName))
|
||||
val rm = new ReplicaManager(
|
||||
metrics = metrics,
|
||||
config = config,
|
||||
time = time,
|
||||
scheduler = new MockScheduler(time),
|
||||
logManager = mockLogMgr,
|
||||
quotaManagers = quotaManager,
|
||||
metadataCache = MetadataCache.zkMetadataCache(config.brokerId),
|
||||
logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size),
|
||||
alterIsrManager = alterIsrManager,
|
||||
threadNamePrefix = Option(this.getClass.getName))
|
||||
try {
|
||||
def callback(responseStatus: Map[TopicPartition, PartitionResponse]) = {
|
||||
assert(responseStatus.values.head.error == Errors.INVALID_REQUIRED_ACKS)
|
||||
|
@ -185,9 +207,16 @@ class ReplicaManagerTest {
|
|||
val aliveBrokers = Seq(new Node(0, "host0", 0), new Node(1, "host1", 1))
|
||||
val metadataCache: MetadataCache = Mockito.mock(classOf[MetadataCache])
|
||||
mockGetAliveBrokerFunctions(metadataCache, aliveBrokers)
|
||||
val rm = new ReplicaManager(config, metrics, time, None, new MockScheduler(time), mockLogMgr,
|
||||
new AtomicBoolean(false), quotaManager, new BrokerTopicStats,
|
||||
metadataCache, new LogDirFailureChannel(config.logDirs.size), alterIsrManager)
|
||||
val rm = new ReplicaManager(
|
||||
metrics = metrics,
|
||||
config = config,
|
||||
time = time,
|
||||
scheduler = new MockScheduler(time),
|
||||
logManager = mockLogMgr,
|
||||
quotaManagers = quotaManager,
|
||||
metadataCache = metadataCache,
|
||||
logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size),
|
||||
alterIsrManager = alterIsrManager)
|
||||
|
||||
try {
|
||||
val brokerList = Seq[Integer](0, 1).asJava
|
||||
|
@ -1797,11 +1826,22 @@ class ReplicaManagerTest {
|
|||
.setLeaderEpoch(leaderEpochFromLeader)
|
||||
.setEndOffset(offsetFromLeader)).asJava,
|
||||
BrokerEndPoint(1, "host1" ,1), time)
|
||||
val replicaManager = new ReplicaManager(config, metrics, time, None, mockScheduler, mockLogMgr,
|
||||
new AtomicBoolean(false), quotaManager, mockBrokerTopicStats,
|
||||
metadataCache, mockLogDirFailureChannel, mockProducePurgatory, mockFetchPurgatory,
|
||||
mockDeleteRecordsPurgatory, mockElectLeaderPurgatory, Option(this.getClass.getName),
|
||||
alterIsrManager) {
|
||||
val replicaManager = new ReplicaManager(
|
||||
metrics = metrics,
|
||||
config = config,
|
||||
time = time,
|
||||
scheduler = mockScheduler,
|
||||
logManager = mockLogMgr,
|
||||
quotaManagers = quotaManager,
|
||||
brokerTopicStats = mockBrokerTopicStats,
|
||||
metadataCache = metadataCache,
|
||||
logDirFailureChannel = mockLogDirFailureChannel,
|
||||
alterIsrManager = alterIsrManager,
|
||||
delayedProducePurgatoryParam = Some(mockProducePurgatory),
|
||||
delayedFetchPurgatoryParam = Some(mockFetchPurgatory),
|
||||
delayedDeleteRecordsPurgatoryParam = Some(mockDeleteRecordsPurgatory),
|
||||
delayedElectLeaderPurgatoryParam = Some(mockElectLeaderPurgatory),
|
||||
threadNamePrefix = Option(this.getClass.getName)) {
|
||||
|
||||
override protected def createReplicaFetcherManager(metrics: Metrics,
|
||||
time: Time,
|
||||
|
@ -1979,11 +2019,21 @@ class ReplicaManagerTest {
|
|||
val mockDelayedElectLeaderPurgatory = new DelayedOperationPurgatory[DelayedElectLeader](
|
||||
purgatoryName = "DelayedElectLeader", timer, reaperEnabled = false)
|
||||
|
||||
new ReplicaManager(config, metrics, time, None, scheduler, mockLogMgr,
|
||||
new AtomicBoolean(false), quotaManager, new BrokerTopicStats,
|
||||
metadataCache, new LogDirFailureChannel(config.logDirs.size), mockProducePurgatory, mockFetchPurgatory,
|
||||
mockDeleteRecordsPurgatory, mockDelayedElectLeaderPurgatory, Option(this.getClass.getName),
|
||||
alterIsrManager) {
|
||||
new ReplicaManager(
|
||||
metrics = metrics,
|
||||
config = config,
|
||||
time = time,
|
||||
scheduler = scheduler,
|
||||
logManager = mockLogMgr,
|
||||
quotaManagers = quotaManager,
|
||||
metadataCache = metadataCache,
|
||||
logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size),
|
||||
alterIsrManager = alterIsrManager,
|
||||
delayedProducePurgatoryParam = Some(mockProducePurgatory),
|
||||
delayedFetchPurgatoryParam = Some(mockFetchPurgatory),
|
||||
delayedDeleteRecordsPurgatoryParam = Some(mockDeleteRecordsPurgatory),
|
||||
delayedElectLeaderPurgatoryParam = Some(mockDelayedElectLeaderPurgatory),
|
||||
threadNamePrefix = Option(this.getClass.getName)) {
|
||||
|
||||
override protected def createReplicaFetcherManager(
|
||||
metrics: Metrics,
|
||||
|
@ -2207,12 +2257,28 @@ class ReplicaManagerTest {
|
|||
mockGetAliveBrokerFunctions(metadataCache1, aliveBrokers)
|
||||
|
||||
// each replica manager is for a broker
|
||||
val rm0 = new ReplicaManager(config0, metrics, time, None, new MockScheduler(time), mockLogMgr0,
|
||||
new AtomicBoolean(false), quotaManager,
|
||||
brokerTopicStats1, metadataCache0, new LogDirFailureChannel(config0.logDirs.size), alterIsrManager)
|
||||
val rm1 = new ReplicaManager(config1, metrics, time, None, new MockScheduler(time), mockLogMgr1,
|
||||
new AtomicBoolean(false), quotaManager,
|
||||
brokerTopicStats2, metadataCache1, new LogDirFailureChannel(config1.logDirs.size), alterIsrManager)
|
||||
val rm0 = new ReplicaManager(
|
||||
metrics = metrics,
|
||||
config = config0,
|
||||
time = time,
|
||||
scheduler = new MockScheduler(time),
|
||||
logManager = mockLogMgr0,
|
||||
quotaManagers = quotaManager,
|
||||
brokerTopicStats = brokerTopicStats1,
|
||||
metadataCache = metadataCache0,
|
||||
logDirFailureChannel = new LogDirFailureChannel(config0.logDirs.size),
|
||||
alterIsrManager = alterIsrManager)
|
||||
val rm1 = new ReplicaManager(
|
||||
metrics = metrics,
|
||||
config = config1,
|
||||
time = time,
|
||||
scheduler = new MockScheduler(time),
|
||||
logManager = mockLogMgr1,
|
||||
quotaManagers = quotaManager,
|
||||
brokerTopicStats = brokerTopicStats2,
|
||||
metadataCache = metadataCache1,
|
||||
logDirFailureChannel = new LogDirFailureChannel(config1.logDirs.size),
|
||||
alterIsrManager = alterIsrManager)
|
||||
|
||||
(rm0, rm1)
|
||||
}
|
||||
|
@ -2453,9 +2519,16 @@ class ReplicaManagerTest {
|
|||
val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect)
|
||||
val config = KafkaConfig.fromProps(props)
|
||||
val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)))
|
||||
new ReplicaManager(config, metrics, time, None, new MockScheduler(time), mockLogMgr,
|
||||
new AtomicBoolean(false), quotaManager, new BrokerTopicStats,
|
||||
MetadataCache.zkMetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), alterIsrManager) {
|
||||
new ReplicaManager(
|
||||
metrics = metrics,
|
||||
config = config,
|
||||
time = time,
|
||||
scheduler = new MockScheduler(time),
|
||||
logManager = mockLogMgr,
|
||||
quotaManagers = quotaManager,
|
||||
metadataCache = MetadataCache.zkMetadataCache(config.brokerId),
|
||||
logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size),
|
||||
alterIsrManager = alterIsrManager) {
|
||||
override def getPartitionOrException(topicPartition: TopicPartition): Partition = {
|
||||
throw Errors.NOT_LEADER_OR_FOLLOWER.exception()
|
||||
}
|
||||
|
|
|
@ -17,7 +17,6 @@
|
|||
package kafka.server.epoch
|
||||
|
||||
import java.io.File
|
||||
import java.util.concurrent.atomic.AtomicBoolean
|
||||
|
||||
import kafka.log.{UnifiedLog, LogManager}
|
||||
import kafka.server.QuotaFactory.QuotaManagers
|
||||
|
@ -65,9 +64,16 @@ class OffsetsForLeaderEpochTest {
|
|||
replay(mockLog, logManager)
|
||||
|
||||
// create a replica manager with 1 partition that has 1 replica
|
||||
replicaManager = new ReplicaManager(config, metrics, time, None, null, logManager, new AtomicBoolean(false),
|
||||
quotaManager, new BrokerTopicStats,
|
||||
MetadataCache.zkMetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), alterIsrManager)
|
||||
replicaManager = new ReplicaManager(
|
||||
metrics = metrics,
|
||||
config = config,
|
||||
time = time,
|
||||
scheduler = null,
|
||||
logManager = logManager,
|
||||
quotaManagers = quotaManager,
|
||||
metadataCache = MetadataCache.zkMetadataCache(config.brokerId),
|
||||
logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size),
|
||||
alterIsrManager = alterIsrManager)
|
||||
val partition = replicaManager.createPartition(tp)
|
||||
partition.setLog(mockLog, isFutureLog = false)
|
||||
partition.leaderReplicaIdOpt = Some(config.brokerId)
|
||||
|
@ -88,9 +94,16 @@ class OffsetsForLeaderEpochTest {
|
|||
replay(logManager)
|
||||
|
||||
//create a replica manager with 1 partition that has 0 replica
|
||||
replicaManager = new ReplicaManager(config, metrics, time, None, null, logManager, new AtomicBoolean(false),
|
||||
quotaManager, new BrokerTopicStats,
|
||||
MetadataCache.zkMetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), alterIsrManager)
|
||||
replicaManager = new ReplicaManager(
|
||||
metrics = metrics,
|
||||
config = config,
|
||||
time = time,
|
||||
scheduler = null,
|
||||
logManager = logManager,
|
||||
quotaManagers = quotaManager,
|
||||
metadataCache = MetadataCache.zkMetadataCache(config.brokerId),
|
||||
logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size),
|
||||
alterIsrManager = alterIsrManager)
|
||||
replicaManager.createPartition(tp)
|
||||
|
||||
//Given
|
||||
|
@ -113,9 +126,16 @@ class OffsetsForLeaderEpochTest {
|
|||
replay(logManager)
|
||||
|
||||
//create a replica manager with 0 partition
|
||||
replicaManager = new ReplicaManager(config, metrics, time, None, null, logManager, new AtomicBoolean(false),
|
||||
quotaManager, new BrokerTopicStats,
|
||||
MetadataCache.zkMetadataCache(config.brokerId), new LogDirFailureChannel(config.logDirs.size), alterIsrManager)
|
||||
replicaManager = new ReplicaManager(
|
||||
metrics = metrics,
|
||||
config = config,
|
||||
time = time,
|
||||
scheduler = null,
|
||||
logManager = logManager,
|
||||
quotaManagers = quotaManager,
|
||||
metadataCache = MetadataCache.zkMetadataCache(config.brokerId),
|
||||
logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size),
|
||||
alterIsrManager = alterIsrManager)
|
||||
|
||||
//Given
|
||||
val epochRequested: Integer = 5
|
||||
|
|
|
@ -42,6 +42,8 @@ import kafka.server.ReplicaFetcherThread;
|
|||
import kafka.server.ReplicaManager;
|
||||
import kafka.server.ReplicaQuota;
|
||||
import kafka.server.ZkMetadataCache;
|
||||
import kafka.server.builders.LogManagerBuilder;
|
||||
import kafka.server.builders.ReplicaManagerBuilder;
|
||||
import kafka.server.checkpoints.OffsetCheckpoints;
|
||||
import kafka.server.metadata.MockConfigRepository;
|
||||
import kafka.utils.KafkaScheduler;
|
||||
|
@ -80,10 +82,6 @@ import org.openjdk.jmh.annotations.Setup;
|
|||
import org.openjdk.jmh.annotations.State;
|
||||
import org.openjdk.jmh.annotations.TearDown;
|
||||
import org.openjdk.jmh.annotations.Warmup;
|
||||
import scala.Option;
|
||||
import scala.collection.Iterator;
|
||||
import scala.collection.JavaConverters;
|
||||
import scala.collection.Map;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
|
@ -96,7 +94,9 @@ import java.util.List;
|
|||
import java.util.Properties;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import scala.Option;
|
||||
import scala.collection.Iterator;
|
||||
import scala.collection.Map;
|
||||
|
||||
@State(Scope.Benchmark)
|
||||
@Fork(value = 1)
|
||||
|
@ -129,26 +129,28 @@ public class ReplicaFetcherThreadBenchmark {
|
|||
KafkaConfig config = new KafkaConfig(props);
|
||||
LogConfig logConfig = createLogConfig();
|
||||
|
||||
List<File> logDirs = Collections.singletonList(logDir);
|
||||
BrokerTopicStats brokerTopicStats = new BrokerTopicStats();
|
||||
LogDirFailureChannel logDirFailureChannel = Mockito.mock(LogDirFailureChannel.class);
|
||||
logManager = new LogManager(JavaConverters.asScalaIteratorConverter(logDirs.iterator()).asScala().toSeq(),
|
||||
JavaConverters.asScalaIteratorConverter(new ArrayList<File>().iterator()).asScala().toSeq(),
|
||||
new MockConfigRepository(),
|
||||
logConfig,
|
||||
new CleanerConfig(0, 0, 0, 0, 0, 0.0, 0, false, "MD5"),
|
||||
1,
|
||||
1000L,
|
||||
10000L,
|
||||
10000L,
|
||||
1000L,
|
||||
60000,
|
||||
ApiVersion.latestVersion(),
|
||||
scheduler,
|
||||
brokerTopicStats,
|
||||
logDirFailureChannel,
|
||||
Time.SYSTEM,
|
||||
true);
|
||||
List<File> logDirs = Collections.singletonList(logDir);
|
||||
logManager = new LogManagerBuilder().
|
||||
setLogDirs(logDirs).
|
||||
setInitialOfflineDirs(Collections.emptyList()).
|
||||
setConfigRepository(new MockConfigRepository()).
|
||||
setInitialDefaultConfig(logConfig).
|
||||
setCleanerConfig(new CleanerConfig(0, 0, 0, 0, 0, 0.0, 0, false, "MD5")).
|
||||
setRecoveryThreadsPerDataDir(1).
|
||||
setFlushCheckMs(1000L).
|
||||
setFlushRecoveryOffsetCheckpointMs(10000L).
|
||||
setFlushStartOffsetCheckpointMs(10000L).
|
||||
setRetentionCheckMs(1000L).
|
||||
setMaxPidExpirationMs(60000).
|
||||
setInterBrokerProtocolVersion(ApiVersion.latestVersion()).
|
||||
setScheduler(scheduler).
|
||||
setBrokerTopicStats(brokerTopicStats).
|
||||
setLogDirFailureChannel(logDirFailureChannel).
|
||||
setTime(Time.SYSTEM).
|
||||
setKeepPartitionMetadataFile(true).
|
||||
build();
|
||||
|
||||
LinkedHashMap<TopicPartition, FetchResponseData.PartitionData> initialFetched = new LinkedHashMap<>();
|
||||
HashMap<String, Uuid> topicIds = new HashMap<>();
|
||||
|
@ -213,8 +215,19 @@ public class ReplicaFetcherThreadBenchmark {
|
|||
ZkMetadataCache metadataCache = new ZkMetadataCache(0);
|
||||
metadataCache.updateMetadata(0, updateMetadataRequest);
|
||||
|
||||
replicaManager = new ReplicaManager(config, metrics, new MockTime(), Option.apply(Mockito.mock(KafkaZkClient.class)), scheduler, logManager, new AtomicBoolean(false),
|
||||
Mockito.mock(QuotaFactory.QuotaManagers.class), brokerTopicStats, metadataCache, new LogDirFailureChannel(logDirs.size()), TestUtils.createAlterIsrManager(), Option.empty());
|
||||
replicaManager = new ReplicaManagerBuilder().
|
||||
setConfig(config).
|
||||
setMetrics(metrics).
|
||||
setTime(new MockTime()).
|
||||
setZkClient(Mockito.mock(KafkaZkClient.class)).
|
||||
setScheduler(scheduler).
|
||||
setLogManager(logManager).
|
||||
setQuotaManagers(Mockito.mock(QuotaFactory.QuotaManagers.class)).
|
||||
setBrokerTopicStats(brokerTopicStats).
|
||||
setMetadataCache(metadataCache).
|
||||
setLogDirFailureChannel(new LogDirFailureChannel(logDirs.size())).
|
||||
setAlterIsrManager(TestUtils.createAlterIsrManager()).
|
||||
build();
|
||||
fetcher = new ReplicaFetcherBenchThread(config, replicaManager, pool);
|
||||
fetcher.addPartitions(initialFetchStates);
|
||||
// force a pass to move partitions to fetching state. We do this in the setup phase
|
||||
|
|
|
@ -39,6 +39,7 @@ import kafka.server.ReplicationQuotaManager;
|
|||
import kafka.server.SimpleApiVersionManager;
|
||||
import kafka.server.ZkAdminManager;
|
||||
import kafka.server.ZkSupport;
|
||||
import kafka.server.builders.KafkaApisBuilder;
|
||||
import kafka.server.metadata.MockConfigRepository;
|
||||
import kafka.zk.KafkaZkClient;
|
||||
import org.apache.kafka.common.memory.MemoryPool;
|
||||
|
@ -56,7 +57,7 @@ import org.apache.kafka.common.requests.RequestHeader;
|
|||
import org.apache.kafka.common.requests.UpdateMetadataRequest;
|
||||
import org.apache.kafka.common.security.auth.KafkaPrincipal;
|
||||
import org.apache.kafka.common.security.auth.SecurityProtocol;
|
||||
import org.apache.kafka.common.utils.SystemTime;
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
import org.mockito.Mockito;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
import org.openjdk.jmh.annotations.BenchmarkMode;
|
||||
|
@ -78,6 +79,7 @@ import java.util.Arrays;
|
|||
import java.util.Collections;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.Properties;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.stream.IntStream;
|
||||
|
@ -173,25 +175,27 @@ public class MetadataRequestBenchmark {
|
|||
kafkaProps.put(KafkaConfig$.MODULE$.ZkConnectProp(), "zk");
|
||||
kafkaProps.put(KafkaConfig$.MODULE$.BrokerIdProp(), brokerId + "");
|
||||
KafkaConfig config = new KafkaConfig(kafkaProps);
|
||||
return new KafkaApis(requestChannel,
|
||||
new ZkSupport(adminManager, kafkaController, kafkaZkClient, Option.empty(), metadataCache),
|
||||
replicaManager,
|
||||
groupCoordinator,
|
||||
transactionCoordinator,
|
||||
autoTopicCreationManager,
|
||||
brokerId,
|
||||
config,
|
||||
new MockConfigRepository(),
|
||||
metadataCache,
|
||||
metrics,
|
||||
Option.empty(),
|
||||
quotaManagers,
|
||||
fetchManager,
|
||||
brokerTopicStats,
|
||||
"clusterId",
|
||||
new SystemTime(),
|
||||
null,
|
||||
new SimpleApiVersionManager(ApiMessageType.ListenerType.ZK_BROKER));
|
||||
return new KafkaApisBuilder().
|
||||
setRequestChannel(requestChannel).
|
||||
setMetadataSupport(new ZkSupport(adminManager, kafkaController, kafkaZkClient, Option.empty(), metadataCache)).
|
||||
setReplicaManager(replicaManager).
|
||||
setGroupCoordinator(groupCoordinator).
|
||||
setTxnCoordinator(transactionCoordinator).
|
||||
setAutoTopicCreationManager(autoTopicCreationManager).
|
||||
setBrokerId(brokerId).
|
||||
setConfig(config).
|
||||
setConfigRepository(new MockConfigRepository()).
|
||||
setMetadataCache(metadataCache).
|
||||
setMetrics(metrics).
|
||||
setAuthorizer(Optional.empty()).
|
||||
setQuotas(quotaManagers).
|
||||
setFetchManager(fetchManager).
|
||||
setBrokerTopicStats(brokerTopicStats).
|
||||
setClusterId("clusterId").
|
||||
setTime(Time.SYSTEM).
|
||||
setTokenManager(null).
|
||||
setApiVersionManager(new SimpleApiVersionManager(ApiMessageType.ListenerType.ZK_BROKER)).
|
||||
build();
|
||||
}
|
||||
|
||||
@TearDown(Level.Trial)
|
||||
|
|
|
@ -30,6 +30,7 @@ import kafka.server.AlterIsrManager;
|
|||
import kafka.server.BrokerTopicStats;
|
||||
import kafka.server.LogDirFailureChannel;
|
||||
import kafka.server.MetadataCache;
|
||||
import kafka.server.builders.LogManagerBuilder;
|
||||
import kafka.server.checkpoints.OffsetCheckpoints;
|
||||
import kafka.server.metadata.MockConfigRepository;
|
||||
import kafka.utils.KafkaScheduler;
|
||||
|
@ -54,14 +55,10 @@ import org.openjdk.jmh.annotations.Setup;
|
|||
import org.openjdk.jmh.annotations.State;
|
||||
import org.openjdk.jmh.annotations.TearDown;
|
||||
import org.openjdk.jmh.annotations.Warmup;
|
||||
import scala.Option;
|
||||
import scala.collection.JavaConverters;
|
||||
import scala.compat.java8.OptionConverters;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
@ -71,6 +68,8 @@ import java.util.UUID;
|
|||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import scala.Option;
|
||||
import scala.compat.java8.OptionConverters;
|
||||
|
||||
@State(Scope.Benchmark)
|
||||
@Fork(value = 1)
|
||||
|
@ -98,26 +97,26 @@ public class PartitionMakeFollowerBenchmark {
|
|||
scheduler.startup();
|
||||
LogConfig logConfig = createLogConfig();
|
||||
|
||||
List<File> logDirs = Collections.singletonList(logDir);
|
||||
BrokerTopicStats brokerTopicStats = new BrokerTopicStats();
|
||||
LogDirFailureChannel logDirFailureChannel = Mockito.mock(LogDirFailureChannel.class);
|
||||
logManager = new LogManager(JavaConverters.asScalaIteratorConverter(logDirs.iterator()).asScala().toSeq(),
|
||||
JavaConverters.asScalaIteratorConverter(new ArrayList<File>().iterator()).asScala().toSeq(),
|
||||
new MockConfigRepository(),
|
||||
logConfig,
|
||||
new CleanerConfig(0, 0, 0, 0, 0, 0.0, 0, false, "MD5"),
|
||||
1,
|
||||
1000L,
|
||||
10000L,
|
||||
10000L,
|
||||
1000L,
|
||||
60000,
|
||||
ApiVersion.latestVersion(),
|
||||
scheduler,
|
||||
brokerTopicStats,
|
||||
logDirFailureChannel,
|
||||
Time.SYSTEM,
|
||||
true);
|
||||
logManager = new LogManagerBuilder().
|
||||
setLogDirs(Collections.singletonList(logDir)).
|
||||
setInitialOfflineDirs(Collections.emptyList()).
|
||||
setConfigRepository(new MockConfigRepository()).
|
||||
setInitialDefaultConfig(logConfig).
|
||||
setCleanerConfig(new CleanerConfig(0, 0, 0, 0, 0, 0.0, 0, false, "MD5")).
|
||||
setRecoveryThreadsPerDataDir(1).
|
||||
setFlushCheckMs(1000L).
|
||||
setFlushRecoveryOffsetCheckpointMs(10000L).
|
||||
setFlushStartOffsetCheckpointMs(10000L).
|
||||
setRetentionCheckMs(1000L).
|
||||
setMaxPidExpirationMs(60000).
|
||||
setInterBrokerProtocolVersion(ApiVersion.latestVersion()).
|
||||
setScheduler(scheduler).
|
||||
setBrokerTopicStats(brokerTopicStats).
|
||||
setLogDirFailureChannel(logDirFailureChannel).
|
||||
setTime(Time.SYSTEM).setKeepPartitionMetadataFile(true).
|
||||
build();
|
||||
|
||||
TopicPartition tp = new TopicPartition("topic", 0);
|
||||
topicId = OptionConverters.toScala(Optional.of(Uuid.randomUuid()));
|
||||
|
|
|
@ -31,6 +31,7 @@ import kafka.server.BrokerTopicStats;
|
|||
import kafka.server.LogDirFailureChannel;
|
||||
import kafka.server.LogOffsetMetadata;
|
||||
import kafka.server.MetadataCache;
|
||||
import kafka.server.builders.LogManagerBuilder;
|
||||
import kafka.server.checkpoints.OffsetCheckpoints;
|
||||
import kafka.server.metadata.MockConfigRepository;
|
||||
import kafka.utils.KafkaScheduler;
|
||||
|
@ -51,9 +52,6 @@ import org.openjdk.jmh.annotations.Setup;
|
|||
import org.openjdk.jmh.annotations.State;
|
||||
import org.openjdk.jmh.annotations.TearDown;
|
||||
import org.openjdk.jmh.annotations.Warmup;
|
||||
import scala.Option;
|
||||
import scala.collection.JavaConverters;
|
||||
import scala.compat.java8.OptionConverters;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.ArrayList;
|
||||
|
@ -63,6 +61,8 @@ import java.util.Optional;
|
|||
import java.util.Properties;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import scala.Option;
|
||||
import scala.compat.java8.OptionConverters;
|
||||
|
||||
@State(Scope.Benchmark)
|
||||
@Fork(value = 1)
|
||||
|
@ -85,24 +85,25 @@ public class UpdateFollowerFetchStateBenchmark {
|
|||
public void setUp() {
|
||||
scheduler.startup();
|
||||
LogConfig logConfig = createLogConfig();
|
||||
List<File> logDirs = Collections.singletonList(logDir);
|
||||
logManager = new LogManager(JavaConverters.asScalaIteratorConverter(logDirs.iterator()).asScala().toSeq(),
|
||||
JavaConverters.asScalaIteratorConverter(new ArrayList<File>().iterator()).asScala().toSeq(),
|
||||
new MockConfigRepository(),
|
||||
logConfig,
|
||||
new CleanerConfig(0, 0, 0, 0, 0, 0.0, 0, false, "MD5"),
|
||||
1,
|
||||
1000L,
|
||||
10000L,
|
||||
10000L,
|
||||
1000L,
|
||||
60000,
|
||||
ApiVersion.latestVersion(),
|
||||
scheduler,
|
||||
brokerTopicStats,
|
||||
logDirFailureChannel,
|
||||
Time.SYSTEM,
|
||||
true);
|
||||
logManager = new LogManagerBuilder().
|
||||
setLogDirs(Collections.singletonList(logDir)).
|
||||
setInitialOfflineDirs(Collections.emptyList()).
|
||||
setConfigRepository(new MockConfigRepository()).
|
||||
setInitialDefaultConfig(logConfig).
|
||||
setCleanerConfig(new CleanerConfig(0, 0, 0, 0, 0, 0.0, 0, false, "MD5")).
|
||||
setRecoveryThreadsPerDataDir(1).
|
||||
setFlushCheckMs(1000L).
|
||||
setFlushRecoveryOffsetCheckpointMs(10000L).
|
||||
setFlushStartOffsetCheckpointMs(10000L).
|
||||
setRetentionCheckMs(1000L).
|
||||
setMaxPidExpirationMs(60000).
|
||||
setInterBrokerProtocolVersion(ApiVersion.latestVersion()).
|
||||
setScheduler(scheduler).
|
||||
setBrokerTopicStats(brokerTopicStats).
|
||||
setLogDirFailureChannel(logDirFailureChannel).
|
||||
setTime(Time.SYSTEM).
|
||||
setKeepPartitionMetadataFile(true).
|
||||
build();
|
||||
OffsetCheckpoints offsetCheckpoints = Mockito.mock(OffsetCheckpoints.class);
|
||||
Mockito.when(offsetCheckpoints.fetch(logDir.getAbsolutePath(), topicPartition)).thenReturn(Option.apply(0L));
|
||||
DelayedOperations delayedOperations = new DelayedOperationsMock();
|
||||
|
|
|
@ -28,6 +28,7 @@ import kafka.server.LogDirFailureChannel;
|
|||
import kafka.server.MetadataCache;
|
||||
import kafka.server.QuotaFactory;
|
||||
import kafka.server.ReplicaManager;
|
||||
import kafka.server.builders.ReplicaManagerBuilder;
|
||||
import kafka.server.checkpoints.OffsetCheckpoints;
|
||||
import kafka.server.metadata.MockConfigRepository;
|
||||
import kafka.utils.KafkaScheduler;
|
||||
|
@ -55,7 +56,6 @@ import java.io.File;
|
|||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import scala.collection.JavaConverters;
|
||||
|
@ -118,20 +118,18 @@ public class CheckpointBench {
|
|||
this.time, "");
|
||||
|
||||
this.alterIsrManager = TestUtils.createAlterIsrManager();
|
||||
this.replicaManager = new ReplicaManager(
|
||||
this.brokerProperties,
|
||||
this.metrics,
|
||||
this.time,
|
||||
Option.empty(),
|
||||
this.scheduler,
|
||||
this.logManager,
|
||||
new AtomicBoolean(false),
|
||||
this.quotaManagers,
|
||||
brokerTopicStats,
|
||||
metadataCache,
|
||||
this.failureChannel,
|
||||
alterIsrManager,
|
||||
Option.empty());
|
||||
this.replicaManager = new ReplicaManagerBuilder().
|
||||
setConfig(brokerProperties).
|
||||
setMetrics(metrics).
|
||||
setTime(time).
|
||||
setScheduler(scheduler).
|
||||
setLogManager(logManager).
|
||||
setQuotaManagers(quotaManagers).
|
||||
setBrokerTopicStats(brokerTopicStats).
|
||||
setMetadataCache(metadataCache).
|
||||
setLogDirFailureChannel(failureChannel).
|
||||
setAlterIsrManager(alterIsrManager).
|
||||
build();
|
||||
replicaManager.startup();
|
||||
|
||||
List<TopicPartition> topicPartitions = new ArrayList<>();
|
||||
|
|
|
@ -16,8 +16,6 @@
|
|||
*/
|
||||
package org.apache.kafka.jmh.server;
|
||||
|
||||
import java.util.Properties;
|
||||
|
||||
import kafka.api.ApiVersion;
|
||||
import kafka.cluster.Partition;
|
||||
import kafka.log.CleanerConfig;
|
||||
|
@ -28,10 +26,11 @@ import kafka.server.AlterIsrManager;
|
|||
import kafka.server.BrokerTopicStats;
|
||||
import kafka.server.KafkaConfig;
|
||||
import kafka.server.LogDirFailureChannel;
|
||||
import kafka.server.MetadataCache;
|
||||
import kafka.server.QuotaFactory;
|
||||
import kafka.server.ReplicaManager;
|
||||
import kafka.server.ZkMetadataCache;
|
||||
import kafka.server.builders.LogManagerBuilder;
|
||||
import kafka.server.builders.ReplicaManagerBuilder;
|
||||
import kafka.server.checkpoints.OffsetCheckpoints;
|
||||
import kafka.server.metadata.ConfigRepository;
|
||||
import kafka.server.metadata.MockConfigRepository;
|
||||
|
@ -62,13 +61,13 @@ import org.openjdk.jmh.annotations.Warmup;
|
|||
|
||||
import java.io.File;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Properties;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import scala.collection.JavaConverters;
|
||||
import scala.Option;
|
||||
import scala.collection.JavaConverters;
|
||||
|
||||
@Warmup(iterations = 5)
|
||||
@Measurement(iterations = 5)
|
||||
|
@ -92,6 +91,7 @@ public class PartitionCreationBench {
|
|||
|
||||
private ReplicaManager replicaManager;
|
||||
private QuotaFactory.QuotaManagers quotaManagers;
|
||||
private KafkaZkClient zkClient;
|
||||
private LogDirFailureChannel failureChannel;
|
||||
private LogManager logManager;
|
||||
private AlterIsrManager alterIsrManager;
|
||||
|
@ -122,52 +122,47 @@ public class PartitionCreationBench {
|
|||
Double.MAX_VALUE, 15 * 1000, true, "MD5");
|
||||
|
||||
ConfigRepository configRepository = new MockConfigRepository();
|
||||
this.logManager = new LogManager(JavaConverters.asScalaIteratorConverter(files.iterator()).asScala().toSeq(),
|
||||
JavaConverters.asScalaIteratorConverter(new ArrayList<File>().iterator()).asScala().toSeq(),
|
||||
configRepository,
|
||||
createLogConfig(),
|
||||
cleanerConfig,
|
||||
1,
|
||||
1000L,
|
||||
10000L,
|
||||
10000L,
|
||||
1000L,
|
||||
60000,
|
||||
ApiVersion.latestVersion(),
|
||||
scheduler,
|
||||
brokerTopicStats,
|
||||
failureChannel,
|
||||
Time.SYSTEM,
|
||||
true);
|
||||
this.logManager = new LogManagerBuilder().
|
||||
setLogDirs(files).
|
||||
setInitialOfflineDirs(Collections.emptyList()).
|
||||
setConfigRepository(configRepository).
|
||||
setInitialDefaultConfig(createLogConfig()).
|
||||
setCleanerConfig(cleanerConfig).
|
||||
setRecoveryThreadsPerDataDir(1).
|
||||
setFlushCheckMs(1000L).
|
||||
setFlushRecoveryOffsetCheckpointMs(10000L).
|
||||
setFlushStartOffsetCheckpointMs(10000L).
|
||||
setRetentionCheckMs(1000L).
|
||||
setMaxPidExpirationMs(60000).
|
||||
setInterBrokerProtocolVersion(ApiVersion.latestVersion()).
|
||||
setScheduler(scheduler).
|
||||
setBrokerTopicStats(brokerTopicStats).
|
||||
setLogDirFailureChannel(failureChannel).
|
||||
setTime(Time.SYSTEM).
|
||||
setKeepPartitionMetadataFile(true).
|
||||
build();
|
||||
scheduler.startup();
|
||||
final MetadataCache metadataCache =
|
||||
new ZkMetadataCache(this.brokerProperties.brokerId());
|
||||
this.quotaManagers =
|
||||
QuotaFactory.instantiate(this.brokerProperties,
|
||||
this.metrics,
|
||||
this.time, "");
|
||||
|
||||
KafkaZkClient zkClient = new KafkaZkClient(null, false, Time.SYSTEM) {
|
||||
this.quotaManagers = QuotaFactory.instantiate(this.brokerProperties, this.metrics, this.time, "");
|
||||
this.zkClient = new KafkaZkClient(null, false, Time.SYSTEM) {
|
||||
@Override
|
||||
public Properties getEntityConfigs(String rootEntityType, String sanitizedEntityName) {
|
||||
return new Properties();
|
||||
}
|
||||
};
|
||||
this.alterIsrManager = TestUtils.createAlterIsrManager();
|
||||
this.replicaManager = new ReplicaManager(
|
||||
this.brokerProperties,
|
||||
this.metrics,
|
||||
this.time,
|
||||
Option.apply(zkClient),
|
||||
this.scheduler,
|
||||
this.logManager,
|
||||
new AtomicBoolean(false),
|
||||
this.quotaManagers,
|
||||
brokerTopicStats,
|
||||
metadataCache,
|
||||
this.failureChannel,
|
||||
alterIsrManager,
|
||||
Option.empty());
|
||||
this.replicaManager = new ReplicaManagerBuilder().
|
||||
setConfig(brokerProperties).
|
||||
setMetrics(metrics).
|
||||
setTime(time).
|
||||
setZkClient(zkClient).
|
||||
setScheduler(scheduler).
|
||||
setLogManager(logManager).
|
||||
setQuotaManagers(quotaManagers).
|
||||
setBrokerTopicStats(brokerTopicStats).
|
||||
setMetadataCache(new ZkMetadataCache(this.brokerProperties.brokerId())).
|
||||
setLogDirFailureChannel(failureChannel).
|
||||
setAlterIsrManager(alterIsrManager).
|
||||
build();
|
||||
replicaManager.startup();
|
||||
replicaManager.checkpointHighWatermarks();
|
||||
}
|
||||
|
@ -182,6 +177,7 @@ public class PartitionCreationBench {
|
|||
for (File dir : JavaConverters.asJavaCollection(logManager.liveLogDirs())) {
|
||||
Utils.delete(dir);
|
||||
}
|
||||
this.zkClient.close();
|
||||
}
|
||||
|
||||
private static LogConfig createLogConfig() {
|
||||
|
|
Loading…
Reference in New Issue