You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2021/09/15 14:33:09 UTC

[GitHub] [kafka] mumrah commented on a change in pull request #11320: MINOR: Make ReplicaManager, LogManager, KafkaApis easier to construct

mumrah commented on a change in pull request #11320:
URL: https://github.com/apache/kafka/pull/11320#discussion_r709241900



##########
File path: core/src/test/scala/unit/kafka/log/LogLoaderTest.scala
##########
@@ -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,

Review comment:
       nit: we should make the no-arg constructor calls consistent wrt parens

##########
File path: core/src/main/java/kafka/server/builders/KafkaApisBuilder.java
##########
@@ -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,

Review comment:
       Does this indentation follow our style guide? I can never remember what all we allow for continuations...

##########
File path: core/src/main/scala/kafka/server/ReplicaManager.scala
##########
@@ -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 {
-
-  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 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,

Review comment:
       nit: dangling comma




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org