[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-19 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r579565540



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
##
@@ -0,0 +1,900 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.ElectionType;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.BrokerIdNotRegisteredException;
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.errors.InvalidRequestException;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment;
+import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData.TopicPartitions;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.PartitionChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.controller.BrokersToIsrs.TopicPartition;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistration;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Random;
+
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET;
+import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC;
+
+
+/**
+ * The ReplicationControlManager is the part of the controller which deals 
with topics
+ * and partitions. It is responsible for managing the in-sync replica set and 
leader
+ * of each partition, as well as administrative tasks like creating or 
deleting topics.
+ */
+public class ReplicationControlManager {
+static class TopicControlInfo {
+private final Uuid id;
+private final TimelineHashMap parts;
+
+TopicControlInfo(SnapshotRegistry snapshotRegistry, Uuid id) {
+this.id = id;
+this.parts = new TimelineHashMap<>(snapshotRegistry, 0);
+}
+}
+
+static class PartitionControlInfo {
+private final int[] replicas;
+private final int[] isr;
+private final int[] removingReplicas;
+private final int[] addingReplicas;
+private final int leader;
+private final int leaderEpoch;
+private final int partitionEpoch;
+
+ 

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-19 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r579564654



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
##
@@ -0,0 +1,920 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.ApiException;
+import org.apache.kafka.common.errors.NotControllerException;
+import org.apache.kafka.common.errors.UnknownServerException;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.ConfigRecord;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.IsrChangeRecord;
+import org.apache.kafka.common.metadata.MetadataRecordType;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.QuotaRecord;
+import org.apache.kafka.common.metadata.RegisterBrokerRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.protocol.ApiMessage;
+import org.apache.kafka.common.quota.ClientQuotaAlteration;
+import org.apache.kafka.common.quota.ClientQuotaEntity;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureManager;
+import org.apache.kafka.metadata.VersionRange;
+import org.apache.kafka.metalog.MetaLogLeader;
+import org.apache.kafka.metalog.MetaLogListener;
+import org.apache.kafka.metalog.MetaLogManager;
+import org.apache.kafka.queue.EventQueue.EarliestDeadlineFunction;
+import org.apache.kafka.queue.EventQueue;
+import org.apache.kafka.queue.KafkaEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+
+public final class QuorumController implements Controller {
+/**
+ * A builder class which creates the QuorumController.
+ */
+static public class Builder {
+private final int nodeId;
+private Time time = Time.SYSTEM;
+private String threadNamePrefix = null;
+private LogContext logContext = null;
+private Map configDefs = 
Collections.emptyMap();
+private MetaLogManager logManager = null;
+private Map supportedFeatures = 
Collections.emptyMap();
+private short defaultReplicationFactor = 3;
+private int defaultNumPartitions = 1;
+private ReplicaPlacementPolicy replicaPlacementPolicy =
+new SimpleReplicaPlacementPolicy(new Random());
+private long sessionTimeoutNs = TimeUnit.NANOSECONDS.convert(18, 
TimeUnit.SECONDS);
+
+public Builder(int nodeId) {
+this.nodeId = nodeId;
+}
+
+public B

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-19 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r579564579



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
##
@@ -0,0 +1,920 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.ApiException;
+import org.apache.kafka.common.errors.NotControllerException;
+import org.apache.kafka.common.errors.UnknownServerException;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.ConfigRecord;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.IsrChangeRecord;
+import org.apache.kafka.common.metadata.MetadataRecordType;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.QuotaRecord;
+import org.apache.kafka.common.metadata.RegisterBrokerRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.protocol.ApiMessage;
+import org.apache.kafka.common.quota.ClientQuotaAlteration;
+import org.apache.kafka.common.quota.ClientQuotaEntity;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureManager;
+import org.apache.kafka.metadata.VersionRange;
+import org.apache.kafka.metalog.MetaLogLeader;
+import org.apache.kafka.metalog.MetaLogListener;
+import org.apache.kafka.metalog.MetaLogManager;
+import org.apache.kafka.queue.EventQueue.EarliestDeadlineFunction;
+import org.apache.kafka.queue.EventQueue;
+import org.apache.kafka.queue.KafkaEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+
+public final class QuorumController implements Controller {
+/**
+ * A builder class which creates the QuorumController.
+ */
+static public class Builder {
+private final int nodeId;
+private Time time = Time.SYSTEM;
+private String threadNamePrefix = null;
+private LogContext logContext = null;
+private Map configDefs = 
Collections.emptyMap();
+private MetaLogManager logManager = null;
+private Map supportedFeatures = 
Collections.emptyMap();
+private short defaultReplicationFactor = 3;
+private int defaultNumPartitions = 1;
+private ReplicaPlacementPolicy replicaPlacementPolicy =
+new SimpleReplicaPlacementPolicy(new Random());
+private long sessionTimeoutNs = TimeUnit.NANOSECONDS.convert(18, 
TimeUnit.SECONDS);
+
+public Builder(int nodeId) {
+this.nodeId = nodeId;
+}
+
+public B

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-19 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r579564268



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java
##
@@ -0,0 +1,136 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import org.apache.kafka.common.metadata.FeatureLevelRecord;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.FeatureMap;
+import org.apache.kafka.metadata.FeatureMapAndEpoch;
+import org.apache.kafka.metadata.VersionRange;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.apache.kafka.timeline.TimelineHashSet;
+
+
+public class FeatureControlManager {
+/**
+ * The features supported by this controller's software.
+ */
+private final Map supportedFeatures;
+
+/**
+ * Maps feature names to finalized version ranges.
+ */
+private final TimelineHashMap finalizedVersions;
+
+/**
+ * The latest feature epoch.
+ */
+private final TimelineHashSet epoch;
+
+FeatureControlManager(Map supportedFeatures,
+  SnapshotRegistry snapshotRegistry) {
+this.supportedFeatures = supportedFeatures;
+this.finalizedVersions = new TimelineHashMap<>(snapshotRegistry, 0);
+this.epoch = new TimelineHashSet<>(snapshotRegistry, 0);
+}
+
+ControllerResult> updateFeatures(
+Map updates, Set downgradeables,
+Map> brokerFeatures) {
+TreeMap results = new TreeMap<>();
+List records = new ArrayList<>();
+for (Entry entry : updates.entrySet()) {
+results.put(entry.getKey(), updateFeature(entry.getKey(), 
entry.getValue(),
+downgradeables.contains(entry.getKey()), brokerFeatures, 
records));
+}
+return new ControllerResult<>(records, results);
+}
+
+private ApiError updateFeature(String featureName,
+   VersionRange newRange,
+   boolean downgradeable,
+   Map> 
brokerFeatures,
+   List records) {
+if (newRange.min() <= 0) {
+return new ApiError(Errors.INVALID_UPDATE_VERSION,
+"The lower value for the new range cannot be less than 1.");
+}
+if (newRange.max() <= 0) {
+return new ApiError(Errors.INVALID_UPDATE_VERSION,
+"The upper value for the new range cannot be less than 1.");
+}
+VersionRange localRange = supportedFeatures.get(featureName);

Review comment:
   let's revisit this after 2.8





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.

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




[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-19 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r579547910



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
##
@@ -0,0 +1,894 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.ElectionType;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.BrokerIdNotRegisteredException;
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.errors.InvalidRequestException;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment;
+import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData.TopicPartitions;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.PartitionChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.controller.BrokersToIsrs.TopicPartition;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistration;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Random;
+
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET;
+import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC;
+
+
+/**
+ * The ReplicationControlManager is the part of the controller which deals 
with topics
+ * and partitions. It is responsible for managing the in-sync replica set and 
leader
+ * of each partition, as well as administrative tasks like creating or 
deleting topics.
+ */
+public class ReplicationControlManager {
+static class TopicControlInfo {
+private final Uuid id;
+private final TimelineHashMap parts;
+
+TopicControlInfo(SnapshotRegistry snapshotRegistry, Uuid id) {
+this.id = id;
+this.parts = new TimelineHashMap<>(snapshotRegistry, 0);
+}
+}
+
+static class PartitionControlInfo {
+private final int[] replicas;
+private final int[] isr;
+private final int[] removingReplicas;
+private final int[] addingReplicas;
+private final int leader;
+private final int leaderEpoch;
+private fin

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-19 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r579547825



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
##
@@ -0,0 +1,894 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.ElectionType;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.BrokerIdNotRegisteredException;
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.errors.InvalidRequestException;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment;
+import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData.TopicPartitions;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.PartitionChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.controller.BrokersToIsrs.TopicPartition;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistration;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Random;
+
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET;
+import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC;
+
+
+/**
+ * The ReplicationControlManager is the part of the controller which deals 
with topics
+ * and partitions. It is responsible for managing the in-sync replica set and 
leader
+ * of each partition, as well as administrative tasks like creating or 
deleting topics.
+ */
+public class ReplicationControlManager {
+static class TopicControlInfo {
+private final Uuid id;
+private final TimelineHashMap parts;
+
+TopicControlInfo(SnapshotRegistry snapshotRegistry, Uuid id) {
+this.id = id;
+this.parts = new TimelineHashMap<>(snapshotRegistry, 0);
+}
+}
+
+static class PartitionControlInfo {
+private final int[] replicas;
+private final int[] isr;
+private final int[] removingReplicas;
+private final int[] addingReplicas;
+private final int leader;
+private final int leaderEpoch;
+private fin

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-19 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r579547731



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
##
@@ -0,0 +1,894 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.ElectionType;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.BrokerIdNotRegisteredException;
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.errors.InvalidRequestException;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment;
+import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData.TopicPartitions;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.PartitionChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.controller.BrokersToIsrs.TopicPartition;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistration;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Random;
+
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET;
+import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC;
+
+
+/**
+ * The ReplicationControlManager is the part of the controller which deals 
with topics
+ * and partitions. It is responsible for managing the in-sync replica set and 
leader
+ * of each partition, as well as administrative tasks like creating or 
deleting topics.
+ */
+public class ReplicationControlManager {
+static class TopicControlInfo {
+private final Uuid id;
+private final TimelineHashMap parts;
+
+TopicControlInfo(SnapshotRegistry snapshotRegistry, Uuid id) {
+this.id = id;
+this.parts = new TimelineHashMap<>(snapshotRegistry, 0);
+}
+}
+
+static class PartitionControlInfo {
+private final int[] replicas;
+private final int[] isr;
+private final int[] removingReplicas;
+private final int[] addingReplicas;
+private final int leader;
+private final int leaderEpoch;
+private fin

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-19 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r579547404



##
File path: metadata/src/main/resources/common/metadata/PartitionRecord.json
##
@@ -34,6 +34,8 @@
 { "name": "Leader", "type": "int32", "versions": "0+", "default": "-1",
   "about": "The lead replica, or -1 if there is no leader." },
 { "name": "LeaderEpoch", "type": "int32", "versions": "0+", "default": 
"-1",
-  "about": "An epoch that gets incremented each time we change the ISR." }
+  "about": "An epoch that gets incremented each time we change the ISR." },
+{ "name": "PartitionEpoch", "type": "int32", "versions": "0+", "default": 
"-1",

Review comment:
   filed KAFKA-12349





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.

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




[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-19 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r579547098



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
##
@@ -0,0 +1,908 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.ElectionType;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.BrokerIdNotRegisteredException;
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.errors.InvalidRequestException;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment;
+import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData.TopicPartitions;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.PartitionChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.controller.BrokersToIsrs.TopicPartition;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistration;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Random;
+
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET;
+import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC;
+
+
+/**
+ * The ReplicationControlManager is the part of the controller which deals 
with topics
+ * and partitions. It is responsible for managing the in-sync replica set and 
leader
+ * of each partition, as well as administrative tasks like creating or 
deleting topics.
+ */
+public class ReplicationControlManager {
+/**
+ * A special value used to represent the leader for a partition with no 
leader. 
+ */
+public static final int NO_LEADER = -1;
+
+/**
+ * A special value used to represent a PartitionChangeRecord that does not 
change the
+ * partition leader.
+ */
+public static final int NO_LEADER_CHANGE = -2;
+
+static class TopicControlInfo {
+private final Uuid id;
+private final TimelineHashMap parts;
+
+TopicControlInfo(SnapshotRegistry snapshotRegistry, Uuid id) {
+this.id = id;
+this.parts = new TimelineHashMap<>(s

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-19 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r579522827



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
##
@@ -0,0 +1,894 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.ElectionType;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.BrokerIdNotRegisteredException;
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.errors.InvalidRequestException;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment;
+import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData.TopicPartitions;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.PartitionChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.controller.BrokersToIsrs.TopicPartition;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistration;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Random;
+
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET;
+import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC;
+
+
+/**
+ * The ReplicationControlManager is the part of the controller which deals 
with topics
+ * and partitions. It is responsible for managing the in-sync replica set and 
leader
+ * of each partition, as well as administrative tasks like creating or 
deleting topics.
+ */
+public class ReplicationControlManager {
+static class TopicControlInfo {
+private final Uuid id;
+private final TimelineHashMap parts;
+
+TopicControlInfo(SnapshotRegistry snapshotRegistry, Uuid id) {
+this.id = id;
+this.parts = new TimelineHashMap<>(snapshotRegistry, 0);
+}
+}
+
+static class PartitionControlInfo {
+private final int[] replicas;
+private final int[] isr;
+private final int[] removingReplicas;
+private final int[] addingReplicas;
+private final int leader;
+private final int leaderEpoch;
+private fin

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-19 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r579522201



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
##
@@ -0,0 +1,894 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.ElectionType;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.BrokerIdNotRegisteredException;
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.errors.InvalidRequestException;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment;
+import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData.TopicPartitions;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.PartitionChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.controller.BrokersToIsrs.TopicPartition;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistration;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Random;
+
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET;
+import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC;
+
+
+/**
+ * The ReplicationControlManager is the part of the controller which deals 
with topics
+ * and partitions. It is responsible for managing the in-sync replica set and 
leader
+ * of each partition, as well as administrative tasks like creating or 
deleting topics.
+ */
+public class ReplicationControlManager {
+static class TopicControlInfo {
+private final Uuid id;
+private final TimelineHashMap parts;
+
+TopicControlInfo(SnapshotRegistry snapshotRegistry, Uuid id) {
+this.id = id;
+this.parts = new TimelineHashMap<>(snapshotRegistry, 0);
+}
+}
+
+static class PartitionControlInfo {
+private final int[] replicas;
+private final int[] isr;
+private final int[] removingReplicas;
+private final int[] addingReplicas;
+private final int leader;
+private final int leaderEpoch;
+private fin

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-19 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r579464755



##
File path: core/src/main/scala/kafka/server/BrokerServer.scala
##
@@ -397,6 +397,9 @@ class BrokerServer(
   info("shutting down")
 
   if (config.controlledShutdownEnable) {
+// Shut down the broker metadata listener, so that we don't get added 
to any
+// more ISRs.
+brokerMetadataListener.beginShutdown()

Review comment:
   It does need to be because there are some paths through the code that 
don't go through here.  In general calling `beginShutdown` or `close` multiple 
times is harmless-- only the first time has an effect.





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.

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




[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-19 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r579463797



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
##
@@ -0,0 +1,894 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.ElectionType;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.BrokerIdNotRegisteredException;
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.errors.InvalidRequestException;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment;
+import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData.TopicPartitions;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.PartitionChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.controller.BrokersToIsrs.TopicPartition;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistration;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Random;
+
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET;
+import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC;
+
+
+/**
+ * The ReplicationControlManager is the part of the controller which deals 
with topics
+ * and partitions. It is responsible for managing the in-sync replica set and 
leader
+ * of each partition, as well as administrative tasks like creating or 
deleting topics.
+ */
+public class ReplicationControlManager {
+static class TopicControlInfo {
+private final Uuid id;
+private final TimelineHashMap parts;
+
+TopicControlInfo(SnapshotRegistry snapshotRegistry, Uuid id) {
+this.id = id;
+this.parts = new TimelineHashMap<>(snapshotRegistry, 0);
+}
+}
+
+static class PartitionControlInfo {
+private final int[] replicas;
+private final int[] isr;
+private final int[] removingReplicas;
+private final int[] addingReplicas;
+private final int leader;
+private final int leaderEpoch;
+private fin

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-19 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r579463797



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
##
@@ -0,0 +1,894 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.ElectionType;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.BrokerIdNotRegisteredException;
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.errors.InvalidRequestException;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment;
+import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData.TopicPartitions;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.PartitionChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.controller.BrokersToIsrs.TopicPartition;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistration;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Random;
+
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET;
+import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC;
+
+
+/**
+ * The ReplicationControlManager is the part of the controller which deals 
with topics
+ * and partitions. It is responsible for managing the in-sync replica set and 
leader
+ * of each partition, as well as administrative tasks like creating or 
deleting topics.
+ */
+public class ReplicationControlManager {
+static class TopicControlInfo {
+private final Uuid id;
+private final TimelineHashMap parts;
+
+TopicControlInfo(SnapshotRegistry snapshotRegistry, Uuid id) {
+this.id = id;
+this.parts = new TimelineHashMap<>(snapshotRegistry, 0);
+}
+}
+
+static class PartitionControlInfo {
+private final int[] replicas;
+private final int[] isr;
+private final int[] removingReplicas;
+private final int[] addingReplicas;
+private final int leader;
+private final int leaderEpoch;
+private fin

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-19 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r579462462



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
##
@@ -0,0 +1,894 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.ElectionType;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.BrokerIdNotRegisteredException;
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.errors.InvalidRequestException;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment;
+import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData.TopicPartitions;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.PartitionChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.controller.BrokersToIsrs.TopicPartition;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistration;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Random;
+
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET;
+import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC;
+
+
+/**
+ * The ReplicationControlManager is the part of the controller which deals 
with topics
+ * and partitions. It is responsible for managing the in-sync replica set and 
leader
+ * of each partition, as well as administrative tasks like creating or 
deleting topics.
+ */
+public class ReplicationControlManager {
+static class TopicControlInfo {
+private final Uuid id;
+private final TimelineHashMap parts;
+
+TopicControlInfo(SnapshotRegistry snapshotRegistry, Uuid id) {
+this.id = id;
+this.parts = new TimelineHashMap<>(snapshotRegistry, 0);
+}
+}
+
+static class PartitionControlInfo {
+private final int[] replicas;
+private final int[] isr;
+private final int[] removingReplicas;
+private final int[] addingReplicas;
+private final int leader;
+private final int leaderEpoch;
+private fin

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-19 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r579458466



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
##
@@ -0,0 +1,894 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.ElectionType;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.BrokerIdNotRegisteredException;
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.errors.InvalidRequestException;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment;
+import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData.TopicPartitions;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.PartitionChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.controller.BrokersToIsrs.TopicPartition;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistration;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Random;
+
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET;
+import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC;
+
+
+/**
+ * The ReplicationControlManager is the part of the controller which deals 
with topics
+ * and partitions. It is responsible for managing the in-sync replica set and 
leader
+ * of each partition, as well as administrative tasks like creating or 
deleting topics.
+ */
+public class ReplicationControlManager {
+static class TopicControlInfo {
+private final Uuid id;
+private final TimelineHashMap parts;
+
+TopicControlInfo(SnapshotRegistry snapshotRegistry, Uuid id) {
+this.id = id;
+this.parts = new TimelineHashMap<>(snapshotRegistry, 0);
+}
+}
+
+static class PartitionControlInfo {
+private final int[] replicas;
+private final int[] isr;
+private final int[] removingReplicas;
+private final int[] addingReplicas;
+private final int leader;
+private final int leaderEpoch;
+private fin

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-19 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r579378306



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
##
@@ -0,0 +1,875 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.ElectionType;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.BrokerIdNotRegisteredException;
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.errors.InvalidRequestException;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment;
+import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData.TopicPartitions;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.PartitionChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.controller.BrokersToIsrs.TopicPartition;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistration;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Random;
+
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET;
+import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC;
+
+
+/**
+ * The ReplicationControlManager is the part of the controller which deals 
with topics
+ * and partitions. It is responsible for managing the in-sync replica set and 
leader
+ * of each partition, as well as administrative tasks like creating or 
deleting topics.
+ */
+public class ReplicationControlManager {
+static class TopicControlInfo {
+private final Uuid id;
+private final TimelineHashMap parts;
+
+TopicControlInfo(SnapshotRegistry snapshotRegistry, Uuid id) {
+this.id = id;
+this.parts = new TimelineHashMap<>(snapshotRegistry, 0);
+}
+}
+
+static class PartitionControlInfo {
+private final int[] replicas;
+private final int[] isr;
+private final int[] removingReplicas;
+private final int[] addingReplicas;
+private final int leader;
+private final int leaderEpoch;
+private final int partitionEpoch;
+
+ 

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-19 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r579374802



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
##
@@ -0,0 +1,875 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.ElectionType;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.BrokerIdNotRegisteredException;
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.errors.InvalidRequestException;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment;
+import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData.TopicPartitions;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.PartitionChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.controller.BrokersToIsrs.TopicPartition;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistration;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Random;
+
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET;
+import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC;
+
+
+/**
+ * The ReplicationControlManager is the part of the controller which deals 
with topics
+ * and partitions. It is responsible for managing the in-sync replica set and 
leader
+ * of each partition, as well as administrative tasks like creating or 
deleting topics.
+ */
+public class ReplicationControlManager {
+static class TopicControlInfo {
+private final Uuid id;
+private final TimelineHashMap parts;
+
+TopicControlInfo(SnapshotRegistry snapshotRegistry, Uuid id) {
+this.id = id;
+this.parts = new TimelineHashMap<>(snapshotRegistry, 0);
+}
+}
+
+static class PartitionControlInfo {
+private final int[] replicas;
+private final int[] isr;
+private final int[] removingReplicas;
+private final int[] addingReplicas;
+private final int leader;
+private final int leaderEpoch;
+private final int partitionEpoch;
+
+ 

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-18 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r578857828



##
File path: 
metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java
##
@@ -0,0 +1,179 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData.Listener;
+import 
org.apache.kafka.common.message.BrokerRegistrationRequestData.ListenerCollection;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData;
+import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.controller.BrokersToIsrs.TopicPartition;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metalog.LocalLogManagerTestEnv;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET;
+import static 
org.apache.kafka.controller.ConfigurationControlManagerTest.BROKER0;
+import static 
org.apache.kafka.controller.ConfigurationControlManagerTest.CONFIGS;
+import static 
org.apache.kafka.controller.ConfigurationControlManagerTest.entry;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+
+@Timeout(value = 40)
+public class QuorumControllerTest {
+private static final Logger log =
+LoggerFactory.getLogger(QuorumControllerTest.class);
+
+/**
+ * Test creating a new QuorumController and closing it.
+ */
+@Test
+public void testCreateAndClose() throws Throwable {
+try (LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv(1)) {
+try (QuorumControllerTestEnv controlEnv =
+ new QuorumControllerTestEnv(logEnv, __ -> { })) {
+}
+}
+}
+
+/**
+ * Test setting some configuration values and reading them back.
+ */
+@Test
+public void testConfigurationOperations() throws Throwable {
+try (LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv(1)) {
+try (QuorumControllerTestEnv controlEnv =
+ new QuorumControllerTestEnv(logEnv, b -> 
b.setConfigDefs(CONFIGS))) {
+testConfigurationOperations(controlEnv.activeController());
+}
+}
+}
+
+private void testConfigurationOperations(QuorumController controller) 
throws Throwable {
+assertEquals(Collections.singletonMap(BROKER0, ApiError.NONE),
+controller.incrementalAlterConfigs(Collections.singletonMap(
+BROKER0, Collections.singletonMap("baz", entry(SET, "123"))), 
true).get());
+assertEquals(Collections.singletonMap(BROKER0,
+new ResultOrError<>(Collections.emptyMap())),
+controller.describeConfigs(Collections.singletonMap(
+BROKER0, Collections.emptyList())).get());
+assertEquals(Collections.singletonMap(BROKER0, ApiError.NONE),
+controller.incrementalAlterConfigs(Collections.singletonMap(
+BROKER0, Collections.singletonMap("baz", entry(SET, "123"))), 
false).get());
+assertEquals(Collections.singletonMap(BROKER0, new 
ResultOrError<>(Collections.
+singletonMap("baz", "123"))),
+controller.describeConfigs(Collections.singletonMap(
+BROKER0, Collections.emptyList())).get

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-18 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r578850804



##
File path: 
metadata/src/test/java/org/apache/kafka/controller/MockControllerMetrics.java
##
@@ -0,0 +1,47 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+
+public final class MockControllerMetrics implements ControllerMetrics {

Review comment:
   It's used in unit tests





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.

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




[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-18 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r578850598



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java
##
@@ -0,0 +1,136 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import org.apache.kafka.common.metadata.FeatureLevelRecord;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.FeatureMap;
+import org.apache.kafka.metadata.FeatureMapAndEpoch;
+import org.apache.kafka.metadata.VersionRange;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.apache.kafka.timeline.TimelineHashSet;
+
+
+public class FeatureControlManager {
+/**
+ * The features supported by this controller's software.
+ */
+private final Map supportedFeatures;
+
+/**
+ * Maps feature names to finalized version ranges.
+ */
+private final TimelineHashMap finalizedVersions;
+
+/**
+ * The latest feature epoch.
+ */
+private final TimelineHashSet epoch;
+
+FeatureControlManager(Map supportedFeatures,
+  SnapshotRegistry snapshotRegistry) {
+this.supportedFeatures = supportedFeatures;
+this.finalizedVersions = new TimelineHashMap<>(snapshotRegistry, 0);
+this.epoch = new TimelineHashSet<>(snapshotRegistry, 0);
+}
+
+ControllerResult> updateFeatures(
+Map updates, Set downgradeables,
+Map> brokerFeatures) {
+TreeMap results = new TreeMap<>();
+List records = new ArrayList<>();
+for (Entry entry : updates.entrySet()) {
+results.put(entry.getKey(), updateFeature(entry.getKey(), 
entry.getValue(),
+downgradeables.contains(entry.getKey()), brokerFeatures, 
records));
+}
+return new ControllerResult<>(records, results);
+}
+
+private ApiError updateFeature(String featureName,
+   VersionRange newRange,
+   boolean downgradeable,
+   Map> 
brokerFeatures,
+   List records) {
+if (newRange.min() <= 0) {
+return new ApiError(Errors.INVALID_UPDATE_VERSION,
+"The lower value for the new range cannot be less than 1.");
+}
+if (newRange.max() <= 0) {
+return new ApiError(Errors.INVALID_UPDATE_VERSION,
+"The upper value for the new range cannot be less than 1.");
+}
+VersionRange localRange = supportedFeatures.get(featureName);

Review comment:
   Hmm... right now, we don't have a good way of finding out what features 
the other controllers support.  Maybe we will have to think more about this 
when we support rolling upgrade in kip-500.





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.

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




[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-18 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r578817527



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java
##
@@ -0,0 +1,367 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.config.ConfigDef.ConfigKey;
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.common.config.ConfigResource.Type;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.metadata.ConfigRecord;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.APPEND;
+
+public class ConfigurationControlManager {
+private final Logger log;
+private final SnapshotRegistry snapshotRegistry;
+private final Map configDefs;
+private final TimelineHashMap> configData;
+
+ConfigurationControlManager(LogContext logContext,
+SnapshotRegistry snapshotRegistry,
+Map 
configDefs) {
+this.log = logContext.logger(ConfigurationControlManager.class);
+this.snapshotRegistry = snapshotRegistry;
+this.configDefs = configDefs;
+this.configData = new TimelineHashMap<>(snapshotRegistry, 0);
+}
+
+/**
+ * Determine the result of applying a batch of incremental configuration 
changes.  Note
+ * that this method does not change the contents of memory.  It just 
generates a
+ * result, that you can replay later if you wish using replay().
+ *
+ * Note that there can only be one result per ConfigResource.  So if you 
try to modify
+ * several keys and one modification fails, the whole ConfigKey fails and 
nothing gets
+ * changed.
+ *
+ * @param configChanges Maps each resource to a map from config keys to
+ *  operation data.
+ * @return  The result.
+ */
+ControllerResult> incrementalAlterConfigs(
+Map>> 
configChanges) {
+List outputRecords = new ArrayList<>();
+Map outputResults = new HashMap<>();
+for (Entry>> 
resourceEntry :
+configChanges.entrySet()) {
+incrementalAlterConfigResource(resourceEntry.getKey(),
+resourceEntry.getValue(),
+outputRecords,
+outputResults);
+}
+return new ControllerResult<>(outputRecords, outputResults);
+}
+
+private void incrementalAlterConfigResource(ConfigResource configResource,
+Map> keysToOps,
+List 
outputRecords,
+Map 
outputResults) {
+ApiError error = checkConfigResource(configResource);
+if (error.isFailure()) {
+outputResults.put(configResource, error);
+return;
+}
+List newRecords = new ArrayList<>();
+for (Entry> keysToOpsEntry : 
keysToOps.entrySet()) {
+String key = keysToOpsEntry.getKey();
+String currentValue = null;
+TimelineHashMap currentConfigs = 
configData.get(configResource);
+if (currentConfigs != null) {
+currentValue = currentConfigs.get(key);
+}
+String newValue = currentValue;
+Entry opTypeAndNewValue = 
keysToOpsEntry.getValue();
+OpType

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-18 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r578817611



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java
##
@@ -0,0 +1,597 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.UsableBroker;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Optional;
+import java.util.TreeSet;
+import java.util.function.Function;
+import java.util.function.Supplier;
+
+import static org.apache.kafka.controller.BrokerControlState.FENCED;
+import static 
org.apache.kafka.controller.BrokerControlState.CONTROLLED_SHUTDOWN;
+import static org.apache.kafka.controller.BrokerControlState.SHUTDOWN_NOW;
+import static org.apache.kafka.controller.BrokerControlState.UNFENCED;
+
+
+/**
+ * The BrokerHeartbeatManager manages all the soft state associated with 
broker heartbeats.
+ * Soft state is state which does not appear in the metadata log.  This state 
includes
+ * things like the last time each broker sent us a heartbeat, and whether the 
broker is
+ * trying to perform a controlled shutdown.
+ *
+ * Only the active controller has a BrokerHeartbeatManager, since only the 
active
+ * controller handles broker heartbeats.  Standby controllers will create a 
heartbeat
+ * manager as part of the process of activating.  This design minimizes the 
size of the
+ * metadata partition by excluding heartbeats from it.  However, it does mean 
that after
+ * a controller failover, we may take some extra time to fence brokers, since 
the new
+ * active controller does not know when the last heartbeats were received from 
each.
+ */
+public class BrokerHeartbeatManager {
+static class BrokerHeartbeatState {
+/**
+ * The broker ID.
+ */
+private final int id;
+
+/**
+ * The last time we received a heartbeat from this broker, in 
monotonic nanoseconds.
+ * When this field is updated, we also may have to update the broker's 
position in
+ * the unfenced list.
+ */
+long lastContactNs;
+
+/**
+ * The last metadata offset which this broker reported.  When this 
field is updated,
+ * we may also have to update the broker's position in the active set.
+ */
+long metadataOffset;
+
+/**
+ * The offset at which the broker should complete its controlled 
shutdown, or -1
+ * if the broker is not performing a controlled shutdown.  When this 
field is
+ * updated, we also have to update the broker's position in the 
shuttingDown set.
+ */
+private long controlledShutDownOffset;
+
+/**
+ * The previous entry in the unfenced list, or null if the broker is 
not in that list.
+ */
+private BrokerHeartbeatState prev;
+
+/**
+ * The next entry in the unfenced list, or null if the broker is not 
in that list.
+ */
+private BrokerHeartbeatState next;
+
+BrokerHeartbeatState(int id) {
+this.id = id;
+this.lastContactNs = 0;
+this.prev = null;
+this.next = null;
+this.metadataOffset = -1;
+this.controlledShutDownOffset = -1;
+}
+
+/**
+ * Returns the broker ID.
+ */
+int id() {
+return id;
+}
+
+/**
+ * Returns true only if the broker is fenced.
+ */
+boolean fenced() {
+return prev == null;
+}
+
+/**
+ * Returns true only if the broker is in controlled shutdown state.
+ */
+boolean shuttingDown() {
+return controlledShutDownOffset >= 0;
+   

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-18 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r578648887



##
File path: 
metadata/src/main/java/org/apache/kafka/metadata/BrokerHeartbeatReply.java
##
@@ -31,16 +31,23 @@
  */
 private final boolean isFenced;
 
+/**
+ * True if the heartbeat reply should tell the broker that it should shut 
down.

Review comment:
   fixed





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.

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




[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-18 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r578630655



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java
##
@@ -0,0 +1,585 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.UsableBroker;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Optional;
+import java.util.TreeSet;
+import java.util.function.Function;
+
+import static org.apache.kafka.controller.BrokerControlState.FENCED;
+import static 
org.apache.kafka.controller.BrokerControlState.CONTROLLED_SHUTDOWN;
+import static org.apache.kafka.controller.BrokerControlState.SHUTDOWN_NOW;
+import static org.apache.kafka.controller.BrokerControlState.UNFENCED;
+
+
+/**
+ * The BrokerHeartbeatManager manages all the soft state associated with 
broker heartbeats.
+ * Soft state is state which does not appear in the metadata log.  This state 
includes
+ * things like the last time each broker sent us a heartbeat, and whether the 
broker is
+ * trying to perform a controlled shutdown.
+ *
+ * Only the active controller has a BrokerHeartbeatManager, since only the 
active
+ * controller handles broker heartbeats.  Standby controllers will create a 
heartbeat
+ * manager as part of the process of activating.  This design minimizes the 
size of the
+ * metadata partition by excluding heartbeats from it.  However, it does mean 
that after
+ * a controller failover, we may take some extra time to fence brokers, since 
the new
+ * active controller does not know when the last heartbeats were received from 
each.
+ */
+public class BrokerHeartbeatManager {
+static class BrokerHeartbeatState {
+/**
+ * The broker ID.
+ */
+private final int id;
+
+/**
+ * The last time we received a heartbeat from this broker, in 
monotonic nanoseconds.
+ * When this field is updated, we also may have to update the broker's 
position in
+ * the unfenced list.
+ */
+long lastContactNs;
+
+/**
+ * The last metadata offset which this broker reported.  When this 
field is updated,
+ * we may also have to update the broker's position in the active set.
+ */
+long metadataOffset;
+
+/**
+ * The offset at which the broker should complete its controlled 
shutdown, or -1
+ * if the broker is not performing a controlled shutdown.  When this 
field is
+ * updated, we also have to update the broker's position in the 
shuttingDown set.
+ */
+private long controlledShutDownOffset;
+
+/**
+ * The previous entry in the unfenced list, or null if the broker is 
not in that list.
+ */
+private BrokerHeartbeatState prev;
+
+/**
+ * The next entry in the unfenced list, or null if the broker is not 
in that list.
+ */
+private BrokerHeartbeatState next;
+
+BrokerHeartbeatState(int id) {
+this.id = id;
+this.lastContactNs = 0;
+this.prev = null;
+this.next = null;
+this.metadataOffset = -1;
+this.controlledShutDownOffset = -1;
+}
+
+/**
+ * Returns the broker ID.
+ */
+int id() {
+return id;
+}
+
+/**
+ * Returns true only if the broker is fenced.
+ */
+boolean fenced() {
+return prev == null;
+}
+
+/**
+ * Returns true only if the broker is in controlled shutdown state.
+ */
+boolean shuttingDown() {
+return controlledShutDownOffset >= 0;
+}
+}
+
+static class Metadat

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-18 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r578629900



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
##
@@ -0,0 +1,900 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.ElectionType;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.BrokerIdNotRegisteredException;
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.errors.InvalidRequestException;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment;
+import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData.TopicPartitions;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.PartitionChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.controller.BrokersToIsrs.TopicPartition;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistration;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Random;
+
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET;
+import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC;
+
+
+/**
+ * The ReplicationControlManager is the part of the controller which deals 
with topics
+ * and partitions. It is responsible for managing the in-sync replica set and 
leader
+ * of each partition, as well as administrative tasks like creating or 
deleting topics.
+ */
+public class ReplicationControlManager {
+static class TopicControlInfo {
+private final Uuid id;
+private final TimelineHashMap parts;
+
+TopicControlInfo(SnapshotRegistry snapshotRegistry, Uuid id) {
+this.id = id;
+this.parts = new TimelineHashMap<>(snapshotRegistry, 0);
+}
+}
+
+static class PartitionControlInfo {
+private final int[] replicas;
+private final int[] isr;
+private final int[] removingReplicas;
+private final int[] addingReplicas;
+private final int leader;
+private final int leaderEpoch;
+private final int partitionEpoch;
+
+ 

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-18 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r578628736



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java
##
@@ -0,0 +1,359 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.common.Endpoint;
+import org.apache.kafka.common.errors.DuplicateBrokerRegistrationException;
+import org.apache.kafka.common.errors.StaleBrokerEpochException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.RegisterBrokerRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerRegistration;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureMapAndEpoch;
+import org.apache.kafka.metadata.VersionRange;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+
+
+public class ClusterControlManager {
+class ReadyBrokersFuture {
+private final CompletableFuture future;
+private final int minBrokers;
+
+ReadyBrokersFuture(CompletableFuture future, int minBrokers) {
+this.future = future;
+this.minBrokers = minBrokers;
+}
+
+boolean check() {
+int numUnfenced = 0;
+for (BrokerRegistration registration : 
brokerRegistrations.values()) {
+if (!registration.fenced()) {
+numUnfenced++;
+}
+if (numUnfenced >= minBrokers) {
+return true;
+}
+}
+return false;
+}
+}
+
+/**
+ * The SLF4J log context.
+ */
+private final LogContext logContext;
+
+/**
+ * The SLF4J log object.
+ */
+private final Logger log;
+
+/**
+ * The Kafka clock object to use.
+ */
+private final Time time;
+
+/**
+ * How long sessions should last, in nanoseconds.
+ */
+private final long sessionTimeoutNs;
+
+/**
+ * The replica placement policy to use.
+ */
+private final ReplicaPlacementPolicy placementPolicy;
+
+/**
+ * Maps broker IDs to broker registrations.
+ */
+private final TimelineHashMap 
brokerRegistrations;
+
+/**
+ * The broker heartbeat manager, or null if this controller is on standby.
+ */
+private BrokerHeartbeatManager heartbeatManager;
+
+/**
+ * A future which is completed as soon as we have the given number of 
brokers
+ * ready.
+ */
+private Optional readyBrokersFuture;
+
+ClusterControlManager(LogContext logContext,
+  Time time,
+  SnapshotRegistry snapshotRegistry,
+  long sessionTimeoutNs,
+  ReplicaPlacementPolicy placementPolicy) {
+this.logContext = logContext;
+this.log = logContext.logger(ClusterControlManager.class);
+this.time = time;
+this.sessionTimeoutNs = sessionTimeoutNs;
+this.placementPolicy = placementPolicy;
+this.brokerRegistrations = new TimelineHashMap<>(snapshotRegistry, 0);
+this.heartbeatManager = null;
+this.readyBrokersFuture = Optional.empty();
+}
+
+/**
+ * Transition this ClusterControlManager to active.
+ */
+public void activate() {
+heartbeatManager = new BrokerHeartbeatManager(logContext, time, 
sessionTimeoutNs);
+  

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-18 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r578627766



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
##
@@ -0,0 +1,900 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.ElectionType;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.BrokerIdNotRegisteredException;
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.errors.InvalidRequestException;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment;
+import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData.TopicPartitions;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.PartitionChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.controller.BrokersToIsrs.TopicPartition;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistration;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Random;
+
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET;
+import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC;
+
+
+/**
+ * The ReplicationControlManager is the part of the controller which deals 
with topics
+ * and partitions. It is responsible for managing the in-sync replica set and 
leader
+ * of each partition, as well as administrative tasks like creating or 
deleting topics.
+ */
+public class ReplicationControlManager {
+static class TopicControlInfo {
+private final Uuid id;
+private final TimelineHashMap parts;
+
+TopicControlInfo(SnapshotRegistry snapshotRegistry, Uuid id) {
+this.id = id;
+this.parts = new TimelineHashMap<>(snapshotRegistry, 0);
+}
+}
+
+static class PartitionControlInfo {
+private final int[] replicas;
+private final int[] isr;
+private final int[] removingReplicas;
+private final int[] addingReplicas;
+private final int leader;
+private final int leaderEpoch;
+private final int partitionEpoch;
+
+ 

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-18 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r578625531



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
##
@@ -0,0 +1,900 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.ElectionType;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.BrokerIdNotRegisteredException;
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.errors.InvalidRequestException;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment;
+import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData.TopicPartitions;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.PartitionChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.controller.BrokersToIsrs.TopicPartition;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistration;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Random;
+
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET;
+import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC;
+
+
+/**
+ * The ReplicationControlManager is the part of the controller which deals 
with topics
+ * and partitions. It is responsible for managing the in-sync replica set and 
leader
+ * of each partition, as well as administrative tasks like creating or 
deleting topics.
+ */
+public class ReplicationControlManager {
+static class TopicControlInfo {
+private final Uuid id;
+private final TimelineHashMap parts;
+
+TopicControlInfo(SnapshotRegistry snapshotRegistry, Uuid id) {
+this.id = id;
+this.parts = new TimelineHashMap<>(snapshotRegistry, 0);
+}
+}
+
+static class PartitionControlInfo {
+private final int[] replicas;
+private final int[] isr;
+private final int[] removingReplicas;
+private final int[] addingReplicas;
+private final int leader;
+private final int leaderEpoch;
+private final int partitionEpoch;
+
+ 

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-18 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r578621621



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java
##
@@ -0,0 +1,585 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.UsableBroker;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Optional;
+import java.util.TreeSet;
+import java.util.function.Function;
+
+import static org.apache.kafka.controller.BrokerControlState.FENCED;
+import static 
org.apache.kafka.controller.BrokerControlState.CONTROLLED_SHUTDOWN;
+import static org.apache.kafka.controller.BrokerControlState.SHUTDOWN_NOW;
+import static org.apache.kafka.controller.BrokerControlState.UNFENCED;
+
+
+/**
+ * The BrokerHeartbeatManager manages all the soft state associated with 
broker heartbeats.
+ * Soft state is state which does not appear in the metadata log.  This state 
includes
+ * things like the last time each broker sent us a heartbeat, and whether the 
broker is
+ * trying to perform a controlled shutdown.
+ *
+ * Only the active controller has a BrokerHeartbeatManager, since only the 
active
+ * controller handles broker heartbeats.  Standby controllers will create a 
heartbeat
+ * manager as part of the process of activating.  This design minimizes the 
size of the
+ * metadata partition by excluding heartbeats from it.  However, it does mean 
that after
+ * a controller failover, we may take some extra time to fence brokers, since 
the new
+ * active controller does not know when the last heartbeats were received from 
each.
+ */
+public class BrokerHeartbeatManager {
+static class BrokerHeartbeatState {
+/**
+ * The broker ID.
+ */
+private final int id;
+
+/**
+ * The last time we received a heartbeat from this broker, in 
monotonic nanoseconds.
+ * When this field is updated, we also may have to update the broker's 
position in
+ * the unfenced list.
+ */
+long lastContactNs;
+
+/**
+ * The last metadata offset which this broker reported.  When this 
field is updated,
+ * we may also have to update the broker's position in the active set.
+ */
+long metadataOffset;
+
+/**
+ * The offset at which the broker should complete its controlled 
shutdown, or -1
+ * if the broker is not performing a controlled shutdown.  When this 
field is
+ * updated, we also have to update the broker's position in the 
shuttingDown set.
+ */
+private long controlledShutDownOffset;
+
+/**
+ * The previous entry in the unfenced list, or null if the broker is 
not in that list.
+ */
+private BrokerHeartbeatState prev;
+
+/**
+ * The next entry in the unfenced list, or null if the broker is not 
in that list.
+ */
+private BrokerHeartbeatState next;
+
+BrokerHeartbeatState(int id) {
+this.id = id;
+this.lastContactNs = 0;
+this.prev = null;
+this.next = null;
+this.metadataOffset = -1;
+this.controlledShutDownOffset = -1;
+}
+
+/**
+ * Returns the broker ID.
+ */
+int id() {
+return id;
+}
+
+/**
+ * Returns true only if the broker is fenced.
+ */
+boolean fenced() {
+return prev == null;
+}
+
+/**
+ * Returns true only if the broker is in controlled shutdown state.
+ */
+boolean shuttingDown() {
+return controlledShutDownOffset >= 0;
+}
+}
+
+static class Metadat

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-18 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r578619727



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java
##
@@ -0,0 +1,585 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.UsableBroker;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Optional;
+import java.util.TreeSet;
+import java.util.function.Function;
+
+import static org.apache.kafka.controller.BrokerControlState.FENCED;
+import static 
org.apache.kafka.controller.BrokerControlState.CONTROLLED_SHUTDOWN;
+import static org.apache.kafka.controller.BrokerControlState.SHUTDOWN_NOW;
+import static org.apache.kafka.controller.BrokerControlState.UNFENCED;
+
+
+/**
+ * The BrokerHeartbeatManager manages all the soft state associated with 
broker heartbeats.
+ * Soft state is state which does not appear in the metadata log.  This state 
includes
+ * things like the last time each broker sent us a heartbeat, and whether the 
broker is
+ * trying to perform a controlled shutdown.
+ *
+ * Only the active controller has a BrokerHeartbeatManager, since only the 
active
+ * controller handles broker heartbeats.  Standby controllers will create a 
heartbeat
+ * manager as part of the process of activating.  This design minimizes the 
size of the
+ * metadata partition by excluding heartbeats from it.  However, it does mean 
that after
+ * a controller failover, we may take some extra time to fence brokers, since 
the new
+ * active controller does not know when the last heartbeats were received from 
each.
+ */
+public class BrokerHeartbeatManager {
+static class BrokerHeartbeatState {
+/**
+ * The broker ID.
+ */
+private final int id;
+
+/**
+ * The last time we received a heartbeat from this broker, in 
monotonic nanoseconds.
+ * When this field is updated, we also may have to update the broker's 
position in
+ * the unfenced list.
+ */
+long lastContactNs;
+
+/**
+ * The last metadata offset which this broker reported.  When this 
field is updated,
+ * we may also have to update the broker's position in the active set.
+ */
+long metadataOffset;
+
+/**
+ * The offset at which the broker should complete its controlled 
shutdown, or -1
+ * if the broker is not performing a controlled shutdown.  When this 
field is
+ * updated, we also have to update the broker's position in the 
shuttingDown set.
+ */
+private long controlledShutDownOffset;
+
+/**
+ * The previous entry in the unfenced list, or null if the broker is 
not in that list.
+ */
+private BrokerHeartbeatState prev;
+
+/**
+ * The next entry in the unfenced list, or null if the broker is not 
in that list.
+ */
+private BrokerHeartbeatState next;
+
+BrokerHeartbeatState(int id) {
+this.id = id;
+this.lastContactNs = 0;
+this.prev = null;
+this.next = null;
+this.metadataOffset = -1;
+this.controlledShutDownOffset = -1;
+}
+
+/**
+ * Returns the broker ID.
+ */
+int id() {
+return id;
+}
+
+/**
+ * Returns true only if the broker is fenced.
+ */
+boolean fenced() {
+return prev == null;
+}
+
+/**
+ * Returns true only if the broker is in controlled shutdown state.
+ */
+boolean shuttingDown() {
+return controlledShutDownOffset >= 0;
+}
+}
+
+static class Metadat

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-18 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r578603931



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/QuorumControllerMetrics.java
##
@@ -0,0 +1,61 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import com.yammer.metrics.core.Gauge;
+import com.yammer.metrics.core.Histogram;
+import com.yammer.metrics.core.MetricName;
+import com.yammer.metrics.core.MetricsRegistry;
+
+
+public final class QuorumControllerMetrics implements ControllerMetrics {
+private final static MetricName ACTIVE_CONTROLLER_COUNT = new MetricName(
+"kafka.controller", "KafkaController", "ActiveControllerCount", null);
+private final static MetricName EVENT_QUEUE_TIME_MS = new MetricName(
+"kafka.controller", "ControllerEventManager", "EventQueueTimeMs", 
null);

Review comment:
   Good point. I will fix it so that EventQueueTimeMs has its original 
meaning.
   
   For now, I have added a metric called EventQueueProcessingTimeMs which deals 
with processing time.  I do want to do the per-state tracking but I don't think 
we have time right now





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.

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




[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-17 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577929403



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
##
@@ -0,0 +1,920 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.ApiException;
+import org.apache.kafka.common.errors.NotControllerException;
+import org.apache.kafka.common.errors.UnknownServerException;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.ConfigRecord;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.IsrChangeRecord;
+import org.apache.kafka.common.metadata.MetadataRecordType;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.QuotaRecord;
+import org.apache.kafka.common.metadata.RegisterBrokerRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.protocol.ApiMessage;
+import org.apache.kafka.common.quota.ClientQuotaAlteration;
+import org.apache.kafka.common.quota.ClientQuotaEntity;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureManager;
+import org.apache.kafka.metadata.VersionRange;
+import org.apache.kafka.metalog.MetaLogLeader;
+import org.apache.kafka.metalog.MetaLogListener;
+import org.apache.kafka.metalog.MetaLogManager;
+import org.apache.kafka.queue.EventQueue.EarliestDeadlineFunction;
+import org.apache.kafka.queue.EventQueue;
+import org.apache.kafka.queue.KafkaEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+
+public final class QuorumController implements Controller {
+/**
+ * A builder class which creates the QuorumController.
+ */
+static public class Builder {
+private final int nodeId;
+private Time time = Time.SYSTEM;
+private String threadNamePrefix = null;
+private LogContext logContext = null;
+private Map configDefs = 
Collections.emptyMap();
+private MetaLogManager logManager = null;
+private Map supportedFeatures = 
Collections.emptyMap();
+private short defaultReplicationFactor = 3;
+private int defaultNumPartitions = 1;
+private ReplicaPlacementPolicy replicaPlacementPolicy =
+new SimpleReplicaPlacementPolicy(new Random());
+private long sessionTimeoutNs = TimeUnit.NANOSECONDS.convert(18, 
TimeUnit.SECONDS);
+
+public Builder(int nodeId) {
+this.nodeId = nodeId;
+}
+
+public B

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-17 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577929118



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
##
@@ -0,0 +1,712 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.ElectionType;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.errors.InvalidRequestException;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment;
+import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData.TopicPartitions;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.IsrChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.controller.BrokersToIsrs.TopicPartition;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Random;
+
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET;
+import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC;
+
+
+public class ReplicationControlManager {
+static class TopicControlInfo {
+private final Uuid id;
+private final TimelineHashMap parts;
+
+TopicControlInfo(SnapshotRegistry snapshotRegistry, Uuid id) {
+this.id = id;
+this.parts = new TimelineHashMap<>(snapshotRegistry, 0);
+}
+}
+
+static class PartitionControlInfo {
+private final int[] replicas;
+private final int[] isr;
+private final int[] removingReplicas;
+private final int[] addingReplicas;
+private final int leader;
+private final int leaderEpoch;
+private final int partitionEpoch;
+
+PartitionControlInfo(PartitionRecord record) {
+this(Replicas.toArray(record.replicas()),
+Replicas.toArray(record.isr()),
+Replicas.toArray(record.removingReplicas()),
+Replicas.toArray(record.addingReplicas()),
+record.leader(),
+record.leaderEpoch(),
+record.partitionEpoch());
+}
+
+PartitionControlInfo(int[] replicas, int[] isr, int[] removingReplicas,
+ int[] addingReplicas, int leader, int leaderEpoch,
+ int partitionEpoch) {
+this.replicas = replicas;
+this.isr = isr;
+this.removingRepli

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-17 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577922831



##
File path: metadata/src/main/java/org/apache/kafka/controller/Controller.java
##
@@ -0,0 +1,180 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.quota.ClientQuotaAlteration;
+import org.apache.kafka.common.quota.ClientQuotaEntity;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureManager;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+
+
+public interface Controller extends AutoCloseable {
+/**
+ * Change partition ISRs.
+ *
+ * @param request   The AlterIsrRequest data.
+ *
+ * @return  A future yielding the response.
+ */
+CompletableFuture alterIsr(AlterIsrRequestData 
request);
+
+/**
+ * Create a batch of topics.
+ *
+ * @param request   The CreateTopicsRequest data.
+ *
+ * @return  A future yielding the response.
+ */
+CompletableFuture
+createTopics(CreateTopicsRequestData request);
+
+/**
+ * Decommission a broker.
+ *
+ * @param brokerId  The broker id to decommission.
+ *
+ * @return  A future that is completed successfully hwne the 
broker is
+ *  decommissioned, or if it is not registered in the 
first place.
+ */
+CompletableFuture decommissionBroker(int brokerId);
+
+/**
+ * Describe the current configuration of various resources.
+ *
+ * @param resources A map from resources to the collection of config 
keys that we
+ *  want to describe for each.  If the collection is 
empty, then
+ *  all configuration keys will be described.
+ *
+ * @return
+ */
+CompletableFuture>>>
+describeConfigs(Map> resources);
+
+/**
+ * Elect new partition leaders.
+ *
+ * @param request   The request.
+ *
+ * @return  A future yielding the elect leaders response.
+ */
+CompletableFuture 
electLeaders(ElectLeadersRequestData request);
+
+/**
+ * Get the current finalized feature ranges for each feature.
+ *
+ * @return  A future yielding the feature ranges.
+ */
+CompletableFuture 
finalizedFeatures();
+
+/**
+ * Perform some incremental configuration changes.
+ *
+ * @param configChanges The changes.
+ * @param validateOnly  True if we should validate the changes but not 
apply them.
+ *
+ * @return  A future yielding a map from partitions to error 
results.
+ */
+CompletableFuture> incrementalAlterConfigs(
+Map>> configChanges,
+boolean validateOnly);
+
+/**
+ * Perform some configuration changes using the legacy API.
+ *
+ * @param newConfigsThe new configuration maps to apply.
+ * @param validateOnly  True if we should validate the changes but not 
apply them.
+ *
+ * @return  A future yielding a map from partitions to error 
results.
+ */
+CompletableFuture> legacyAlterConfigs(
+Map> newConfigs, boolean 
validateOnly);
+
+/**
+ * Process a heartbeat from a broker.
+ *
+ * @param request  The broker heartbeat request.
+ 

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-17 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577922334



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
##
@@ -0,0 +1,920 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.ApiException;
+import org.apache.kafka.common.errors.NotControllerException;
+import org.apache.kafka.common.errors.UnknownServerException;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.ConfigRecord;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.IsrChangeRecord;
+import org.apache.kafka.common.metadata.MetadataRecordType;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.QuotaRecord;
+import org.apache.kafka.common.metadata.RegisterBrokerRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.protocol.ApiMessage;
+import org.apache.kafka.common.quota.ClientQuotaAlteration;
+import org.apache.kafka.common.quota.ClientQuotaEntity;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureManager;
+import org.apache.kafka.metadata.VersionRange;
+import org.apache.kafka.metalog.MetaLogLeader;
+import org.apache.kafka.metalog.MetaLogListener;
+import org.apache.kafka.metalog.MetaLogManager;
+import org.apache.kafka.queue.EventQueue.EarliestDeadlineFunction;
+import org.apache.kafka.queue.EventQueue;
+import org.apache.kafka.queue.KafkaEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+
+public final class QuorumController implements Controller {
+/**
+ * A builder class which creates the QuorumController.
+ */
+static public class Builder {
+private final int nodeId;
+private Time time = Time.SYSTEM;
+private String threadNamePrefix = null;
+private LogContext logContext = null;
+private Map configDefs = 
Collections.emptyMap();
+private MetaLogManager logManager = null;
+private Map supportedFeatures = 
Collections.emptyMap();
+private short defaultReplicationFactor = 3;
+private int defaultNumPartitions = 1;
+private ReplicaPlacementPolicy replicaPlacementPolicy =
+new SimpleReplicaPlacementPolicy(new Random());
+private long sessionTimeoutNs = TimeUnit.NANOSECONDS.convert(18, 
TimeUnit.SECONDS);
+
+public Builder(int nodeId) {
+this.nodeId = nodeId;
+}
+
+public B

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-17 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577921275



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
##
@@ -0,0 +1,920 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.ApiException;
+import org.apache.kafka.common.errors.NotControllerException;
+import org.apache.kafka.common.errors.UnknownServerException;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.ConfigRecord;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.IsrChangeRecord;
+import org.apache.kafka.common.metadata.MetadataRecordType;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.QuotaRecord;
+import org.apache.kafka.common.metadata.RegisterBrokerRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.protocol.ApiMessage;
+import org.apache.kafka.common.quota.ClientQuotaAlteration;
+import org.apache.kafka.common.quota.ClientQuotaEntity;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureManager;
+import org.apache.kafka.metadata.VersionRange;
+import org.apache.kafka.metalog.MetaLogLeader;
+import org.apache.kafka.metalog.MetaLogListener;
+import org.apache.kafka.metalog.MetaLogManager;
+import org.apache.kafka.queue.EventQueue.EarliestDeadlineFunction;
+import org.apache.kafka.queue.EventQueue;
+import org.apache.kafka.queue.KafkaEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+
+public final class QuorumController implements Controller {
+/**
+ * A builder class which creates the QuorumController.
+ */
+static public class Builder {
+private final int nodeId;
+private Time time = Time.SYSTEM;
+private String threadNamePrefix = null;
+private LogContext logContext = null;
+private Map configDefs = 
Collections.emptyMap();
+private MetaLogManager logManager = null;
+private Map supportedFeatures = 
Collections.emptyMap();
+private short defaultReplicationFactor = 3;
+private int defaultNumPartitions = 1;
+private ReplicaPlacementPolicy replicaPlacementPolicy =
+new SimpleReplicaPlacementPolicy(new Random());
+private long sessionTimeoutNs = TimeUnit.NANOSECONDS.convert(18, 
TimeUnit.SECONDS);
+
+public Builder(int nodeId) {
+this.nodeId = nodeId;
+}
+
+public B

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-17 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577920610



##
File path: metadata/src/main/java/org/apache/kafka/controller/BrokersToIsrs.java
##
@@ -0,0 +1,308 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.NoSuchElementException;
+import java.util.Objects;
+
+
+/**
+ * Associates brokers with their in-sync partitions.
+ *
+ * This is useful when we need to remove a broker from all the ISRs, or move 
all leaders
+ * away from a broker.
+ *
+ * We also track all the partitions that currently have no leader.
+ *
+ * The core data structure is a map from broker IDs to topic maps.  Each topic 
map relates
+ * topic UUIDs to arrays of partition IDs.
+ *
+ * Each entry in the array has a high bit which indicates that the broker is 
the leader
+ * for the given partition, as well as 31 low bits which contain the partition 
id.  This
+ * works because partition IDs cannot be negative.
+ */
+public class BrokersToIsrs {
+private final static int[] EMPTY = new int[0];
+
+private final static int LEADER_FLAG = 0x8000_;
+
+private final static int REPLICA_MASK = 0x7fff_;
+
+static class TopicPartition {
+private final Uuid topicId;
+private final int partitionId;
+
+TopicPartition(Uuid topicId, int partitionId) {
+this.topicId = topicId;
+this.partitionId = partitionId;
+}
+
+public Uuid topicId() {
+return topicId;
+}
+
+public int partitionId() {
+return partitionId;
+}
+
+@Override
+public boolean equals(Object o) {
+if (!(o instanceof TopicPartition)) return false;
+TopicPartition other = (TopicPartition) o;
+return other.topicId.equals(topicId) && other.partitionId == 
partitionId;
+}
+
+@Override
+public int hashCode() {
+return Objects.hash(topicId, partitionId);
+}
+
+@Override
+public String toString() {
+return topicId + ":" + partitionId;
+}
+}
+
+static class PartitionsOnReplicaIterator implements 
Iterator {
+private final Iterator> iterator;
+private final boolean leaderOnly;
+private int offset = 0;
+Uuid uuid = Uuid.ZERO_UUID;
+int[] replicas = EMPTY;
+private TopicPartition next = null;
+
+PartitionsOnReplicaIterator(Map topicMap, boolean 
leaderOnly) {
+this.iterator = topicMap.entrySet().iterator();
+this.leaderOnly = leaderOnly;
+}
+
+@Override
+public boolean hasNext() {
+if (next != null) return true;
+while (true) {
+if (offset >= replicas.length) {
+if (!iterator.hasNext()) return false;
+offset = 0;
+Entry entry = iterator.next();
+uuid = entry.getKey();
+replicas = entry.getValue();
+}
+int replica = replicas[offset++];
+if ((!leaderOnly) || (replica & LEADER_FLAG) != 0) {
+next = new TopicPartition(uuid, replica & REPLICA_MASK);
+return true;
+}
+}
+}
+
+@Override
+public TopicPartition next() {
+if (!hasNext()) {
+throw new NoSuchElementException();
+}
+TopicPartition result = next;
+next = null;
+return result;
+}
+}
+
+private final SnapshotRegistry snapshotRegistry;
+
+/**
+ * A map of broker IDs to the partitions that the broker is in the ISR for.
+ * Partitions with no isr members appear in this map under id -1.
+ */
+private final TimelineHashMap> 
isrM

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-17 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577920156



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java
##
@@ -0,0 +1,519 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.UsableBroker;
+import org.slf4j.Logger;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Optional;
+import java.util.TreeSet;
+import java.util.function.Function;
+
+
+/**
+ * The BrokerHeartbeatManager manages all the soft state associated with 
broker heartbeats.
+ * Soft state is state which does not appear in the metadata log.  This state 
includes
+ * things like the last time each broker sent us a heartbeat, and whether the 
broker is
+ * trying to perform a controlled shutdown.
+ *
+ * Only the active controller has a BrokerHeartbeatManager, since only the 
active
+ * controller handles broker heartbeats.  Standby controllers will create a 
heartbeat
+ * manager as part of the process of activating.  This design minimizes the 
size of the
+ * metadata partition by excluding heartbeats from it.  However, it does mean 
that after
+ * a controller failover, we may take some extra time to fence brokers, since 
the new
+ * active controller does not know when the last heartbeats were received from 
each.
+ */
+public class BrokerHeartbeatManager {
+static class BrokerHeartbeatState {
+/**
+ * The broker ID.
+ */
+private final int id;
+
+/**
+ * The last time we received a heartbeat from this broker, in 
monotonic nanoseconds.
+ * When this field is updated, we also may have to update the broker's 
position in
+ * the unfenced list.
+ */
+long lastContactNs;
+
+/**
+ * The last metadata offset which this broker reported.  When this 
field is updated,
+ * we may also have to update the broker's position in the active set.
+ */
+long metadataOffset;
+
+/**
+ * The offset at which the broker should complete its controlled 
shutdown, or -1
+ * if the broker is not performing a controlled shutdown.  When this 
field is
+ * updated, we also have to update the broker's position in the 
shuttingDown set.
+ */
+private long shutDownOffset;
+
+/**
+ * The previous entry in the unfenced list, or null if the broker is 
not in that list.
+ */
+private BrokerHeartbeatState prev;
+
+/**
+ * The next entry in the unfenced list, or null if the broker is not 
in that list.
+ */
+private BrokerHeartbeatState next;
+
+BrokerHeartbeatState(int id) {
+this.id = id;
+this.lastContactNs = 0;
+this.prev = null;
+this.next = null;
+this.metadataOffset = -1;
+this.shutDownOffset = -1;
+}
+
+/**
+ * Returns the broker ID.
+ */
+int id() {
+return id;
+}
+
+/**
+ * Returns true only if the broker is fenced.
+ */
+boolean fenced() {
+return prev == null;
+}
+
+/**
+ * Returns true only if the broker is in controlled shutdown state.
+ */
+boolean shuttingDown() {
+return shutDownOffset >= 0;
+}
+}
+
+static class MetadataOffsetComparator implements 
Comparator {
+static final MetadataOffsetComparator INSTANCE = new 
MetadataOffsetComparator();
+
+@Override
+public int compare(BrokerHeartbeatState a, BrokerHeartbeatState b) {
+if (a.metadataOffset < b.metadataOffset) {
+return -1;
+} else if (a.metadataOffset > b.metadataOffset) {
+return 1;
+} else if (

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-17 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577919831



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java
##
@@ -0,0 +1,456 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.common.Endpoint;
+import org.apache.kafka.common.errors.DuplicateBrokerRegistrationException;
+import org.apache.kafka.common.errors.StaleBrokerEpochException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.RegisterBrokerRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistration;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureManager;
+import org.apache.kafka.metadata.VersionRange;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+
+public class ClusterControlManager {
+class ReadyBrokersFuture {
+private final CompletableFuture future;
+private final int minBrokers;
+
+ReadyBrokersFuture(CompletableFuture future, int minBrokers) {
+this.future = future;
+this.minBrokers = minBrokers;
+}
+
+boolean check() {
+int numUnfenced = 0;
+for (BrokerRegistration registration : 
brokerRegistrations.values()) {
+if (!registration.fenced()) {
+numUnfenced++;
+}
+if (numUnfenced >= minBrokers) {
+return true;
+}
+}
+return false;
+}
+}
+
+/**
+ * The SLF4J log context.
+ */
+private final LogContext logContext;
+
+/**
+ * The SLF4J log object.
+ */
+private final Logger log;
+
+/**
+ * The Kafka clock object to use.
+ */
+private final Time time;
+
+/**
+ * How long sessions should last, in nanoseconds.
+ */
+private final long sessionTimeoutNs;
+
+/**
+ * The replica placement policy to use.
+ */
+private final ReplicaPlacementPolicy placementPolicy;
+
+/**
+ * Maps broker IDs to broker registrations.
+ */
+private final TimelineHashMap 
brokerRegistrations;
+
+/**
+ * The broker heartbeat manager, or null if this controller is on standby.
+ */
+private BrokerHeartbeatManager heartbeatManager;
+
+/**
+ * A future which is completed as soon as we have the given number of 
brokers
+ * ready.
+ */
+private Optional readyBrokersFuture;
+
+ClusterControlManager(LogContext logContext,
+  Time time,
+  SnapshotRegistry snapshotRegistry,
+  long sessionTimeoutNs,
+  ReplicaPlacementPolicy placementPolicy) {
+this.logContext = logContext;
+this.log = logContext.logger(ClusterControlManager.class);
+this.time = time;
+this.sessionTimeoutNs = sessionTimeoutNs;
+this.placementPolicy = placementPolicy;
+this.brokerRegistrations = new TimelineHashMap<>(snapshotRegistry, 0);
+this.heartbeatManager = null;
+this.readyBrokersFuture = Optional.empty();
+}
+
+/**
+ * Transition thi

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-17 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577919102



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
##
@@ -0,0 +1,920 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.ApiException;
+import org.apache.kafka.common.errors.NotControllerException;
+import org.apache.kafka.common.errors.UnknownServerException;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.ConfigRecord;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.IsrChangeRecord;
+import org.apache.kafka.common.metadata.MetadataRecordType;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.QuotaRecord;
+import org.apache.kafka.common.metadata.RegisterBrokerRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.protocol.ApiMessage;
+import org.apache.kafka.common.quota.ClientQuotaAlteration;
+import org.apache.kafka.common.quota.ClientQuotaEntity;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureManager;
+import org.apache.kafka.metadata.VersionRange;
+import org.apache.kafka.metalog.MetaLogLeader;
+import org.apache.kafka.metalog.MetaLogListener;
+import org.apache.kafka.metalog.MetaLogManager;
+import org.apache.kafka.queue.EventQueue.EarliestDeadlineFunction;
+import org.apache.kafka.queue.EventQueue;
+import org.apache.kafka.queue.KafkaEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+
+public final class QuorumController implements Controller {
+/**
+ * A builder class which creates the QuorumController.
+ */
+static public class Builder {
+private final int nodeId;
+private Time time = Time.SYSTEM;
+private String threadNamePrefix = null;
+private LogContext logContext = null;
+private Map configDefs = 
Collections.emptyMap();
+private MetaLogManager logManager = null;
+private Map supportedFeatures = 
Collections.emptyMap();
+private short defaultReplicationFactor = 3;
+private int defaultNumPartitions = 1;
+private ReplicaPlacementPolicy replicaPlacementPolicy =
+new SimpleReplicaPlacementPolicy(new Random());
+private long sessionTimeoutNs = TimeUnit.NANOSECONDS.convert(18, 
TimeUnit.SECONDS);
+
+public Builder(int nodeId) {
+this.nodeId = nodeId;
+}
+
+public B

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-17 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577918400



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java
##
@@ -0,0 +1,519 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.UsableBroker;
+import org.slf4j.Logger;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Optional;
+import java.util.TreeSet;
+import java.util.function.Function;
+
+
+/**
+ * The BrokerHeartbeatManager manages all the soft state associated with 
broker heartbeats.
+ * Soft state is state which does not appear in the metadata log.  This state 
includes
+ * things like the last time each broker sent us a heartbeat, and whether the 
broker is
+ * trying to perform a controlled shutdown.
+ *
+ * Only the active controller has a BrokerHeartbeatManager, since only the 
active
+ * controller handles broker heartbeats.  Standby controllers will create a 
heartbeat
+ * manager as part of the process of activating.  This design minimizes the 
size of the
+ * metadata partition by excluding heartbeats from it.  However, it does mean 
that after
+ * a controller failover, we may take some extra time to fence brokers, since 
the new
+ * active controller does not know when the last heartbeats were received from 
each.
+ */
+public class BrokerHeartbeatManager {
+static class BrokerHeartbeatState {
+/**
+ * The broker ID.
+ */
+private final int id;
+
+/**
+ * The last time we received a heartbeat from this broker, in 
monotonic nanoseconds.
+ * When this field is updated, we also may have to update the broker's 
position in
+ * the unfenced list.
+ */
+long lastContactNs;
+
+/**
+ * The last metadata offset which this broker reported.  When this 
field is updated,
+ * we may also have to update the broker's position in the active set.
+ */
+long metadataOffset;
+
+/**
+ * The offset at which the broker should complete its controlled 
shutdown, or -1
+ * if the broker is not performing a controlled shutdown.  When this 
field is
+ * updated, we also have to update the broker's position in the 
shuttingDown set.
+ */
+private long shutDownOffset;
+
+/**
+ * The previous entry in the unfenced list, or null if the broker is 
not in that list.
+ */
+private BrokerHeartbeatState prev;
+
+/**
+ * The next entry in the unfenced list, or null if the broker is not 
in that list.
+ */
+private BrokerHeartbeatState next;
+
+BrokerHeartbeatState(int id) {
+this.id = id;
+this.lastContactNs = 0;
+this.prev = null;
+this.next = null;
+this.metadataOffset = -1;
+this.shutDownOffset = -1;
+}
+
+/**
+ * Returns the broker ID.
+ */
+int id() {
+return id;
+}
+
+/**
+ * Returns true only if the broker is fenced.
+ */
+boolean fenced() {
+return prev == null;
+}
+
+/**
+ * Returns true only if the broker is in controlled shutdown state.
+ */
+boolean shuttingDown() {
+return shutDownOffset >= 0;
+}
+}
+
+static class MetadataOffsetComparator implements 
Comparator {
+static final MetadataOffsetComparator INSTANCE = new 
MetadataOffsetComparator();
+
+@Override
+public int compare(BrokerHeartbeatState a, BrokerHeartbeatState b) {
+if (a.metadataOffset < b.metadataOffset) {
+return -1;
+} else if (a.metadataOffset > b.metadataOffset) {
+return 1;
+} else if (

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-17 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577916988



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java
##
@@ -0,0 +1,456 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.common.Endpoint;
+import org.apache.kafka.common.errors.DuplicateBrokerRegistrationException;
+import org.apache.kafka.common.errors.StaleBrokerEpochException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.RegisterBrokerRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistration;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureManager;
+import org.apache.kafka.metadata.VersionRange;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+
+public class ClusterControlManager {
+class ReadyBrokersFuture {
+private final CompletableFuture future;
+private final int minBrokers;
+
+ReadyBrokersFuture(CompletableFuture future, int minBrokers) {
+this.future = future;
+this.minBrokers = minBrokers;
+}
+
+boolean check() {
+int numUnfenced = 0;
+for (BrokerRegistration registration : 
brokerRegistrations.values()) {
+if (!registration.fenced()) {
+numUnfenced++;
+}
+if (numUnfenced >= minBrokers) {
+return true;
+}
+}
+return false;
+}
+}
+
+/**
+ * The SLF4J log context.
+ */
+private final LogContext logContext;
+
+/**
+ * The SLF4J log object.
+ */
+private final Logger log;
+
+/**
+ * The Kafka clock object to use.
+ */
+private final Time time;
+
+/**
+ * How long sessions should last, in nanoseconds.
+ */
+private final long sessionTimeoutNs;
+
+/**
+ * The replica placement policy to use.
+ */
+private final ReplicaPlacementPolicy placementPolicy;
+
+/**
+ * Maps broker IDs to broker registrations.
+ */
+private final TimelineHashMap 
brokerRegistrations;
+
+/**
+ * The broker heartbeat manager, or null if this controller is on standby.
+ */
+private BrokerHeartbeatManager heartbeatManager;
+
+/**
+ * A future which is completed as soon as we have the given number of 
brokers
+ * ready.
+ */
+private Optional readyBrokersFuture;
+
+ClusterControlManager(LogContext logContext,
+  Time time,
+  SnapshotRegistry snapshotRegistry,
+  long sessionTimeoutNs,
+  ReplicaPlacementPolicy placementPolicy) {
+this.logContext = logContext;
+this.log = logContext.logger(ClusterControlManager.class);
+this.time = time;
+this.sessionTimeoutNs = sessionTimeoutNs;
+this.placementPolicy = placementPolicy;
+this.brokerRegistrations = new TimelineHashMap<>(snapshotRegistry, 0);
+this.heartbeatManager = null;
+this.readyBrokersFuture = Optional.empty();
+}
+
+/**
+ * Transition thi

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-17 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577916633



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java
##
@@ -0,0 +1,456 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.common.Endpoint;
+import org.apache.kafka.common.errors.DuplicateBrokerRegistrationException;
+import org.apache.kafka.common.errors.StaleBrokerEpochException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.RegisterBrokerRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistration;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureManager;
+import org.apache.kafka.metadata.VersionRange;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+
+public class ClusterControlManager {
+class ReadyBrokersFuture {
+private final CompletableFuture future;
+private final int minBrokers;
+
+ReadyBrokersFuture(CompletableFuture future, int minBrokers) {
+this.future = future;
+this.minBrokers = minBrokers;
+}
+
+boolean check() {
+int numUnfenced = 0;
+for (BrokerRegistration registration : 
brokerRegistrations.values()) {
+if (!registration.fenced()) {
+numUnfenced++;
+}
+if (numUnfenced >= minBrokers) {
+return true;
+}
+}
+return false;
+}
+}
+
+/**
+ * The SLF4J log context.
+ */
+private final LogContext logContext;
+
+/**
+ * The SLF4J log object.
+ */
+private final Logger log;
+
+/**
+ * The Kafka clock object to use.
+ */
+private final Time time;
+
+/**
+ * How long sessions should last, in nanoseconds.
+ */
+private final long sessionTimeoutNs;
+
+/**
+ * The replica placement policy to use.
+ */
+private final ReplicaPlacementPolicy placementPolicy;
+
+/**
+ * Maps broker IDs to broker registrations.
+ */
+private final TimelineHashMap 
brokerRegistrations;
+
+/**
+ * The broker heartbeat manager, or null if this controller is on standby.
+ */
+private BrokerHeartbeatManager heartbeatManager;
+
+/**
+ * A future which is completed as soon as we have the given number of 
brokers
+ * ready.
+ */
+private Optional readyBrokersFuture;
+
+ClusterControlManager(LogContext logContext,
+  Time time,
+  SnapshotRegistry snapshotRegistry,
+  long sessionTimeoutNs,
+  ReplicaPlacementPolicy placementPolicy) {
+this.logContext = logContext;
+this.log = logContext.logger(ClusterControlManager.class);
+this.time = time;
+this.sessionTimeoutNs = sessionTimeoutNs;
+this.placementPolicy = placementPolicy;
+this.brokerRegistrations = new TimelineHashMap<>(snapshotRegistry, 0);
+this.heartbeatManager = null;
+this.readyBrokersFuture = Optional.empty();
+}
+
+/**
+ * Transition thi

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-17 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577916179



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
##
@@ -0,0 +1,920 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.ApiException;
+import org.apache.kafka.common.errors.NotControllerException;
+import org.apache.kafka.common.errors.UnknownServerException;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.ConfigRecord;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.IsrChangeRecord;
+import org.apache.kafka.common.metadata.MetadataRecordType;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.QuotaRecord;
+import org.apache.kafka.common.metadata.RegisterBrokerRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.protocol.ApiMessage;
+import org.apache.kafka.common.quota.ClientQuotaAlteration;
+import org.apache.kafka.common.quota.ClientQuotaEntity;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureManager;
+import org.apache.kafka.metadata.VersionRange;
+import org.apache.kafka.metalog.MetaLogLeader;
+import org.apache.kafka.metalog.MetaLogListener;
+import org.apache.kafka.metalog.MetaLogManager;
+import org.apache.kafka.queue.EventQueue.EarliestDeadlineFunction;
+import org.apache.kafka.queue.EventQueue;
+import org.apache.kafka.queue.KafkaEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+
+public final class QuorumController implements Controller {
+/**
+ * A builder class which creates the QuorumController.
+ */
+static public class Builder {
+private final int nodeId;
+private Time time = Time.SYSTEM;
+private String threadNamePrefix = null;
+private LogContext logContext = null;
+private Map configDefs = 
Collections.emptyMap();
+private MetaLogManager logManager = null;
+private Map supportedFeatures = 
Collections.emptyMap();
+private short defaultReplicationFactor = 3;
+private int defaultNumPartitions = 1;
+private ReplicaPlacementPolicy replicaPlacementPolicy =
+new SimpleReplicaPlacementPolicy(new Random());
+private long sessionTimeoutNs = TimeUnit.NANOSECONDS.convert(18, 
TimeUnit.SECONDS);
+
+public Builder(int nodeId) {
+this.nodeId = nodeId;
+}
+
+public B

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-17 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577915565



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
##
@@ -0,0 +1,920 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.ApiException;
+import org.apache.kafka.common.errors.NotControllerException;
+import org.apache.kafka.common.errors.UnknownServerException;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.ConfigRecord;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.IsrChangeRecord;
+import org.apache.kafka.common.metadata.MetadataRecordType;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.QuotaRecord;
+import org.apache.kafka.common.metadata.RegisterBrokerRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.protocol.ApiMessage;
+import org.apache.kafka.common.quota.ClientQuotaAlteration;
+import org.apache.kafka.common.quota.ClientQuotaEntity;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureManager;
+import org.apache.kafka.metadata.VersionRange;
+import org.apache.kafka.metalog.MetaLogLeader;
+import org.apache.kafka.metalog.MetaLogListener;
+import org.apache.kafka.metalog.MetaLogManager;
+import org.apache.kafka.queue.EventQueue.EarliestDeadlineFunction;
+import org.apache.kafka.queue.EventQueue;
+import org.apache.kafka.queue.KafkaEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+
+public final class QuorumController implements Controller {
+/**
+ * A builder class which creates the QuorumController.
+ */
+static public class Builder {
+private final int nodeId;
+private Time time = Time.SYSTEM;
+private String threadNamePrefix = null;
+private LogContext logContext = null;
+private Map configDefs = 
Collections.emptyMap();
+private MetaLogManager logManager = null;
+private Map supportedFeatures = 
Collections.emptyMap();
+private short defaultReplicationFactor = 3;
+private int defaultNumPartitions = 1;
+private ReplicaPlacementPolicy replicaPlacementPolicy =
+new SimpleReplicaPlacementPolicy(new Random());
+private long sessionTimeoutNs = TimeUnit.NANOSECONDS.convert(18, 
TimeUnit.SECONDS);
+
+public Builder(int nodeId) {
+this.nodeId = nodeId;
+}
+
+public B

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-17 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577914392



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
##
@@ -0,0 +1,712 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.ElectionType;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.errors.InvalidRequestException;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment;
+import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData.TopicPartitions;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.IsrChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.controller.BrokersToIsrs.TopicPartition;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Random;
+
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET;
+import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC;
+
+
+public class ReplicationControlManager {
+static class TopicControlInfo {
+private final Uuid id;
+private final TimelineHashMap parts;
+
+TopicControlInfo(SnapshotRegistry snapshotRegistry, Uuid id) {
+this.id = id;
+this.parts = new TimelineHashMap<>(snapshotRegistry, 0);
+}
+}
+
+static class PartitionControlInfo {
+private final int[] replicas;
+private final int[] isr;
+private final int[] removingReplicas;
+private final int[] addingReplicas;
+private final int leader;
+private final int leaderEpoch;
+private final int partitionEpoch;
+
+PartitionControlInfo(PartitionRecord record) {
+this(Replicas.toArray(record.replicas()),
+Replicas.toArray(record.isr()),
+Replicas.toArray(record.removingReplicas()),
+Replicas.toArray(record.addingReplicas()),
+record.leader(),
+record.leaderEpoch(),
+record.partitionEpoch());
+}
+
+PartitionControlInfo(int[] replicas, int[] isr, int[] removingReplicas,
+ int[] addingReplicas, int leader, int leaderEpoch,
+ int partitionEpoch) {
+this.replicas = replicas;
+this.isr = isr;
+this.removingRepli

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-17 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577895640



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
##
@@ -0,0 +1,920 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.ApiException;
+import org.apache.kafka.common.errors.NotControllerException;
+import org.apache.kafka.common.errors.UnknownServerException;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.ConfigRecord;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.IsrChangeRecord;
+import org.apache.kafka.common.metadata.MetadataRecordType;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.QuotaRecord;
+import org.apache.kafka.common.metadata.RegisterBrokerRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.protocol.ApiMessage;
+import org.apache.kafka.common.quota.ClientQuotaAlteration;
+import org.apache.kafka.common.quota.ClientQuotaEntity;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureManager;
+import org.apache.kafka.metadata.VersionRange;
+import org.apache.kafka.metalog.MetaLogLeader;
+import org.apache.kafka.metalog.MetaLogListener;
+import org.apache.kafka.metalog.MetaLogManager;
+import org.apache.kafka.queue.EventQueue.EarliestDeadlineFunction;
+import org.apache.kafka.queue.EventQueue;
+import org.apache.kafka.queue.KafkaEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+
+public final class QuorumController implements Controller {
+/**
+ * A builder class which creates the QuorumController.
+ */
+static public class Builder {
+private final int nodeId;
+private Time time = Time.SYSTEM;
+private String threadNamePrefix = null;
+private LogContext logContext = null;
+private Map configDefs = 
Collections.emptyMap();
+private MetaLogManager logManager = null;
+private Map supportedFeatures = 
Collections.emptyMap();
+private short defaultReplicationFactor = 3;
+private int defaultNumPartitions = 1;
+private ReplicaPlacementPolicy replicaPlacementPolicy =
+new SimpleReplicaPlacementPolicy(new Random());
+private long sessionTimeoutNs = TimeUnit.NANOSECONDS.convert(18, 
TimeUnit.SECONDS);
+
+public Builder(int nodeId) {
+this.nodeId = nodeId;
+}
+
+public B

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-17 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577894071



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java
##
@@ -0,0 +1,456 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.common.Endpoint;
+import org.apache.kafka.common.errors.DuplicateBrokerRegistrationException;
+import org.apache.kafka.common.errors.StaleBrokerEpochException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.RegisterBrokerRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistration;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureManager;
+import org.apache.kafka.metadata.VersionRange;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+
+public class ClusterControlManager {
+class ReadyBrokersFuture {
+private final CompletableFuture future;
+private final int minBrokers;
+
+ReadyBrokersFuture(CompletableFuture future, int minBrokers) {
+this.future = future;
+this.minBrokers = minBrokers;
+}
+
+boolean check() {
+int numUnfenced = 0;
+for (BrokerRegistration registration : 
brokerRegistrations.values()) {
+if (!registration.fenced()) {
+numUnfenced++;
+}
+if (numUnfenced >= minBrokers) {
+return true;
+}
+}
+return false;
+}
+}
+
+/**
+ * The SLF4J log context.
+ */
+private final LogContext logContext;
+
+/**
+ * The SLF4J log object.
+ */
+private final Logger log;
+
+/**
+ * The Kafka clock object to use.
+ */
+private final Time time;
+
+/**
+ * How long sessions should last, in nanoseconds.
+ */
+private final long sessionTimeoutNs;
+
+/**
+ * The replica placement policy to use.
+ */
+private final ReplicaPlacementPolicy placementPolicy;
+
+/**
+ * Maps broker IDs to broker registrations.
+ */
+private final TimelineHashMap 
brokerRegistrations;
+
+/**
+ * The broker heartbeat manager, or null if this controller is on standby.
+ */
+private BrokerHeartbeatManager heartbeatManager;
+
+/**
+ * A future which is completed as soon as we have the given number of 
brokers
+ * ready.
+ */
+private Optional readyBrokersFuture;
+
+ClusterControlManager(LogContext logContext,
+  Time time,
+  SnapshotRegistry snapshotRegistry,
+  long sessionTimeoutNs,
+  ReplicaPlacementPolicy placementPolicy) {
+this.logContext = logContext;
+this.log = logContext.logger(ClusterControlManager.class);
+this.time = time;
+this.sessionTimeoutNs = sessionTimeoutNs;
+this.placementPolicy = placementPolicy;
+this.brokerRegistrations = new TimelineHashMap<>(snapshotRegistry, 0);
+this.heartbeatManager = null;
+this.readyBrokersFuture = Optional.empty();
+}
+
+/**
+ * Transition thi

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-17 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577868843



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
##
@@ -0,0 +1,712 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.ElectionType;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.errors.InvalidRequestException;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment;
+import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData.TopicPartitions;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.IsrChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.controller.BrokersToIsrs.TopicPartition;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Random;
+
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET;
+import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC;
+
+
+public class ReplicationControlManager {
+static class TopicControlInfo {
+private final Uuid id;
+private final TimelineHashMap parts;
+
+TopicControlInfo(SnapshotRegistry snapshotRegistry, Uuid id) {
+this.id = id;
+this.parts = new TimelineHashMap<>(snapshotRegistry, 0);
+}
+}
+
+static class PartitionControlInfo {
+private final int[] replicas;
+private final int[] isr;
+private final int[] removingReplicas;
+private final int[] addingReplicas;
+private final int leader;
+private final int leaderEpoch;
+private final int partitionEpoch;
+
+PartitionControlInfo(PartitionRecord record) {
+this(Replicas.toArray(record.replicas()),
+Replicas.toArray(record.isr()),
+Replicas.toArray(record.removingReplicas()),
+Replicas.toArray(record.addingReplicas()),
+record.leader(),
+record.leaderEpoch(),
+record.partitionEpoch());
+}
+
+PartitionControlInfo(int[] replicas, int[] isr, int[] removingReplicas,
+ int[] addingReplicas, int leader, int leaderEpoch,
+ int partitionEpoch) {
+this.replicas = replicas;
+this.isr = isr;
+this.removingRepli

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-17 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577866850



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
##
@@ -0,0 +1,920 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.ApiException;
+import org.apache.kafka.common.errors.NotControllerException;
+import org.apache.kafka.common.errors.UnknownServerException;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.ConfigRecord;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.IsrChangeRecord;
+import org.apache.kafka.common.metadata.MetadataRecordType;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.QuotaRecord;
+import org.apache.kafka.common.metadata.RegisterBrokerRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.protocol.ApiMessage;
+import org.apache.kafka.common.quota.ClientQuotaAlteration;
+import org.apache.kafka.common.quota.ClientQuotaEntity;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureManager;
+import org.apache.kafka.metadata.VersionRange;
+import org.apache.kafka.metalog.MetaLogLeader;
+import org.apache.kafka.metalog.MetaLogListener;
+import org.apache.kafka.metalog.MetaLogManager;
+import org.apache.kafka.queue.EventQueue.EarliestDeadlineFunction;
+import org.apache.kafka.queue.EventQueue;
+import org.apache.kafka.queue.KafkaEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+
+public final class QuorumController implements Controller {
+/**
+ * A builder class which creates the QuorumController.
+ */
+static public class Builder {
+private final int nodeId;
+private Time time = Time.SYSTEM;
+private String threadNamePrefix = null;
+private LogContext logContext = null;
+private Map configDefs = 
Collections.emptyMap();
+private MetaLogManager logManager = null;
+private Map supportedFeatures = 
Collections.emptyMap();
+private short defaultReplicationFactor = 3;
+private int defaultNumPartitions = 1;
+private ReplicaPlacementPolicy replicaPlacementPolicy =
+new SimpleReplicaPlacementPolicy(new Random());
+private long sessionTimeoutNs = TimeUnit.NANOSECONDS.convert(18, 
TimeUnit.SECONDS);
+
+public Builder(int nodeId) {
+this.nodeId = nodeId;
+}
+
+public B

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-17 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577865782



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
##
@@ -0,0 +1,917 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.ApiException;
+import org.apache.kafka.common.errors.NotControllerException;
+import org.apache.kafka.common.errors.UnknownServerException;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.ConfigRecord;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.IsrChangeRecord;
+import org.apache.kafka.common.metadata.MetadataRecordType;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.QuotaRecord;
+import org.apache.kafka.common.metadata.RegisterBrokerRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.protocol.ApiMessage;
+import org.apache.kafka.common.quota.ClientQuotaAlteration;
+import org.apache.kafka.common.quota.ClientQuotaEntity;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureManager;
+import org.apache.kafka.metadata.VersionRange;
+import org.apache.kafka.metalog.MetaLogLeader;
+import org.apache.kafka.metalog.MetaLogListener;
+import org.apache.kafka.metalog.MetaLogManager;
+import org.apache.kafka.queue.EventQueue.EarliestDeadlineFunction;
+import org.apache.kafka.queue.EventQueue;
+import org.apache.kafka.queue.KafkaEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+
+public final class QuorumController implements Controller {
+/**
+ * A builder class which creates the QuorumController.
+ */
+static public class Builder {
+private final int nodeId;
+private Time time = Time.SYSTEM;
+private String threadNamePrefix = null;
+private LogContext logContext = null;
+private Map configDefs = 
Collections.emptyMap();
+private MetaLogManager logManager = null;
+private Map supportedFeatures = 
Collections.emptyMap();
+private short defaultReplicationFactor = 3;
+private int defaultNumPartitions = 1;
+private ReplicaPlacementPolicy replicaPlacementPolicy =
+new SimpleReplicaPlacementPolicy(new Random());
+private long sessionTimeoutNs = TimeUnit.NANOSECONDS.convert(18, 
TimeUnit.SECONDS);
+
+public Builder(int nodeId) {
+this.nodeId = nodeId;
+}
+
+public B

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-16 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577217605



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
##
@@ -0,0 +1,920 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.ApiException;
+import org.apache.kafka.common.errors.NotControllerException;
+import org.apache.kafka.common.errors.UnknownServerException;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.ConfigRecord;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.IsrChangeRecord;
+import org.apache.kafka.common.metadata.MetadataRecordType;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.QuotaRecord;
+import org.apache.kafka.common.metadata.RegisterBrokerRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.protocol.ApiMessage;
+import org.apache.kafka.common.quota.ClientQuotaAlteration;
+import org.apache.kafka.common.quota.ClientQuotaEntity;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureManager;
+import org.apache.kafka.metadata.VersionRange;
+import org.apache.kafka.metalog.MetaLogLeader;
+import org.apache.kafka.metalog.MetaLogListener;
+import org.apache.kafka.metalog.MetaLogManager;
+import org.apache.kafka.queue.EventQueue.EarliestDeadlineFunction;
+import org.apache.kafka.queue.EventQueue;
+import org.apache.kafka.queue.KafkaEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+
+public final class QuorumController implements Controller {
+/**
+ * A builder class which creates the QuorumController.
+ */
+static public class Builder {
+private final int nodeId;
+private Time time = Time.SYSTEM;
+private String threadNamePrefix = null;
+private LogContext logContext = null;
+private Map configDefs = 
Collections.emptyMap();
+private MetaLogManager logManager = null;
+private Map supportedFeatures = 
Collections.emptyMap();
+private short defaultReplicationFactor = 3;
+private int defaultNumPartitions = 1;
+private ReplicaPlacementPolicy replicaPlacementPolicy =
+new SimpleReplicaPlacementPolicy(new Random());
+private long sessionTimeoutNs = TimeUnit.NANOSECONDS.convert(18, 
TimeUnit.SECONDS);
+
+public Builder(int nodeId) {
+this.nodeId = nodeId;
+}
+
+public B

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-16 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577214559



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java
##
@@ -0,0 +1,456 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.common.Endpoint;
+import org.apache.kafka.common.errors.DuplicateBrokerRegistrationException;
+import org.apache.kafka.common.errors.StaleBrokerEpochException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.RegisterBrokerRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistration;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureManager;
+import org.apache.kafka.metadata.VersionRange;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+
+public class ClusterControlManager {
+class ReadyBrokersFuture {
+private final CompletableFuture future;
+private final int minBrokers;
+
+ReadyBrokersFuture(CompletableFuture future, int minBrokers) {
+this.future = future;
+this.minBrokers = minBrokers;
+}
+
+boolean check() {
+int numUnfenced = 0;
+for (BrokerRegistration registration : 
brokerRegistrations.values()) {
+if (!registration.fenced()) {
+numUnfenced++;
+}
+if (numUnfenced >= minBrokers) {
+return true;
+}
+}
+return false;
+}
+}
+
+/**
+ * The SLF4J log context.
+ */
+private final LogContext logContext;
+
+/**
+ * The SLF4J log object.
+ */
+private final Logger log;
+
+/**
+ * The Kafka clock object to use.
+ */
+private final Time time;
+
+/**
+ * How long sessions should last, in nanoseconds.
+ */
+private final long sessionTimeoutNs;
+
+/**
+ * The replica placement policy to use.
+ */
+private final ReplicaPlacementPolicy placementPolicy;
+
+/**
+ * Maps broker IDs to broker registrations.
+ */
+private final TimelineHashMap 
brokerRegistrations;
+
+/**
+ * The broker heartbeat manager, or null if this controller is on standby.
+ */
+private BrokerHeartbeatManager heartbeatManager;
+
+/**
+ * A future which is completed as soon as we have the given number of 
brokers
+ * ready.
+ */
+private Optional readyBrokersFuture;
+
+ClusterControlManager(LogContext logContext,
+  Time time,
+  SnapshotRegistry snapshotRegistry,
+  long sessionTimeoutNs,
+  ReplicaPlacementPolicy placementPolicy) {
+this.logContext = logContext;
+this.log = logContext.logger(ClusterControlManager.class);
+this.time = time;
+this.sessionTimeoutNs = sessionTimeoutNs;
+this.placementPolicy = placementPolicy;
+this.brokerRegistrations = new TimelineHashMap<>(snapshotRegistry, 0);
+this.heartbeatManager = null;
+this.readyBrokersFuture = Optional.empty();
+}
+
+/**
+ * Transition thi

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-16 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577188202



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
##
@@ -0,0 +1,920 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.ApiException;
+import org.apache.kafka.common.errors.NotControllerException;
+import org.apache.kafka.common.errors.UnknownServerException;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.ConfigRecord;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.IsrChangeRecord;
+import org.apache.kafka.common.metadata.MetadataRecordType;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.QuotaRecord;
+import org.apache.kafka.common.metadata.RegisterBrokerRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.protocol.ApiMessage;
+import org.apache.kafka.common.quota.ClientQuotaAlteration;
+import org.apache.kafka.common.quota.ClientQuotaEntity;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureManager;
+import org.apache.kafka.metadata.VersionRange;
+import org.apache.kafka.metalog.MetaLogLeader;
+import org.apache.kafka.metalog.MetaLogListener;
+import org.apache.kafka.metalog.MetaLogManager;
+import org.apache.kafka.queue.EventQueue.EarliestDeadlineFunction;
+import org.apache.kafka.queue.EventQueue;
+import org.apache.kafka.queue.KafkaEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+
+public final class QuorumController implements Controller {
+/**
+ * A builder class which creates the QuorumController.
+ */
+static public class Builder {
+private final int nodeId;
+private Time time = Time.SYSTEM;
+private String threadNamePrefix = null;
+private LogContext logContext = null;
+private Map configDefs = 
Collections.emptyMap();
+private MetaLogManager logManager = null;
+private Map supportedFeatures = 
Collections.emptyMap();
+private short defaultReplicationFactor = 3;
+private int defaultNumPartitions = 1;
+private ReplicaPlacementPolicy replicaPlacementPolicy =
+new SimpleReplicaPlacementPolicy(new Random());
+private long sessionTimeoutNs = TimeUnit.NANOSECONDS.convert(18, 
TimeUnit.SECONDS);
+
+public Builder(int nodeId) {
+this.nodeId = nodeId;
+}
+
+public B

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-16 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577187004



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java
##
@@ -0,0 +1,456 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.common.Endpoint;
+import org.apache.kafka.common.errors.DuplicateBrokerRegistrationException;
+import org.apache.kafka.common.errors.StaleBrokerEpochException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.RegisterBrokerRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistration;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureManager;
+import org.apache.kafka.metadata.VersionRange;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+
+public class ClusterControlManager {
+class ReadyBrokersFuture {
+private final CompletableFuture future;
+private final int minBrokers;
+
+ReadyBrokersFuture(CompletableFuture future, int minBrokers) {
+this.future = future;
+this.minBrokers = minBrokers;
+}
+
+boolean check() {
+int numUnfenced = 0;
+for (BrokerRegistration registration : 
brokerRegistrations.values()) {
+if (!registration.fenced()) {
+numUnfenced++;
+}
+if (numUnfenced >= minBrokers) {
+return true;
+}
+}
+return false;
+}
+}
+
+/**
+ * The SLF4J log context.
+ */
+private final LogContext logContext;
+
+/**
+ * The SLF4J log object.
+ */
+private final Logger log;
+
+/**
+ * The Kafka clock object to use.
+ */
+private final Time time;
+
+/**
+ * How long sessions should last, in nanoseconds.
+ */
+private final long sessionTimeoutNs;
+
+/**
+ * The replica placement policy to use.
+ */
+private final ReplicaPlacementPolicy placementPolicy;
+
+/**
+ * Maps broker IDs to broker registrations.
+ */
+private final TimelineHashMap 
brokerRegistrations;
+
+/**
+ * The broker heartbeat manager, or null if this controller is on standby.
+ */
+private BrokerHeartbeatManager heartbeatManager;
+
+/**
+ * A future which is completed as soon as we have the given number of 
brokers
+ * ready.
+ */
+private Optional readyBrokersFuture;
+
+ClusterControlManager(LogContext logContext,
+  Time time,
+  SnapshotRegistry snapshotRegistry,
+  long sessionTimeoutNs,
+  ReplicaPlacementPolicy placementPolicy) {
+this.logContext = logContext;
+this.log = logContext.logger(ClusterControlManager.class);
+this.time = time;
+this.sessionTimeoutNs = sessionTimeoutNs;
+this.placementPolicy = placementPolicy;
+this.brokerRegistrations = new TimelineHashMap<>(snapshotRegistry, 0);
+this.heartbeatManager = null;
+this.readyBrokersFuture = Optional.empty();
+}
+
+/**
+ * Transition thi

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-16 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577187004



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java
##
@@ -0,0 +1,456 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.common.Endpoint;
+import org.apache.kafka.common.errors.DuplicateBrokerRegistrationException;
+import org.apache.kafka.common.errors.StaleBrokerEpochException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.RegisterBrokerRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistration;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureManager;
+import org.apache.kafka.metadata.VersionRange;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+
+public class ClusterControlManager {
+class ReadyBrokersFuture {
+private final CompletableFuture future;
+private final int minBrokers;
+
+ReadyBrokersFuture(CompletableFuture future, int minBrokers) {
+this.future = future;
+this.minBrokers = minBrokers;
+}
+
+boolean check() {
+int numUnfenced = 0;
+for (BrokerRegistration registration : 
brokerRegistrations.values()) {
+if (!registration.fenced()) {
+numUnfenced++;
+}
+if (numUnfenced >= minBrokers) {
+return true;
+}
+}
+return false;
+}
+}
+
+/**
+ * The SLF4J log context.
+ */
+private final LogContext logContext;
+
+/**
+ * The SLF4J log object.
+ */
+private final Logger log;
+
+/**
+ * The Kafka clock object to use.
+ */
+private final Time time;
+
+/**
+ * How long sessions should last, in nanoseconds.
+ */
+private final long sessionTimeoutNs;
+
+/**
+ * The replica placement policy to use.
+ */
+private final ReplicaPlacementPolicy placementPolicy;
+
+/**
+ * Maps broker IDs to broker registrations.
+ */
+private final TimelineHashMap 
brokerRegistrations;
+
+/**
+ * The broker heartbeat manager, or null if this controller is on standby.
+ */
+private BrokerHeartbeatManager heartbeatManager;
+
+/**
+ * A future which is completed as soon as we have the given number of 
brokers
+ * ready.
+ */
+private Optional readyBrokersFuture;
+
+ClusterControlManager(LogContext logContext,
+  Time time,
+  SnapshotRegistry snapshotRegistry,
+  long sessionTimeoutNs,
+  ReplicaPlacementPolicy placementPolicy) {
+this.logContext = logContext;
+this.log = logContext.logger(ClusterControlManager.class);
+this.time = time;
+this.sessionTimeoutNs = sessionTimeoutNs;
+this.placementPolicy = placementPolicy;
+this.brokerRegistrations = new TimelineHashMap<>(snapshotRegistry, 0);
+this.heartbeatManager = null;
+this.readyBrokersFuture = Optional.empty();
+}
+
+/**
+ * Transition thi

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-16 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577178197



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java
##
@@ -0,0 +1,456 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.common.Endpoint;
+import org.apache.kafka.common.errors.DuplicateBrokerRegistrationException;
+import org.apache.kafka.common.errors.StaleBrokerEpochException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.RegisterBrokerRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistration;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureManager;
+import org.apache.kafka.metadata.VersionRange;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+
+public class ClusterControlManager {
+class ReadyBrokersFuture {
+private final CompletableFuture future;
+private final int minBrokers;
+
+ReadyBrokersFuture(CompletableFuture future, int minBrokers) {
+this.future = future;
+this.minBrokers = minBrokers;
+}
+
+boolean check() {
+int numUnfenced = 0;
+for (BrokerRegistration registration : 
brokerRegistrations.values()) {
+if (!registration.fenced()) {
+numUnfenced++;
+}
+if (numUnfenced >= minBrokers) {
+return true;
+}
+}
+return false;
+}
+}
+
+/**
+ * The SLF4J log context.
+ */
+private final LogContext logContext;
+
+/**
+ * The SLF4J log object.
+ */
+private final Logger log;
+
+/**
+ * The Kafka clock object to use.
+ */
+private final Time time;
+
+/**
+ * How long sessions should last, in nanoseconds.
+ */
+private final long sessionTimeoutNs;
+
+/**
+ * The replica placement policy to use.
+ */
+private final ReplicaPlacementPolicy placementPolicy;
+
+/**
+ * Maps broker IDs to broker registrations.
+ */
+private final TimelineHashMap 
brokerRegistrations;
+
+/**
+ * The broker heartbeat manager, or null if this controller is on standby.
+ */
+private BrokerHeartbeatManager heartbeatManager;
+
+/**
+ * A future which is completed as soon as we have the given number of 
brokers
+ * ready.
+ */
+private Optional readyBrokersFuture;
+
+ClusterControlManager(LogContext logContext,
+  Time time,
+  SnapshotRegistry snapshotRegistry,
+  long sessionTimeoutNs,
+  ReplicaPlacementPolicy placementPolicy) {
+this.logContext = logContext;
+this.log = logContext.logger(ClusterControlManager.class);
+this.time = time;
+this.sessionTimeoutNs = sessionTimeoutNs;
+this.placementPolicy = placementPolicy;
+this.brokerRegistrations = new TimelineHashMap<>(snapshotRegistry, 0);
+this.heartbeatManager = null;
+this.readyBrokersFuture = Optional.empty();
+}
+
+/**
+ * Transition thi

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-16 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577176042



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
##
@@ -0,0 +1,712 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.ElectionType;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.errors.InvalidRequestException;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment;
+import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData.TopicPartitions;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.IsrChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.controller.BrokersToIsrs.TopicPartition;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Random;
+
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET;
+import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC;
+
+
+public class ReplicationControlManager {
+static class TopicControlInfo {
+private final Uuid id;
+private final TimelineHashMap parts;
+
+TopicControlInfo(SnapshotRegistry snapshotRegistry, Uuid id) {
+this.id = id;
+this.parts = new TimelineHashMap<>(snapshotRegistry, 0);
+}
+}
+
+static class PartitionControlInfo {
+private final int[] replicas;
+private final int[] isr;
+private final int[] removingReplicas;
+private final int[] addingReplicas;
+private final int leader;
+private final int leaderEpoch;
+private final int partitionEpoch;
+
+PartitionControlInfo(PartitionRecord record) {
+this(Replicas.toArray(record.replicas()),
+Replicas.toArray(record.isr()),
+Replicas.toArray(record.removingReplicas()),
+Replicas.toArray(record.addingReplicas()),
+record.leader(),
+record.leaderEpoch(),
+record.partitionEpoch());
+}
+
+PartitionControlInfo(int[] replicas, int[] isr, int[] removingReplicas,
+ int[] addingReplicas, int leader, int leaderEpoch,
+ int partitionEpoch) {
+this.replicas = replicas;
+this.isr = isr;
+this.removingRepli

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-16 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577176042



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
##
@@ -0,0 +1,712 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.ElectionType;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.errors.InvalidRequestException;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment;
+import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData.TopicPartitions;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.IsrChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.controller.BrokersToIsrs.TopicPartition;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Random;
+
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET;
+import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC;
+
+
+public class ReplicationControlManager {
+static class TopicControlInfo {
+private final Uuid id;
+private final TimelineHashMap parts;
+
+TopicControlInfo(SnapshotRegistry snapshotRegistry, Uuid id) {
+this.id = id;
+this.parts = new TimelineHashMap<>(snapshotRegistry, 0);
+}
+}
+
+static class PartitionControlInfo {
+private final int[] replicas;
+private final int[] isr;
+private final int[] removingReplicas;
+private final int[] addingReplicas;
+private final int leader;
+private final int leaderEpoch;
+private final int partitionEpoch;
+
+PartitionControlInfo(PartitionRecord record) {
+this(Replicas.toArray(record.replicas()),
+Replicas.toArray(record.isr()),
+Replicas.toArray(record.removingReplicas()),
+Replicas.toArray(record.addingReplicas()),
+record.leader(),
+record.leaderEpoch(),
+record.partitionEpoch());
+}
+
+PartitionControlInfo(int[] replicas, int[] isr, int[] removingReplicas,
+ int[] addingReplicas, int leader, int leaderEpoch,
+ int partitionEpoch) {
+this.replicas = replicas;
+this.isr = isr;
+this.removingRepli

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-16 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577167941



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java
##
@@ -0,0 +1,367 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.config.ConfigDef.ConfigKey;
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.common.config.ConfigResource.Type;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.metadata.ConfigRecord;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.APPEND;
+
+public class ConfigurationControlManager {
+private final Logger log;
+private final SnapshotRegistry snapshotRegistry;
+private final Map configDefs;
+private final TimelineHashMap> configData;
+
+ConfigurationControlManager(LogContext logContext,
+SnapshotRegistry snapshotRegistry,
+Map 
configDefs) {
+this.log = logContext.logger(ConfigurationControlManager.class);
+this.snapshotRegistry = snapshotRegistry;
+this.configDefs = configDefs;
+this.configData = new TimelineHashMap<>(snapshotRegistry, 0);
+}
+
+/**
+ * Determine the result of applying a batch of incremental configuration 
changes.  Note
+ * that this method does not change the contents of memory.  It just 
generates a
+ * result, that you can replay later if you wish using replay().
+ *
+ * Note that there can only be one result per ConfigResource.  So if you 
try to modify
+ * several keys and one modification fails, the whole ConfigKey fails and 
nothing gets
+ * changed.
+ *
+ * @param configChanges Maps each resource to a map from config keys to
+ *  operation data.
+ * @return  The result.
+ */
+ControllerResult> incrementalAlterConfigs(
+Map>> 
configChanges) {
+List outputRecords = new ArrayList<>();
+Map outputResults = new HashMap<>();
+for (Entry>> 
resourceEntry :
+configChanges.entrySet()) {
+incrementalAlterConfigResource(resourceEntry.getKey(),
+resourceEntry.getValue(),
+outputRecords,
+outputResults);
+}
+return new ControllerResult<>(outputRecords, outputResults);
+}
+
+private void incrementalAlterConfigResource(ConfigResource configResource,
+Map> keysToOps,
+List 
outputRecords,
+Map 
outputResults) {
+ApiError error = checkConfigResource(configResource);
+if (error.isFailure()) {
+outputResults.put(configResource, error);
+return;
+}
+List newRecords = new ArrayList<>();
+for (Entry> keysToOpsEntry : 
keysToOps.entrySet()) {
+String key = keysToOpsEntry.getKey();
+String currentValue = null;
+TimelineHashMap currentConfigs = 
configData.get(configResource);
+if (currentConfigs != null) {
+currentValue = currentConfigs.get(key);
+}
+String newValue = currentValue;
+Entry opTypeAndNewValue = 
keysToOpsEntry.getValue();
+OpType

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-16 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577167409



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
##
@@ -0,0 +1,712 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.ElectionType;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.errors.InvalidRequestException;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment;
+import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData.TopicPartitions;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.IsrChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.controller.BrokersToIsrs.TopicPartition;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Random;
+
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET;
+import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC;
+
+
+public class ReplicationControlManager {
+static class TopicControlInfo {
+private final Uuid id;
+private final TimelineHashMap parts;
+
+TopicControlInfo(SnapshotRegistry snapshotRegistry, Uuid id) {
+this.id = id;
+this.parts = new TimelineHashMap<>(snapshotRegistry, 0);
+}
+}
+
+static class PartitionControlInfo {
+private final int[] replicas;
+private final int[] isr;
+private final int[] removingReplicas;
+private final int[] addingReplicas;
+private final int leader;
+private final int leaderEpoch;
+private final int partitionEpoch;
+
+PartitionControlInfo(PartitionRecord record) {
+this(Replicas.toArray(record.replicas()),
+Replicas.toArray(record.isr()),
+Replicas.toArray(record.removingReplicas()),
+Replicas.toArray(record.addingReplicas()),
+record.leader(),
+record.leaderEpoch(),
+record.partitionEpoch());
+}
+
+PartitionControlInfo(int[] replicas, int[] isr, int[] removingReplicas,
+ int[] addingReplicas, int leader, int leaderEpoch,
+ int partitionEpoch) {
+this.replicas = replicas;
+this.isr = isr;
+this.removingRepli

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-16 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577163457



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
##
@@ -0,0 +1,712 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.ElectionType;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.errors.InvalidRequestException;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment;
+import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData.TopicPartitions;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.IsrChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.controller.BrokersToIsrs.TopicPartition;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Random;
+
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET;
+import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC;
+
+
+public class ReplicationControlManager {
+static class TopicControlInfo {
+private final Uuid id;
+private final TimelineHashMap parts;
+
+TopicControlInfo(SnapshotRegistry snapshotRegistry, Uuid id) {
+this.id = id;
+this.parts = new TimelineHashMap<>(snapshotRegistry, 0);
+}
+}
+
+static class PartitionControlInfo {
+private final int[] replicas;
+private final int[] isr;
+private final int[] removingReplicas;
+private final int[] addingReplicas;
+private final int leader;
+private final int leaderEpoch;
+private final int partitionEpoch;
+
+PartitionControlInfo(PartitionRecord record) {
+this(Replicas.toArray(record.replicas()),
+Replicas.toArray(record.isr()),
+Replicas.toArray(record.removingReplicas()),
+Replicas.toArray(record.addingReplicas()),
+record.leader(),
+record.leaderEpoch(),
+record.partitionEpoch());
+}
+
+PartitionControlInfo(int[] replicas, int[] isr, int[] removingReplicas,
+ int[] addingReplicas, int leader, int leaderEpoch,
+ int partitionEpoch) {
+this.replicas = replicas;
+this.isr = isr;
+this.removingRepli

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-16 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577117523



##
File path: metadata/src/main/java/org/apache/kafka/metalog/LocalLogManager.java
##
@@ -0,0 +1,379 @@
+/*
+ * 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 org.apache.kafka.metalog;
+
+import org.apache.kafka.common.protocol.ApiMessage;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.queue.EventQueue;
+import org.apache.kafka.queue.KafkaEventQueue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.TreeMap;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.stream.Collectors;
+
+
+/**
+ * The LocalLogManager is a test implementation that relies on the contents of 
memory.

Review comment:
   Yes, it is.  I will move it to the test directory.





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.

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




[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-16 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577117523



##
File path: metadata/src/main/java/org/apache/kafka/metalog/LocalLogManager.java
##
@@ -0,0 +1,379 @@
+/*
+ * 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 org.apache.kafka.metalog;
+
+import org.apache.kafka.common.protocol.ApiMessage;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.queue.EventQueue;
+import org.apache.kafka.queue.KafkaEventQueue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.TreeMap;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.stream.Collectors;
+
+
+/**
+ * The LocalLogManager is a test implementation that relies on the contents of 
memory.

Review comment:
   Yes, it is is.  I will move it to the test directory.





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.

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




[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-16 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577148107



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
##
@@ -0,0 +1,920 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.ApiException;
+import org.apache.kafka.common.errors.NotControllerException;
+import org.apache.kafka.common.errors.UnknownServerException;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.ConfigRecord;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.IsrChangeRecord;
+import org.apache.kafka.common.metadata.MetadataRecordType;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.QuotaRecord;
+import org.apache.kafka.common.metadata.RegisterBrokerRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.protocol.ApiMessage;
+import org.apache.kafka.common.quota.ClientQuotaAlteration;
+import org.apache.kafka.common.quota.ClientQuotaEntity;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureManager;
+import org.apache.kafka.metadata.VersionRange;
+import org.apache.kafka.metalog.MetaLogLeader;
+import org.apache.kafka.metalog.MetaLogListener;
+import org.apache.kafka.metalog.MetaLogManager;
+import org.apache.kafka.queue.EventQueue.EarliestDeadlineFunction;
+import org.apache.kafka.queue.EventQueue;
+import org.apache.kafka.queue.KafkaEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+
+public final class QuorumController implements Controller {
+/**
+ * A builder class which creates the QuorumController.
+ */
+static public class Builder {
+private final int nodeId;
+private Time time = Time.SYSTEM;
+private String threadNamePrefix = null;
+private LogContext logContext = null;
+private Map configDefs = 
Collections.emptyMap();
+private MetaLogManager logManager = null;
+private Map supportedFeatures = 
Collections.emptyMap();
+private short defaultReplicationFactor = 3;
+private int defaultNumPartitions = 1;
+private ReplicaPlacementPolicy replicaPlacementPolicy =
+new SimpleReplicaPlacementPolicy(new Random());
+private long sessionTimeoutNs = TimeUnit.NANOSECONDS.convert(18, 
TimeUnit.SECONDS);
+
+public Builder(int nodeId) {
+this.nodeId = nodeId;
+}
+
+public B

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-16 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577143791



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
##
@@ -0,0 +1,712 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.ElectionType;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.errors.InvalidRequestException;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment;
+import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData.TopicPartitions;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.IsrChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.controller.BrokersToIsrs.TopicPartition;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Random;
+
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET;
+import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC;
+
+
+public class ReplicationControlManager {
+static class TopicControlInfo {
+private final Uuid id;
+private final TimelineHashMap parts;
+
+TopicControlInfo(SnapshotRegistry snapshotRegistry, Uuid id) {
+this.id = id;
+this.parts = new TimelineHashMap<>(snapshotRegistry, 0);
+}
+}
+
+static class PartitionControlInfo {
+private final int[] replicas;
+private final int[] isr;
+private final int[] removingReplicas;
+private final int[] addingReplicas;
+private final int leader;
+private final int leaderEpoch;
+private final int partitionEpoch;
+
+PartitionControlInfo(PartitionRecord record) {
+this(Replicas.toArray(record.replicas()),
+Replicas.toArray(record.isr()),
+Replicas.toArray(record.removingReplicas()),
+Replicas.toArray(record.addingReplicas()),
+record.leader(),
+record.leaderEpoch(),
+record.partitionEpoch());
+}
+
+PartitionControlInfo(int[] replicas, int[] isr, int[] removingReplicas,
+ int[] addingReplicas, int leader, int leaderEpoch,
+ int partitionEpoch) {
+this.replicas = replicas;
+this.isr = isr;
+this.removingRepli

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-16 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577146421



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java
##
@@ -0,0 +1,456 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.common.Endpoint;
+import org.apache.kafka.common.errors.DuplicateBrokerRegistrationException;
+import org.apache.kafka.common.errors.StaleBrokerEpochException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.RegisterBrokerRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistration;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureManager;
+import org.apache.kafka.metadata.VersionRange;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+
+
+public class ClusterControlManager {
+class ReadyBrokersFuture {
+private final CompletableFuture future;
+private final int minBrokers;
+
+ReadyBrokersFuture(CompletableFuture future, int minBrokers) {
+this.future = future;
+this.minBrokers = minBrokers;
+}
+
+boolean check() {
+int numUnfenced = 0;
+for (BrokerRegistration registration : 
brokerRegistrations.values()) {
+if (!registration.fenced()) {
+numUnfenced++;
+}
+if (numUnfenced >= minBrokers) {
+return true;
+}
+}
+return false;
+}
+}
+
+/**
+ * The SLF4J log context.
+ */
+private final LogContext logContext;
+
+/**
+ * The SLF4J log object.
+ */
+private final Logger log;
+
+/**
+ * The Kafka clock object to use.
+ */
+private final Time time;
+
+/**
+ * How long sessions should last, in nanoseconds.
+ */
+private final long sessionTimeoutNs;
+
+/**
+ * The replica placement policy to use.
+ */
+private final ReplicaPlacementPolicy placementPolicy;
+
+/**
+ * Maps broker IDs to broker registrations.
+ */
+private final TimelineHashMap 
brokerRegistrations;
+
+/**
+ * The broker heartbeat manager, or null if this controller is on standby.
+ */
+private BrokerHeartbeatManager heartbeatManager;
+
+/**
+ * A future which is completed as soon as we have the given number of 
brokers
+ * ready.
+ */
+private Optional readyBrokersFuture;
+
+ClusterControlManager(LogContext logContext,
+  Time time,
+  SnapshotRegistry snapshotRegistry,
+  long sessionTimeoutNs,
+  ReplicaPlacementPolicy placementPolicy) {
+this.logContext = logContext;
+this.log = logContext.logger(ClusterControlManager.class);
+this.time = time;
+this.sessionTimeoutNs = sessionTimeoutNs;
+this.placementPolicy = placementPolicy;
+this.brokerRegistrations = new TimelineHashMap<>(snapshotRegistry, 0);
+this.heartbeatManager = null;
+this.readyBrokersFuture = Optional.empty();
+}
+
+/**
+ * Transition thi

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-16 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577143791



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
##
@@ -0,0 +1,712 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.ElectionType;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.errors.InvalidRequestException;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment;
+import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData.TopicPartitions;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.IsrChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.controller.BrokersToIsrs.TopicPartition;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Random;
+
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET;
+import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC;
+
+
+public class ReplicationControlManager {
+static class TopicControlInfo {
+private final Uuid id;
+private final TimelineHashMap parts;
+
+TopicControlInfo(SnapshotRegistry snapshotRegistry, Uuid id) {
+this.id = id;
+this.parts = new TimelineHashMap<>(snapshotRegistry, 0);
+}
+}
+
+static class PartitionControlInfo {
+private final int[] replicas;
+private final int[] isr;
+private final int[] removingReplicas;
+private final int[] addingReplicas;
+private final int leader;
+private final int leaderEpoch;
+private final int partitionEpoch;
+
+PartitionControlInfo(PartitionRecord record) {
+this(Replicas.toArray(record.replicas()),
+Replicas.toArray(record.isr()),
+Replicas.toArray(record.removingReplicas()),
+Replicas.toArray(record.addingReplicas()),
+record.leader(),
+record.leaderEpoch(),
+record.partitionEpoch());
+}
+
+PartitionControlInfo(int[] replicas, int[] isr, int[] removingReplicas,
+ int[] addingReplicas, int leader, int leaderEpoch,
+ int partitionEpoch) {
+this.replicas = replicas;
+this.isr = isr;
+this.removingRepli

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-16 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577136289



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
##
@@ -0,0 +1,920 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.ApiException;
+import org.apache.kafka.common.errors.NotControllerException;
+import org.apache.kafka.common.errors.UnknownServerException;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.ConfigRecord;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.IsrChangeRecord;
+import org.apache.kafka.common.metadata.MetadataRecordType;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.QuotaRecord;
+import org.apache.kafka.common.metadata.RegisterBrokerRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.protocol.ApiMessage;
+import org.apache.kafka.common.quota.ClientQuotaAlteration;
+import org.apache.kafka.common.quota.ClientQuotaEntity;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureManager;
+import org.apache.kafka.metadata.VersionRange;
+import org.apache.kafka.metalog.MetaLogLeader;
+import org.apache.kafka.metalog.MetaLogListener;
+import org.apache.kafka.metalog.MetaLogManager;
+import org.apache.kafka.queue.EventQueue.EarliestDeadlineFunction;
+import org.apache.kafka.queue.EventQueue;
+import org.apache.kafka.queue.KafkaEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+
+public final class QuorumController implements Controller {
+/**
+ * A builder class which creates the QuorumController.
+ */
+static public class Builder {
+private final int nodeId;
+private Time time = Time.SYSTEM;
+private String threadNamePrefix = null;
+private LogContext logContext = null;
+private Map configDefs = 
Collections.emptyMap();
+private MetaLogManager logManager = null;
+private Map supportedFeatures = 
Collections.emptyMap();
+private short defaultReplicationFactor = 3;
+private int defaultNumPartitions = 1;
+private ReplicaPlacementPolicy replicaPlacementPolicy =
+new SimpleReplicaPlacementPolicy(new Random());
+private long sessionTimeoutNs = TimeUnit.NANOSECONDS.convert(18, 
TimeUnit.SECONDS);
+
+public Builder(int nodeId) {
+this.nodeId = nodeId;
+}
+
+public B

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-16 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577135430



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
##
@@ -0,0 +1,712 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.ElectionType;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.errors.InvalidRequestException;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment;
+import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData.TopicPartitions;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.IsrChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.controller.BrokersToIsrs.TopicPartition;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Random;
+
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET;
+import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC;
+
+
+public class ReplicationControlManager {
+static class TopicControlInfo {
+private final Uuid id;
+private final TimelineHashMap parts;
+
+TopicControlInfo(SnapshotRegistry snapshotRegistry, Uuid id) {
+this.id = id;
+this.parts = new TimelineHashMap<>(snapshotRegistry, 0);
+}
+}
+
+static class PartitionControlInfo {
+private final int[] replicas;
+private final int[] isr;
+private final int[] removingReplicas;
+private final int[] addingReplicas;
+private final int leader;
+private final int leaderEpoch;
+private final int partitionEpoch;
+
+PartitionControlInfo(PartitionRecord record) {
+this(Replicas.toArray(record.replicas()),
+Replicas.toArray(record.isr()),
+Replicas.toArray(record.removingReplicas()),
+Replicas.toArray(record.addingReplicas()),
+record.leader(),
+record.leaderEpoch(),
+record.partitionEpoch());
+}
+
+PartitionControlInfo(int[] replicas, int[] isr, int[] removingReplicas,
+ int[] addingReplicas, int leader, int leaderEpoch,
+ int partitionEpoch) {
+this.replicas = replicas;
+this.isr = isr;
+this.removingRepli

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-16 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577127231



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java
##
@@ -0,0 +1,712 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.ElectionType;
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.errors.InvalidRequestException;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.internals.Topic;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment;
+import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic;
+import 
org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import 
org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData.TopicPartitions;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult;
+import 
org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.IsrChangeRecord;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.controller.BrokersToIsrs.TopicPartition;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.apache.kafka.timeline.TimelineHashMap;
+import org.slf4j.Logger;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Random;
+
+import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET;
+import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC;
+
+
+public class ReplicationControlManager {
+static class TopicControlInfo {
+private final Uuid id;
+private final TimelineHashMap parts;
+
+TopicControlInfo(SnapshotRegistry snapshotRegistry, Uuid id) {
+this.id = id;
+this.parts = new TimelineHashMap<>(snapshotRegistry, 0);
+}
+}
+
+static class PartitionControlInfo {
+private final int[] replicas;
+private final int[] isr;
+private final int[] removingReplicas;
+private final int[] addingReplicas;
+private final int leader;
+private final int leaderEpoch;
+private final int partitionEpoch;
+
+PartitionControlInfo(PartitionRecord record) {
+this(Replicas.toArray(record.replicas()),
+Replicas.toArray(record.isr()),
+Replicas.toArray(record.removingReplicas()),
+Replicas.toArray(record.addingReplicas()),
+record.leader(),
+record.leaderEpoch(),
+record.partitionEpoch());
+}
+
+PartitionControlInfo(int[] replicas, int[] isr, int[] removingReplicas,
+ int[] addingReplicas, int leader, int leaderEpoch,
+ int partitionEpoch) {
+this.replicas = replicas;
+this.isr = isr;
+this.removingRepli

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-16 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577125115



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/QuorumController.java
##
@@ -0,0 +1,920 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.clients.admin.AlterConfigOp.OpType;
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.errors.ApiException;
+import org.apache.kafka.common.errors.NotControllerException;
+import org.apache.kafka.common.errors.UnknownServerException;
+import org.apache.kafka.common.message.AlterIsrRequestData;
+import org.apache.kafka.common.message.AlterIsrResponseData;
+import org.apache.kafka.common.message.BrokerHeartbeatRequestData;
+import org.apache.kafka.common.message.BrokerRegistrationRequestData;
+import org.apache.kafka.common.message.CreateTopicsRequestData;
+import org.apache.kafka.common.message.CreateTopicsResponseData;
+import org.apache.kafka.common.message.ElectLeadersRequestData;
+import org.apache.kafka.common.message.ElectLeadersResponseData;
+import org.apache.kafka.common.metadata.ConfigRecord;
+import org.apache.kafka.common.metadata.FenceBrokerRecord;
+import org.apache.kafka.common.metadata.IsrChangeRecord;
+import org.apache.kafka.common.metadata.MetadataRecordType;
+import org.apache.kafka.common.metadata.PartitionRecord;
+import org.apache.kafka.common.metadata.QuotaRecord;
+import org.apache.kafka.common.metadata.RegisterBrokerRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.common.metadata.UnfenceBrokerRecord;
+import org.apache.kafka.common.metadata.UnregisterBrokerRecord;
+import org.apache.kafka.common.protocol.ApiMessage;
+import org.apache.kafka.common.quota.ClientQuotaAlteration;
+import org.apache.kafka.common.quota.ClientQuotaEntity;
+import org.apache.kafka.common.requests.ApiError;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.metadata.BrokerHeartbeatReply;
+import org.apache.kafka.metadata.BrokerRegistrationReply;
+import org.apache.kafka.metadata.FeatureManager;
+import org.apache.kafka.metadata.VersionRange;
+import org.apache.kafka.metalog.MetaLogLeader;
+import org.apache.kafka.metalog.MetaLogListener;
+import org.apache.kafka.metalog.MetaLogManager;
+import org.apache.kafka.queue.EventQueue.EarliestDeadlineFunction;
+import org.apache.kafka.queue.EventQueue;
+import org.apache.kafka.queue.KafkaEventQueue;
+import org.apache.kafka.timeline.SnapshotRegistry;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+
+public final class QuorumController implements Controller {
+/**
+ * A builder class which creates the QuorumController.
+ */
+static public class Builder {
+private final int nodeId;
+private Time time = Time.SYSTEM;
+private String threadNamePrefix = null;
+private LogContext logContext = null;
+private Map configDefs = 
Collections.emptyMap();
+private MetaLogManager logManager = null;
+private Map supportedFeatures = 
Collections.emptyMap();
+private short defaultReplicationFactor = 3;
+private int defaultNumPartitions = 1;
+private ReplicaPlacementPolicy replicaPlacementPolicy =
+new SimpleReplicaPlacementPolicy(new Random());
+private long sessionTimeoutNs = TimeUnit.NANOSECONDS.convert(18, 
TimeUnit.SECONDS);
+
+public Builder(int nodeId) {
+this.nodeId = nodeId;
+}
+
+public B

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-16 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r577117523



##
File path: metadata/src/main/java/org/apache/kafka/metalog/LocalLogManager.java
##
@@ -0,0 +1,379 @@
+/*
+ * 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 org.apache.kafka.metalog;
+
+import org.apache.kafka.common.protocol.ApiMessage;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.ApiMessageAndVersion;
+import org.apache.kafka.queue.EventQueue;
+import org.apache.kafka.queue.KafkaEventQueue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.TreeMap;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.stream.Collectors;
+
+
+/**
+ * The LocalLogManager is a test implementation that relies on the contents of 
memory.

Review comment:
   Yes.  I have moved it to the test directory.





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.

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




[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-09 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r573256853



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java
##
@@ -0,0 +1,491 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.UsableBroker;
+import org.slf4j.Logger;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Optional;
+import java.util.TreeSet;
+import java.util.function.Function;
+
+
+/**
+ * The BrokerHeartbeatManager manages all the soft state associated with 
broker heartbeats.
+ * Soft state is state which does not appear in the metadata log.  This state 
includes
+ * things like the last time each broker sent us a heartbeat, and whether the 
broker is
+ * trying to perform a controlled shutdown.
+ *
+ * Only the active controller has a BrokerHeartbeatManager, since only the 
active
+ * controller handles broker heartbeats.  Standby controllers will create a 
heartbeat
+ * manager as part of the process of activating.  This design minimizes the 
size of the
+ * metadata partition by excluding heartbeats from it.  However, it does mean 
that after
+ * a controller failover, we may take some extra time to fence brokers, since 
the new
+ * active controller does not know when the last heartbeats were received from 
each.
+ */
+public class BrokerHeartbeatManager {
+static class BrokerHeartbeatState {
+/**
+ * The broker ID.
+ */
+private final int id;
+
+/**
+ * The last time we received a heartbeat from this broker, in 
monotonic nanoseconds.
+ * When this field is updated, we also may have to update the broker's 
position in
+ * the unfenced list.
+ */
+long lastContactNs;
+
+/**
+ * The last metadata offset which this broker reported.  When this 
field is updated,
+ * we may also have to update the broker's position in the active set.
+ */
+long metadataOffset;
+
+/**
+ * The offset at which the broker should complete its controlled 
shutdown, or -1
+ * if the broker is not performing a controlled shutdown.  When this 
field is
+ * updated, we also have to update the broker's position in the 
shuttingDown set.
+ */
+private long shutdownOffset;
+
+/**
+ * The previous entry in the unfenced list, or null if the broker is 
not in that list.
+ */
+private BrokerHeartbeatState prev;
+
+/**
+ * The next entry in the unfenced list, or null if the broker is not 
in that list.
+ */
+private BrokerHeartbeatState next;
+
+BrokerHeartbeatState(int id) {
+this.id = id;
+this.lastContactNs = 0;
+this.prev = null;
+this.next = null;
+this.metadataOffset = -1;
+this.shutdownOffset = -1;
+}
+
+/**
+ * Returns the broker ID.
+ */
+int id() {
+return id;
+}
+
+/**
+ * Returns true only if the broker is fenced.
+ */
+boolean fenced() {
+return prev == null;
+}
+
+/**
+ * Returns true only if the broker is in controlled shutdown state.
+ */
+boolean shuttingDown() {
+return shutdownOffset >= 0;
+}
+}
+
+static class MetadataOffsetComparator implements 
Comparator {
+static final MetadataOffsetComparator INSTANCE = new 
MetadataOffsetComparator();
+
+@Override
+public int compare(BrokerHeartbeatState a, BrokerHeartbeatState b) {
+if (a.metadataOffset < b.metadataOffset) {
+return -1;
+} else if (a.metadataOffset > b.metadataOffset) {
+return 1;
+} else if (

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-09 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r573256612



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java
##
@@ -0,0 +1,491 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.UsableBroker;
+import org.slf4j.Logger;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Optional;
+import java.util.TreeSet;
+import java.util.function.Function;
+
+
+/**
+ * The BrokerHeartbeatManager manages all the soft state associated with 
broker heartbeats.
+ * Soft state is state which does not appear in the metadata log.  This state 
includes
+ * things like the last time each broker sent us a heartbeat, and whether the 
broker is
+ * trying to perform a controlled shutdown.
+ *
+ * Only the active controller has a BrokerHeartbeatManager, since only the 
active
+ * controller handles broker heartbeats.  Standby controllers will create a 
heartbeat
+ * manager as part of the process of activating.  This design minimizes the 
size of the
+ * metadata partition by excluding heartbeats from it.  However, it does mean 
that after
+ * a controller failover, we may take some extra time to fence brokers, since 
the new
+ * active controller does not know when the last heartbeats were received from 
each.
+ */
+public class BrokerHeartbeatManager {
+static class BrokerHeartbeatState {
+/**
+ * The broker ID.
+ */
+private final int id;
+
+/**
+ * The last time we received a heartbeat from this broker, in 
monotonic nanoseconds.
+ * When this field is updated, we also may have to update the broker's 
position in
+ * the unfenced list.
+ */
+long lastContactNs;
+
+/**
+ * The last metadata offset which this broker reported.  When this 
field is updated,
+ * we may also have to update the broker's position in the active set.
+ */
+long metadataOffset;
+
+/**
+ * The offset at which the broker should complete its controlled 
shutdown, or -1
+ * if the broker is not performing a controlled shutdown.  When this 
field is
+ * updated, we also have to update the broker's position in the 
shuttingDown set.
+ */
+private long shutdownOffset;
+
+/**
+ * The previous entry in the unfenced list, or null if the broker is 
not in that list.
+ */
+private BrokerHeartbeatState prev;
+
+/**
+ * The next entry in the unfenced list, or null if the broker is not 
in that list.
+ */
+private BrokerHeartbeatState next;
+
+BrokerHeartbeatState(int id) {
+this.id = id;
+this.lastContactNs = 0;
+this.prev = null;
+this.next = null;
+this.metadataOffset = -1;
+this.shutdownOffset = -1;
+}
+
+/**
+ * Returns the broker ID.
+ */
+int id() {
+return id;
+}
+
+/**
+ * Returns true only if the broker is fenced.
+ */
+boolean fenced() {
+return prev == null;
+}
+
+/**
+ * Returns true only if the broker is in controlled shutdown state.
+ */
+boolean shuttingDown() {
+return shutdownOffset >= 0;
+}
+}
+
+static class MetadataOffsetComparator implements 
Comparator {
+static final MetadataOffsetComparator INSTANCE = new 
MetadataOffsetComparator();
+
+@Override
+public int compare(BrokerHeartbeatState a, BrokerHeartbeatState b) {
+if (a.metadataOffset < b.metadataOffset) {
+return -1;
+} else if (a.metadataOffset > b.metadataOffset) {
+return 1;
+} else if (

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-09 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r573256321



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java
##
@@ -0,0 +1,491 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.UsableBroker;
+import org.slf4j.Logger;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Optional;
+import java.util.TreeSet;
+import java.util.function.Function;
+
+
+/**
+ * The BrokerHeartbeatManager manages all the soft state associated with 
broker heartbeats.
+ * Soft state is state which does not appear in the metadata log.  This state 
includes
+ * things like the last time each broker sent us a heartbeat, and whether the 
broker is
+ * trying to perform a controlled shutdown.
+ *
+ * Only the active controller has a BrokerHeartbeatManager, since only the 
active
+ * controller handles broker heartbeats.  Standby controllers will create a 
heartbeat
+ * manager as part of the process of activating.  This design minimizes the 
size of the
+ * metadata partition by excluding heartbeats from it.  However, it does mean 
that after
+ * a controller failover, we may take some extra time to fence brokers, since 
the new
+ * active controller does not know when the last heartbeats were received from 
each.
+ */
+public class BrokerHeartbeatManager {
+static class BrokerHeartbeatState {
+/**
+ * The broker ID.
+ */
+private final int id;
+
+/**
+ * The last time we received a heartbeat from this broker, in 
monotonic nanoseconds.
+ * When this field is updated, we also may have to update the broker's 
position in
+ * the unfenced list.
+ */
+long lastContactNs;
+
+/**
+ * The last metadata offset which this broker reported.  When this 
field is updated,
+ * we may also have to update the broker's position in the active set.
+ */
+long metadataOffset;
+
+/**
+ * The offset at which the broker should complete its controlled 
shutdown, or -1
+ * if the broker is not performing a controlled shutdown.  When this 
field is
+ * updated, we also have to update the broker's position in the 
shuttingDown set.
+ */
+private long shutdownOffset;
+
+/**
+ * The previous entry in the unfenced list, or null if the broker is 
not in that list.
+ */
+private BrokerHeartbeatState prev;
+
+/**
+ * The next entry in the unfenced list, or null if the broker is not 
in that list.
+ */
+private BrokerHeartbeatState next;
+
+BrokerHeartbeatState(int id) {
+this.id = id;
+this.lastContactNs = 0;
+this.prev = null;
+this.next = null;
+this.metadataOffset = -1;
+this.shutdownOffset = -1;
+}
+
+/**
+ * Returns the broker ID.
+ */
+int id() {
+return id;
+}
+
+/**
+ * Returns true only if the broker is fenced.
+ */
+boolean fenced() {
+return prev == null;
+}
+
+/**
+ * Returns true only if the broker is in controlled shutdown state.
+ */
+boolean shuttingDown() {
+return shutdownOffset >= 0;
+}
+}
+
+static class MetadataOffsetComparator implements 
Comparator {
+static final MetadataOffsetComparator INSTANCE = new 
MetadataOffsetComparator();
+
+@Override
+public int compare(BrokerHeartbeatState a, BrokerHeartbeatState b) {
+if (a.metadataOffset < b.metadataOffset) {
+return -1;
+} else if (a.metadataOffset > b.metadataOffset) {
+return 1;
+} else if (

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-09 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r573254808



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java
##
@@ -0,0 +1,491 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.UsableBroker;
+import org.slf4j.Logger;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Optional;
+import java.util.TreeSet;
+import java.util.function.Function;
+
+
+/**
+ * The BrokerHeartbeatManager manages all the soft state associated with 
broker heartbeats.
+ * Soft state is state which does not appear in the metadata log.  This state 
includes
+ * things like the last time each broker sent us a heartbeat, and whether the 
broker is
+ * trying to perform a controlled shutdown.
+ *
+ * Only the active controller has a BrokerHeartbeatManager, since only the 
active
+ * controller handles broker heartbeats.  Standby controllers will create a 
heartbeat
+ * manager as part of the process of activating.  This design minimizes the 
size of the
+ * metadata partition by excluding heartbeats from it.  However, it does mean 
that after
+ * a controller failover, we may take some extra time to fence brokers, since 
the new
+ * active controller does not know when the last heartbeats were received from 
each.
+ */
+public class BrokerHeartbeatManager {
+static class BrokerHeartbeatState {
+/**
+ * The broker ID.
+ */
+private final int id;
+
+/**
+ * The last time we received a heartbeat from this broker, in 
monotonic nanoseconds.
+ * When this field is updated, we also may have to update the broker's 
position in
+ * the unfenced list.
+ */
+long lastContactNs;
+
+/**
+ * The last metadata offset which this broker reported.  When this 
field is updated,
+ * we may also have to update the broker's position in the active set.
+ */
+long metadataOffset;
+
+/**
+ * The offset at which the broker should complete its controlled 
shutdown, or -1
+ * if the broker is not performing a controlled shutdown.  When this 
field is
+ * updated, we also have to update the broker's position in the 
shuttingDown set.
+ */
+private long shutdownOffset;
+
+/**
+ * The previous entry in the unfenced list, or null if the broker is 
not in that list.
+ */
+private BrokerHeartbeatState prev;
+
+/**
+ * The next entry in the unfenced list, or null if the broker is not 
in that list.
+ */
+private BrokerHeartbeatState next;
+
+BrokerHeartbeatState(int id) {
+this.id = id;
+this.lastContactNs = 0;
+this.prev = null;
+this.next = null;
+this.metadataOffset = -1;
+this.shutdownOffset = -1;
+}
+
+/**
+ * Returns the broker ID.
+ */
+int id() {
+return id;
+}
+
+/**
+ * Returns true only if the broker is fenced.
+ */
+boolean fenced() {
+return prev == null;
+}
+
+/**
+ * Returns true only if the broker is in controlled shutdown state.
+ */
+boolean shuttingDown() {
+return shutdownOffset >= 0;
+}
+}
+
+static class MetadataOffsetComparator implements 
Comparator {
+static final MetadataOffsetComparator INSTANCE = new 
MetadataOffsetComparator();
+
+@Override
+public int compare(BrokerHeartbeatState a, BrokerHeartbeatState b) {
+if (a.metadataOffset < b.metadataOffset) {
+return -1;
+} else if (a.metadataOffset > b.metadataOffset) {
+return 1;
+} else if (

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-09 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r573238990



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java
##
@@ -0,0 +1,491 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.UsableBroker;
+import org.slf4j.Logger;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Optional;
+import java.util.TreeSet;
+import java.util.function.Function;
+
+
+/**
+ * The BrokerHeartbeatManager manages all the soft state associated with 
broker heartbeats.
+ * Soft state is state which does not appear in the metadata log.  This state 
includes
+ * things like the last time each broker sent us a heartbeat, and whether the 
broker is
+ * trying to perform a controlled shutdown.
+ *
+ * Only the active controller has a BrokerHeartbeatManager, since only the 
active
+ * controller handles broker heartbeats.  Standby controllers will create a 
heartbeat
+ * manager as part of the process of activating.  This design minimizes the 
size of the
+ * metadata partition by excluding heartbeats from it.  However, it does mean 
that after
+ * a controller failover, we may take some extra time to fence brokers, since 
the new
+ * active controller does not know when the last heartbeats were received from 
each.
+ */
+public class BrokerHeartbeatManager {
+static class BrokerHeartbeatState {
+/**
+ * The broker ID.
+ */
+private final int id;
+
+/**
+ * The last time we received a heartbeat from this broker, in 
monotonic nanoseconds.
+ * When this field is updated, we also may have to update the broker's 
position in
+ * the unfenced list.
+ */
+long lastContactNs;
+
+/**
+ * The last metadata offset which this broker reported.  When this 
field is updated,
+ * we may also have to update the broker's position in the active set.
+ */
+long metadataOffset;
+
+/**
+ * The offset at which the broker should complete its controlled 
shutdown, or -1
+ * if the broker is not performing a controlled shutdown.  When this 
field is
+ * updated, we also have to update the broker's position in the 
shuttingDown set.
+ */
+private long shutdownOffset;
+
+/**
+ * The previous entry in the unfenced list, or null if the broker is 
not in that list.
+ */
+private BrokerHeartbeatState prev;
+
+/**
+ * The next entry in the unfenced list, or null if the broker is not 
in that list.
+ */
+private BrokerHeartbeatState next;
+
+BrokerHeartbeatState(int id) {
+this.id = id;
+this.lastContactNs = 0;
+this.prev = null;
+this.next = null;
+this.metadataOffset = -1;
+this.shutdownOffset = -1;
+}
+
+/**
+ * Returns the broker ID.
+ */
+int id() {
+return id;
+}
+
+/**
+ * Returns true only if the broker is fenced.
+ */
+boolean fenced() {
+return prev == null;
+}
+
+/**
+ * Returns true only if the broker is in controlled shutdown state.
+ */
+boolean shuttingDown() {
+return shutdownOffset >= 0;
+}
+}
+
+static class MetadataOffsetComparator implements 
Comparator {
+static final MetadataOffsetComparator INSTANCE = new 
MetadataOffsetComparator();
+
+@Override
+public int compare(BrokerHeartbeatState a, BrokerHeartbeatState b) {
+if (a.metadataOffset < b.metadataOffset) {
+return -1;
+} else if (a.metadataOffset > b.metadataOffset) {
+return 1;
+} else if (

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-09 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r573235373



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java
##
@@ -0,0 +1,491 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.UsableBroker;
+import org.slf4j.Logger;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Optional;
+import java.util.TreeSet;
+import java.util.function.Function;
+
+
+/**
+ * The BrokerHeartbeatManager manages all the soft state associated with 
broker heartbeats.
+ * Soft state is state which does not appear in the metadata log.  This state 
includes
+ * things like the last time each broker sent us a heartbeat, and whether the 
broker is
+ * trying to perform a controlled shutdown.
+ *
+ * Only the active controller has a BrokerHeartbeatManager, since only the 
active
+ * controller handles broker heartbeats.  Standby controllers will create a 
heartbeat
+ * manager as part of the process of activating.  This design minimizes the 
size of the
+ * metadata partition by excluding heartbeats from it.  However, it does mean 
that after
+ * a controller failover, we may take some extra time to fence brokers, since 
the new
+ * active controller does not know when the last heartbeats were received from 
each.
+ */
+public class BrokerHeartbeatManager {
+static class BrokerHeartbeatState {
+/**
+ * The broker ID.
+ */
+private final int id;
+
+/**
+ * The last time we received a heartbeat from this broker, in 
monotonic nanoseconds.
+ * When this field is updated, we also may have to update the broker's 
position in
+ * the unfenced list.
+ */
+long lastContactNs;
+
+/**
+ * The last metadata offset which this broker reported.  When this 
field is updated,
+ * we may also have to update the broker's position in the active set.
+ */
+long metadataOffset;
+
+/**
+ * The offset at which the broker should complete its controlled 
shutdown, or -1
+ * if the broker is not performing a controlled shutdown.  When this 
field is
+ * updated, we also have to update the broker's position in the 
shuttingDown set.
+ */
+private long shutdownOffset;
+
+/**
+ * The previous entry in the unfenced list, or null if the broker is 
not in that list.
+ */
+private BrokerHeartbeatState prev;
+
+/**
+ * The next entry in the unfenced list, or null if the broker is not 
in that list.
+ */
+private BrokerHeartbeatState next;
+
+BrokerHeartbeatState(int id) {
+this.id = id;
+this.lastContactNs = 0;
+this.prev = null;
+this.next = null;
+this.metadataOffset = -1;
+this.shutdownOffset = -1;
+}
+
+/**
+ * Returns the broker ID.
+ */
+int id() {
+return id;
+}
+
+/**
+ * Returns true only if the broker is fenced.
+ */
+boolean fenced() {
+return prev == null;
+}
+
+/**
+ * Returns true only if the broker is in controlled shutdown state.
+ */
+boolean shuttingDown() {
+return shutdownOffset >= 0;
+}
+}
+
+static class MetadataOffsetComparator implements 
Comparator {
+static final MetadataOffsetComparator INSTANCE = new 
MetadataOffsetComparator();
+
+@Override
+public int compare(BrokerHeartbeatState a, BrokerHeartbeatState b) {
+if (a.metadataOffset < b.metadataOffset) {
+return -1;
+} else if (a.metadataOffset > b.metadataOffset) {
+return 1;
+} else if (

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-09 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r573236986



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java
##
@@ -0,0 +1,491 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.UsableBroker;
+import org.slf4j.Logger;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Optional;
+import java.util.TreeSet;
+import java.util.function.Function;
+
+
+/**
+ * The BrokerHeartbeatManager manages all the soft state associated with 
broker heartbeats.
+ * Soft state is state which does not appear in the metadata log.  This state 
includes
+ * things like the last time each broker sent us a heartbeat, and whether the 
broker is
+ * trying to perform a controlled shutdown.
+ *
+ * Only the active controller has a BrokerHeartbeatManager, since only the 
active
+ * controller handles broker heartbeats.  Standby controllers will create a 
heartbeat
+ * manager as part of the process of activating.  This design minimizes the 
size of the
+ * metadata partition by excluding heartbeats from it.  However, it does mean 
that after
+ * a controller failover, we may take some extra time to fence brokers, since 
the new
+ * active controller does not know when the last heartbeats were received from 
each.
+ */
+public class BrokerHeartbeatManager {
+static class BrokerHeartbeatState {
+/**
+ * The broker ID.
+ */
+private final int id;
+
+/**
+ * The last time we received a heartbeat from this broker, in 
monotonic nanoseconds.
+ * When this field is updated, we also may have to update the broker's 
position in
+ * the unfenced list.
+ */
+long lastContactNs;
+
+/**
+ * The last metadata offset which this broker reported.  When this 
field is updated,
+ * we may also have to update the broker's position in the active set.
+ */
+long metadataOffset;
+
+/**
+ * The offset at which the broker should complete its controlled 
shutdown, or -1
+ * if the broker is not performing a controlled shutdown.  When this 
field is
+ * updated, we also have to update the broker's position in the 
shuttingDown set.
+ */
+private long shutdownOffset;
+
+/**
+ * The previous entry in the unfenced list, or null if the broker is 
not in that list.
+ */
+private BrokerHeartbeatState prev;
+
+/**
+ * The next entry in the unfenced list, or null if the broker is not 
in that list.
+ */
+private BrokerHeartbeatState next;
+
+BrokerHeartbeatState(int id) {
+this.id = id;
+this.lastContactNs = 0;
+this.prev = null;
+this.next = null;
+this.metadataOffset = -1;
+this.shutdownOffset = -1;
+}
+
+/**
+ * Returns the broker ID.
+ */
+int id() {
+return id;
+}
+
+/**
+ * Returns true only if the broker is fenced.
+ */
+boolean fenced() {
+return prev == null;
+}
+
+/**
+ * Returns true only if the broker is in controlled shutdown state.
+ */
+boolean shuttingDown() {
+return shutdownOffset >= 0;
+}
+}
+
+static class MetadataOffsetComparator implements 
Comparator {
+static final MetadataOffsetComparator INSTANCE = new 
MetadataOffsetComparator();
+
+@Override
+public int compare(BrokerHeartbeatState a, BrokerHeartbeatState b) {
+if (a.metadataOffset < b.metadataOffset) {
+return -1;
+} else if (a.metadataOffset > b.metadataOffset) {
+return 1;
+} else if (

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-09 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r573236986



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java
##
@@ -0,0 +1,491 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.UsableBroker;
+import org.slf4j.Logger;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Optional;
+import java.util.TreeSet;
+import java.util.function.Function;
+
+
+/**
+ * The BrokerHeartbeatManager manages all the soft state associated with 
broker heartbeats.
+ * Soft state is state which does not appear in the metadata log.  This state 
includes
+ * things like the last time each broker sent us a heartbeat, and whether the 
broker is
+ * trying to perform a controlled shutdown.
+ *
+ * Only the active controller has a BrokerHeartbeatManager, since only the 
active
+ * controller handles broker heartbeats.  Standby controllers will create a 
heartbeat
+ * manager as part of the process of activating.  This design minimizes the 
size of the
+ * metadata partition by excluding heartbeats from it.  However, it does mean 
that after
+ * a controller failover, we may take some extra time to fence brokers, since 
the new
+ * active controller does not know when the last heartbeats were received from 
each.
+ */
+public class BrokerHeartbeatManager {
+static class BrokerHeartbeatState {
+/**
+ * The broker ID.
+ */
+private final int id;
+
+/**
+ * The last time we received a heartbeat from this broker, in 
monotonic nanoseconds.
+ * When this field is updated, we also may have to update the broker's 
position in
+ * the unfenced list.
+ */
+long lastContactNs;
+
+/**
+ * The last metadata offset which this broker reported.  When this 
field is updated,
+ * we may also have to update the broker's position in the active set.
+ */
+long metadataOffset;
+
+/**
+ * The offset at which the broker should complete its controlled 
shutdown, or -1
+ * if the broker is not performing a controlled shutdown.  When this 
field is
+ * updated, we also have to update the broker's position in the 
shuttingDown set.
+ */
+private long shutdownOffset;
+
+/**
+ * The previous entry in the unfenced list, or null if the broker is 
not in that list.
+ */
+private BrokerHeartbeatState prev;
+
+/**
+ * The next entry in the unfenced list, or null if the broker is not 
in that list.
+ */
+private BrokerHeartbeatState next;
+
+BrokerHeartbeatState(int id) {
+this.id = id;
+this.lastContactNs = 0;
+this.prev = null;
+this.next = null;
+this.metadataOffset = -1;
+this.shutdownOffset = -1;
+}
+
+/**
+ * Returns the broker ID.
+ */
+int id() {
+return id;
+}
+
+/**
+ * Returns true only if the broker is fenced.
+ */
+boolean fenced() {
+return prev == null;
+}
+
+/**
+ * Returns true only if the broker is in controlled shutdown state.
+ */
+boolean shuttingDown() {
+return shutdownOffset >= 0;
+}
+}
+
+static class MetadataOffsetComparator implements 
Comparator {
+static final MetadataOffsetComparator INSTANCE = new 
MetadataOffsetComparator();
+
+@Override
+public int compare(BrokerHeartbeatState a, BrokerHeartbeatState b) {
+if (a.metadataOffset < b.metadataOffset) {
+return -1;
+} else if (a.metadataOffset > b.metadataOffset) {
+return 1;
+} else if (

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-09 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r573235373



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java
##
@@ -0,0 +1,491 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.UsableBroker;
+import org.slf4j.Logger;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Optional;
+import java.util.TreeSet;
+import java.util.function.Function;
+
+
+/**
+ * The BrokerHeartbeatManager manages all the soft state associated with 
broker heartbeats.
+ * Soft state is state which does not appear in the metadata log.  This state 
includes
+ * things like the last time each broker sent us a heartbeat, and whether the 
broker is
+ * trying to perform a controlled shutdown.
+ *
+ * Only the active controller has a BrokerHeartbeatManager, since only the 
active
+ * controller handles broker heartbeats.  Standby controllers will create a 
heartbeat
+ * manager as part of the process of activating.  This design minimizes the 
size of the
+ * metadata partition by excluding heartbeats from it.  However, it does mean 
that after
+ * a controller failover, we may take some extra time to fence brokers, since 
the new
+ * active controller does not know when the last heartbeats were received from 
each.
+ */
+public class BrokerHeartbeatManager {
+static class BrokerHeartbeatState {
+/**
+ * The broker ID.
+ */
+private final int id;
+
+/**
+ * The last time we received a heartbeat from this broker, in 
monotonic nanoseconds.
+ * When this field is updated, we also may have to update the broker's 
position in
+ * the unfenced list.
+ */
+long lastContactNs;
+
+/**
+ * The last metadata offset which this broker reported.  When this 
field is updated,
+ * we may also have to update the broker's position in the active set.
+ */
+long metadataOffset;
+
+/**
+ * The offset at which the broker should complete its controlled 
shutdown, or -1
+ * if the broker is not performing a controlled shutdown.  When this 
field is
+ * updated, we also have to update the broker's position in the 
shuttingDown set.
+ */
+private long shutdownOffset;
+
+/**
+ * The previous entry in the unfenced list, or null if the broker is 
not in that list.
+ */
+private BrokerHeartbeatState prev;
+
+/**
+ * The next entry in the unfenced list, or null if the broker is not 
in that list.
+ */
+private BrokerHeartbeatState next;
+
+BrokerHeartbeatState(int id) {
+this.id = id;
+this.lastContactNs = 0;
+this.prev = null;
+this.next = null;
+this.metadataOffset = -1;
+this.shutdownOffset = -1;
+}
+
+/**
+ * Returns the broker ID.
+ */
+int id() {
+return id;
+}
+
+/**
+ * Returns true only if the broker is fenced.
+ */
+boolean fenced() {
+return prev == null;
+}
+
+/**
+ * Returns true only if the broker is in controlled shutdown state.
+ */
+boolean shuttingDown() {
+return shutdownOffset >= 0;
+}
+}
+
+static class MetadataOffsetComparator implements 
Comparator {
+static final MetadataOffsetComparator INSTANCE = new 
MetadataOffsetComparator();
+
+@Override
+public int compare(BrokerHeartbeatState a, BrokerHeartbeatState b) {
+if (a.metadataOffset < b.metadataOffset) {
+return -1;
+} else if (a.metadataOffset > b.metadataOffset) {
+return 1;
+} else if (

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-09 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r573235167



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java
##
@@ -0,0 +1,491 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.UsableBroker;
+import org.slf4j.Logger;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Optional;
+import java.util.TreeSet;
+import java.util.function.Function;
+
+
+/**
+ * The BrokerHeartbeatManager manages all the soft state associated with 
broker heartbeats.
+ * Soft state is state which does not appear in the metadata log.  This state 
includes
+ * things like the last time each broker sent us a heartbeat, and whether the 
broker is
+ * trying to perform a controlled shutdown.
+ *
+ * Only the active controller has a BrokerHeartbeatManager, since only the 
active
+ * controller handles broker heartbeats.  Standby controllers will create a 
heartbeat
+ * manager as part of the process of activating.  This design minimizes the 
size of the
+ * metadata partition by excluding heartbeats from it.  However, it does mean 
that after
+ * a controller failover, we may take some extra time to fence brokers, since 
the new
+ * active controller does not know when the last heartbeats were received from 
each.
+ */
+public class BrokerHeartbeatManager {
+static class BrokerHeartbeatState {
+/**
+ * The broker ID.
+ */
+private final int id;
+
+/**
+ * The last time we received a heartbeat from this broker, in 
monotonic nanoseconds.
+ * When this field is updated, we also may have to update the broker's 
position in
+ * the unfenced list.
+ */
+long lastContactNs;
+
+/**
+ * The last metadata offset which this broker reported.  When this 
field is updated,
+ * we may also have to update the broker's position in the active set.
+ */
+long metadataOffset;
+
+/**
+ * The offset at which the broker should complete its controlled 
shutdown, or -1
+ * if the broker is not performing a controlled shutdown.  When this 
field is
+ * updated, we also have to update the broker's position in the 
shuttingDown set.
+ */
+private long shutdownOffset;
+
+/**
+ * The previous entry in the unfenced list, or null if the broker is 
not in that list.
+ */
+private BrokerHeartbeatState prev;
+
+/**
+ * The next entry in the unfenced list, or null if the broker is not 
in that list.
+ */
+private BrokerHeartbeatState next;
+
+BrokerHeartbeatState(int id) {
+this.id = id;
+this.lastContactNs = 0;
+this.prev = null;
+this.next = null;
+this.metadataOffset = -1;
+this.shutdownOffset = -1;
+}
+
+/**
+ * Returns the broker ID.
+ */
+int id() {
+return id;
+}
+
+/**
+ * Returns true only if the broker is fenced.
+ */
+boolean fenced() {
+return prev == null;
+}
+
+/**
+ * Returns true only if the broker is in controlled shutdown state.
+ */
+boolean shuttingDown() {
+return shutdownOffset >= 0;
+}
+}
+
+static class MetadataOffsetComparator implements 
Comparator {
+static final MetadataOffsetComparator INSTANCE = new 
MetadataOffsetComparator();
+
+@Override
+public int compare(BrokerHeartbeatState a, BrokerHeartbeatState b) {
+if (a.metadataOffset < b.metadataOffset) {
+return -1;
+} else if (a.metadataOffset > b.metadataOffset) {
+return 1;
+} else if (

[GitHub] [kafka] cmccabe commented on a change in pull request #10070: KAFKA-12276: Add the quorum controller code

2021-02-09 Thread GitBox


cmccabe commented on a change in pull request #10070:
URL: https://github.com/apache/kafka/pull/10070#discussion_r573234880



##
File path: 
metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java
##
@@ -0,0 +1,491 @@
+/*
+ * 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 org.apache.kafka.controller;
+
+import org.apache.kafka.common.errors.InvalidReplicationFactorException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.metadata.UsableBroker;
+import org.slf4j.Logger;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Optional;
+import java.util.TreeSet;
+import java.util.function.Function;
+
+
+/**
+ * The BrokerHeartbeatManager manages all the soft state associated with 
broker heartbeats.
+ * Soft state is state which does not appear in the metadata log.  This state 
includes
+ * things like the last time each broker sent us a heartbeat, and whether the 
broker is
+ * trying to perform a controlled shutdown.
+ *
+ * Only the active controller has a BrokerHeartbeatManager, since only the 
active
+ * controller handles broker heartbeats.  Standby controllers will create a 
heartbeat
+ * manager as part of the process of activating.  This design minimizes the 
size of the
+ * metadata partition by excluding heartbeats from it.  However, it does mean 
that after
+ * a controller failover, we may take some extra time to fence brokers, since 
the new
+ * active controller does not know when the last heartbeats were received from 
each.
+ */
+public class BrokerHeartbeatManager {
+static class BrokerHeartbeatState {
+/**
+ * The broker ID.
+ */
+private final int id;
+
+/**
+ * The last time we received a heartbeat from this broker, in 
monotonic nanoseconds.
+ * When this field is updated, we also may have to update the broker's 
position in
+ * the unfenced list.
+ */
+long lastContactNs;
+
+/**
+ * The last metadata offset which this broker reported.  When this 
field is updated,
+ * we may also have to update the broker's position in the active set.
+ */
+long metadataOffset;
+
+/**
+ * The offset at which the broker should complete its controlled 
shutdown, or -1
+ * if the broker is not performing a controlled shutdown.  When this 
field is
+ * updated, we also have to update the broker's position in the 
shuttingDown set.
+ */
+private long shutdownOffset;
+
+/**
+ * The previous entry in the unfenced list, or null if the broker is 
not in that list.
+ */
+private BrokerHeartbeatState prev;
+
+/**
+ * The next entry in the unfenced list, or null if the broker is not 
in that list.
+ */
+private BrokerHeartbeatState next;
+
+BrokerHeartbeatState(int id) {
+this.id = id;
+this.lastContactNs = 0;
+this.prev = null;
+this.next = null;
+this.metadataOffset = -1;
+this.shutdownOffset = -1;
+}
+
+/**
+ * Returns the broker ID.
+ */
+int id() {
+return id;
+}
+
+/**
+ * Returns true only if the broker is fenced.
+ */
+boolean fenced() {
+return prev == null;
+}
+
+/**
+ * Returns true only if the broker is in controlled shutdown state.
+ */
+boolean shuttingDown() {
+return shutdownOffset >= 0;
+}
+}
+
+static class MetadataOffsetComparator implements 
Comparator {
+static final MetadataOffsetComparator INSTANCE = new 
MetadataOffsetComparator();
+
+@Override
+public int compare(BrokerHeartbeatState a, BrokerHeartbeatState b) {
+if (a.metadataOffset < b.metadataOffset) {
+return -1;
+} else if (a.metadataOffset > b.metadataOffset) {
+return 1;
+} else if (