diff --git a/cluster-operator/src/main/java/io/strimzi/operator/cluster/ClusterOperatorConfig.java b/cluster-operator/src/main/java/io/strimzi/operator/cluster/ClusterOperatorConfig.java index 9b7eeebfd1..c094a5ecf2 100644 --- a/cluster-operator/src/main/java/io/strimzi/operator/cluster/ClusterOperatorConfig.java +++ b/cluster-operator/src/main/java/io/strimzi/operator/cluster/ClusterOperatorConfig.java @@ -145,6 +145,11 @@ public class ClusterOperatorConfig { */ public static final ConfigParameter OPERATION_TIMEOUT_MS = new ConfigParameter<>("STRIMZI_OPERATION_TIMEOUT_MS", LONG, "300000", CONFIG_VALUES); + /** + * The maximum number of broker nodes that can be restarted at once + */ + public static final ConfigParameter MAX_RESTART_BATCH_SIZE = new ConfigParameter<>("STRIMZI_MAX_RESTART_BATCH_SIZE", INTEGER, "1", CONFIG_VALUES); + /** * Timeout used to wait for a Kafka Connect builds to finish */ @@ -465,6 +470,13 @@ public long getOperationTimeoutMs() { return get(OPERATION_TIMEOUT_MS); } + /** + * @return how many broker nodes can be restarted in parallel + */ + public int getMaxRestartBatchSize() { + return get(MAX_RESTART_BATCH_SIZE); + } + /** * @return How many milliseconds should we wait for Kafka Connect build to complete */ diff --git a/cluster-operator/src/main/java/io/strimzi/operator/cluster/model/RestartReasons.java b/cluster-operator/src/main/java/io/strimzi/operator/cluster/model/RestartReasons.java index 5173485b4f..80e8c7b7e9 100644 --- a/cluster-operator/src/main/java/io/strimzi/operator/cluster/model/RestartReasons.java +++ b/cluster-operator/src/main/java/io/strimzi/operator/cluster/model/RestartReasons.java @@ -146,4 +146,12 @@ public int hashCode() { public String toString() { return reasons.keySet().toString(); } + + /** + * @param reason The reason to test. + * @return true if these reasons are just the single given reason. + */ + public boolean isSingletonOf(RestartReason reason) { + return reasons.size() == 1 && reasons.containsKey(reason); + } } diff --git a/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/assembly/KafkaReconciler.java b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/assembly/KafkaReconciler.java index 820cc4372c..172f384eaa 100644 --- a/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/assembly/KafkaReconciler.java +++ b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/assembly/KafkaReconciler.java @@ -47,7 +47,7 @@ import io.strimzi.operator.cluster.model.RestartReasons; import io.strimzi.operator.cluster.operator.resource.ConcurrentDeletionException; import io.strimzi.operator.cluster.operator.resource.KafkaAgentClientProvider; -import io.strimzi.operator.cluster.operator.resource.KafkaRoller; +//import io.strimzi.operator.cluster.operator.resource.KafkaRoller; import io.strimzi.operator.cluster.operator.resource.ResourceOperatorSupplier; import io.strimzi.operator.cluster.operator.resource.events.KubernetesRestartEventPublisher; import io.strimzi.operator.cluster.operator.resource.kubernetes.ClusterRoleBindingOperator; @@ -65,9 +65,10 @@ import io.strimzi.operator.cluster.operator.resource.kubernetes.ServiceOperator; import io.strimzi.operator.cluster.operator.resource.kubernetes.StorageClassOperator; import io.strimzi.operator.cluster.operator.resource.kubernetes.StrimziPodSetOperator; +import io.strimzi.operator.cluster.operator.resource.rolling.RackRolling; import io.strimzi.operator.common.AdminClientProvider; import io.strimzi.operator.common.Annotations; -import io.strimzi.operator.common.BackOff; +//import io.strimzi.operator.common.e; import io.strimzi.operator.common.Reconciliation; import io.strimzi.operator.common.ReconciliationLogger; import io.strimzi.operator.common.Util; @@ -112,6 +113,7 @@ public class KafkaReconciler { // Various settings private final long operationTimeoutMs; + private final int maxRestartBatchSize; private final boolean isNetworkPolicyGeneration; private final boolean isPodDisruptionBudgetGeneration; private final boolean isKafkaNodePoolsEnabled; @@ -195,6 +197,7 @@ public KafkaReconciler( this.reconciliation = reconciliation; this.vertx = vertx; this.operationTimeoutMs = config.getOperationTimeoutMs(); + this.maxRestartBatchSize = config.getMaxRestartBatchSize(); this.kafkaNodePoolCrs = nodePools; this.kafka = kafka; @@ -466,23 +469,47 @@ protected Future maybeRollKafka( Map> kafkaAdvertisedPorts, boolean allowReconfiguration ) { - return new KafkaRoller( - reconciliation, - vertx, - podOperator, - 1_000, - operationTimeoutMs, - () -> new BackOff(250, 2, 10), - nodes, - this.coTlsPemIdentity, - adminClientProvider, - kafkaAgentClientProvider, - brokerId -> kafka.generatePerBrokerConfiguration(brokerId, kafkaAdvertisedHostnames, kafkaAdvertisedPorts), - logging, - kafka.getKafkaVersion(), - allowReconfiguration, - eventsPublisher - ).rollingRestart(podNeedsRestart); + Function kafkaConfigProvider = nodeId -> kafka.generatePerBrokerConfiguration(nodeId, kafkaAdvertisedHostnames, kafkaAdvertisedPorts); + //TODO: Change this logic to run the new roller if the feature gate for it is enabled (also add feature gate). + + var rr = RackRolling.rollingRestart( + podOperator, + nodes, + reconciliation, + // Remap the function from pod to RestartReasons to nodeId to RestartReasons + nodeId -> podNeedsRestart.apply(podOperator.get(reconciliation.namespace(), nodes.stream().filter(nodeRef -> nodeRef.nodeId() == nodeId).collect(Collectors.toList()).get(0).podName())), + this.coTlsPemIdentity, + adminClientProvider, + kafkaAgentClientProvider, + kafkaConfigProvider, + allowReconfiguration, + kafka.getKafkaVersion(), + logging, + operationTimeoutMs, + maxRestartBatchSize, + 3, + 3, + 10, + eventsPublisher); + + return rr.executeRollingAsync(vertx); +// return new KafkaRoller( +// reconciliation, +// vertx, +// podOperator, +// 1_000, +// operationTimeoutMs, +// () -> new BackOff(250, 2, 10), +// nodes, +// this.coTlsPemIdentity, +// adminClientProvider, +// kafkaAgentClientProvider, +// kafkaConfigProvider, +// logging, +// kafka.getKafkaVersion(), +// allowReconfiguration, +// eventsPublisher +// ).rollingRestart(podNeedsRestart); } /** diff --git a/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/BrokerState.java b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/BrokerState.java index f67dcf08f3..6e319a5569 100644 --- a/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/BrokerState.java +++ b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/BrokerState.java @@ -12,7 +12,7 @@ /** * Java representation of the JSON response from the /v1/broker-state endpoint of the KafkaAgent */ -class BrokerState { +public class BrokerState { private static final int BROKER_RECOVERY_STATE = 2; private final int code; diff --git a/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/KafkaBrokerConfigurationDiff.java b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/KafkaBrokerConfigurationDiff.java index c70df52cf3..43f9a65ef3 100644 --- a/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/KafkaBrokerConfigurationDiff.java +++ b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/KafkaBrokerConfigurationDiff.java @@ -74,7 +74,11 @@ public class KafkaBrokerConfigurationDiff extends AbstractJsonDiff { * @param kafkaVersion Kafka version * @param brokerNodeRef Broker node reference */ - protected KafkaBrokerConfigurationDiff(Reconciliation reconciliation, Config brokerConfigs, String desired, KafkaVersion kafkaVersion, NodeRef brokerNodeRef) { + public KafkaBrokerConfigurationDiff(Reconciliation reconciliation, + Config brokerConfigs, + String desired, + KafkaVersion kafkaVersion, + NodeRef brokerNodeRef) { this.reconciliation = reconciliation; this.configModel = KafkaConfiguration.readConfigModel(kafkaVersion); this.brokerConfigDiff = diff(brokerNodeRef, desired, brokerConfigs, configModel); @@ -83,7 +87,7 @@ protected KafkaBrokerConfigurationDiff(Reconciliation reconciliation, Config bro /** * @return Returns true if the configuration can be updated dynamically */ - protected boolean canBeUpdatedDynamically() { + public boolean canBeUpdatedDynamically() { boolean result = true; for (AlterConfigOp entry : brokerConfigDiff) { if (isEntryReadOnly(entry.configEntry())) { @@ -107,7 +111,7 @@ private boolean isEntryReadOnly(ConfigEntry entry) { * Returns configuration difference * @return Collection of AlterConfigOp containing difference between current and desired configuration */ - protected Collection getConfigDiff() { + public Collection getConfigDiff() { return brokerConfigDiff; } diff --git a/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/KafkaBrokerLoggingConfigurationDiff.java b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/KafkaBrokerLoggingConfigurationDiff.java index c080246532..6b72aa7aab 100644 --- a/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/KafkaBrokerLoggingConfigurationDiff.java +++ b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/KafkaBrokerLoggingConfigurationDiff.java @@ -38,7 +38,7 @@ public class KafkaBrokerLoggingConfigurationDiff extends AbstractJsonDiff { * @param brokerConfigs Current broker configuration from Kafka Admin API * @param desired Desired logging configuration */ - protected KafkaBrokerLoggingConfigurationDiff(Reconciliation reconciliation, Config brokerConfigs, String desired) { + public KafkaBrokerLoggingConfigurationDiff(Reconciliation reconciliation, Config brokerConfigs, String desired) { this.reconciliation = reconciliation; this.diff = diff(desired, brokerConfigs); } @@ -47,7 +47,7 @@ protected KafkaBrokerLoggingConfigurationDiff(Reconciliation reconciliation, Con * Returns logging difference * @return Collection of AlterConfigOp containing difference between current and desired logging configuration */ - protected Collection getLoggingDiff() { + public Collection getLoggingDiff() { return diff; } diff --git a/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/kubernetes/PodOperator.java b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/kubernetes/PodOperator.java index e521d4402f..da39b25ca2 100644 --- a/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/kubernetes/PodOperator.java +++ b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/kubernetes/PodOperator.java @@ -51,7 +51,7 @@ public Future restart(Reconciliation reconciliation, Pod pod, long timeout String namespace = pod.getMetadata().getNamespace(); String podName = pod.getMetadata().getName(); Promise deleteFinished = Promise.promise(); - LOGGER.infoCr(reconciliation, "Rolling pod {}", podName); + LOGGER.infoCr(reconciliation, "Deleting pod {}", podName); // Determine generation of deleted pod String deleted = getPodUid(pod); @@ -66,7 +66,7 @@ public Future restart(Reconciliation reconciliation, Pod pod, long timeout boolean done = !deleted.equals(newUid); if (done) { - LOGGER.debugCr(reconciliation, "Rolling pod {} finished", podName); + LOGGER.debugCr(reconciliation, "Deleting pod {} finished", podName); } return done; diff --git a/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/AgentClient.java b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/AgentClient.java new file mode 100644 index 0000000000..08628eb50f --- /dev/null +++ b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/AgentClient.java @@ -0,0 +1,16 @@ +/* + * Copyright Strimzi authors. + * License: Apache License 2.0 (see the file LICENSE or http://apache.org/licenses/LICENSE-2.0.html). + */ +package io.strimzi.operator.cluster.operator.resource.rolling; + +import io.strimzi.operator.cluster.model.NodeRef; + +/** + * An abstraction over a KafkaAgent client. + */ +interface AgentClient { + + /** @return The broker state, according to the Kafka Agent */ + BrokerState getBrokerState(NodeRef nodeRef); +} diff --git a/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/AgentClientImpl.java b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/AgentClientImpl.java new file mode 100644 index 0000000000..d0a8bcfa85 --- /dev/null +++ b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/AgentClientImpl.java @@ -0,0 +1,26 @@ +/* + * Copyright Strimzi authors. + * License: Apache License 2.0 (see the file LICENSE or http://apache.org/licenses/LICENSE-2.0.html). + */ +package io.strimzi.operator.cluster.operator.resource.rolling; + +import io.strimzi.operator.cluster.model.NodeRef; +import io.strimzi.operator.cluster.operator.resource.KafkaAgentClient; + +class AgentClientImpl implements AgentClient { + private final KafkaAgentClient kafkaAgentClient; + + AgentClientImpl(KafkaAgentClient kafkaAgentClient) { + this.kafkaAgentClient = kafkaAgentClient; + + } + + @Override + public BrokerState getBrokerState(NodeRef nodeRef) { + var result = kafkaAgentClient.getBrokerState(nodeRef.podName()); + BrokerState brokerState = BrokerState.fromValue((byte) result.code()); + brokerState.setRemainingSegmentsToRecover(result.remainingSegmentsToRecover()); + brokerState.setRemainingLogsToRecover(result.remainingLogsToRecover()); + return brokerState; + } +} diff --git a/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/Alarm.java b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/Alarm.java new file mode 100644 index 0000000000..19f65ac9ed --- /dev/null +++ b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/Alarm.java @@ -0,0 +1,115 @@ +/* + * Copyright Strimzi authors. + * License: Apache License 2.0 (see the file LICENSE or http://apache.org/licenses/LICENSE-2.0.html). + */ +package io.strimzi.operator.cluster.operator.resource.rolling; + +import io.strimzi.operator.common.UncheckedInterruptedException; + +import java.util.concurrent.TimeoutException; +import java.util.function.BooleanSupplier; +import java.util.function.Supplier; + +/** + * Timing utility for polling loops which allows to set an alarm (in terms of a duration from "now") and + * subsequently sleep the executing thread. If the alarm duration is exceeded the call to sleep will throw a + * {@link TimeoutException}. This can be used to simplify writing polling logic like the following + *
{@code
+ * long timeoutMs = 60_000
+ * long pollIntervalMs = 1_000;
+ * Alarm alarm = Alarm.start(time, timeoutMs);
+ * while (true) {
+ *   // do some processing
+ *   if (processingSuccess) {
+ *       timeoutMs = alarm.remainingMs();
+ *       // we might want to use the remaining timeout when
+ *       // a single timeout is used for a sequence of polling tasks
+ *       break;
+ *   }
+ *   alarm.sleep(pollIntervalMs);
+ * }
+ * }
+ * This logic is encapsulated in the {@link #poll(long, BooleanSupplier)} method. + */ +public class Alarm { + + final Time time; + final long deadline; + private final Supplier timeoutMessageSupplier; + + private Alarm(Time time, long deadline, Supplier timeoutMessageSupplier) { + this.time = time; + this.deadline = deadline; + this.timeoutMessageSupplier = timeoutMessageSupplier; + } + + /** + * Creates an Alerm + * @param time The source of time + * @param timeoutMs The timeout for this alarm. + * @param timeoutMessageSupplier The exception message + * @return The alarm + */ + public static Alarm timer(Time time, long timeoutMs, Supplier timeoutMessageSupplier) { + if (timeoutMs < 0) { + throw new IllegalArgumentException(); + } + long deadline = time.nanoTime() + 1_000_000 * timeoutMs; + return new Alarm(time, deadline, timeoutMessageSupplier); + } + + /** + * @return The remaining number of milliseconds until the deadline passed + */ + public long remainingMs() { + return Math.max(deadline - time.nanoTime(), 0) / 1_000_000L; + } + + /** + * Sleep the current thread for at most at least {@code ms} milliseconds, according to + * (and subject to the precision and accuracy of) the configured {@link Time} instance. + * The actual sleep time will be less than {@code ms} if using {@code ms} would exceed this + * alarm's deadline. + * The thread does not lose ownership of any monitors. + * @param ms The number of milliseconds to sleep for. + * @throws TimeoutException If the Alarm's deadline has passed + * @throws InterruptedException If the current thread is interrupted + */ + public void sleep(long ms) throws TimeoutException, InterruptedException { + if (ms < 0) { + throw new IllegalArgumentException(); + } + long sleepNs = Math.min(1_000_000L * ms, deadline - time.nanoTime()); + if (sleepNs <= 0) { + throw new TimeoutException(timeoutMessageSupplier.get()); + } + time.sleep(sleepNs / 1_000_000L, (int) (sleepNs % 1_000_000L)); + } + + /** + * Test {@code done} at least once, returning when it returns true, and otherwise sleeping for at most approximately + * {@code pollIntervalMs} before repeating, throwing {@link TimeoutException} should this + * alarm expire before {@code done} returns true. + * + * @param pollIntervalMs The polling interval + * @param done A predicate function to detecting when the polling loop is complete. + * @return The remaining time left for this alarm, in ms. + * @throws UncheckedInterruptedException The thread was interrupted + * @throws TimeoutException The {@link #remainingMs()} has reached zero. + */ + public long poll(long pollIntervalMs, BooleanSupplier done) throws TimeoutException { + if (pollIntervalMs <= 0) { + throw new IllegalArgumentException(); + } + try { + while (true) { + if (done.getAsBoolean()) { + return this.remainingMs(); + } + this.sleep(pollIntervalMs); + } + } catch (InterruptedException e) { + throw new UncheckedInterruptedException(e); + } + } +} diff --git a/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/Availability.java b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/Availability.java new file mode 100644 index 0000000000..438579bcb8 --- /dev/null +++ b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/Availability.java @@ -0,0 +1,103 @@ +/* + * Copyright Strimzi authors. + * License: Apache License 2.0 (see the file LICENSE or http://apache.org/licenses/LICENSE-2.0.html). + */ +package io.strimzi.operator.cluster.operator.resource.rolling; + +import io.strimzi.operator.common.Reconciliation; +import io.strimzi.operator.common.ReconciliationLogger; +import org.apache.kafka.clients.admin.TopicDescription; +import org.apache.kafka.clients.admin.TopicListing; + +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * Determines whether the given broker can be rolled without affecting + * producers with acks=all publishing to topics with a {@code min.in.sync.replicas}. + */ +class Availability { + + private static final ReconciliationLogger LOGGER = ReconciliationLogger.create(Availability.class.getName()); + + private final Reconciliation reconciliation; + + private final List topicDescriptions; + + private final Map minIsrByTopic; + + Availability(Reconciliation reconciliation, RollClient rollClient) { + this.reconciliation = reconciliation; + // 1. Get all topics + Collection topicListings = rollClient.listTopics(); + // 2. Get topic descriptions + topicDescriptions = rollClient.describeTopics(topicListings.stream().map(TopicListing::topicId).toList()); + // 2. Get topic minISR configurations + minIsrByTopic = rollClient.describeTopicMinIsrs(rollClient.listTopics().stream().map(TopicListing::name).toList()); + } + + protected boolean anyPartitionWouldBeUnderReplicated(int nodeId) { + var replicas = getReplicasForNode(nodeId); + for (var replica : replicas) { + var topicName = replica.topicName(); + Integer minIsr = minIsrByTopic.get(topicName); + if (wouldBeUnderReplicated(minIsr, replica)) { + return true; + } + } + return false; + } + + protected Set getReplicasForNode(int nodeId) { + Set replicas = new HashSet<>(); + topicDescriptions.forEach(topicDescription -> { + topicDescription.partitions().forEach(topicPartitionInfo -> { + topicPartitionInfo.replicas().forEach(replicatingBroker -> { + if (replicatingBroker.id() == nodeId) { + replicas.add(new Replica( + replicatingBroker, + topicDescription.name(), + topicPartitionInfo.partition(), + topicPartitionInfo.isr() + )); + } + }); + }); + }); + return replicas; + } + + private static boolean wouldBeUnderReplicated(Integer minIsr, Replica replica) { + final boolean wouldByUnderReplicated; + if (minIsr == null) { + // if topic doesn't have minISR then it's fine + wouldByUnderReplicated = false; + } else { + //TODO: check if minISR can be set to equal to or greater than replica size + + // else topic has minISR + // compute spare = size(ISR) - minISR + int sizeIsr = replica.isrSize(); + int spare = sizeIsr - minIsr; + if (spare > 0) { + // if (spare > 0) then we can restart the broker hosting this replica + // without the topic being under-replicated + wouldByUnderReplicated = false; + } else if (spare == 0) { + // if we restart this broker this replica would be under-replicated if it's currently in the ISR + // if it's not in the ISR then restarting the server won't make a difference + wouldByUnderReplicated = replica.isInIsr(); + } else { + // this partition is already under-replicated + // if it's not in the ISR then restarting the server won't make a difference + // but in this case since it's already under-replicated let's + // not possible prolong the time to this server rejoining the ISR + wouldByUnderReplicated = true; + } + } + return wouldByUnderReplicated; + } +} diff --git a/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/Batching.java b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/Batching.java new file mode 100644 index 0000000000..8937adc08f --- /dev/null +++ b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/Batching.java @@ -0,0 +1,194 @@ +/* + * Copyright Strimzi authors. + * License: Apache License 2.0 (see the file LICENSE or http://apache.org/licenses/LICENSE-2.0.html). + */ +package io.strimzi.operator.cluster.operator.resource.rolling; + +import io.strimzi.operator.common.Reconciliation; +import io.strimzi.operator.common.ReconciliationLogger; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Comparator; +import java.util.HashSet; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Deal with the batching mechanism for the new Kafka roller + */ +public class Batching { + + private static final ReconciliationLogger LOGGER = ReconciliationLogger.create(Batching.class); + + /** + * Partition the given {@code brokers} + * into cells that can be rolled in parallel because they + * contain no replicas in common. + */ + static List> cells(Reconciliation reconciliation, + Collection brokers) { + + // find brokers that are individually rollable + var rollable = new LinkedHashSet<>(brokers); + // partition the set under the equivalence relation "shares a partition with" + Set> disjoint = partitionByHasAnyReplicasInCommon(reconciliation, rollable); + // disjoint cannot be empty, because rollable isn't empty, and disjoint is a partitioning or rollable + // We find the biggest set of brokers which can parallel-rolled + return disjoint.stream().sorted(Comparator.>comparingInt(Set::size).reversed()).toList(); + } + + private static Set> partitionByHasAnyReplicasInCommon(Reconciliation reconciliation, Set rollable) { + Set> disjoint = new HashSet<>(); + for (var node : rollable) { + var nodeReplicas = node.replicas(); + Set> merge = new HashSet<>(); + for (Set cell : disjoint) { + if (!containsAny(reconciliation, node, nodeReplicas, cell)) { + LOGGER.debugCr(reconciliation, "Add {} to {{}}", node.id(), idsOf(cell)); + merge.add(cell); + merge.add(Set.of(node)); + // problem is here, we're iterating over all cells (ones which we've decided should be disjoint) + // and we merged them in violation of that + // we could break here at the end of the if block (which would be correct) + // but it might not be optimal (in the sense of forming large cells) + break; + } + } + if (merge.isEmpty()) { + LOGGER.debugCr(reconciliation, "New cell: {{}}", node.id()); + disjoint.add(Set.of(node)); + } else { + LOGGER.debugCr(reconciliation, "Merge {}", idsOf2(merge)); + for (Set r : merge) { + LOGGER.debugCr(reconciliation, "Remove cell: {}", idsOf(r)); + disjoint.remove(r); + } + Set newCell = union(merge); + LOGGER.debugCr(reconciliation, "New cell: {{}}", idsOf(newCell)); + disjoint.add(newCell); + } + LOGGER.debugCr(reconciliation, "Disjoint cells now: {}", idsOf2(disjoint)); + } + return disjoint; + } + + /** + * Split the given cells into batches, + * taking account of {@code acks=all} availability and the given maxBatchSize + */ + static List> batchCells(Reconciliation reconciliation, + List> cells, + Availability availability, + int maxBatchSize) { + List> result = new ArrayList<>(); + Set unavail = new HashSet<>(); + for (var cell : cells) { + List> availBatches = new ArrayList<>(); + for (var kafkaNode : cell) { + if (!availability.anyPartitionWouldBeUnderReplicated(kafkaNode.id())) { + LOGGER.debugCr(reconciliation, "No replicas of node {} will be unavailable => add to batch", + kafkaNode.id()); + var currentBatch = availBatches.isEmpty() ? null : availBatches.get(availBatches.size() - 1); + if (currentBatch == null || currentBatch.size() >= maxBatchSize) { + currentBatch = new HashSet<>(); + availBatches.add(currentBatch); + } + currentBatch.add(kafkaNode.id()); + } else { + LOGGER.debugCr(reconciliation, "Some replicas of node {} will be unavailable => do not add to batch", kafkaNode.id()); + unavail.add(kafkaNode.id()); + } + } + result.addAll(availBatches); + } + if (result.isEmpty() && !unavail.isEmpty()) { + LOGGER.warnCr(reconciliation, "Cannot restart nodes {} without violating some topics' min.in.sync.replicas", nodeIdsToString(unavail)); + } + return result; + } + + static T elementInIntersection(Set set, Set set2) { + for (T t : set) { + if (set2.contains(t)) { + return t; + } + } + return null; + } + + static boolean containsAny(Reconciliation reconciliation, + KafkaNode node, + Set nodeReplicas, + Set cell) { + for (var b : cell) { + var commonReplica = elementInIntersection(b.replicas(), nodeReplicas); + if (commonReplica != null) { + LOGGER.debugCr(reconciliation, "Nodes {} and {} have at least {} in common", + node.id(), b.id(), commonReplica); + return true; + } + } + LOGGER.debugCr(reconciliation, "Node {} has no replicas in common with any of the nodes in {}", + node.id(), idsOf(cell)); + return false; + } + + private static String idsOf(Collection cell) { + return cell.stream() + .map(kafkaNode -> Integer.toString(kafkaNode.id())) + .collect(Collectors.joining(",", "{", "}")); + } + + private static String idsOf2(Collection> merge) { + return merge.stream() + .map(Batching::idsOf) + .collect(Collectors.joining(",", "{", "}")); + } + + /** Returns a new set that is the union of each of the sets in the given {@code merge}. I.e. flatten without duplicates. */ + private static Set union(Set> merge) { + HashSet result = new HashSet<>(); + for (var x : merge) { + result.addAll(x); + } + return result; + } + + + static String nodeIdsToString2(Collection> merge) { + return merge.stream() + .map(Batching::nodeIdsToString) + .collect(Collectors.joining(",", "{", "}")); + } + + static String nodeIdsToString(Collection cell) { + return cell.stream() + .map(String::valueOf) + .collect(Collectors.joining(",", "{", "}")); + } + + /** + * Pick the "best" batch to be restarted. + * This is the largest batch of available servers + * @return the "best" batch to be restarted + */ + static Set pickBestBatchForRestart(List> batches) { + if (batches.size() < 1) { + return Set.of(); + } + + // if none of the batches contain more than 1 node, return a set with unready node if there is any, otherwise the first set. + if (batches.stream().filter(set -> set.size() > 1).count() < 1) { + return batches.iterator().next(); + } + + var sorted = batches.stream().sorted(Comparator.comparing(Set::size)).toList(); + return sorted.get(sorted.size() - 1); + } + + + +} diff --git a/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/BrokerState.java b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/BrokerState.java new file mode 100644 index 0000000000..549bac6726 --- /dev/null +++ b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/BrokerState.java @@ -0,0 +1,97 @@ +/* + * Copyright Strimzi authors. + * License: Apache License 2.0 (see the file LICENSE or http://apache.org/licenses/LICENSE-2.0.html). + */ + +package io.strimzi.operator.cluster.operator.resource.rolling; + + +import java.util.HashMap; +import java.util.Map; + +/** + * Enumerates the states published by a Kafka node via the broker state metric. + */ +enum BrokerState { + /** + * The state the broker is in when it first starts up. + */ + NOT_RUNNING((byte) 0), + + /** + * The state the broker is in when it is catching up with cluster metadata. + */ + STARTING((byte) 1), + + /** + * The broker has caught up with cluster metadata, but has not yet + * been unfenced by the controller. + */ + RECOVERY((byte) 2), + + /** + * The state the broker is in when it has registered at least once, and is + * accepting client requests. + */ + RUNNING((byte) 3), + + /** + * The state the broker is in when it is attempting to perform a controlled + * shutdown. + */ + PENDING_CONTROLLED_SHUTDOWN((byte) 6), + + /** + * The state the broker is in when it is shutting down. + */ + SHUTTING_DOWN((byte) 7), + + /** + * The broker is in an unknown state. + */ + UNKNOWN((byte) 127); + + private final static Map VALUES_TO_ENUMS = new HashMap<>(); + + static { + for (BrokerState state : BrokerState.values()) { + VALUES_TO_ENUMS.put(state.value(), state); + } + } + + private final byte value; + private int remainingLogsToRecover; + private int remainingSegmentsToRecover; + + BrokerState(byte value) { + this.value = value; + } + + public static BrokerState fromValue(byte value) { + BrokerState state = VALUES_TO_ENUMS.get(value); + if (state == null) { + return UNKNOWN; + } + return state; + } + + public byte value() { + return value; + } + + void setRemainingLogsToRecover(int value) { + remainingLogsToRecover = value; + } + + void setRemainingSegmentsToRecover(int value) { + remainingSegmentsToRecover = value; + } + + int remainingLogsToRecover() { + return remainingLogsToRecover; + } + + int remainingSegmentsToRecover() { + return remainingSegmentsToRecover; + } +} \ No newline at end of file diff --git a/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/Configs.java b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/Configs.java new file mode 100644 index 0000000000..a83d98501d --- /dev/null +++ b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/Configs.java @@ -0,0 +1,18 @@ +/* + * Copyright Strimzi authors. + * License: Apache License 2.0 (see the file LICENSE or http://apache.org/licenses/LICENSE-2.0.html). + */ +package io.strimzi.operator.cluster.operator.resource.rolling; + +import org.apache.kafka.clients.admin.Config; + +/** + * Holds Kafka node configs and logger configs returned from Kafka Admin API. + * + * @param nodeConfigs Broker/Controller configs + * @param nodeLoggerConfigs Broker/Controller logging config + */ +record Configs(Config nodeConfigs, Config nodeLoggerConfigs) { + +} + diff --git a/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/Context.java b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/Context.java new file mode 100644 index 0000000000..8a804d90d6 --- /dev/null +++ b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/Context.java @@ -0,0 +1,144 @@ +/* + * Copyright Strimzi authors. + * License: Apache License 2.0 (see the file LICENSE or http://apache.org/licenses/LICENSE-2.0.html). + */ +package io.strimzi.operator.cluster.operator.resource.rolling; + +import io.strimzi.operator.cluster.model.NodeRef; +import io.strimzi.operator.cluster.model.RestartReasons; +import io.strimzi.operator.cluster.operator.resource.KafkaBrokerConfigurationDiff; +import io.strimzi.operator.cluster.operator.resource.KafkaBrokerLoggingConfigurationDiff; + +import java.time.Instant; +import java.util.function.Function; + +/** + * Per-server context information during a rolling restart/reconfigure + */ +final class Context { + /** The node this context refers to */ + private final NodeRef nodeRef; + /** The process roles currently assigned to the node */ + private final NodeRoles currentRoles; + /** The state of the node the last time it was observed */ + private State state; + /** The time of the last state transition */ + private long lastTransition; + /** The reasons this node needs to be restarted or reconfigured */ + private RestartReasons reason; + /** The number of restarts done so far. */ + private int numRestarts; + /** The number of reconfigurations done so far. */ + private int numReconfigs; + /** The number of operational attempts so far. */ + private int numAttempts; + /** The difference between the current logging config and the desired logging config */ + private KafkaBrokerLoggingConfigurationDiff loggingDiff; + /** The difference between the current node config and the desired node config */ + private KafkaBrokerConfigurationDiff brokerConfigDiff; + + private Context(NodeRef nodeRef, NodeRoles currentRoles, State state, long lastTransition, RestartReasons reason, int numRestarts, int numReconfigs, int numAttempts) { + this.nodeRef = nodeRef; + this.currentRoles = currentRoles; + this.state = state; + this.lastTransition = lastTransition; + this.reason = reason; + this.numRestarts = numRestarts; + this.numReconfigs = numReconfigs; + this.numAttempts = numAttempts; + } + + static Context start(NodeRef nodeRef, + NodeRoles nodeRoles, + Function predicate, + Time time) { + return new Context(nodeRef, nodeRoles, State.UNKNOWN, time.systemTimeMillis(), predicate.apply(nodeRef.nodeId()), 0, 0, 1); + } + + State transitionTo(State state, Time time) { + if (this.state() == state) { + return state; + } + this.state = state; + + this.lastTransition = time.systemTimeMillis(); + return state; + } + + public int nodeId() { + return nodeRef.nodeId(); + } + + public NodeRef nodeRef() { + return nodeRef; + } + + public NodeRoles currentRoles() { + return currentRoles; + } + + public State state() { + return state; + } + + public long lastTransition() { + return lastTransition; + } + + public RestartReasons reason() { + return reason; + } + + public int numRestarts() { + return numRestarts; + } + + public int numReconfigs() { + return numReconfigs; + } + + public int numAttempts() { + return numAttempts; + } + + public void incrementNumAttempts() { + this.numAttempts++; + } + + public void incrementNumRestarts() { + this.numRestarts++; + } + + public void incrementNumReconfigs() { + this.numReconfigs++; + } + + @Override + public String toString() { + + return "Context[" + + "nodeRef=" + nodeRef + ", " + + "currentRoles=" + currentRoles + ", " + + "state=" + state + ", " + + "lastTransition=" + Instant.ofEpochMilli(lastTransition) + ", " + + "reason=" + reason + ", " + + "numRestarts=" + numRestarts + ", " + + "numReconfigs=" + numReconfigs + ", " + + "numAttempts=" + numAttempts + ']'; + } + + public void brokerConfigDiff(KafkaBrokerConfigurationDiff diff) { + this.brokerConfigDiff = diff; + } + public void loggingDiff(KafkaBrokerLoggingConfigurationDiff loggingDiff) { + this.loggingDiff = loggingDiff; + } + + public KafkaBrokerLoggingConfigurationDiff loggingDiff() { + return loggingDiff; + } + + public KafkaBrokerConfigurationDiff brokerConfigDiff() { + return brokerConfigDiff; + } +} diff --git a/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/KafkaNode.java b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/KafkaNode.java new file mode 100644 index 0000000000..28697dd2f1 --- /dev/null +++ b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/KafkaNode.java @@ -0,0 +1,15 @@ +/* + * Copyright Strimzi authors. + * License: Apache License 2.0 (see the file LICENSE or http://apache.org/licenses/LICENSE-2.0.html). + */ +package io.strimzi.operator.cluster.operator.resource.rolling; + +import java.util.Set; + +/** + * Information about a Kafka node (which may be a broker, controller, or both) and its replicas. + * @param id The id of the server + * @param replicas The replicas on this server + */ +record KafkaNode(int id, Set replicas) { +} diff --git a/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/MaxAttemptsExceededException.java b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/MaxAttemptsExceededException.java new file mode 100644 index 0000000000..3950cb9047 --- /dev/null +++ b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/MaxAttemptsExceededException.java @@ -0,0 +1,18 @@ +/* + * Copyright Strimzi authors. + * License: Apache License 2.0 (see the file LICENSE or http://apache.org/licenses/LICENSE-2.0.html). + */ +package io.strimzi.operator.cluster.operator.resource.rolling; + +/** + * Indicates failure of the roller to make progress + */ +public class MaxAttemptsExceededException extends RuntimeException { + /** + * Constructor + * @param message The message + */ + public MaxAttemptsExceededException(String message) { + super(message); + } +} diff --git a/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/MaxRestartsExceededException.java b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/MaxRestartsExceededException.java new file mode 100644 index 0000000000..01c0424078 --- /dev/null +++ b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/MaxRestartsExceededException.java @@ -0,0 +1,18 @@ +/* + * Copyright Strimzi authors. + * License: Apache License 2.0 (see the file LICENSE or http://apache.org/licenses/LICENSE-2.0.html). + */ +package io.strimzi.operator.cluster.operator.resource.rolling; + +/** + * Indicates failure of the roller to make progress + */ +public class MaxRestartsExceededException extends RuntimeException { + /** + * Constructor + * @param message The message + */ + public MaxRestartsExceededException(String message) { + super(message); + } +} diff --git a/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/NodeRoles.java b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/NodeRoles.java new file mode 100644 index 0000000000..366b54b3e8 --- /dev/null +++ b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/NodeRoles.java @@ -0,0 +1,16 @@ +/* + * Copyright Strimzi authors. + * License: Apache License 2.0 (see the file LICENSE or http://apache.org/licenses/LICENSE-2.0.html). + */ +package io.strimzi.operator.cluster.operator.resource.rolling; + +/** + * Holds the current process roles for a node. + * This is different from the broker and controller information in NodeRef. + * NodeRef holds the desired roles for a node rather than the roles currently assigned to the node. + * @param controller set to true if the node has controller role + * @param broker set to true if the node has broker role + */ +record NodeRoles(boolean controller, boolean broker) { +} + diff --git a/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/PlatformClient.java b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/PlatformClient.java new file mode 100644 index 0000000000..1390ef3432 --- /dev/null +++ b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/PlatformClient.java @@ -0,0 +1,51 @@ +/* + * Copyright Strimzi authors. + * License: Apache License 2.0 (see the file LICENSE or http://apache.org/licenses/LICENSE-2.0.html). + */ +package io.strimzi.operator.cluster.operator.resource.rolling; + +import io.strimzi.operator.cluster.model.NodeRef; +import io.strimzi.operator.cluster.model.RestartReasons; + +/** + * Abstraction over the platform (i.e. kubernetes). + */ +public interface PlatformClient { + + /** + * State of the node + */ + enum NodeState { + /** + * The pod/process is not running. This includes + * the pod has {@code status.phase=="Pending"} and condition {@code c} in {@code status.conditions} + * with {@code c.type=="PodScheduled" && c.status=="False" && c.reason=="Unschedulable"} + * and any of the containers in the waiting state with ready ImagePullBackoff or CrashLoopBackoff + */ + NOT_RUNNING, + /** The pod/process is not {@link #NOT_RUNNING}, but is lacks a "Ready" condition with status "True" */ + NOT_READY, + /** The pod/process is running and ready */ + READY + } + + /** + * @param nodeRef Node reference + * @return NodeState according to the platform + * */ + NodeState nodeState(NodeRef nodeRef); + + /** + * Initiate the restart of the corresponding Kafka server. + * @param nodeRef The node. + * @param reasons Reasons for restarting the node to emit as an event + */ + void restartNode(NodeRef nodeRef, RestartReasons reasons); + + /** + * @param nodeRef Node reference + * @return Kafka process roles for this node according to the platform. + * This could differ from the roles that the running process actually has (for instance if the process needs to be restarted to pick up its current roles). + */ + NodeRoles nodeRoles(NodeRef nodeRef); +} diff --git a/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/PlatformClientImpl.java b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/PlatformClientImpl.java new file mode 100644 index 0000000000..24d17d5bba --- /dev/null +++ b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/PlatformClientImpl.java @@ -0,0 +1,110 @@ +/* + * Copyright Strimzi authors. + * License: Apache License 2.0 (see the file LICENSE or http://apache.org/licenses/LICENSE-2.0.html). + */ +package io.strimzi.operator.cluster.operator.resource.rolling; + +import io.fabric8.kubernetes.api.model.Pod; +import io.strimzi.operator.cluster.model.NodeRef; +import io.strimzi.operator.cluster.model.RestartReasons; +import io.strimzi.operator.cluster.operator.resource.events.KubernetesRestartEventPublisher; +import io.strimzi.operator.cluster.operator.resource.kubernetes.PodOperator; +import io.strimzi.operator.common.Reconciliation; +import io.strimzi.operator.common.model.Labels; + +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +/** + * Implementation of PlatformClient in terms of Kubernetes Pods + */ +public class PlatformClientImpl implements PlatformClient { + + private final PodOperator podOps; + private final String namespace; + + private final Reconciliation reconciliation; + + private final long operationTimeoutMs; + private final KubernetesRestartEventPublisher eventPublisher; + + PlatformClientImpl(PodOperator podOps, String namespace, Reconciliation reconciliation, long operationTimeoutMs, KubernetesRestartEventPublisher eventPublisher) { + this.podOps = podOps; + this.namespace = namespace; + this.reconciliation = reconciliation; + this.operationTimeoutMs = operationTimeoutMs; + this.eventPublisher = eventPublisher; + } + + @Override + public NodeState nodeState(NodeRef nodeRef) { + var pod = podOps.get(namespace, nodeRef.podName()); + if (pod == null) { + throw new UnrestartableNodesException("Pod " + nodeRef.podName() + " does not exist: "); + } else if (pod.getStatus() == null) { + return NodeState.NOT_RUNNING; + } else { + if (podOps.isReady(namespace, nodeRef.podName())) { + return NodeState.READY; + } else { + if (pendingAndUnschedulable(pod)) { + return NodeState.NOT_RUNNING; // NOT_RUNNING is more of a "likely stuck in not ready" + } else if (hasWaitingContainerWithReason(pod, Set.of("CrashLoopBackoff", "ImagePullBackoff"))) { + return NodeState.NOT_RUNNING; + } + return NodeState.NOT_READY; + } + } + + } + + private static boolean hasWaitingContainerWithReason(Pod pod, Set reasons) { + return pod.getStatus().getContainerStatuses().stream().anyMatch(cs -> { + if (cs.getState() != null && cs.getState().getWaiting() != null) { + var waitingReason = cs.getState().getWaiting().getReason(); + return reasons.contains(waitingReason); + } else { + return false; + } + }); + } + + private static boolean pendingAndUnschedulable(Pod pod) { + return "Pending".equals(pod.getStatus().getPhase()) && pod.getStatus().getConditions().stream().anyMatch( + c -> "PodScheduled".equals(c.getType()) + && "False".equals(c.getStatus()) + && "Unschedulable".equals(c.getReason())); + } + + @Override + public void restartNode(NodeRef nodeRef, RestartReasons reasons) { + var pod = podOps.get(namespace, nodeRef.podName()); + CompletableFuture cf = new CompletableFuture<>(); + podOps.restart(reconciliation, pod, operationTimeoutMs) + .onComplete(i -> { + eventPublisher.publishRestartEvents(pod, reasons); + cf.complete(true); + }); + + try { + cf.get(operationTimeoutMs, TimeUnit.MILLISECONDS); + } catch (InterruptedException | ExecutionException | TimeoutException e) { + throw new RuntimeException(e); + } + } + + @Override + public NodeRoles nodeRoles(NodeRef nodeRef) { + Pod pod = podOps.get(namespace, nodeRef.podName()); + if (pod != null) { + var podLabels = pod.getMetadata().getLabels(); + return new NodeRoles(Boolean.parseBoolean(podLabels.get(Labels.STRIMZI_CONTROLLER_ROLE_LABEL)), + Boolean.parseBoolean(podLabels.get(Labels.STRIMZI_BROKER_ROLE_LABEL))); + } else { + throw new RuntimeException("Could not find pod " + nodeRef.podName()); + } + } +} diff --git a/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/RackRolling.java b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/RackRolling.java new file mode 100644 index 0000000000..247a3af113 --- /dev/null +++ b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/RackRolling.java @@ -0,0 +1,970 @@ +/* + * Copyright Strimzi authors. + * License: Apache License 2.0 (see the file LICENSE or http://apache.org/licenses/LICENSE-2.0.html). + */ +package io.strimzi.operator.cluster.operator.resource.rolling; + +import io.strimzi.operator.cluster.model.KafkaVersion; +import io.strimzi.operator.cluster.model.NodeRef; +import io.strimzi.operator.cluster.model.RestartReason; +import io.strimzi.operator.cluster.model.RestartReasons; +import io.strimzi.operator.cluster.operator.resource.KafkaAgentClientProvider; +import io.strimzi.operator.cluster.operator.resource.KafkaBrokerConfigurationDiff; +import io.strimzi.operator.cluster.operator.resource.KafkaBrokerLoggingConfigurationDiff; +import io.strimzi.operator.cluster.operator.resource.events.KubernetesRestartEventPublisher; +import io.strimzi.operator.cluster.operator.resource.kubernetes.PodOperator; +import io.strimzi.operator.common.AdminClientProvider; +import io.strimzi.operator.common.Reconciliation; +import io.strimzi.operator.common.ReconciliationLogger; +import io.strimzi.operator.common.UncheckedExecutionException; +import io.strimzi.operator.common.UncheckedInterruptedException; +import io.strimzi.operator.common.auth.TlsPemIdentity; +import io.vertx.core.Future; +import io.vertx.core.Promise; +import io.vertx.core.Vertx; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeoutException; +import java.util.function.BiConsumer; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * RackRolling + */ +@SuppressWarnings({"ParameterNumber" }) +public class RackRolling { + + private static final ReconciliationLogger LOGGER = ReconciliationLogger.create(RackRolling.class); + private static final String CONTROLLER_QUORUM_FETCH_TIMEOUT_MS_CONFIG_NAME = "controller.quorum.fetch.timeout.ms"; + private static final long CONTROLLER_QUORUM_FETCH_TIMEOUT_MS_CONFIG_DEFAULT = 2000L; + private final List contexts; + + enum Action { + // Used for brokers that are initially healthy and require neither restart no reconfigure + NOP, + // Used in for nodes that are not healthy and require neither restart + WAIT_FOR_READINESS, + WAIT_FOR_LOG_RECOVERY, + // Used in {@link #initialPlan(List, RollClient)} for nodes that require reconfigure + // before we know whether the actual config changes are reconfigurable + MAYBE_RECONFIGURE, + // Used in {@link #refinePlanForReconfigurability(Reconciliation, KafkaVersion, Function, String, RollClient, Map)} + // once we know a MAYBE_RECONFIGURE node can actually be reconfigured + RECONFIGURE, + RESTART_UNHEALTHY, + RESTART, + } + + /** + * Constructs RackRolling instance and initializes contexts for given {@code nodes} + * to do a rolling restart (or reconfigure) of them. + * + * @param podOperator Pod operator for managing pods + * @param nodes The nodes (not all of which may need restarting). + * @param reconciliation Reconciliation marker + * @param predicate The predicate used to determine whether to restart a particular node + * @param coTlsPemIdentity Cluster operator PEM identity + * @param adminClientProvider Kafka Admin client provider + * @param kafkaAgentClientProvider Kafka Agent client provider + * @param kafkaVersion Kafka version + * @param allowReconfiguration Flag indicting whether reconfiguration is allowed or not + * @param kafkaConfigProvider Kafka configuration provider + * @param kafkaLogging Kafka logging configuration + * @param postOperationTimeoutMs The maximum time in milliseconds to wait after a restart or reconfigure + * @param maxRestartBatchSize The maximum number of nodes that might be restarted at once + * @param maxRestarts The maximum number of restart that can be done for a node + * @param maxReconfigs The maximum number of reconfiguration that can be done for a node + * @param maxAttempts The maximum number to operational attempt that can be done for a node + * @param eventPublisher Kubernetes Events publisher for publishing events about node restarts + * @return RackRolling instance + */ + public static RackRolling rollingRestart(PodOperator podOperator, + Collection nodes, + Reconciliation reconciliation, + Function predicate, + TlsPemIdentity coTlsPemIdentity, + AdminClientProvider adminClientProvider, + KafkaAgentClientProvider kafkaAgentClientProvider, + Function kafkaConfigProvider, + boolean allowReconfiguration, + KafkaVersion kafkaVersion, + String kafkaLogging, + long postOperationTimeoutMs, + int maxRestartBatchSize, + int maxRestarts, + int maxReconfigs, + int maxAttempts, + KubernetesRestartEventPublisher eventPublisher) { + PlatformClient platformClient = new PlatformClientImpl(podOperator, reconciliation.namespace(), reconciliation, postOperationTimeoutMs, eventPublisher); + Time time = Time.SYSTEM_TIME; + final var contexts = nodes.stream().map(node -> Context.start(node, platformClient.nodeRoles(node), predicate, time)).collect(Collectors.toList()); + + RollClient rollClient = new RollClientImpl(reconciliation, coTlsPemIdentity, adminClientProvider); + AgentClient agentClient = new AgentClientImpl(kafkaAgentClientProvider.createKafkaAgentClient(reconciliation, coTlsPemIdentity)); + + return new RackRolling(time, + platformClient, + rollClient, + agentClient, + reconciliation, + kafkaVersion, + allowReconfiguration, + kafkaConfigProvider, + kafkaLogging, + postOperationTimeoutMs, + maxRestartBatchSize, + maxRestarts, + maxReconfigs, + maxAttempts, + contexts); + } + + // visible for testing + protected static RackRolling rollingRestart(Time time, + PlatformClient platformClient, + RollClient rollClient, + AgentClient agentClient, + Collection nodes, + Function predicate, + Reconciliation reconciliation, + KafkaVersion kafkaVersion, + boolean allowReconfiguration, + Function kafkaConfigProvider, + String desiredLogging, + long postOperationTimeoutMs, + int maxRestartBatchSize, + int maxRestarts, + int maxReconfigs, + int maxAttempts) { + final var contexts = nodes.stream().map(node -> Context.start(node, platformClient.nodeRoles(node), predicate, time)).collect(Collectors.toList()); + + return new RackRolling(time, + platformClient, + rollClient, + agentClient, + reconciliation, + kafkaVersion, + allowReconfiguration, + kafkaConfigProvider, + desiredLogging, + postOperationTimeoutMs, + maxRestartBatchSize, + maxRestarts, + maxReconfigs, + maxAttempts, + contexts); + } + + private final Time time; + private final PlatformClient platformClient; + private final RollClient rollClient; + private final AgentClient agentClient; + private final Reconciliation reconciliation; + private final KafkaVersion kafkaVersion; + private final boolean allowReconfiguration; + private final Function kafkaConfigProvider; + private final String desiredLogging; + private final long postOperationTimeoutMs; + private final int maxRestartBatchSize; + private final int maxRestarts; + private final int maxReconfigs; + private final int maxAttempts; + + /** + * Constructor for RackRolling instance + * @param time initial time to set for context + * @param platformClient client for platform calls + * @param rollClient client for kafka cluster admin calls + * @param agentClient client for kafka agent calls + * @param reconciliation Reconciliation marker + * @param kafkaVersion Kafka version + * @param allowReconfiguration Flag indicting whether reconfiguration is allowed or not + * @param kafkaConfigProvider Kafka configuration provider + * @param desiredLogging Kafka logging configuration + * @param postOperationTimeoutMs The maximum time in milliseconds to wait after a restart or reconfigure + * @param maxRestartBatchSize The maximum number of nodes that might be restarted at once + * @param maxRestarts The maximum number of restart that can be done for a node + * @param maxReconfigs The maximum number of reconfiguration that can be done for a node + * @param maxAttempts The maximum number to operational attempt that can be done for a node + * @param contexts List of context for each node + */ + public RackRolling(Time time, + PlatformClient platformClient, + RollClient rollClient, + AgentClient agentClient, + Reconciliation reconciliation, + KafkaVersion kafkaVersion, + boolean allowReconfiguration, + Function kafkaConfigProvider, + String desiredLogging, + long postOperationTimeoutMs, + int maxRestartBatchSize, + int maxRestarts, + int maxReconfigs, + int maxAttempts, + List contexts) { + this.time = time; + this.platformClient = platformClient; + this.rollClient = rollClient; + this.agentClient = agentClient; + this.reconciliation = reconciliation; + this.kafkaVersion = kafkaVersion; + this.kafkaConfigProvider = kafkaConfigProvider; + this.desiredLogging = desiredLogging; + this.postOperationTimeoutMs = postOperationTimeoutMs; + this.maxRestartBatchSize = maxRestartBatchSize; + this.maxRestarts = maxRestarts; + this.maxReconfigs = maxReconfigs; + this.contexts = contexts; + this.maxAttempts = maxAttempts; + this.allowReconfiguration = allowReconfiguration; + } + + /** + * Runs the roller via single thread Executor + * + * @param vertx Vertx instance + * @return a future based on the rolling outcome. + */ + public Future executeRollingAsync( + Vertx vertx) { + + Promise result = Promise.promise(); + var singleExecutor = Executors.newSingleThreadScheduledExecutor( + runnable -> new Thread(runnable, "kafka-roller")); + try { + singleExecutor.submit(() -> { + try { + executeRolling(); + vertx.runOnContext(ig -> result.complete()); + } catch (Exception e) { + LOGGER.debugCr(reconciliation, "Something went wrong when trying to do a rolling restart", e); + vertx.runOnContext(ig -> result.fail(e)); + } + }); + } finally { + try { + System.out.println("Closing controller admin client"); + rollClient.closeControllerAdminClient(); + } catch (RuntimeException e) { + LOGGER.debugCr(reconciliation, "Exception closing controller admin client", e); + } + + try { + System.out.println("Closing broker admin client"); + rollClient.closeBrokerAdminClient(); + } catch (RuntimeException e) { + LOGGER.debugCr(reconciliation, "Exception closing broker admin client", e); + } + + singleExecutor.shutdown(); + } + return result.future(); + } + + private void executeRolling() throws InterruptedException, ExecutionException { + List nodesToRestart; + do { + nodesToRestart = loop(); + } while (!nodesToRestart.isEmpty()); + } + + /** + * Process each context to determine which nodes need restarting. + * Nodes that are not ready (in the Kubernetes sense) will always be considered for restart before any others. + * The given {@code predicate} will be called for each of the remaining nodes and those for which the function returns a non-empty + * list of reasons will be restarted. + * + * The expected worst case execution time of this function is approximately + * {@code (timeoutMs * maxRestarts + postOperationTimeoutMs) * size(nodes)}. + * This is reached when: + *
    + *
  1. We initially attempt to reconfigure the nodes that have configuration changes
  2. + *
  3. If reconfigurations fail after {@code maxReconfigs}, so we resort to restarts
  4. + *
  5. We require {@code maxRestarts} restarts for each node, and each restart uses the + * maximum {@code timeoutMs}.
  6. + *
+ * + * If a broker node is restarted by this method (because the {@code predicate} function returned empty), then + * it will be elected as a leader of all its preferred replicas if it's not leading them yet. + * However, if failed to lead them within a certain time, this will result in a warning log and moving onto the next node. + * + * This method is executed repeatedly until there is no nodes left to restart or reconfigure or max attempt is reached for any node. + * If this method completes normally then all initially unready nodes and the nodes for which the {@code predicate} function returned + * a non-empty list of reasons (which may be no nodes) will have been successfully restarted and + * nodes that have configurations changed will have been reconfigured. + * In other words, successful return from this method indicates that all nodes seem to be up and + * "functioning normally". + * If a node fails to become ready after a restart (e.g. recovering its logs) within a certain time, it will be retried for a restart or wait + * until the maximum restart or maximum attempt has reached. + * + * If the maximum restart reached for any node, this method will throw MaxRestartsExceededException. + * If the maximum attempt reached for any node, this method will throw MaxAttemptsExceededException. + * If any node is not running but has an up-to-date revision, this method will throw UnrestartableNodesException. + * + * @return list of nodes to retry + * @throws InterruptedException UncheckedInterruptionException The thread was interrupted + * @throws ExecutionException UncheckedExecutionException Execution exception from clients + **/ + public List loop() throws InterruptedException, ExecutionException { + try { + // Observe current state and update the contexts + for (var context : contexts) { + context.transitionTo(observe(reconciliation, platformClient, agentClient, context.nodeRef()), time); + } + + // We want to give nodes chance to get ready before we try to connect to the or consider them for rolling. + // This is important especially for nodes which were just started. + waitForNodeReadiness(contexts.stream().filter(context -> context.state().equals(State.NOT_READY)).collect(Collectors.toList()), + (c, e) -> { }); + + var byPlan = initialPlan(contexts, rollClient); + LOGGER.debugCr(reconciliation, "Initial plan: {}", byPlan.entrySet().stream().map(plan -> String.format("\n %s=%s", plan.getKey(), plan.getValue())).collect(Collectors.toSet())); + + if (!byPlan.getOrDefault(Action.WAIT_FOR_LOG_RECOVERY, List.of()).isEmpty()) { + return waitForLogRecovery(byPlan.get(Action.WAIT_FOR_LOG_RECOVERY)); + } + + // Restart any initially unready nodes + if (!byPlan.getOrDefault(Action.RESTART_UNHEALTHY, List.of()).isEmpty()) { + return restartUnhealthyNodes(byPlan.get(Action.RESTART_UNHEALTHY)); + } + + if (!byPlan.getOrDefault(Action.WAIT_FOR_READINESS, List.of()).isEmpty()) { + return waitForNodeReadiness(byPlan.get(Action.WAIT_FOR_READINESS)); + } + + rollClient.initialiseControllerAdmin(contexts.stream().filter(c -> c.currentRoles().controller()).map(Context::nodeRef).collect(Collectors.toSet())); + rollClient.initialiseBrokerAdmin(contexts.stream().filter(c -> c.currentRoles().broker()).map(Context::nodeRef).collect(Collectors.toSet())); + + // Refine the plan, reassigning nodes under MAYBE_RECONFIGURE to either RECONFIGURE or RESTART + // based on whether they have only reconfiguration config changes + List maybeConfigureNodes = byPlan.getOrDefault(Action.MAYBE_RECONFIGURE, List.of()); + if (allowReconfiguration && !maybeConfigureNodes.isEmpty()) { + var nodeConfigs = getNodeConfigs(rollClient, maxAttempts, maybeConfigureNodes); + if (nodeConfigs == null) { + return maybeConfigureNodes.stream().map(Context::nodeId).toList(); + } + + byPlan = refinePlanForReconfigurability(reconciliation, + kafkaVersion, + kafkaConfigProvider, + desiredLogging, + maybeConfigureNodes, + nodeConfigs, + byPlan); + LOGGER.debugCr(reconciliation, "Refined plan: {}", byPlan.entrySet().stream().map(plan -> String.format("\n %s=%s", plan.getKey(), plan.getValue())).collect(Collectors.toSet())); + } + + // Reconfigure any reconfigurable nodes + if (!byPlan.getOrDefault(Action.RECONFIGURE, List.of()).isEmpty()) { + return reconfigureNodes(byPlan.get(Action.RECONFIGURE)); + } + + // If we get this far then all remaining nodes require a restart + if (!byPlan.getOrDefault(Action.RESTART, List.of()).isEmpty()) { + return restartNodes(byPlan.get(Action.RESTART)); + } + + if (contexts.stream().allMatch(context -> context.state().equals(State.READY) && context.numRestarts() > 1)) { + LOGGER.debugCr(reconciliation, "Reconciliation completed successfully: All nodes are ready after restart"); + return List.of(); + } + + return contexts.stream() + .filter(c -> !c.state().equals(State.READY)) + .map(c -> c.nodeRef().nodeId()) + .collect(Collectors.toList()); + } catch (UncheckedInterruptedException e) { + throw e.getCause(); + } catch (UncheckedExecutionException e) { + throw e.getCause(); + } + } + + /** + * Makes observations of server of the given context, and return the corresponding state. + * @param nodeRef The node + * @return The state + */ + private static State observe(Reconciliation reconciliation, PlatformClient platformClient, AgentClient agentClient, NodeRef nodeRef) { + State state; + var nodeState = platformClient.nodeState(nodeRef); + LOGGER.debugCr(reconciliation, "Node {}: nodeState is {}", nodeRef, nodeState); + switch (nodeState) { + case NOT_RUNNING: + state = State.NOT_RUNNING; + break; + case READY: + state = State.READY; + break; + case NOT_READY: + default: + try { + var bs = agentClient.getBrokerState(nodeRef); + LOGGER.debugCr(reconciliation, "Node {}: brokerState is {}", nodeRef, bs); + if (bs.value() >= BrokerState.RUNNING.value() && bs.value() != BrokerState.UNKNOWN.value()) { + state = State.READY; + } else if (bs.value() == BrokerState.RECOVERY.value()) { + LOGGER.warnCr(reconciliation, "Node {} is in log recovery. There are {} logs and {} segments left to recover", nodeRef.nodeId(), bs.remainingLogsToRecover(), bs.remainingSegmentsToRecover()); + state = State.RECOVERING; + } else { + state = State.NOT_READY; + } + } catch (Exception e) { + LOGGER.warnCr(reconciliation, "Could not get broker state for node {}. This might be temporary if a node was just restarted", nodeRef, e.getCause()); + state = State.NOT_READY; + } + } + LOGGER.debugCr(reconciliation, "Node {}: observation outcome is {}", nodeRef, state); + return state; + } + + private List waitForNodeReadiness(List contexts, BiConsumer timeoutHandler) { + long remainingTimeoutMs = postOperationTimeoutMs; + for (Context context : contexts) { + try { + remainingTimeoutMs = awaitState(reconciliation, time, platformClient, agentClient, context, State.READY, remainingTimeoutMs); + } catch (TimeoutException e) { + timeoutHandler.accept(context, e); + } + } + return contexts.stream().map(Context::nodeId).collect(Collectors.toList()); + } + + private static long awaitState(Reconciliation reconciliation, + Time time, + PlatformClient platformClient, + AgentClient agentClient, + Context context, + State targetState, + long timeoutMs) throws TimeoutException { + LOGGER.debugCr(reconciliation, "Node {}: Waiting for node to enter state {}", context, targetState); + return Alarm.timer( + time, + timeoutMs, + () -> "Failed to reach " + targetState + " within " + timeoutMs + " ms: " + context + ).poll(1_000, () -> { + var state = context.transitionTo(observe(reconciliation, platformClient, agentClient, context.nodeRef()), time); + return state == targetState; + }); + } + + private Map> initialPlan(List contexts, RollClient rollClient) { + return contexts.stream().collect(Collectors.groupingBy(context -> { + if (context.state() == State.NOT_RUNNING) { + LOGGER.debugCr(reconciliation, "{} is in {} state therefore may get restarted first", context.nodeRef(), context.state()); + return Action.RESTART_UNHEALTHY; + + } else if (context.state() == State.RECOVERING) { + LOGGER.debugCr(reconciliation, "{} is in log recovery therefore will not be restarted", context.nodeRef()); + return Action.WAIT_FOR_LOG_RECOVERY; + + } else if (!rollClient.canConnectToNode(context.nodeRef(), context.currentRoles().controller())) { + LOGGER.debugCr(reconciliation, "{} will be restarted because it does not seem to responding to connection attempt", context.nodeRef()); + context.reason().add(RestartReason.POD_UNRESPONSIVE); + return Action.RESTART_UNHEALTHY; + + } else { + var reasons = context.reason(); + if (reasons.getReasons().isEmpty()) { + //TODO: When controller node can be reconfigure, always return Action.MAYBE_RECONFIGURE when node state is ready. + return context.state().equals(State.READY) ? + context.currentRoles().controller() && !context.currentRoles().broker() ? Action.NOP : Action.MAYBE_RECONFIGURE + : Action.WAIT_FOR_READINESS; + } + + if (context.numRestarts() > 0) { + return context.state().equals(State.READY) ? Action.NOP : Action.WAIT_FOR_READINESS; + } + + return Action.RESTART; + } + })); + } + + private List waitForLogRecovery(List contexts) { + return waitForNodeReadiness(contexts, (c, e) -> { + var brokerState = agentClient.getBrokerState(c.nodeRef()); + LOGGER.debugCr(reconciliation, "Node {} is still in log recovery. There are {} logs and {} segments left to recover.", c.nodeRef(), brokerState.remainingLogsToRecover(), brokerState.remainingSegmentsToRecover()); + if (c.numAttempts() >= maxAttempts) { + throw new MaxAttemptsExceededException("The max attempts (" + maxAttempts + ") to wait for this node " + c.nodeRef() + " to finish performing log recovery has been reached. " + + "There are " + brokerState.remainingLogsToRecover() + " logs and " + brokerState.remainingSegmentsToRecover() + " segments left to recover."); + } + c.incrementNumAttempts(); + }); + } + + private List restartUnhealthyNodes(List contexts) { + Set notRunningControllers = new HashSet<>(); + Set pureControllerNodesToRestart = new HashSet<>(); + Set combinedNodesToRestart = new HashSet<>(); + + for (var c : contexts) { + if (c.state() == State.NOT_RUNNING) { + if (!c.reason().contains(RestartReason.POD_HAS_OLD_REVISION)) { + // If the node is not running (e.g. unschedulable) then restarting it, likely won't make any difference. + // Proceeding and deleting another node may result in it not running too. Avoid restarting it unless it has an old revision. + throw new UnrestartableNodesException("Pod is unschedulable or is not starting"); + } + + // Collect all not running controllers to restart them in parallel + if (c.currentRoles().controller()) { + notRunningControllers.add(c); + continue; + } + } + + if (c.currentRoles().controller()) { + if (c.currentRoles().broker()) { + combinedNodesToRestart.add(c); + } else { + // we always restart a pure controller first, so if we have any, we exit the loop here + pureControllerNodesToRestart.add(c); + break; + } + } + } + + if (!notRunningControllers.isEmpty()) { + LOGGER.debugCr(reconciliation, "There are multiple controllers {} that are not running, which runs a risk of losing the quorum. Restarting them in parallel", notRunningControllers); + restartInParallel(reconciliation, time, platformClient, rollClient, agentClient, notRunningControllers, postOperationTimeoutMs, maxRestarts); + return notRunningControllers.stream().map(Context::nodeId).collect(Collectors.toList()); + } + + // restart in the following order: pure controllers, combined nodes and brokers + Context nodeToRestart = !pureControllerNodesToRestart.isEmpty() ? pureControllerNodesToRestart.iterator().next() + : !combinedNodesToRestart.isEmpty() ? combinedNodesToRestart.iterator().next() + : contexts.get(0); + + restartInParallel(reconciliation, time, platformClient, rollClient, agentClient, Collections.singleton(nodeToRestart), postOperationTimeoutMs, maxRestarts); + return Collections.singletonList(nodeToRestart.nodeId()); + } + + private void restartInParallel(Reconciliation reconciliation, + Time time, + PlatformClient platformClient, + RollClient rollClient, + AgentClient agentClient, + Set batch, + long timeoutMs, + int maxRestarts) { + for (Context context : batch) { + restartNode(reconciliation, time, platformClient, context, maxRestarts); + } + //TODO: Apply post restart delay here + long remainingTimeoutMs = timeoutMs; + for (Context context : batch) { + try { + remainingTimeoutMs = awaitState(reconciliation, time, platformClient, agentClient, context, State.READY, remainingTimeoutMs); + if (context.currentRoles().broker()) { + awaitPreferred(reconciliation, time, rollClient, context, remainingTimeoutMs); + } + } catch (TimeoutException e) { + LOGGER.warnCr(reconciliation, "Timed out waiting for node {} to become ready after a restart", context.nodeRef()); + if (context.numAttempts() >= maxAttempts) { + throw new MaxAttemptsExceededException("Cannot restart node " + context.nodeRef() + + " because they violate quorum health or topic availability. " + + "The max attempts (" + maxAttempts + ") to retry the nodes has been reached."); + } else { + context.incrementNumAttempts(); + return; + } + } + } + } + + private static void restartNode(Reconciliation reconciliation, + Time time, + PlatformClient platformClient, + Context context, + int maxRestarts) { + if (context.numRestarts() >= maxRestarts) { + throw new MaxRestartsExceededException("Node " + context.nodeRef() + " has been restarted " + maxRestarts + " times"); + } + LOGGER.debugCr(reconciliation, "Node {}: Restarting", context.nodeRef()); + try { + platformClient.restartNode(context.nodeRef(), context.reason()); + } catch (RuntimeException e) { + LOGGER.warnCr(reconciliation, "An exception thrown during the restart of the node {}", context.nodeRef(), e); + } + context.incrementNumRestarts(); + context.transitionTo(State.UNKNOWN, time); + LOGGER.debugCr(reconciliation, "Node {}: Restarted", context.nodeRef()); + } + + private static void awaitPreferred(Reconciliation reconciliation, + Time time, + RollClient rollClient, + Context context, + long timeoutMs) { + // TODO: apply configured delay (via env variable) before triggering leader election. + // This should be probably passed to tryElectAllPreferredLeaders so that delay is only applied + // if there are topic partitions to elect, otherwise no point of delaying the process + time.sleep(10000L, 0); + LOGGER.debugCr(reconciliation, "Node {}: Waiting for node to be leader of all its preferred replicas", context); + try { + Alarm.timer(time, + timeoutMs, + () -> "Failed to elect the preferred leader " + context + " for topic partitions within " + timeoutMs) + .poll(1_000, () -> rollClient.tryElectAllPreferredLeaders(context.nodeRef()) == 0); + } catch (TimeoutException e) { + LOGGER.warnCr(reconciliation, "Timed out waiting for node to be leader for all its preferred replicas"); + } + } + + private List waitForNodeReadiness(List unreadyNodes) { + if (!unreadyNodes.isEmpty()) { + LOGGER.debugCr(reconciliation, "Waiting for nodes {} to become ready before initialising plan in case they just started", unreadyNodes); + waitForNodeReadiness(unreadyNodes, (c, e) -> { + if (c.numAttempts() >= maxAttempts) { + String restartedState = c.numRestarts() > 0 ? "restarted" : "non-restarted"; + throw new MaxAttemptsExceededException("The max attempts (" + maxAttempts + ") to wait for " + restartedState + " node " + c.nodeRef() + " to become ready has been reached."); + } + c.incrementNumAttempts(); + }); + } + return unreadyNodes.stream().map(Context::nodeId).collect(Collectors.toList()); + } + + private Map getNodeConfigs(RollClient rollClient, int maxAttempts, List contexts) { + //TODO: deal with controller node config here + Map nodeConfigs = null; + try { + nodeConfigs = rollClient.describeBrokerConfigs(contexts.stream() + .map(Context::nodeRef).toList()); + } catch (Exception e) { + LOGGER.errorCr(reconciliation, "Error getting configs for : " + contexts, e.getCause()); + contexts.forEach(c -> { + if (c.numAttempts() < maxAttempts) { + c.incrementNumAttempts(); + } else { + c.reason().add(RestartReason.POD_FORCE_RESTART_ON_ERROR); + } + }); + } + + return nodeConfigs; + } + + private static Map> refinePlanForReconfigurability(Reconciliation reconciliation, + KafkaVersion kafkaVersion, + Function kafkaConfigProvider, + String desiredLogging, + List contexts, + Map nodeConfigs, + Map> byPlan) { + var refinedPlan = contexts.stream().collect(Collectors.groupingBy(context -> { + Configs configPair = nodeConfigs.get(context.nodeId()); + + var diff = new KafkaBrokerConfigurationDiff(reconciliation, + configPair.nodeConfigs(), + kafkaConfigProvider.apply(context.nodeId()), + kafkaVersion, + context.nodeRef()); + context.brokerConfigDiff(diff); + + var loggingDiff = new KafkaBrokerLoggingConfigurationDiff(reconciliation, configPair.nodeLoggerConfigs(), desiredLogging); + context.loggingDiff(loggingDiff); + + if (!diff.isEmpty()) { + if (diff.canBeUpdatedDynamically() || !loggingDiff.isEmpty()) { + return Action.RECONFIGURE; + } else { + context.reason().add(RestartReason.CONFIG_CHANGE_REQUIRES_RESTART); + return Action.RESTART; + } + } + return Action.NOP; + })); + + return Map.of( + Action.RESTART, Stream.concat(byPlan.getOrDefault(Action.RESTART, List.of()).stream(), refinedPlan.getOrDefault(Action.RESTART, List.of()).stream()).toList(), + Action.RECONFIGURE, refinedPlan.getOrDefault(Action.RECONFIGURE, List.of()), + Action.NOP, Stream.concat(byPlan.getOrDefault(Action.NOP, List.of()).stream(), refinedPlan.getOrDefault(Action.NOP, List.of()).stream()).toList() + ); + } + + private List reconfigureNodes(List contexts) { + //TODO: reconfigure controller node + List reconfiguredNode = new ArrayList<>(); + for (var context : contexts) { + if (context.numReconfigs() >= maxReconfigs) { + LOGGER.warnCr(reconciliation, "The maximum number of configuration attempt reached for node {}, will be restarted.", context.nodeRef()); + context.reason().add(RestartReason.CONFIG_CHANGE_REQUIRES_RESTART); + return List.of(context.nodeId()); + } + + try { + reconfigureNode(reconciliation, time, rollClient, context); + } catch (Exception e) { + LOGGER.warnCr(reconciliation, "Failed to reconfigure {} due to {}", context.nodeRef(), e); + context.incrementNumReconfigs(); + return List.of(context.nodeId()); + } + + time.sleep(postOperationTimeoutMs / 2, 0); + waitForNodeReadiness(Collections.singletonList(context)); + // TODO decide whether we need an explicit healthcheck here + // or at least to know that the kube health check probe will have failed at the time + // we break to OUTER (We need to test a scenario of breaking configuration change, does this sleep catch it?) + reconfiguredNode.add(context.nodeId()); + break; + } + return reconfiguredNode; + } + + private static void reconfigureNode(Reconciliation reconciliation, + Time time, + RollClient rollClient, + Context context) { + LOGGER.debugCr(reconciliation, "Node {}: Reconfiguring", context.nodeRef()); + rollClient.reconfigureNode(context.nodeRef(), context.brokerConfigDiff(), context.loggingDiff()); + context.incrementNumReconfigs(); + context.transitionTo(State.UNKNOWN, time); + LOGGER.debugCr(reconciliation, "Node {}: Reconfigured", context.nodeRef()); + } + + private List restartNodes(List nodesToRestart) { + // determine batches of nodes to be restarted together + // for controller nodes, a batch with a single node will be returned + var batch = nextBatch(reconciliation, rollClient, nodesToRestart, maxRestartBatchSize); + + // Empty batch means, there is no node that can safely restarted without violating quorum health or availability. + if (batch.isEmpty()) { + // check if the maxAttempt for any of the nodes has reached + nodesToRestart.forEach(c -> { + if (c.numAttempts() >= maxAttempts) { + throw new MaxAttemptsExceededException("Cannot restart nodes " + nodesToRestart.stream().map(Context::nodeRef).toList() + + " because they violate quorum health or topic availability. " + + "The max attempts (" + maxAttempts + ") to retry the nodes has been reached."); + } + c.incrementNumAttempts(); + }); + // sleep and retry the nodes + time.sleep(postOperationTimeoutMs, 0); + return nodesToRestart.stream().map(Context::nodeId).collect(Collectors.toList()); + } + + LOGGER.debugCr(reconciliation, "Restart batch: {}", batch); + // restart a batch + restartInParallel(reconciliation, time, platformClient, rollClient, agentClient, batch, postOperationTimeoutMs, maxRestarts); + return batch.stream().map(Context::nodeId).collect(Collectors.toList()); + } + + /** + * Figures out a batch of nodes that can be restarted together. + * This method enforces the following roll order: + *
    + *
  1. Pure controller
  2. + *
  3. Combined node
  4. + *
  5. Active controller
  6. + *
  7. Broker (only this case is parallelizable)
  8. + *
+ * + * @param rollClient The roll client + * @param nodesToRestart The ids of the nodes which need to be restarted + * @param maxRestartBatchSize The maximum allowed size for a batch + * @return The nodes corresponding to a subset of {@code nodeIdsNeedingRestart} that can safely be rolled together + */ + private Set nextBatch(Reconciliation reconciliation, + RollClient rollClient, + List nodesToRestart, + int maxRestartBatchSize) { + + var controllersToRestart = nodesToRestart.stream().filter(c -> c.currentRoles().controller()).collect(Collectors.toList()); + if (!controllersToRestart.isEmpty()) { + return nextControllerToRestart(reconciliation, rollClient, controllersToRestart); + } + + return nextBatchBrokers(reconciliation, rollClient, nodesToRestart, maxRestartBatchSize); + } + + /** + * @param reconciliation The roll client + * @param controllersToStart controllers to restarts + * @return The first one from the given list of nodes that can be restarted without impacting the quorum health. + * If there is no node that doesn't have an impact on the quorum health, an empty set is returned. + */ + private Set nextControllerToRestart(Reconciliation reconciliation, RollClient rollClient, + List controllersToStart) { + + int activeControllerId = rollClient.activeController(); + LOGGER.debugCr(reconciliation, "The active controller is {}", activeControllerId); + + if (activeControllerId < 0) { + // if we can't determine the active controller, we cannot safely restart a controller node + // To retry, we return an empty set + return Collections.emptySet(); + } + + var orderedNodes = controllersToStart.stream().sorted(Comparator.comparing((Context c) -> c.state().equals(State.READY)) // Sort by the state (ready goes to the back) + .thenComparing(c -> c.currentRoles().broker())); // Sort by the roles (combined goes to the back))) + LOGGER.debugCr(reconciliation, "Checking controllers in the following order to restart: {}", controllersToStart); + + Set nextNodeToRestart = new HashSet<>(); + orderedNodes.anyMatch(c -> { + if (c.nodeId() == activeControllerId && controllersToStart.size() != 1) { + LOGGER.debugCr(reconciliation, "Controller node {} is the active controller, there are other controller nodes to restart", c.nodeId()); + return false; + } + + if (isQuorumHealthyWithoutNode(reconciliation, c.nodeId(), activeControllerId, rollClient)) { + // if this node is combined, then we have to check the availability as well + if (c.currentRoles().broker()) { + Availability availability = new Availability(reconciliation, rollClient); + if (availability.anyPartitionWouldBeUnderReplicated(c.nodeId())) { + LOGGER.debugCr(reconciliation, "Combined node {} cannot be safely restarted without impacting the availability", c.nodeId()); + return false; + } + } + + LOGGER.debugCr(reconciliation, "Controller node {} can be safely restarted", c.nodeId()); + nextNodeToRestart.add(c); + return true; + } else { + LOGGER.debugCr(reconciliation, "Controller node {} cannot be safely restarted without impacting quorum health", c.nodeId()); + } + return false; + }); + + if (nextNodeToRestart.isEmpty()) LOGGER.warnCr(reconciliation, "None of the following controller nodes can be safely restarted: {}", controllersToStart); + return nextNodeToRestart; + } + + /** + * Returns true if the majority of the controllers' lastCaughtUpTimestamps are within + * the controller.quorum.fetch.timeout.ms based on the given quorum info. + * The given controllerNeedRestarting is the one being considered to restart, therefore excluded from the check. + * + * The total number of controller is passed in to this method rather than using the size of the quorum followers + * returned from the Admin. This is because when scaling down controllers, the returned quorum info from them + * could contain inconsistent number of followers. + */ + private boolean isQuorumHealthyWithoutNode(Reconciliation reconciliation, + int controllerNeedRestarting, + int activeControllerId, + RollClient rollClient) { + LOGGER.debugCr(reconciliation, "Determining the impact of restarting controller {} on quorum health", controllerNeedRestarting); + Map quorumFollowerStates = rollClient.quorumLastCaughtUpTimestamps(contexts.stream().filter(c -> c.nodeId() == activeControllerId).map(Context::nodeRef).collect(Collectors.toSet())); + int controllerCount = quorumFollowerStates.size(); + if (controllerCount == 1) { + LOGGER.warnCr(reconciliation, "Performing rolling update on controller quorum with a single node. The cluster may be " + + "in a defective state once the rolling update is complete. It is recommended that a minimum of three controllers are used."); + return true; + } + + var activeController = contexts.stream().filter(c -> c.nodeId() == activeControllerId).map(Context::nodeRef).toList(); + var config = rollClient.describeControllerConfigs(activeController); + var nodeConfigs = (config != null) && (config.get(0) != null) ? config.get(0).nodeConfigs() : null; + var controllerQuorumFetchTimeoutValue = (nodeConfigs != null) ? nodeConfigs.get(CONTROLLER_QUORUM_FETCH_TIMEOUT_MS_CONFIG_NAME) : null; + var controllerQuorumFetchTimeout = controllerQuorumFetchTimeoutValue == null ? + CONTROLLER_QUORUM_FETCH_TIMEOUT_MS_CONFIG_DEFAULT : Long.parseLong(controllerQuorumFetchTimeoutValue.value()); + + long leaderLastCaughtUpTimestamp = quorumFollowerStates.get(activeControllerId); + + long numOfCaughtUpControllers = quorumFollowerStates.entrySet().stream().filter(entry -> { + int nodeId = entry.getKey(); + long lastCaughtUpTimestamp = entry.getValue(); + if (lastCaughtUpTimestamp < 0) { + LOGGER.errorCr(reconciliation, "No valid lastCaughtUpTimestamp is found for controller {} ", nodeId); + } else { + LOGGER.debugCr(reconciliation, "The lastCaughtUpTimestamp for controller {} is {}", nodeId, lastCaughtUpTimestamp); + if (nodeId == activeControllerId || (leaderLastCaughtUpTimestamp - lastCaughtUpTimestamp) < controllerQuorumFetchTimeout) { + if (nodeId != controllerNeedRestarting) { + return true; + } + LOGGER.debugCr(reconciliation, "Controller {} has caught up with the controller quorum leader", nodeId); + } else { + LOGGER.debugCr(reconciliation, "Controller {} has fallen behind the controller quorum leader", nodeId); + } + } + return false; + }).count(); + + if (controllerCount == 2) { + // Only roll the controller if the other one in the quorum has caught up or is the active controller. + if (numOfCaughtUpControllers == 1) { + LOGGER.warnCr(reconciliation, "Performing rolling update on a controller quorum with 2 nodes. The cluster may be " + + "in a defective state once the rolling update is complete. It is recommended that a minimum of three controllers are used."); + return true; + } else { + return false; + } + } else { + boolean result = numOfCaughtUpControllers >= (controllerCount + 2) / 2; + if (!result) { + LOGGER.debugCr(reconciliation, "Controller {} cannot be restarted without impacting quorum health", controllerNeedRestarting); + } + return result; + } + } + + /** + * Returns a batch of broker nodes that have no topic partitions in common and have no impact on cluster availability if restarted. + */ + private Set nextBatchBrokers(Reconciliation reconciliation, + RollClient rollClient, + List nodesNeedingRestart, + int maxRestartBatchSize) { + + if (nodesNeedingRestart.size() == 1) { + return Collections.singleton(nodesNeedingRestart.get(0)); + } + + if (nodesNeedingRestart.size() < 1) { + return Collections.emptySet(); + } + + Availability availability = new Availability(reconciliation, rollClient); + + //TODO: Implement dry run for batch rolling + + // If maxRestartBatchSize is set to 1, no point executing batching algorithm so + // return the next available node that is ordered by the readiness state + if (maxRestartBatchSize == 1) { + List eligibleNodes = nodesNeedingRestart.stream() + .filter(context -> !availability.anyPartitionWouldBeUnderReplicated(context.nodeId())) + .sorted(Comparator.comparing((Context c) -> c.state().equals(State.READY))) + .toList(); + return eligibleNodes.size() > 0 ? Set.of(eligibleNodes.get(0)) : Set.of(); + } + + LOGGER.debugCr(reconciliation, "Parallel batching of broker nodes is enabled. Max batch size is {}"); + List nodes = nodesNeedingRestart.stream() + .map(c -> new KafkaNode(c.nodeId(), availability.getReplicasForNode(c.nodeId()))) + .collect(Collectors.toList()); + + // Split the set of all brokers into subsets of brokers that can be rolled in parallel + var cells = Batching.cells(reconciliation, nodes); + int cellNum = 0; + for (var cell: cells) { + LOGGER.debugCr(reconciliation, "Cell {}: {}", ++cellNum, cell); + } + + cellNum = 0; + for (var cell: cells) { + LOGGER.debugCr(reconciliation, "Restart-eligible cell {}: {}", ++cellNum, cell); + } + + var batches = Batching.batchCells(reconciliation, cells, availability, maxRestartBatchSize); + LOGGER.debugCr(reconciliation, "Batches {}", Batching.nodeIdsToString2(batches)); + + var bestBatch = Batching.pickBestBatchForRestart(batches); + LOGGER.debugCr(reconciliation, "Best batch {}", Batching.nodeIdsToString(bestBatch)); + + return nodesNeedingRestart.stream().filter(c -> bestBatch.contains(c.nodeId())).collect(Collectors.toSet()); + } + +} diff --git a/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/Replica.java b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/Replica.java new file mode 100644 index 0000000000..a043da1437 --- /dev/null +++ b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/Replica.java @@ -0,0 +1,47 @@ +/* + * Copyright Strimzi authors. + * License: Apache License 2.0 (see the file LICENSE or http://apache.org/licenses/LICENSE-2.0.html). + */ +package io.strimzi.operator.cluster.operator.resource.rolling; + +import org.apache.kafka.common.Node; + +import java.util.Collection; + +/** + * A replica on a particular {@link KafkaNode}. + * + * @param topicName The name of the topic + * @param partitionId The partition id + * @param isrSize If the broker hosting this replica is in the ISR for the partition of this replica + * this is the size of the ISR. + * If the broker hosting this replica is NOT in the ISR for the partition of this replica + * this is the negative of the size of the ISR. + * In other words, the magnitude is the size of the ISR and the sign will be negative + * if the broker hosting this replica is not in the ISR. + */ +record Replica(String topicName, int partitionId, short isrSize) { + + public Replica(Node broker, String topicName, int partitionId, Collection isr) { + this(topicName, partitionId, (short) (isr.contains(broker) ? isr.size() : -isr.size())); + } + + @Override + public String toString() { + return topicName + "-" + partitionId; + } + + /** + * @return The size of the ISR for the partition of this replica. + */ + public short isrSize() { + return (short) Math.abs(isrSize); + } + + /** + * @return true if the broker hosting this replica is in the ISR for the partition of this replica. + */ + public boolean isInIsr() { + return isrSize > 0; + } +} diff --git a/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/RollClient.java b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/RollClient.java new file mode 100644 index 0000000000..8a57bb9804 --- /dev/null +++ b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/RollClient.java @@ -0,0 +1,132 @@ +/* + * Copyright Strimzi authors. + * License: Apache License 2.0 (see the file LICENSE or http://apache.org/licenses/LICENSE-2.0.html). + */ +package io.strimzi.operator.cluster.operator.resource.rolling; + +import io.strimzi.operator.cluster.model.NodeRef; +import io.strimzi.operator.cluster.operator.resource.KafkaBrokerConfigurationDiff; +import io.strimzi.operator.cluster.operator.resource.KafkaBrokerLoggingConfigurationDiff; +import org.apache.kafka.clients.admin.TopicDescription; +import org.apache.kafka.clients.admin.TopicListing; +import org.apache.kafka.common.Uuid; + +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * An abstraction over a Kafka Admin client. + */ +interface RollClient { + /** + * Sets admin client for brokers. + **/ + void initialiseBrokerAdmin(Set brokerNodes); + + /** + * Sets admin client for controllers. + **/ + void initialiseControllerAdmin(Set controllerNodes); + + /** + * Closes controller admin client + **/ + void closeControllerAdminClient(); + + /** + * Closes broker admin client + **/ + void closeBrokerAdminClient(); + + /** + * Checks if node is responsive by connecting to it via Admin API + * @param nodeRef The node ref + * @param controller a boolean value informing if it's a controller node + * @return true if node is responsive, otherwise false + */ + boolean canConnectToNode(NodeRef nodeRef, boolean controller); + + /** + * @return All the topics in the cluster, including internal topics. + * @throws io.strimzi.operator.common.UncheckedExecutionException Execution exception from clients + * @throws io.strimzi.operator.common.UncheckedInterruptedException The thread was interrupted + */ + Collection listTopics(); + + /** + * Describe the topics with the given ids. + * If the given {@code topicIds} is large, multiple requests (to different brokers) may be used. + * @param topicIds The topic ids. + * @return The topic descriptions. + * @throws io.strimzi.operator.common.UncheckedExecutionException Execution exception from clients + * @throws io.strimzi.operator.common.UncheckedInterruptedException The thread was interrupted + */ + List describeTopics(List topicIds); + + /** + * Get the {@code min.insync.replicas} of each of the topics in the given {@code topicNames} list. + * @param topicNames The names of the topics to get the {@code min.insync.replicas} of. + * @return A map from topic name to its {@code min.insync.replicas}. + * @throws io.strimzi.operator.common.UncheckedExecutionException Execution exception from clients + * @throws io.strimzi.operator.common.UncheckedInterruptedException The thread was interrupted + */ + Map describeTopicMinIsrs(List topicNames); + + /** + * Describe the metadata quorum info. + * + * @param activeControllerNodeRef A set containing active controller NodeRef. + * @return A map from controller quorum followers to their {@code lastCaughtUpTimestamps}. + * @throws io.strimzi.operator.common.UncheckedExecutionException Execution exception from clients + * @throws io.strimzi.operator.common.UncheckedInterruptedException The thread was interrupted + */ + Map quorumLastCaughtUpTimestamps(Set activeControllerNodeRef); + + /** + * @return The id of the node that is the active controller of the cluster. + * If there is no active controller or failed to get quorum information, + * return -1 as the default value. + * @throws io.strimzi.operator.common.UncheckedExecutionException Execution exception from clients + * @throws io.strimzi.operator.common.UncheckedInterruptedException The thread was interrupted + */ + int activeController(); + + /** + * Reconfigure the given server with the given configs + * + * @param nodeRef The node + * @param kafkaBrokerConfigurationDiff The broker config diff + * @param kafkaBrokerLoggingConfigurationDiff The broker logging diff + */ + void reconfigureNode(NodeRef nodeRef, KafkaBrokerConfigurationDiff kafkaBrokerConfigurationDiff, KafkaBrokerLoggingConfigurationDiff kafkaBrokerLoggingConfigurationDiff); + + /** + * Try to elect the given server as the leader for all the replicas on the server where it's not already + * the preferred leader. + * @param nodeRef The node + * @return The number of replicas on the server which it is not leading, but is preferred leader + * @throws io.strimzi.operator.common.UncheckedExecutionException Execution exception from clients + * @throws io.strimzi.operator.common.UncheckedInterruptedException The thread was interrupted + */ + int tryElectAllPreferredLeaders(NodeRef nodeRef); + + /** + * Return the Kafka broker configs and logger configs for each of the given nodes + * @param toList The nodes to get the configs for + * @return A map from node id to configs + * @throws io.strimzi.operator.common.UncheckedExecutionException Execution exception from clients + * @throws io.strimzi.operator.common.UncheckedInterruptedException The thread was interrupted + */ + Map describeBrokerConfigs(List toList); + + /** + * Return the Kafka controller configs for each of the given nodes + * @param toList The nodes to get the configs for + * @return A map from node id to configs + * @throws io.strimzi.operator.common.UncheckedExecutionException Execution exception from clients + * @throws io.strimzi.operator.common.UncheckedInterruptedException The thread was interrupted + */ + Map describeControllerConfigs(List toList); +} diff --git a/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/RollClientImpl.java b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/RollClientImpl.java new file mode 100644 index 0000000000..4d140cc8ef --- /dev/null +++ b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/RollClientImpl.java @@ -0,0 +1,346 @@ +/* + * Copyright Strimzi authors. + * License: Apache License 2.0 (see the file LICENSE or http://apache.org/licenses/LICENSE-2.0.html). + */ +package io.strimzi.operator.cluster.operator.resource.rolling; + +import io.strimzi.api.kafka.model.kafka.KafkaResources; +import io.strimzi.operator.cluster.model.DnsNameGenerator; +import io.strimzi.operator.cluster.model.KafkaCluster; +import io.strimzi.operator.cluster.model.NodeRef; +import io.strimzi.operator.cluster.operator.resource.KafkaBrokerConfigurationDiff; +import io.strimzi.operator.cluster.operator.resource.KafkaBrokerLoggingConfigurationDiff; +import io.strimzi.operator.common.AdminClientProvider; +import io.strimzi.operator.common.Reconciliation; +import io.strimzi.operator.common.ReconciliationLogger; +import io.strimzi.operator.common.UncheckedExecutionException; +import io.strimzi.operator.common.UncheckedInterruptedException; +import io.strimzi.operator.common.Util; +import io.strimzi.operator.common.auth.PemAuthIdentity; +import io.strimzi.operator.common.auth.PemTrustSet; +import io.strimzi.operator.common.auth.TlsPemIdentity; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.AlterConfigOp; +import org.apache.kafka.clients.admin.AlterConfigsOptions; +import org.apache.kafka.clients.admin.AlterConfigsResult; +import org.apache.kafka.clients.admin.Config; +import org.apache.kafka.clients.admin.DescribeMetadataQuorumOptions; +import org.apache.kafka.clients.admin.DescribeMetadataQuorumResult; +import org.apache.kafka.clients.admin.ListTopicsOptions; +import org.apache.kafka.clients.admin.QuorumInfo; +import org.apache.kafka.clients.admin.TopicDescription; +import org.apache.kafka.clients.admin.TopicListing; +import org.apache.kafka.common.ElectionType; +import org.apache.kafka.common.KafkaFuture; +import org.apache.kafka.common.TopicCollection; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.TopicPartitionInfo; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.config.ConfigResource; +import org.apache.kafka.common.config.TopicConfig; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +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; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; + +class RollClientImpl implements RollClient { + + private final static ReconciliationLogger LOGGER = ReconciliationLogger.create(RackRolling.class); + private final static int ADMIN_BATCH_SIZE = 200; + // TODO: set to the same value of the thread blocking limit for now but we need to decide whether we need a dedicated thread for RackRolling + private final static long ADMIN_CALL_TIMEOUT = 2000L; + private Admin brokerAdmin = null; + + private Admin controllerAdmin = null; + + private final PemAuthIdentity pemAuthIdentity; + + private final PemTrustSet pemTrustSet; + + private final Reconciliation reconciliation; + + private final AdminClientProvider adminClientProvider; + RollClientImpl(Reconciliation reconciliation, + TlsPemIdentity coTlsPemIdentity, + AdminClientProvider adminClientProvider) { + this.pemTrustSet = coTlsPemIdentity.pemTrustSet(); + this.pemAuthIdentity = coTlsPemIdentity.pemAuthIdentity(); + this.reconciliation = reconciliation; + this.adminClientProvider = adminClientProvider; + } + + /** Return a future that completes when all the given futures complete */ + @SuppressWarnings("rawtypes") + private static CompletableFuture allOf(List> futures) { + CompletableFuture[] ts = futures.toArray(new CompletableFuture[0]); + return CompletableFuture.allOf(ts); + } + + /** Splits the given {@code items} into batches no larger than {@code maxBatchSize}. */ + private static Set> batch(List items, int maxBatchSize) { + Set> allBatches = new HashSet<>(); + List currentBatch = null; + for (var topicId : items) { + if (currentBatch == null || currentBatch.size() > maxBatchSize) { + currentBatch = new ArrayList<>(); + allBatches.add(currentBatch); + } + currentBatch.add(topicId); + } + return allBatches; + } + + @Override + public void initialiseBrokerAdmin(Set brokerNodes) { + if (this.brokerAdmin == null) this.brokerAdmin = createBrokerAdminClient(brokerNodes); + } + + @Override + public void initialiseControllerAdmin(Set controllerNodes) { + if (this.controllerAdmin == null) this.controllerAdmin = createControllerAdminClient(controllerNodes); + } + + @Override + public void closeControllerAdminClient() { + if (this.controllerAdmin != null) { + this.controllerAdmin.close(Duration.ofSeconds(30)); + } + } + + @Override + public void closeBrokerAdminClient() { + if (this.brokerAdmin != null) { + this.brokerAdmin.close(Duration.ofSeconds(30)); + } + } + + @Override + public boolean canConnectToNode(NodeRef nodeRef, boolean controller) { + try (Admin ignored = controller ? createControllerAdminClient(Collections.singleton(nodeRef)) : createBrokerAdminClient(Collections.singleton(nodeRef))) { + return true; + } catch (Exception e) { + LOGGER.errorCr(reconciliation, "Cannot create an admin client connection to {}", nodeRef, e); + return false; + } + } + + private Admin createControllerAdminClient(Set controllerNodes) { + String bootstrapHostnames = controllerNodes.stream().map(node -> DnsNameGenerator.podDnsName(reconciliation.namespace(), KafkaResources.brokersServiceName(reconciliation.name()), node.podName()) + ":" + KafkaCluster.CONTROLPLANE_PORT).collect(Collectors.joining(",")); + LOGGER.debugCr(reconciliation, "Creating an admin client with {}", bootstrapHostnames); + try { + return adminClientProvider.createControllerAdminClient(bootstrapHostnames, pemTrustSet, pemAuthIdentity); + } catch (RuntimeException e) { + throw new RuntimeException("Failed to create controller admin client", e.getCause()); + } + } + + private Admin createBrokerAdminClient(Set brokerNodes) { + String bootstrapHostnames = brokerNodes.stream().map(node -> DnsNameGenerator.podDnsName(reconciliation.namespace(), KafkaResources.brokersServiceName(reconciliation.name()), node.podName()) + ":" + KafkaCluster.REPLICATION_PORT).collect(Collectors.joining(",")); + LOGGER.debugCr(reconciliation, "Creating an admin client with {}", bootstrapHostnames); + try { + return adminClientProvider.createAdminClient(bootstrapHostnames, pemTrustSet, pemAuthIdentity); + } catch (RuntimeException e) { + throw new RuntimeException("Failed to create broker admin client", e.getCause()); + } + } + + @Override + public Collection listTopics() { + try { + return brokerAdmin.listTopics(new ListTopicsOptions().listInternal(true)).listings().get(ADMIN_CALL_TIMEOUT, TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + throw new UncheckedInterruptedException(e); + } catch (ExecutionException e) { + throw new UncheckedExecutionException(e); + } catch (TimeoutException e) { + throw new RuntimeException(e); + } + } + + @Override + public List describeTopics(List topicIds) { + try { + var topicIdBatches = batch(topicIds, ADMIN_BATCH_SIZE); + var futures = new ArrayList>>(); + for (var topicIdBatch : topicIdBatches) { + var mapKafkaFuture = brokerAdmin.describeTopics(TopicCollection.ofTopicIds(topicIdBatch)).allTopicIds().toCompletionStage().toCompletableFuture(); + futures.add(mapKafkaFuture); + } + allOf(futures).get(ADMIN_CALL_TIMEOUT, TimeUnit.MILLISECONDS); + var topicDescriptions = futures.stream().flatMap(cf -> { + try { + return cf.get().values().stream(); + } catch (InterruptedException e) { + throw new UncheckedInterruptedException(e); + } catch (ExecutionException e) { + throw new UncheckedExecutionException(e); + } + }); + return topicDescriptions.toList(); + } catch (InterruptedException e) { + throw new UncheckedInterruptedException(e); + } catch (ExecutionException e) { + throw new UncheckedExecutionException(e); + } catch (TimeoutException e) { + throw new RuntimeException(e); + } + } + + @Override + public Map quorumLastCaughtUpTimestamps(Set activeControllerNodeRef) { + // TODO: this should be changed to controllerAdmin when once KAFKA-18230 is completed and supported in Strimzi + // in which case, this method does not need the parameter, activeControllerNodeRef + + try (Admin admin = createControllerAdminClient(activeControllerNodeRef)) { + DescribeMetadataQuorumResult dmqr = admin.describeMetadataQuorum(new DescribeMetadataQuorumOptions()); + return dmqr.quorumInfo().get().voters().stream().collect(Collectors.toMap( + QuorumInfo.ReplicaState::replicaId, + state -> state.lastCaughtUpTimestamp().orElse(-1))); + } catch (InterruptedException e) { + throw new UncheckedInterruptedException(e); + } catch (ExecutionException e) { + throw new UncheckedExecutionException(e); + } + } + + @Override + public int activeController() { + try { + return controllerAdmin.describeCluster().controller().get().id(); + } catch (ExecutionException | InterruptedException e) { + throw new RuntimeException(e); + } + } + + @Override + public Map describeTopicMinIsrs(List topicNames) { + try { + var topicIdBatches = batch(topicNames, ADMIN_BATCH_SIZE); + var futures = new ArrayList>>(); + for (var topicIdBatch : topicIdBatches) { + var mapKafkaFuture = brokerAdmin.describeConfigs(topicIdBatch.stream().map(name -> new ConfigResource(ConfigResource.Type.TOPIC, name)).collect(Collectors.toSet())).all().toCompletionStage().toCompletableFuture(); + futures.add(mapKafkaFuture); + } + allOf(futures).get(ADMIN_CALL_TIMEOUT, TimeUnit.MILLISECONDS); + var topicDescriptions = futures.stream().flatMap(cf -> { + try { + return cf.get().entrySet().stream(); + } catch (InterruptedException e) { + throw new UncheckedInterruptedException(e); + } catch (ExecutionException e) { + throw new UncheckedExecutionException(e); + } + }); + return topicDescriptions.collect(Collectors.toMap( + entry -> entry.getKey().name(), + entry -> Integer.parseInt(entry.getValue().get(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG).value()))); + } catch (InterruptedException e) { + throw new UncheckedInterruptedException(e); + } catch (ExecutionException e) { + throw new UncheckedExecutionException(e); + } catch (TimeoutException e) { + throw new RuntimeException(e); + } + } + + @Override + public void reconfigureNode(NodeRef nodeRef, KafkaBrokerConfigurationDiff kafkaBrokerConfigurationDiff, KafkaBrokerLoggingConfigurationDiff kafkaBrokerLoggingConfigurationDiff) { + Map> updatedConfig = new HashMap<>(2); + updatedConfig.put(Util.getBrokersConfig(nodeRef.nodeId()), kafkaBrokerConfigurationDiff.getConfigDiff()); + updatedConfig.put(Util.getBrokersLogging(nodeRef.nodeId()), kafkaBrokerLoggingConfigurationDiff.getLoggingDiff()); + + AlterConfigsResult alterConfigResult = brokerAdmin.incrementalAlterConfigs(updatedConfig, new AlterConfigsOptions().timeoutMs(2000)); + KafkaFuture brokerConfigFuture = alterConfigResult.values().get(Util.getBrokersConfig(nodeRef.nodeId())); + KafkaFuture brokerLoggingConfigFuture = alterConfigResult.values().get(Util.getBrokersLogging(nodeRef.nodeId())); + + try { + brokerConfigFuture.get(); + brokerLoggingConfigFuture.get(); + } catch (InterruptedException e) { + throw new UncheckedInterruptedException(e); + } catch (ExecutionException e) { + throw new UncheckedExecutionException(e); + } + } + + @Override + public int tryElectAllPreferredLeaders(NodeRef nodeRef) { + // If brokerAdmin has not been initialised yet, create an admin with the given node + // this could happen, if there is any not_running nodes in the new reconciliation (not_running nodes get restarted before initialising an admin client) + try (Admin admin = brokerAdmin == null ? createBrokerAdminClient(Collections.singleton(nodeRef)) : brokerAdmin) { + // find all partitions where the node is the preferred leader + // we could do listTopics then describe all the topics, but that would scale poorly with number of topics + // using describe log dirs should be more efficient + var topicsOnNode = admin.describeLogDirs(List.of(nodeRef.nodeId())).allDescriptions().get() + .getOrDefault(nodeRef, Map.of()).values().stream() + .flatMap(x -> x.replicaInfos().keySet().stream()) + .map(TopicPartition::topic) + .collect(Collectors.toSet()); + + var topicDescriptionsOnNode = admin.describeTopics(topicsOnNode).allTopicNames().get(ADMIN_CALL_TIMEOUT, TimeUnit.MILLISECONDS).values(); + var toElect = new HashSet(); + for (TopicDescription td : topicDescriptionsOnNode) { + for (TopicPartitionInfo topicPartitionInfo : td.partitions()) { + if (!topicPartitionInfo.replicas().isEmpty() + && topicPartitionInfo.replicas().get(0).id() == nodeRef.nodeId() // this node is preferred leader + && topicPartitionInfo.leader().id() != nodeRef.nodeId()) { // this onde is not current leader + toElect.add(new TopicPartition(td.name(), topicPartitionInfo.partition())); + } + } + } + + var electionResults = admin.electLeaders(ElectionType.PREFERRED, toElect).partitions().get(); + + long count = electionResults.values().stream() + .filter(Optional::isPresent) + .count(); + return count > Integer.MAX_VALUE ? Integer.MAX_VALUE : (int) count; + } catch (InterruptedException e) { + throw new UncheckedInterruptedException(e); + } catch (ExecutionException e) { + throw new UncheckedExecutionException(e); + } catch (TimeoutException e) { + throw new RuntimeException(e); + } + } + + @Override + public Map describeBrokerConfigs(List toList) { + return describeNodeConfigs(brokerAdmin, toList); + } + + @Override + public Map describeControllerConfigs(List toList) { + return describeNodeConfigs(controllerAdmin, toList); + } + + private Map describeNodeConfigs(Admin admin, List toList) { + try { + var dc = admin.describeConfigs(toList.stream().map(nodeRef -> new ConfigResource(ConfigResource.Type.BROKER, String.valueOf(nodeRef.nodeId()))).toList()); + var result = dc.all().get(ADMIN_CALL_TIMEOUT, TimeUnit.MILLISECONDS); + + return toList.stream().collect(Collectors.toMap(NodeRef::nodeId, + nodeRef -> new Configs(result.get(new ConfigResource(ConfigResource.Type.BROKER, String.valueOf(nodeRef.nodeId()))), + result.get(new ConfigResource(ConfigResource.Type.BROKER_LOGGER, String.valueOf(nodeRef.nodeId()))) + ))); + } catch (InterruptedException e) { + throw new UncheckedInterruptedException(e); + } catch (ExecutionException e) { + throw new UncheckedExecutionException(e); + } catch (TimeoutException e) { + throw new RuntimeException(e); + } + } +} diff --git a/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/State.java b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/State.java new file mode 100644 index 0000000000..768b832549 --- /dev/null +++ b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/State.java @@ -0,0 +1,16 @@ +/* + * Copyright Strimzi authors. + * License: Apache License 2.0 (see the file LICENSE or http://apache.org/licenses/LICENSE-2.0.html). + */ +package io.strimzi.operator.cluster.operator.resource.rolling; + +/** + * Enumerates the possible "rolling states" of a Kafka node + */ +enum State { + UNKNOWN, // the initial state + NOT_RUNNING, // The pod/process is not running. + NOT_READY, // decided to restart right now or broker state < 2 OR == 127 + RECOVERING, // broker state == 2 + READY, // broker state >= 3 AND != 127 +} diff --git a/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/Time.java b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/Time.java new file mode 100644 index 0000000000..34e86e67f3 --- /dev/null +++ b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/Time.java @@ -0,0 +1,159 @@ +/* + * Copyright Strimzi authors. + * License: Apache License 2.0 (see the file LICENSE or http://apache.org/licenses/LICENSE-2.0.html). + */ +package io.strimzi.operator.cluster.operator.resource.rolling; + +import io.strimzi.operator.common.UncheckedInterruptedException; + +/** + * An abstraction of time. + * A Time instance represents some way of measuring durations (the {@link #nanoTime()} method) and some way of suspending + * the execution of a thread for some duration consistent with such measurements {@link #sleep(long, int)}. + *

In practice a real system would use {@link #SYSTEM_TIME}.

+ *

In testing you can use {@link Time.TestTime} so that tests don't depend on actually sleeping threads.

+ */ +public interface Time { + + /** The system's time */ + public final Time SYSTEM_TIME = new Time() { + @Override + public long nanoTime() { + return System.nanoTime(); + } + + @Override + public long systemTimeMillis() { + return System.currentTimeMillis(); + } + + @Override + public void sleep(long millis, int nanos) { + try { + Thread.sleep(millis, nanos); + } catch (InterruptedException e) { + throw new UncheckedInterruptedException(e); + } + } + + @Override + public String toString() { + return "SYSTEM_TIME"; + } + }; + + /** + * A {@code Time} implementation used for testing. + */ + public static class TestTime implements Time { + private final long autoAdvanceNs; + long time = 0; + + /** + * Constructs a new {@code Time} where time will only advance via calls to {@link #tickNanos(long)}. + */ + public TestTime() { + this(0); + } + + /** + * Constructs a new {@code Time} where time will advance either by calls to {@link #tickNanos(long)} or + * as a side-effect of calls to {@link #nanoTime()}. + * This can be useful in tests where the code under test needs to see time advance, + * does not call {@link #sleep(long, int)}, and difficult or impossible for the test code to call + * {@link #tickNanos(long)}. + * @param autoAdvanceNs How much time will advance when {@link #nanoTime()} is called. + */ + public TestTime(long autoAdvanceNs) { + if (autoAdvanceNs < 0) { + throw new IllegalArgumentException(); + } + this.autoAdvanceNs = autoAdvanceNs; + } + + /** + * Advance the time by the given number of nanoseconds. + * @param advanceNs The number of nanoseconds by which the time should be advanced. + */ + public void tickNanos(long advanceNs) { + if (advanceNs < 0) { + throw new IllegalArgumentException(); + } + time += advanceNs; + } + + @Override + public long nanoTime() { + var result = time; + time += autoAdvanceNs; + return result; + } + + @Override + public long systemTimeMillis() { + return time / 1_000_000; + } + + @Override + public void sleep(long millis, int nanos) { + if (millis < 0 || nanos < 0 || nanos > 999_999) { + throw new IllegalArgumentException(); + } + time += 1_000_000 * millis + nanos; + } + + @Override + public String toString() { + return "TestTime"; + } + } + + + /** + * The number of nanoseconds since some unknown epoch. + * Different {@code Time} instances may use different epochs. + * This is only useful for measuring and calculating elapsed time against the same Time instance. + * The return value is guaranteed to be monotonic. + * + *

When the instance is {@link Time#SYSTEM_TIME} this corresponds to a call to {@link System#nanoTime()}.

+ * + * @return The number of nanoseconds since some unknown epoch. + */ + long nanoTime(); + + /** + * The system time, defined as the number of milliseconds since the epoch, midnight, January 1, 1970 UTC. + * Different {@code Time} instances all use the same epoch. + * + * There are no guarantees about precision or accuracy and the return value is not guaranteed to be monotonic. + * + * When the instance is {@link Time#SYSTEM_TIME} this corresponds to a call to {@link System#currentTimeMillis()}. + * @return The number of milliseconds since the epoch. + */ + long systemTimeMillis(); + + /** + * Causes the current thread to sleep (suspend execution) for the given number of milliseconds + * plus the given number of nanoseconds, relative to this Time instance and subject to its precision and accuracy. + * In other words after successful execution of the following code + *
{@code
+     * Time time = ...
+     * long start = time.nanoTime();
+     * time.sleep(millis, nanos)
+     * long end = time.nanoTime();
+     * long diff = (1_000_000 * millis + nanos) - (end - start)
+     * }
+ * we would expect {@code diff} to be small in magnitude. + * + *

The thread does not lose ownership of any monitors.

+ * + *

When the instance is {@link Time#SYSTEM_TIME} this corresponds to a call to {@link Thread#sleep(long, int)}.

+ * + * @param millis The number of milliseconds + * @param nanos The number of nanoseconds + * @throws io.strimzi.operator.common.UncheckedInterruptedException If the sleep was interrupted before the given time has elapsed. + */ + void sleep(long millis, int nanos); + +} + diff --git a/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/UnrestartableNodesException.java b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/UnrestartableNodesException.java new file mode 100644 index 0000000000..c0412020ea --- /dev/null +++ b/cluster-operator/src/main/java/io/strimzi/operator/cluster/operator/resource/rolling/UnrestartableNodesException.java @@ -0,0 +1,17 @@ +/* + * Copyright Strimzi authors. + * License: Apache License 2.0 (see the file LICENSE or http://apache.org/licenses/LICENSE-2.0.html). + */ +package io.strimzi.operator.cluster.operator.resource.rolling; + +class UnrestartableNodesException extends RuntimeException { + + /** + * This exception indicates that a node cannot be attempted to restart which could be + * because of not satisfying the safety conditions or the maximum number of retry attempt has been reached. + * @param message the detail message. The detail message is saved for later retrieval by the getMessage() method + */ + public UnrestartableNodesException(String message) { + super(message); + } +} diff --git a/cluster-operator/src/test/java/io/strimzi/operator/cluster/operator/resource/rolling/AlarmTest.java b/cluster-operator/src/test/java/io/strimzi/operator/cluster/operator/resource/rolling/AlarmTest.java new file mode 100644 index 0000000000..a2686f893c --- /dev/null +++ b/cluster-operator/src/test/java/io/strimzi/operator/cluster/operator/resource/rolling/AlarmTest.java @@ -0,0 +1,50 @@ +/* + * Copyright Strimzi authors. + * License: Apache License 2.0 (see the file LICENSE or http://apache.org/licenses/LICENSE-2.0.html). + */ +package io.strimzi.operator.cluster.operator.resource.rolling; + +import org.junit.jupiter.api.Test; + +import java.util.concurrent.TimeoutException; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + +class AlarmTest { + + private final Time.TestTime time = new Time.TestTime(); + + @Test + public void testRemaining() throws InterruptedException, TimeoutException { + var tx = time.nanoTime(); + var alarm = Alarm.timer(time, 1_000, () -> "message"); + assertEquals(1_000, alarm.remainingMs()); + time.tickNanos(100_000_000); + assertEquals(900, alarm.remainingMs()); + time.tickNanos(50_000_000); + assertEquals(850, alarm.remainingMs()); + + alarm.sleep(100); + assertEquals(750, alarm.remainingMs()); + alarm.sleep(700); + assertEquals(50, alarm.remainingMs()); + + // We wake up one final time with 0ns left, even if we would have slept longer + var t0 = time.nanoTime(); + alarm.sleep(51); + assertEquals(50_000_000, time.nanoTime() - t0, "Expected to sleep for exactly 5,000,000 ns"); + assertEquals(0, alarm.remainingMs(), "Expect no time remaining"); + assertEquals(1_000_000_000, time.nanoTime() - tx, "Expected timeout after 1,000 ms"); + + // Once the time advances past the dealine + time.tickNanos(1); + + // We expect subsequent attampts to sleep to throw + assertEquals("message", + assertThrows(TimeoutException.class, () -> alarm.sleep(0)).getMessage()); + assertEquals("message", + assertThrows(TimeoutException.class, () -> alarm.sleep(1)).getMessage()); + } + +} \ No newline at end of file diff --git a/cluster-operator/src/test/java/io/strimzi/operator/cluster/operator/resource/rolling/BatchingTest.java b/cluster-operator/src/test/java/io/strimzi/operator/cluster/operator/resource/rolling/BatchingTest.java new file mode 100644 index 0000000000..c7f205308a --- /dev/null +++ b/cluster-operator/src/test/java/io/strimzi/operator/cluster/operator/resource/rolling/BatchingTest.java @@ -0,0 +1,94 @@ +/* + * Copyright Strimzi authors. + * License: Apache License 2.0 (see the file LICENSE or http://apache.org/licenses/LICENSE-2.0.html). + */ +package io.strimzi.operator.cluster.operator.resource.rolling; + +import io.strimzi.operator.common.Reconciliation; +import org.apache.kafka.common.Node; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; + +public class BatchingTest { + + @Test + public void testCellCreation() { + List> cells = Batching.cells(Reconciliation.DUMMY_RECONCILIATION, + Set.of(addKafkaNode(0, addReplicas(0, "my-topic", 0, 1, 0)), + addKafkaNode(1, Set.of()), + addKafkaNode(2, addReplicas(2, "my-topic", 0, 1, 2)))); + + assertEquals(cells.size(), 2); + assertEquals("[KafkaNode[id=0, replicas=[my-topic-0]], KafkaNode[id=1, replicas=[]]]", cells.get(0).toString()); + assertEquals("[KafkaNode[id=2, replicas=[my-topic-0]]]", cells.get(1).toString()); + } + + @Test + public void testBatchCreationWithSingleTopic() { + Set kafkaNodes = Set.of(addKafkaNode(0, addReplicas(0, "my-topic", 0, 1, 0)), + addKafkaNode(1, Set.of()), + addKafkaNode(2, addReplicas(2, "my-topic", 0, 1, 2))); + List> cells = Batching.cells(Reconciliation.DUMMY_RECONCILIATION, kafkaNodes); + + Availability availability = mock(Availability.class); + doReturn(true).when(availability).anyPartitionWouldBeUnderReplicated(0); + doReturn(true).when(availability).anyPartitionWouldBeUnderReplicated(2); + + List> batch = Batching.batchCells(Reconciliation.DUMMY_RECONCILIATION, cells, availability, 2); + + assertEquals(batch.size(), 1); + assertEquals("[1]", batch.get(0).toString()); + } + + @Test + public void testBatchCreationWithMultipleTopics() { + + List> cells = Batching.cells(Reconciliation.DUMMY_RECONCILIATION, + Set.of(addKafkaNode(0, addReplicas(0, "my-topic", 0, 0, 3)), + addKafkaNode(3, addReplicas(3, "my-topic", 0, 0, 3)), + addKafkaNode(1, addReplicas(1, "my-topic-1", 1, 1, 2)), + addKafkaNode(2, addReplicas(2, "my-topic-1", 1, 1, 2)))); + + Availability availability = mock(Availability.class); + doReturn(true).when(availability).anyPartitionWouldBeUnderReplicated(0); + doReturn(true).when(availability).anyPartitionWouldBeUnderReplicated(1); + doReturn(true).when(availability).anyPartitionWouldBeUnderReplicated(2); + + List> batch = Batching.batchCells(Reconciliation.DUMMY_RECONCILIATION, cells, availability, 2); + + + System.out.println(batch); + assertEquals(batch.size(), 1); + assertEquals("[3]", batch.get(0).toString()); + } + + static public Set addReplicas(int brokerid, String topicName, int partition, int... isrIds) { + Set replicaSet = new HashSet<>(); + + replicaSet.add(new Replica(addNodes(brokerid).get(0), topicName, partition, addNodes(isrIds))); + + return replicaSet; + } + + static List addNodes(int... isrIds) { + List nodes = new ArrayList<>(); + + for (int id : isrIds) { + nodes.add(new Node(id, "pool-kafka-" + id, 9092)); + } + + return nodes; + } + + static public KafkaNode addKafkaNode(int nodeId, Set replicas) { + return new KafkaNode(nodeId, replicas); + } +} diff --git a/cluster-operator/src/test/java/io/strimzi/operator/cluster/operator/resource/rolling/RackRollingTest.java b/cluster-operator/src/test/java/io/strimzi/operator/cluster/operator/resource/rolling/RackRollingTest.java new file mode 100644 index 0000000000..101bdf07ad --- /dev/null +++ b/cluster-operator/src/test/java/io/strimzi/operator/cluster/operator/resource/rolling/RackRollingTest.java @@ -0,0 +1,1611 @@ +/* + * Copyright Strimzi authors. + * License: Apache License 2.0 (see the file LICENSE or http://apache.org/licenses/LICENSE-2.0.html). + */ +package io.strimzi.operator.cluster.operator.resource.rolling; + +import io.strimzi.operator.cluster.KafkaVersionTestUtils; +import io.strimzi.operator.cluster.model.NodeRef; +import io.strimzi.operator.cluster.model.RestartReason; +import io.strimzi.operator.cluster.model.RestartReasons; +import io.strimzi.operator.cluster.operator.resource.KafkaBrokerConfigurationDiff; +import io.strimzi.operator.cluster.operator.resource.KafkaBrokerLoggingConfigurationDiff; +import io.strimzi.operator.common.Reconciliation; +import org.apache.kafka.clients.admin.AlterConfigOp; +import org.apache.kafka.clients.admin.Config; +import org.apache.kafka.clients.admin.ConfigEntry; +import org.apache.kafka.clients.admin.TopicDescription; +import org.apache.kafka.clients.admin.TopicListing; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.TopicPartitionInfo; +import org.apache.kafka.common.Uuid; +import org.junit.jupiter.api.Test; +import org.mockito.Mockito; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; + +public class RackRollingTest { + + static final Function EMPTY_CONFIG_SUPPLIER = serverId -> ""; + + private final Time time = new Time.TestTime(1_000_000_000L); + + static RestartReasons noReasons(int serverId) { + return RestartReasons.empty(); + } + + private static RestartReasons manualRolling(int serverId) { + return RestartReasons.of(RestartReason.MANUAL_ROLLING_UPDATE); + } + + private static RestartReasons podUnresponsive(int serverId) { + return RestartReasons.of(RestartReason.POD_UNRESPONSIVE); + } + + private static RestartReasons podHasOldRevision(int serverId) { + return RestartReasons.of(RestartReason.POD_HAS_OLD_REVISION); + } + + private PlatformClient mockedPlatformClient() { + PlatformClient platformClient = mock(PlatformClient.class); + doReturn(PlatformClient.NodeState.READY) + .when(platformClient) + .nodeState(any()); + + return platformClient; + } + + private RollClient mockedRollClient() { + RollClient rollClient = mock(RollClient.class); + + doReturn(Collections.emptyMap()) + .when(rollClient) + .describeBrokerConfigs(any()); + doReturn(Collections.emptyMap()) + .when(rollClient) + .describeControllerConfigs(any()); + + doReturn(0) + .when(rollClient) + .tryElectAllPreferredLeaders(any()); + + doReturn(true).when(rollClient).canConnectToNode(any(), anyBoolean()); + + doReturn(-1).when(rollClient).activeController(); + return rollClient; + } + + static class MockBuilder { + private final Map nodeRefs = new LinkedHashMap<>(); + private final Map nodes = new LinkedHashMap<>(); + private final Set topicListing = new HashSet<>(); + private final Map topicMinIsrs = new HashMap<>(); + private final Map topicDescriptions = new HashMap<>(); + private final Map configPair = new HashMap<>(); + + + MockBuilder addNode(PlatformClient platformClient, boolean controller, boolean broker, int nodeId) { + if (nodeRefs.containsKey(nodeId)) { + throw new RuntimeException(); + } + if (!controller && !broker) { + throw new RuntimeException(); + } + + nodeRefs.put(nodeId, new NodeRef("pool-kafka-" + nodeId, nodeId, "pool", controller, broker)); + nodes.put(nodeId, new Node(nodeId, "pool-kafka-" + nodeId, 9092)); + doReturn(new NodeRoles(controller, broker)).when(platformClient).nodeRoles(nodeRefs.get(nodeId)); + return this; + } + + MockBuilder addNodes(PlatformClient platformClient, boolean controller, boolean broker, int... nodeIds) { + for (int nodeId : nodeIds) { + addNode(platformClient, controller, broker, nodeId); + } + return this; + } + + MockBuilder mockNodeState(PlatformClient platformClient, List nodeStates, int nodeId) { + doReturn(nodeStates.get(0), nodeStates.size() == 1 ? new Object[0] : nodeStates.subList(1, nodeStates.size()).toArray()) + .when(platformClient) + .nodeState(nodeRefs.get(nodeId)); + return this; + } + + MockBuilder mockHealthyNodes(PlatformClient platformClient, int... nodeIds) { + for (var nodeId : nodeIds) { + mockHealthyNode(platformClient, nodeId); + } + return this; + } + + private void mockHealthyNode(PlatformClient platformClient, int nodeId) { + doReturn(PlatformClient.NodeState.READY) + .when(platformClient) + .nodeState(nodeRefs.get(nodeId)); + } + + MockBuilder mockNotRunningNodes(PlatformClient platformClient, int... nodeIds) { + for (var nodeId : nodeIds) { + mockNotRunningNode(platformClient, nodeId); + } + return this; + } + + private void mockNotRunningNode(PlatformClient platformClient, int nodeId) { + doReturn(PlatformClient.NodeState.NOT_RUNNING, PlatformClient.NodeState.READY) + .when(platformClient) + .nodeState(nodeRefs.get(nodeId)); + } + + MockBuilder addTopic(String topicName, int leaderId, List replicaIds, List isrIds) { + return addTopic(topicName, leaderId, replicaIds, isrIds, null); + } + + MockBuilder addTopic(String topicName, int leaderId, List replicaIds, List isrIds, Integer minIsr) { + if (!replicaIds.contains(leaderId)) { + throw new RuntimeException("Leader is not a replica"); + } + for (var isrId : isrIds) { + if (!replicaIds.contains(isrId)) { + throw new RuntimeException("ISR is not a subset of replicas"); + } + } + if (topicListing.stream().anyMatch(tl -> tl.name().equals(topicName))) { + throw new RuntimeException("Topic " + topicName + " already exists"); + } + Uuid topicId = Uuid.randomUuid(); + topicListing.add(new TopicListing(topicName, topicId, false)); + + Node leaderNode = nodes.get(leaderId); + List replicas = replicaIds.stream().map(nodes::get).toList(); + List isr = isrIds.stream().map(nodes::get).toList(); + topicDescriptions.put(topicId, new TopicDescription(topicName, false, + List.of(new TopicPartitionInfo(0, + leaderNode, replicas, isr)))); + + topicMinIsrs.put(topicName, minIsr); + return this; + } + + MockBuilder mockTopics(RollClient client) { + doReturn(topicListing) + .when(client) + .listTopics(); + + doAnswer(i -> { + List topicIds = i.getArgument(0); + return topicIds.stream().map(topicDescriptions::get).toList(); + }) + .when(client) + .describeTopics(any()); + doAnswer(i -> { + List topicNames = i.getArgument(0); + Map map = new HashMap<>(); + for (String topicName : topicNames) { + if (map.put(topicName, topicMinIsrs.get(topicName)) != null) { + throw new IllegalStateException("Duplicate key"); + } + } + return map; + }) + .when(client) + .describeTopicMinIsrs(any()); + return this; + } + + MockBuilder mockDescribeConfigs(RollClient rollClient, Set nodeConfigs, Set loggerConfigs, int... nodeIds) { + for (var nodeId : nodeIds) { + if (!configPair.containsKey(nodeId)) configPair.put(nodeId, new Configs(new Config(nodeConfigs), new Config(loggerConfigs))); + } + + doReturn(configPair) + .when(rollClient) + .describeBrokerConfigs(any()); + doReturn(configPair) + .when(rollClient) + .describeControllerConfigs(any()); + + return this; + } + + MockBuilder mockReconfigureConfigs(RollClient rollClient) { + doAnswer(i -> { + NodeRef nodeRef = i.getArgument(0); + KafkaBrokerConfigurationDiff diff = i.getArgument(1); + KafkaBrokerLoggingConfigurationDiff loggingDiff = i.getArgument(2); + var nodeConfigs = diff.getConfigDiff().stream().map(AlterConfigOp::configEntry).collect(Collectors.toSet()); + var loggerConfigs = loggingDiff.getLoggingDiff().stream().map(AlterConfigOp::configEntry).collect(Collectors.toSet()); + configPair.put(nodeRef.nodeId(), new Configs(new Config(nodeConfigs), new Config(loggerConfigs))); + return null; + }) + .when(rollClient) + .reconfigureNode(any(), any(), any()); + return this; + } + + MockBuilder mockQuorumLastCaughtUpTimestamps(RollClient rollClient, Map quorumState) { + doReturn(quorumState) + .when(rollClient) + .quorumLastCaughtUpTimestamps(any()); + return this; + } + + public MockBuilder mockElectLeaders(RollClient rollClient, int... nodeIds) { + return mockElectLeaders(rollClient, List.of(0), nodeIds); + } + + MockBuilder mockSuccessfulConnection(RollClient rollClient, int... nodeIds) { + for (var nodeId : nodeIds) { + mockConnectionToNode(rollClient, nodeId); + } + return this; + } + + private void mockConnectionToNode(RollClient rollClient, int nodeId) { + doReturn(true).when(rollClient).canConnectToNode(eq(nodeRefs.get(nodeId)), anyBoolean()); + } + + MockBuilder mockConnectionToNode(RollClient rollClient, List connectionStates, int nodeId) { + doReturn(connectionStates.get(0), connectionStates.size() == 1 ? new Object[0] : connectionStates.subList(1, connectionStates.size()).toArray()) + .when(rollClient) + .canConnectToNode(eq(nodeRefs.get(nodeId)), anyBoolean()); + return this; + } + + MockBuilder mockElectLeaders(RollClient rollClient, List results, int... nodeIds) { + for (var nodeId : nodeIds) { + doReturn(results.get(0), results.subList(1, results.size()).toArray()) + .when(rollClient) + .tryElectAllPreferredLeaders(nodeRefs.get(nodeId)); + } + return this; + } + + Map done() { + return nodeRefs; + } + + MockBuilder mockLeader(RollClient rollClient, int leaderId) { + doReturn(leaderId).when(rollClient).activeController(); + return this; + } + } + + + private static void assertNodesRestarted(PlatformClient platformClient, + RollClient rollClient, + Map nodeRefs, + RackRolling rr, + int... nodeIds) throws InterruptedException, ExecutionException { + for (var nodeId : nodeIds) { + Mockito.verify(platformClient, never()).restartNode(eq(nodeRefs.get(nodeId)), any()); + Mockito.verify(rollClient, never()).tryElectAllPreferredLeaders(eq(nodeRefs.get(nodeId))); + } + + List restartedNodes = rr.loop(); + List expectedRestartedNodes = IntStream.of(nodeIds).boxed().toList(); + assertThat(expectedRestartedNodes.size(), is(restartedNodes.size())); + assertEquals(new HashSet<>(expectedRestartedNodes), new HashSet<>(restartedNodes)); //Sets may have different orders + for (var nodeId : nodeIds) { + if (nodeRefs.get(nodeId).broker()) { + Mockito.verify(rollClient, times(1)).tryElectAllPreferredLeaders(eq(nodeRefs.get(nodeId))); + } else { + Mockito.verify(rollClient, never()).tryElectAllPreferredLeaders(eq(nodeRefs.get(nodeId))); + } + Mockito.verify(platformClient, times(1)).restartNode(eq(nodeRefs.get(nodeId)), any()); + } + Mockito.verify(rollClient, never()).reconfigureNode(any(), any(), any()); + } + + private RackRolling newRollingRestart(PlatformClient platformClient, + RollClient rollClient, + AgentClient agentClient, + Collection nodeRefList, + Function reason, + Function kafkaConfigProvider, + boolean allowReconfiguration, + int maxRestartsBatchSize) { + return RackRolling.rollingRestart(time, + platformClient, + rollClient, + agentClient, + nodeRefList, + reason, + Reconciliation.DUMMY_RECONCILIATION, + KafkaVersionTestUtils.getLatestVersion(), + allowReconfiguration, + kafkaConfigProvider, + null, + 1_000, + maxRestartsBatchSize, + 1, + 1, + 3); + } + + + private void doRollingRestart(PlatformClient platformClient, + RollClient rollClient, + AgentClient agentClient, + Collection nodeRefList, + Function reason, + Function kafkaConfigProvider, + int maxRestartsBatchSize, + int maxRestart) throws ExecutionException, InterruptedException { + + var rr = RackRolling.rollingRestart(time, + platformClient, + rollClient, + agentClient, + nodeRefList, + reason, + Reconciliation.DUMMY_RECONCILIATION, + KafkaVersionTestUtils.getLatestVersion(), + true, + kafkaConfigProvider, + null, + 120_000, + maxRestartsBatchSize, + maxRestart, + 1, + 3); + List nodes; + do { + nodes = rr.loop(); + } while (!nodes.isEmpty()); + } + + ////////////////////////////////////////////////////// + /// Test scenarios we expect restarts /// + ////////////////////////////////////////////////////// + + @Test + public void shouldRestartManualRollingUpdate() throws ExecutionException, InterruptedException { + // given + PlatformClient platformClient = mockedPlatformClient(); + RollClient rollClient = mockedRollClient(); + Map quorumState = Map.of(3, 10_000L, 4, 10_000L, 5, 10_000L); + var nodeRefs = new MockBuilder() + .addNodes(platformClient, false, true, 0, 1, 2) + .addNodes(platformClient, true, false, 3, 4, 5) + .mockLeader(rollClient, 3) + .mockHealthyNodes(platformClient, 0, 1, 2, 3, 4, 5) + .mockSuccessfulConnection(rollClient, 0, 1, 2, 3, 4, 5) + .addTopic("topic-A", 0, List.of(0, 1, 2), List.of(0, 1, 2)) + .addTopic("topic-B", 1, List.of(1, 2, 0), List.of(1, 2, 0)) + .addTopic("topic-C", 2, List.of(2, 0, 1), List.of(2, 0, 1)) + .mockDescribeConfigs(rollClient, Set.of(), Set.of(), 0, 1) + .mockQuorumLastCaughtUpTimestamps(rollClient, quorumState) + .mockTopics(rollClient) + .mockElectLeaders(rollClient, 0, 1, 2) + .done(); + + var rr = newRollingRestart(platformClient, + rollClient, + null, + nodeRefs.values(), + RackRollingTest::manualRolling, + EMPTY_CONFIG_SUPPLIER, + true, + 1); + + // The order we expect is controllers, active controller, brokers + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 4); + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 5); + + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 3); + + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 0); + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 1); + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 2); + + assertEquals(List.of(), rr.loop()); + } + + @Test + public void shouldRestartCombinedNodesManualRollingUpdate() throws ExecutionException, InterruptedException { + // given + PlatformClient platformClient = mock(PlatformClient.class); + RollClient rollClient = mock(RollClient.class); + Map quorumState = Map.of(0, 10_000L, 1, 10_000L, 2, 10_000L); + var nodeRefs = new MockBuilder() + .addNodes(platformClient, true, true, 0, 1, 2) + .mockLeader(rollClient, 0) + .mockHealthyNodes(platformClient, 0, 1, 2) + .mockSuccessfulConnection(rollClient, 0, 1, 2) + .addTopic("topic-A", 0, List.of(0, 1, 2), List.of(0, 1, 2)) + .addTopic("topic-B", 1, List.of(1, 2, 0), List.of(1, 2, 0)) + .addTopic("topic-C", 2, List.of(2, 0, 1), List.of(2, 0, 1)) + .mockDescribeConfigs(rollClient, Set.of(), Set.of(), 0, 1, 2) + .mockQuorumLastCaughtUpTimestamps(rollClient, quorumState) + .mockTopics(rollClient) + .mockElectLeaders(rollClient, 0, 1, 2) + .done(); + + var rr = newRollingRestart(platformClient, + rollClient, + null, + nodeRefs.values(), + RackRollingTest::manualRolling, + EMPTY_CONFIG_SUPPLIER, + true, + 1); + + // The order we expect is combined nodes and active controller + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 1); + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 2); + + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 0); + + assertEquals(List.of(), rr.loop()); + } + + @Test + void shouldRestartBrokerWithNoTopicIfReasonManualRolling() throws ExecutionException, InterruptedException { + PlatformClient platformClient = mock(PlatformClient.class); + RollClient rollClient = mock(RollClient.class); + var nodeRef = new MockBuilder() + .addNode(platformClient, false, true, 0) + .mockLeader(rollClient, -1) + .mockHealthyNodes(platformClient, 0) + .mockSuccessfulConnection(rollClient, 0) + .mockDescribeConfigs(rollClient, Set.of(), Set.of(), 0) + .done().get(0); + + doRollingRestart(platformClient, rollClient, null, List.of(nodeRef), RackRollingTest::manualRolling, EMPTY_CONFIG_SUPPLIER, 1, 1); + + Mockito.verify(rollClient, never()).reconfigureNode(any(), any(), any()); + Mockito.verify(platformClient, times(1)).restartNode(eq(nodeRef), any()); + Mockito.verify(rollClient, times(1)).tryElectAllPreferredLeaders(eq(nodeRef)); + } + + @Test + public void shouldRestartUnreadyWithManualRollingUpdate() throws ExecutionException, InterruptedException { + PlatformClient platformClient = mock(PlatformClient.class); + RollClient rollClient = mock(RollClient.class); + Map quorumState = Map.of(0, 10_000L, 1, 10_000L, 2, 10_000L); + var nodeRefs = new MockBuilder() + .addNodes(platformClient, true, false, 0) + .addNodes(platformClient, true, false, 1) + .addNodes(platformClient, true, true, 2) + .addNodes(platformClient, false, true, 3) + .addNodes(platformClient, false, true, 4) + .mockLeader(rollClient, 1) + .mockNodeState(platformClient, List.of(PlatformClient.NodeState.NOT_READY, PlatformClient.NodeState.NOT_READY, PlatformClient.NodeState.READY), 2) + .mockNodeState(platformClient, List.of(PlatformClient.NodeState.NOT_READY, PlatformClient.NodeState.NOT_READY, PlatformClient.NodeState.NOT_READY, PlatformClient.NodeState.NOT_READY, PlatformClient.NodeState.NOT_READY, PlatformClient.NodeState.NOT_READY, PlatformClient.NodeState.NOT_READY, PlatformClient.NodeState.NOT_READY, PlatformClient.NodeState.READY), 4) + .mockHealthyNodes(platformClient, 0, 1, 3) + .mockSuccessfulConnection(rollClient, 0, 1, 2, 3, 4) + .mockQuorumLastCaughtUpTimestamps(rollClient, quorumState) + .mockTopics(rollClient) + .mockDescribeConfigs(rollClient, Set.of(), Set.of(), 0) + .mockElectLeaders(rollClient, 2, 3, 4) + .done(); + + var rr = newRollingRestart(platformClient, + rollClient, + null, + nodeRefs.values(), + RackRollingTest::manualRolling, + EMPTY_CONFIG_SUPPLIER, + true, + 1); + + // The order we expect is unready controller (in this case combined), ready controller, active controller, unready broker, ready broker + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 2); + + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 0); + + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 1); + + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 4); + + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 3); + + assertEquals(List.of(), rr.loop()); + } + + @Test + public void shouldRestartNotRunningNodes() throws ExecutionException, InterruptedException { + // given + PlatformClient platformClient = mock(PlatformClient.class); + RollClient rollClient = mock(RollClient.class); + Map quorumState = Map.of(0, 10_000L, 1, 10_000L, 2, 10_000L); + var nodeRefs = new MockBuilder() + .addNodes(platformClient, true, false, 0) + .addNodes(platformClient, true, false, 1) + .addNodes(platformClient, true, true, 2) + .addNodes(platformClient, false, true, 3) + .addNodes(platformClient, false, true, 4) + .mockLeader(rollClient, 1) + .mockNotRunningNodes(platformClient, 0, 2) + .mockNodeState(platformClient, List.of(PlatformClient.NodeState.NOT_RUNNING, PlatformClient.NodeState.NOT_RUNNING, PlatformClient.NodeState.READY), 4) + .mockHealthyNodes(platformClient, 1, 3) + .mockSuccessfulConnection(rollClient, 0, 1, 2, 3, 4) + .mockQuorumLastCaughtUpTimestamps(rollClient, quorumState) + .mockTopics(rollClient) + .mockDescribeConfigs(rollClient, Set.of(), Set.of(), 0) + .mockElectLeaders(rollClient, 2, 3, 4) + .done(); + + var rr = newRollingRestart(platformClient, + rollClient, + null, + nodeRefs.values(), + RackRollingTest::podHasOldRevision, + EMPTY_CONFIG_SUPPLIER, + true, + 1); + + // The order we expect parallel restart of controllers and then broker + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 0, 2); + + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 4); + + // The healthy nodes should restart as normal due to podHasOldRevision reason returned for all nodes + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 1); + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 3); + + assertEquals(List.of(), rr.loop()); + } + + @Test + public void shouldRestartUnresponsiveNodes() throws ExecutionException, InterruptedException { + PlatformClient platformClient = mock(PlatformClient.class); + RollClient rollClient = mock(RollClient.class); + var nodeRefs = new MockBuilder() + .addNodes(platformClient, true, false, 0) + .addNodes(platformClient, true, false, 1) + .addNodes(platformClient, true, true, 2) + .addNodes(platformClient, false, true, 3) + .addNodes(platformClient, false, true, 4) + .mockLeader(rollClient, 1) + .mockHealthyNodes(platformClient, 0, 1, 2, 3, 4) + .mockSuccessfulConnection(rollClient, 1, 3) + .mockConnectionToNode(rollClient, List.of(false, true), 0) + .mockConnectionToNode(rollClient, List.of(false, false, true), 2) + .mockConnectionToNode(rollClient, List.of(false, false, false, true), 4) + .mockDescribeConfigs(rollClient, Set.of(), Set.of(), 1, 3) + .done(); + + var rr = newRollingRestart(platformClient, + rollClient, + null, + nodeRefs.values(), + RackRollingTest::noReasons, + EMPTY_CONFIG_SUPPLIER, + true, + 1); + + // The order we expect parallel unresponsive controller, combined node and broker + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 0); + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 2); + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 4); + + assertEquals(List.of(), rr.loop()); + } + + @Test + public void shouldRestartNonDynamicConfig() throws ExecutionException, InterruptedException { + // given + PlatformClient platformClient = mock(PlatformClient.class); + RollClient rollClient = mock(RollClient.class); + Map quorumState = Map.of(0, 10_000L, 1, 10_000L, 2, 10_000L); + var nodeRefs = new MockBuilder() + .addNodes(platformClient, true, false, 0) + .addNodes(platformClient, true, false, 1) + .addNodes(platformClient, true, true, 2) + .addNodes(platformClient, false, true, 3) + .addNodes(platformClient, false, true, 4) + .mockLeader(rollClient, 1) + .mockHealthyNodes(platformClient, 0, 1, 2, 3, 4) + .mockSuccessfulConnection(rollClient, 0, 1, 2, 3, 4) + .mockQuorumLastCaughtUpTimestamps(rollClient, quorumState) + .mockTopics(rollClient) + .mockDescribeConfigs(rollClient, Set.of(new ConfigEntry("auto.leader.rebalance.enable", "true")), Set.of(), 0, 1, 2, 3, 4) + .done(); + + var rr = newRollingRestart(platformClient, + rollClient, + null, + nodeRefs.values(), + RackRollingTest::noReasons, + nodeId -> "auto.leader.rebalance.enable=false", + true, + 1); + + //TODO: uncomment the asserts for controller node when controller reconfiguration is implemented + // so that the order of restart we expect due to non-dynamic config: pure controller, combined, active controller, brokers +// assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 0); + + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 2); + +// assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 1); + + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 3); + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 4); + + assertEquals(List.of(), rr.loop()); + + for (var nodeRef : nodeRefs.values()) { + Mockito.verify(rollClient, never()).reconfigureNode(eq(nodeRef), any(), any()); + } + } + + @Test + public void shouldRestartBrokerDynamicConfigFailed() throws ExecutionException, InterruptedException { + PlatformClient platformClient = mock(PlatformClient.class); + RollClient rollClient = mock(RollClient.class); + doThrow(new RuntimeException("Configuration update failed")).when(rollClient).reconfigureNode(any(), any(), any()); + + Map quorumState = Map.of(0, 10_000L, 1, 10_000L, 2, 10_000L); + var nodeRefs = new MockBuilder() + .addNodes(platformClient, true, false, 0) + .addNodes(platformClient, true, false, 1) + .addNodes(platformClient, true, true, 2) + .addNodes(platformClient, false, true, 3) + .addNodes(platformClient, false, true, 4) + .mockLeader(rollClient, 1) + .mockHealthyNodes(platformClient, 0, 1, 2, 3, 4) + .mockSuccessfulConnection(rollClient, 0, 1, 2, 3, 4) + .mockQuorumLastCaughtUpTimestamps(rollClient, quorumState) + .mockTopics(rollClient) + .mockDescribeConfigs(rollClient, Set.of(new ConfigEntry("min.insync.replicas", "1")), Set.of(), 0, 1, 2, 3, 4) + .mockElectLeaders(rollClient, 2, 3, 4) + .done(); + + doRollingRestart(platformClient, + rollClient, + null, + nodeRefs.values(), + RackRollingTest::noReasons, + nodeId -> "min.insync.replicas=2", + 1, + 1); + + for (var nodeRef : nodeRefs.values()) { + //TODO: the following should apply to controller nodes as well later, when controller reconfiguration is implemented + if (nodeRef.broker()) { + Mockito.verify(rollClient, times(1)).reconfigureNode(eq(nodeRef), any(), any()); + Mockito.verify(platformClient, times(1)).restartNode(eq(nodeRef), any()); + Mockito.verify(rollClient, times(1)).tryElectAllPreferredLeaders(eq(nodeRef)); + } else { + Mockito.verify(rollClient, never()).reconfigureNode(eq(nodeRef), any(), any()); + Mockito.verify(platformClient, never()).restartNode(eq(nodeRef), any()); + } + } + } + + @Test + public void shouldRestartDescribeConfigFailed() throws ExecutionException, InterruptedException { + PlatformClient platformClient = mock(PlatformClient.class); + RollClient rollClient = mock(RollClient.class); + doThrow(new RuntimeException("Error getting broker config")).when(rollClient).describeBrokerConfigs(any()); + + Map quorumState = Map.of(0, 10_000L, 1, 10_000L, 2, 10_000L); + var nodeRefs = new MockBuilder() + .addNodes(platformClient, true, false, 0) + .addNodes(platformClient, true, false, 1) + .addNodes(platformClient, true, true, 2) + .addNodes(platformClient, false, true, 3) + .addNodes(platformClient, false, true, 4) + .mockLeader(rollClient, 1) + .mockHealthyNodes(platformClient, 0, 1, 2, 3, 4) + .mockSuccessfulConnection(rollClient, 0, 1, 2, 3, 4) + .mockQuorumLastCaughtUpTimestamps(rollClient, quorumState) + .mockTopics(rollClient) + .mockElectLeaders(rollClient, 2, 3, 4) + .done(); + + doRollingRestart(platformClient, + rollClient, + null, + nodeRefs.values(), + RackRollingTest::noReasons, + EMPTY_CONFIG_SUPPLIER, + 1, + 1); + + for (var nodeRef : nodeRefs.values()) { + Mockito.verify(rollClient, never()).reconfigureNode(eq(nodeRef), any(), any()); + if (nodeRef.broker()) { + Mockito.verify(platformClient, times(1)).restartNode(eq(nodeRef), any()); + Mockito.verify(rollClient, times(1)).tryElectAllPreferredLeaders(eq(nodeRef)); + } + } + } + + @Test + void shouldThrowMaxRestartsExceeded() { + PlatformClient platformClient = mock(PlatformClient.class); + RollClient rollClient = mock(RollClient.class); + var nodeRefs = new MockBuilder() + .addNodes(platformClient, false, true, 0) + .addNodes(platformClient, false, true, 1) + .addNodes(platformClient, false, true, 2) + .addTopic("topic-A", 0, List.of(0, 1, 2), List.of(0, 1, 2), 2) + .mockHealthyNodes(platformClient, 0, 1, 2) + .mockTopics(rollClient) + .mockDescribeConfigs(rollClient, Set.of(), Set.of(), 0) + .done(); + + var ex = assertThrows(MaxRestartsExceededException.class, + () -> doRollingRestart(platformClient, + rollClient, + null, + nodeRefs.values(), + RackRollingTest::podUnresponsive, + EMPTY_CONFIG_SUPPLIER, + 1, + 3)); + + assertEquals("Node pool-kafka-0/0 has been restarted 3 times", ex.getMessage()); + Mockito.verify(platformClient, times(3)).restartNode(eq(nodeRefs.get(0)), any()); + Mockito.verify(platformClient, never()).restartNode(eq(nodeRefs.get(1)), any()); + Mockito.verify(platformClient, never()).restartNode(eq(nodeRefs.get(2)), any()); + Mockito.verify(rollClient, never()).tryElectAllPreferredLeaders(eq(nodeRefs.get(1))); + Mockito.verify(rollClient, never()).tryElectAllPreferredLeaders(eq(nodeRefs.get(2))); + } + + @Test + void shouldRestartWhenCannotGetBrokerState() throws ExecutionException, InterruptedException { + PlatformClient platformClient = mock(PlatformClient.class); + RollClient rollClient = mock(RollClient.class); + AgentClient agentClient = mock(AgentClient.class); + Map quorumState = Map.of(0, 10_000L, 1, 10_000L, 2, 10_000L); + var nodeRefs = new MockBuilder() + .addNodes(platformClient, true, false, 0) + .addNodes(platformClient, true, false, 1) + .addNodes(platformClient, true, true, 2) + .addNodes(platformClient, false, true, 3) + .addNodes(platformClient, false, true, 4) + .mockLeader(rollClient, 1) + .mockHealthyNodes(platformClient, 0, 1, 2, 3, 4) + .mockSuccessfulConnection(rollClient, 0, 1, 2, 3, 4) + .mockQuorumLastCaughtUpTimestamps(rollClient, quorumState) + .done(); + + doThrow(new RuntimeException("Cannot get broker state")) + .when(agentClient) + .getBrokerState(any()); + + var rr = newRollingRestart(platformClient, + rollClient, + agentClient, + nodeRefs.values(), + RackRollingTest::manualRolling, + EMPTY_CONFIG_SUPPLIER, + true, + 1); + + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 0); + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 2); + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 1); + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 3); + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 4); + + assertEquals(List.of(), rr.loop()); + } + + + @Test + public void shouldRestartTwoNodesQuorumControllers() throws ExecutionException, InterruptedException { + PlatformClient platformClient = mock(PlatformClient.class); + RollClient rollClient = mock(RollClient.class); + AgentClient agentClient = mock(AgentClient.class); + Map quorumState = Map.of(1, 10_000L, 2, 10_000L); + var nodeRefs = new MockBuilder() + .addNodes(platformClient, true, false, 1, 2) + .mockLeader(rollClient, 1) + .mockHealthyNodes(platformClient, 1, 2) + .mockSuccessfulConnection(rollClient, 1, 2) + .mockQuorumLastCaughtUpTimestamps(rollClient, quorumState) + .done(); + + var rr = newRollingRestart(platformClient, + rollClient, + agentClient, + nodeRefs.values(), + RackRollingTest::manualRolling, + EMPTY_CONFIG_SUPPLIER, + true, + 3); + + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 2); + + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 1); + + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr); + + assertEquals(List.of(), rr.loop()); + } + + @Test + public void shouldRestartTwoNodesQuorumOneControllerBehind() { + PlatformClient platformClient = mock(PlatformClient.class); + RollClient rollClient = mock(RollClient.class); + AgentClient agentClient = mock(AgentClient.class); + Map quorumState = Map.of(1, 10_000L, 2, 7_000L); + var nodeRefs = new MockBuilder() + .addNodes(platformClient, true, false, 1, 2) + .mockLeader(rollClient, 1) + .mockHealthyNodes(platformClient, 1, 2) + .mockSuccessfulConnection(rollClient, 1, 2) + .mockDescribeConfigs(rollClient, Set.of(), Set.of(), 1, 2) + .mockQuorumLastCaughtUpTimestamps(rollClient, quorumState) + .mockTopics(rollClient) + .mockElectLeaders(rollClient, 1, 2) + .done(); + + assertThrows(MaxAttemptsExceededException.class, () -> + doRollingRestart(platformClient, rollClient, agentClient, nodeRefs.values(), RackRollingTest::manualRolling, EMPTY_CONFIG_SUPPLIER, 3, 1), + "Cannot restart nodes [pool-kafka-1/1] because they violate quorum health or topic availability. The max attempts (3) to retry the nodes has been reached."); + //only the controller that has fallen behind should be restarted + Mockito.verify(platformClient, times(1)).restartNode(eq(nodeRefs.get(2)), any()); + Mockito.verify(platformClient, never()).restartNode(eq(nodeRefs.get(1)), any()); + } + + @Test + public void shouldRestartSingleNodeQuorum() throws ExecutionException, InterruptedException { + PlatformClient platformClient = mock(PlatformClient.class); + RollClient rollClient = mock(RollClient.class); + AgentClient agentClient = mock(AgentClient.class); + Map quorumState = Map.of(1, 10_000L); + var nodeRefs = new MockBuilder() + .addNodes(platformClient, true, false, 1) + .addNode(platformClient, false, true, 2) + .mockLeader(rollClient, 1) + .mockHealthyNodes(platformClient, 1, 2) + .mockSuccessfulConnection(rollClient, 1, 2) + .mockDescribeConfigs(rollClient, Set.of(), Set.of(), 1, 2) + .mockQuorumLastCaughtUpTimestamps(rollClient, quorumState) + .mockTopics(rollClient) + .mockElectLeaders(rollClient, 1, 2) + .done(); + + var rr = newRollingRestart(platformClient, + rollClient, + agentClient, + nodeRefs.values(), + RackRollingTest::manualRolling, + EMPTY_CONFIG_SUPPLIER, + true, + 3); + + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 1); + + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 2); + + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr); + + assertEquals(List.of(), rr.loop()); + } + + + ////////////////////////////////////////////////////// + /// Test scenarios we expect no restarts /// + ////////////////////////////////////////////////////// + + @Test + void shouldNotRestartNodesNoReason() throws ExecutionException, InterruptedException { + PlatformClient platformClient = mock(PlatformClient.class); + RollClient rollClient = mock(RollClient.class); + var nodeRefs = new MockBuilder() + .addNodes(platformClient, true, false, 0) + .addNodes(platformClient, true, false, 1) + .addNodes(platformClient, true, true, 2) + .addNodes(platformClient, false, true, 3) + .addNodes(platformClient, false, true, 4) + .mockLeader(rollClient, 1) + .mockHealthyNodes(platformClient, 0, 1, 2, 3, 4) + .mockSuccessfulConnection(rollClient, 0, 1, 2, 3, 4) + .mockTopics(rollClient) + .mockDescribeConfigs(rollClient, Set.of(), Set.of(), 0, 1, 2, 3, 4) + .mockElectLeaders(rollClient, 2, 3, 4) + .done(); + + doRollingRestart(platformClient, + rollClient, + null, + nodeRefs.values(), + RackRollingTest::noReasons, + EMPTY_CONFIG_SUPPLIER, + 1, + 1); + + Mockito.verify(rollClient, never()).reconfigureNode(any(), any(), any()); + Mockito.verify(platformClient, never()).restartNode(any(), any()); + Mockito.verify(rollClient, never()).tryElectAllPreferredLeaders(any()); + } + + @Test + void shouldNotRestartWhenNotReadyAfterRestart() { + PlatformClient platformClient = mock(PlatformClient.class); + RollClient rollClient = mock(RollClient.class); + var nodeRefs = new MockBuilder() + .addNodes(platformClient, false, true, 0) + .addNodes(platformClient, false, true, 1) + .addNodes(platformClient, false, true, 2) + .mockNodeState(platformClient, List.of(PlatformClient.NodeState.READY, PlatformClient.NodeState.NOT_READY), 0) + .mockHealthyNodes(platformClient, 1, 2) + .mockSuccessfulConnection(rollClient, 0, 1, 2) + .mockTopics(rollClient) + .mockDescribeConfigs(rollClient, Set.of(), Set.of(), 0, 1, 2) + .done(); + + var ex = assertThrows(MaxAttemptsExceededException.class, + () -> doRollingRestart(platformClient, + rollClient, + null, + nodeRefs.values(), + nodeId -> manualRolling(0), + EMPTY_CONFIG_SUPPLIER, + 1, + 1)); + + assertEquals("The max attempts (3) to wait for restarted node pool-kafka-0/0 to become ready has been reached.", ex.getMessage()); + + Mockito.verify(platformClient, times(1)).restartNode(eq(nodeRefs.get(0)), any()); + Mockito.verify(platformClient, never()).restartNode(eq(nodeRefs.get(1)), any()); + Mockito.verify(platformClient, never()).restartNode(eq(nodeRefs.get(2)), any()); + Mockito.verify(rollClient, never()).reconfigureNode(any(), any(), any()); + Mockito.verify(rollClient, never()).tryElectAllPreferredLeaders(any()); + } + + @Test + void shouldNotRestartWhenNotReadyNoReason() { + PlatformClient platformClient = mock(PlatformClient.class); + RollClient rollClient = mock(RollClient.class); + var nodeRefs = new MockBuilder() + .addNodes(platformClient, true, false, 0) + .addNodes(platformClient, true, false, 1) + .addNodes(platformClient, true, true, 2) + .mockNodeState(platformClient, List.of(PlatformClient.NodeState.NOT_READY), 0) + .mockHealthyNodes(platformClient, 1, 2) + .mockSuccessfulConnection(rollClient, 0, 1, 2) + .mockTopics(rollClient) + .mockDescribeConfigs(rollClient, Set.of(), Set.of(), 0, 1, 2) + .done(); + + var ex = assertThrows(MaxAttemptsExceededException.class, + () -> doRollingRestart(platformClient, + rollClient, + null, + nodeRefs.values(), + RackRollingTest::noReasons, + EMPTY_CONFIG_SUPPLIER, + 1, + 1)); + + assertEquals("The max attempts (3) to wait for non-restarted node pool-kafka-0/0 to become ready has been reached.", ex.getMessage()); + + Mockito.verify(platformClient, never()).restartNode(any(), any()); + Mockito.verify(rollClient, never()).reconfigureNode(any(), any(), any()); + Mockito.verify(rollClient, never()).tryElectAllPreferredLeaders(any()); + } + + @Test + void shouldNotRestartWhenQuorumCheckFailed() { + PlatformClient platformClient = mock(PlatformClient.class); + RollClient rollClient = mock(RollClient.class); + Map quorumState = Map.of(0, 10_000L, 1, 10_000L, 2, 6000L); + + var nodeRefs = new MockBuilder() + .addNodes(platformClient, true, false, 0) + .addNodes(platformClient, true, false, 1) + .addNodes(platformClient, true, false, 2) + .mockLeader(rollClient, 1) + .mockHealthyNodes(platformClient, 0, 1, 2) + .mockSuccessfulConnection(rollClient, 0, 1, 2) + .mockDescribeConfigs(rollClient, Set.of(new ConfigEntry("controller.quorum.fetch.timeout.ms", "3000")), Set.of(), 0, 1, 2) + .mockQuorumLastCaughtUpTimestamps(rollClient, quorumState) + .done(); + + var ex = assertThrows(MaxAttemptsExceededException.class, + () -> doRollingRestart(platformClient, + rollClient, + null, + nodeRefs.values(), + RackRollingTest::manualRolling, + EMPTY_CONFIG_SUPPLIER, + 1, + 1)); + + // The active controller and the up-to-date follower should not be restarted but fallen-behind follower can be restarted as doesn't impact the quorum health + assertEquals("Cannot restart nodes [pool-kafka-0/0, pool-kafka-1/1] because they violate quorum health or topic availability. The max attempts (3) to retry the nodes has been reached.", ex.getMessage()); + + Mockito.verify(platformClient, times(1)).restartNode(eq(nodeRefs.get(2)), any()); + Mockito.verify(platformClient, never()).restartNode(eq(nodeRefs.get(0)), any()); + Mockito.verify(platformClient, never()).restartNode(eq(nodeRefs.get(1)), any()); + } + + @Test + void shouldNotRestartWhenAvailabilityCheckFailed() { + PlatformClient platformClient = mock(PlatformClient.class); + RollClient rollClient = mock(RollClient.class); + + var nodeRefs = new MockBuilder() + .addNodes(platformClient, false, true, 0) + .addNodes(platformClient, false, true, 1) + .addNodes(platformClient, false, true, 2) + .mockHealthyNodes(platformClient, 0, 1, 2) + .mockSuccessfulConnection(rollClient, 0, 1, 2) + .addTopic("topic-A", 0, List.of(0, 1, 2), List.of(0, 1), 2) + .mockTopics(rollClient) + .done(); + + var ex = assertThrows(MaxAttemptsExceededException.class, + () -> doRollingRestart(platformClient, + rollClient, + null, + nodeRefs.values(), + RackRollingTest::manualRolling, + EMPTY_CONFIG_SUPPLIER, + 1, + 1)); + + // The partition leader and in sync replica should not be restarted but out of sync replica can be restarted as doesn't impact the availability + assertEquals("Cannot restart nodes [pool-kafka-0/0, pool-kafka-1/1] because they violate quorum health or topic availability. The max attempts (3) to retry the nodes has been reached.", ex.getMessage()); + + Mockito.verify(platformClient, times(1)).restartNode(eq(nodeRefs.get(2)), any()); + Mockito.verify(platformClient, never()).restartNode(eq(nodeRefs.get(0)), any()); + Mockito.verify(platformClient, never()).restartNode(eq(nodeRefs.get(1)), any()); + } + + @Test + void shouldNotRestartCombinedNodesWhenQuorumCheckFailed() { + PlatformClient platformClient = mock(PlatformClient.class); + RollClient rollClient = mock(RollClient.class); + Map quorumState = Map.of(0, 10_000L, 1, 10_000L, 2, 7000L); //default fetch timeout is 2000L + var nodeRefs = new MockBuilder() + .addNodes(platformClient, true, true, 0) + .addNodes(platformClient, true, true, 1) + .addNodes(platformClient, true, true, 2) + .mockLeader(rollClient, 1) + .mockHealthyNodes(platformClient, 0, 1, 2) + .mockSuccessfulConnection(rollClient, 0, 1, 2) + .mockQuorumLastCaughtUpTimestamps(rollClient, quorumState) + .addTopic("topic-A", 0, List.of(0, 1, 2), List.of(0, 1, 2), 2) + .mockTopics(rollClient) + .done(); + + var ex = assertThrows(MaxAttemptsExceededException.class, + () -> doRollingRestart(platformClient, + rollClient, + null, + nodeRefs.values(), + RackRollingTest::manualRolling, + EMPTY_CONFIG_SUPPLIER, + 1, + 1)); + + // The partition leader and in sync replica should not be restarted but out of sync replica can be restarted as doesn't impact the availability + assertEquals("Cannot restart nodes [pool-kafka-0/0, pool-kafka-1/1] because they violate quorum health or topic availability. The max attempts (3) to retry the nodes has been reached.", ex.getMessage()); + + Mockito.verify(platformClient, times(1)).restartNode(eq(nodeRefs.get(2)), any()); + Mockito.verify(platformClient, never()).restartNode(eq(nodeRefs.get(0)), any()); + Mockito.verify(platformClient, never()).restartNode(eq(nodeRefs.get(1)), any()); + } + + @Test + void shouldNotRestartCombinedNodesWhenAvailabilityCheckFailed() { + PlatformClient platformClient = mock(PlatformClient.class); + RollClient rollClient = mock(RollClient.class); + Map quorumState = Map.of(0, 10_000L, 1, 10_000L, 2, 10_000L); + var nodeRefs = new MockBuilder() + .addNodes(platformClient, true, true, 0) + .addNodes(platformClient, true, true, 1) + .addNodes(platformClient, true, true, 2) + .mockLeader(rollClient, 1) + .mockHealthyNodes(platformClient, 0, 1, 2) + .mockSuccessfulConnection(rollClient, 0, 1, 2) + .mockQuorumLastCaughtUpTimestamps(rollClient, quorumState) + .addTopic("topic-A", 0, List.of(0, 1, 2), List.of(0, 1), 2) + .mockTopics(rollClient) + .done(); + + var ex = assertThrows(MaxAttemptsExceededException.class, + () -> doRollingRestart(platformClient, + rollClient, + null, + nodeRefs.values(), + RackRollingTest::manualRolling, + EMPTY_CONFIG_SUPPLIER, + 1, + 1)); + + // The partition leader and in sync replica should not be restarted but out of sync replica can be restarted as doesn't impact the availability + assertEquals("Cannot restart nodes [pool-kafka-0/0, pool-kafka-1/1] because they violate quorum health or topic availability. The max attempts (3) to retry the nodes has been reached.", ex.getMessage()); + + Mockito.verify(platformClient, times(1)).restartNode(eq(nodeRefs.get(2)), any()); + Mockito.verify(platformClient, never()).restartNode(eq(nodeRefs.get(0)), any()); + Mockito.verify(platformClient, never()).restartNode(eq(nodeRefs.get(1)), any()); + } + + @Test + void shouldNotRestartBrokerNodeInRecovery() { + PlatformClient platformClient = mock(PlatformClient.class); + RollClient rollClient = mock(RollClient.class); + AgentClient agentClient = mock(AgentClient.class); + + var nodeRefs = new MockBuilder() + .addNodes(platformClient, false, true, 0) + .addNodes(platformClient, false, true, 1) + .addNodes(platformClient, false, true, 2) + .mockHealthyNodes(platformClient, 0, 1) + .mockNodeState(platformClient, List.of(PlatformClient.NodeState.NOT_READY), 2) + .done(); + + var bs = BrokerState.RECOVERY; + bs.setRemainingLogsToRecover(100); + bs.setRemainingSegmentsToRecover(300); + doReturn(bs) + .when(agentClient) + .getBrokerState(nodeRefs.get(2)); + + var ex = assertThrows(MaxAttemptsExceededException.class, + () -> doRollingRestart(platformClient, + rollClient, + agentClient, + nodeRefs.values(), + RackRollingTest::manualRolling, + EMPTY_CONFIG_SUPPLIER, + 1, + 1)); + + assertEquals("The max attempts (3) to wait for this node pool-kafka-2/2 to finish performing log recovery has been reached. There are 100 logs and 300 segments left to recover.", + ex.getMessage()); + + Mockito.verify(platformClient, never()).restartNode(any(), any()); + } + + @Test + void shouldNotRestartControllerNodeInRecovery() { + PlatformClient platformClient = mock(PlatformClient.class); + RollClient rollClient = mock(RollClient.class); + AgentClient agentClient = mock(AgentClient.class); + + var nodeRefs = new MockBuilder() + .addNodes(platformClient, true, false, 0) + .addNodes(platformClient, true, false, 1) + .addNodes(platformClient, true, false, 2) + .mockLeader(rollClient, 1) + .mockHealthyNodes(platformClient, 0, 1) + .mockNodeState(platformClient, List.of(PlatformClient.NodeState.NOT_READY), 2) + .done(); + + var bs = BrokerState.RECOVERY; + bs.setRemainingLogsToRecover(100); + bs.setRemainingSegmentsToRecover(300); + doReturn(bs) + .when(agentClient) + .getBrokerState(nodeRefs.get(2)); + + var ex = assertThrows(MaxAttemptsExceededException.class, + () -> doRollingRestart(platformClient, + rollClient, + agentClient, + nodeRefs.values(), + RackRollingTest::manualRolling, + EMPTY_CONFIG_SUPPLIER, + 1, + 1)); + + assertEquals("The max attempts (3) to wait for this node pool-kafka-2/2 to finish performing log recovery has been reached. There are 100 logs and 300 segments left to recover.", + ex.getMessage()); + + Mockito.verify(platformClient, never()).restartNode(any(), any()); + } + + @Test + public void shouldNotRestartEvenSizedQuorumTwoControllersBehind() { + PlatformClient platformClient = mock(PlatformClient.class); + RollClient rollClient = mock(RollClient.class); + AgentClient agentClient = mock(AgentClient.class); + Map quorumState = Map.of(0, 10_000L, 1, 10_000L, 2, 7000L, 3, 6000L); + var nodeRefs = new MockBuilder() + .addNodes(platformClient, true, true, 0, 1, 2, 4) //combined nodes + .mockLeader(rollClient, 1) + .mockHealthyNodes(platformClient, 0, 1, 2, 4) + .mockSuccessfulConnection(rollClient, 0, 1, 2, 4) + .mockQuorumLastCaughtUpTimestamps(rollClient, quorumState) + .done(); + + var ex = assertThrows(MaxAttemptsExceededException.class, + () -> doRollingRestart(platformClient, + rollClient, + agentClient, + nodeRefs.values(), + RackRollingTest::manualRolling, + EMPTY_CONFIG_SUPPLIER, + 1, + 1)); + + // we should not restart any controllers as the majority have not caught up to the leader + assertEquals("Cannot restart nodes [pool-kafka-0/0, pool-kafka-1/1, pool-kafka-2/2, pool-kafka-4/4] because they violate quorum health or topic availability. The max attempts (3) to retry the nodes has been reached.", + ex.getMessage()); + + Mockito.verify(platformClient, never()).restartNode(any(), any()); + } + + @Test + public void shouldNotRestartControllersWithInvalidTimestamp() { + // given + PlatformClient platformClient = mock(PlatformClient.class); + RollClient rollClient = mock(RollClient.class); + AgentClient agentClient = mock(AgentClient.class); + Map quorumState = Map.of(0, -1L, 1, 10_000L, 2, -1L); + var nodeRefs = new MockBuilder() + .addNodes(platformClient, true, false, 0, 1, 2) + .mockLeader(rollClient, 1) + .mockHealthyNodes(platformClient, 0, 1, 2) + .mockSuccessfulConnection(rollClient, 0, 1, 2) + .mockQuorumLastCaughtUpTimestamps(rollClient, quorumState) + .done(); + + var ex = assertThrows(MaxAttemptsExceededException.class, + () -> doRollingRestart(platformClient, + rollClient, + agentClient, + nodeRefs.values(), + RackRollingTest::manualRolling, + EMPTY_CONFIG_SUPPLIER, + 1, + 1)); + + // we should not restart any controllers as the majority have not caught up to the leader + assertEquals("Cannot restart nodes [pool-kafka-0/0, pool-kafka-1/1, pool-kafka-2/2] because they violate quorum health or topic availability. The max attempts (3) to retry the nodes has been reached.", + ex.getMessage()); + + Mockito.verify(platformClient, never()).restartNode(any(), any()); + } + + @Test + public void shouldNotRollControllersWithInvalidLeader() { + // given + PlatformClient platformClient = mock(PlatformClient.class); + RollClient rollClient = mock(RollClient.class); + AgentClient agentClient = mock(AgentClient.class); + Map quorumState = Map.of(0, 10_000L, 1, 10_000L, 2, 10_000L); + var nodeRefs = new MockBuilder() + .addNodes(platformClient, true, false, 0, 1, 2) + .mockLeader(rollClient, -1) + .mockHealthyNodes(platformClient, 0, 1, 2) + .mockSuccessfulConnection(rollClient, 0, 1, 2) + .mockDescribeConfigs(rollClient, Set.of(), Set.of(), 0, 1, 2) + .mockQuorumLastCaughtUpTimestamps(rollClient, quorumState) + .done(); + + //TODO: Revise how we should handle this (should be similar to the current roller) + + var ex = assertThrows(MaxAttemptsExceededException.class, + () -> doRollingRestart(platformClient, + rollClient, + agentClient, + nodeRefs.values(), + RackRollingTest::manualRolling, + EMPTY_CONFIG_SUPPLIER, + 1, + 1)); + + // we should not restart any controllers as the majority have not caught up to the leader + assertEquals("Cannot restart nodes [pool-kafka-0/0, pool-kafka-1/1, pool-kafka-2/2] because they violate quorum health or topic availability. The max attempts (3) to retry the nodes has been reached.", + ex.getMessage()); + } + + @Test + public void shouldThrowExceptionInitAdminException() { + PlatformClient platformClient = mock(PlatformClient.class); + RollClient rollClient = mock(RollClient.class); + AgentClient agentClient = mock(AgentClient.class); + doThrow(new RuntimeException("Failed to create admin client for brokers")).when(rollClient).initialiseBrokerAdmin(any()); + Map quorumState = Map.of(1, 10_000L); + + var nodeRefs = new MockBuilder() + .addNodes(platformClient, true, false, 1) + .addNode(platformClient, false, true, 2) + .mockLeader(rollClient, 1) + .mockHealthyNodes(platformClient, 1, 2) + .mockSuccessfulConnection(rollClient, 1, 2) + .mockDescribeConfigs(rollClient, Set.of(), Set.of(), 1, 2) + .mockQuorumLastCaughtUpTimestamps(rollClient, quorumState) + .mockTopics(rollClient) + .done(); + + var rr = newRollingRestart(platformClient, + rollClient, + agentClient, + nodeRefs.values(), + RackRollingTest::manualRolling, + EMPTY_CONFIG_SUPPLIER, + true, + 3); + + var ex = assertThrows(RuntimeException.class, rr::loop, + "Expect RuntimeException because admin client cannot be created"); + assertEquals("Failed to create admin client for brokers", ex.getMessage()); + } + + @Test + public void shouldNotReconfigureWhenAllowReconfigurationIsFalse() throws ExecutionException, InterruptedException { + PlatformClient platformClient = mock(PlatformClient.class); + RollClient rollClient = mock(RollClient.class); + AgentClient agentClient = mock(AgentClient.class); + + var nodeRefs = new MockBuilder() + .addNodes(platformClient, true, true, 1) + .mockHealthyNodes(platformClient, 1) + .mockSuccessfulConnection(rollClient, 1) + .mockDescribeConfigs(rollClient, Set.of(new ConfigEntry("compression.type", "zstd")), Set.of(), 1) + .mockTopics(rollClient) + .done(); + + var rr = newRollingRestart(platformClient, + rollClient, + agentClient, + nodeRefs.values(), + RackRollingTest::noReasons, + serverId -> "compression.type=snappy", + false, + 3); + + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr); + + Mockito.verify(rollClient, never()).reconfigureNode(eq(nodeRefs.get(1)), any(), any()); + Mockito.verify(platformClient, never()).restartNode(eq(nodeRefs.get(1)), any()); + Mockito.verify(rollClient, never()).tryElectAllPreferredLeaders(eq(nodeRefs.get(1))); + } + + @Test + public void shouldNotRestartDynamicConfig() throws ExecutionException, InterruptedException { + PlatformClient platformClient = mock(PlatformClient.class); + RollClient rollClient = mock(RollClient.class); + + var nodeRefs = new MockBuilder() + .addNodes(platformClient, true, false, 0) + .addNodes(platformClient, true, false, 1) + .addNodes(platformClient, true, true, 2) + .addNodes(platformClient, false, true, 3) + .addNodes(platformClient, false, true, 4) + .mockLeader(rollClient, 1) + .mockHealthyNodes(platformClient, 0, 1, 2, 3, 4) + .mockSuccessfulConnection(rollClient, 0, 1, 2, 3, 4) + .mockDescribeConfigs(rollClient, Set.of(new ConfigEntry("min.insync.replicas", "1")), Set.of(), 0, 1, 2, 3, 4) + .mockReconfigureConfigs(rollClient) + .done(); + + doRollingRestart(platformClient, + rollClient, + null, + nodeRefs.values(), + RackRollingTest::noReasons, + nodeId -> "min.insync.replicas=2", + 1, + 1); + + for (var nodeRef : nodeRefs.values()) { + //TODO: the following should apply to controller nodes as well later, when controller reconfiguration is implemented + if (nodeRef.broker()) { + Mockito.verify(rollClient, times(1)).reconfigureNode(eq(nodeRef), any(), any()); + Mockito.verify(platformClient, never()).restartNode(eq(nodeRef), any()); + } else { + Mockito.verify(rollClient, never()).reconfigureNode(eq(nodeRef), any(), any()); + Mockito.verify(platformClient, never()).restartNode(eq(nodeRef), any()); + } + } + } + + //TODO: should we test logging config update? + + + ////////////////////////////////////////////////////// + /// Test scenarios for batch rolling /// + ////////////////////////////////////////////////////// + + @Test + public void shouldRestartInExpectedOrderAndBatched() throws ExecutionException, InterruptedException { + // given + PlatformClient platformClient = mock(PlatformClient.class); + RollClient rollClient = mock(RollClient.class); + AgentClient agentClient = mock(AgentClient.class); + Map quorumState = Map.of(0, 10_000L, 1, 10_000L, 2, 10_000L); + var nodeRefs = new MockBuilder() + .addNodes(platformClient, true, false, 0, 1, 2) // controllers + .addNodes(platformClient, false, true, // brokers + 3, 6, // rack X + 4, 7, // rack Y + 5, 8) // rack Z + .mockLeader(rollClient, 1) + .mockHealthyNodes(platformClient, 0, 1, 2, 3, 4, 5, 6, 7, 8) + .mockSuccessfulConnection(rollClient, 0, 1, 2, 3, 4, 5, 6, 7, 8) + .addTopic("topic-A", 3, List.of(3, 4, 5), List.of(3, 4, 5)) + .addTopic("topic-B", 6, List.of(6, 7, 8), List.of(6, 7, 8)) + .addTopic("topic-C", 4, List.of(4, 8, 6), List.of(4, 8, 6)) + .addTopic("topic-D", 7, List.of(7, 3, 5), List.of(7, 3, 5)) + .addTopic("topic-E", 6, List.of(6, 4, 5), List.of(6, 4, 5)) + .mockDescribeConfigs(rollClient, Set.of(), Set.of(), 0, 1, 2, 3, 4, 5, 6, 7, 8) + .mockQuorumLastCaughtUpTimestamps(rollClient, quorumState) + .mockTopics(rollClient) + .mockElectLeaders(rollClient, 3, 4, 5, 6, 7, 8) + .done(); + + // when + var rr = newRollingRestart(platformClient, + rollClient, + agentClient, + nodeRefs.values(), + RackRollingTest::manualRolling, + EMPTY_CONFIG_SUPPLIER, + true, + 3); + + // The expected order is non-active controllers, active controller and batches of brokers that don't have partitions in common + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 0); //non-active controller + + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 2); //non-active controller + + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 1); //active controller + + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 5, 8); + + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 3, 6); + + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 4, 7); + + assertEquals(List.of(), rr.loop()); + + for (var nodeRef : nodeRefs.values()) { + Mockito.verify(rollClient, never()).reconfigureNode(eq(nodeRef), any(), any()); + } + } + + @Test + public void shouldRestartCombinedNodesInExpectedOrderAndBatched() throws ExecutionException, InterruptedException { + // given + PlatformClient platformClient = mock(PlatformClient.class); + RollClient rollClient = mock(RollClient.class); + AgentClient agentClient = mock(AgentClient.class); + Map quorumState = Map.of(3, 10_000L, + 4, 10_000L, + 5, 10_000L, + 6, 10_000L, + 7, 10_000L, + 8, 5_000L); + var nodeRefs = new MockBuilder() + .addNodes(platformClient, true, true, // combined nodes + 3, 6, // rack X + 4, 7, // rack Y + 5, 8) // rack Z + .mockLeader(rollClient, 3) + .mockHealthyNodes(platformClient, 3, 4, 5, 6, 7, 8) + .mockSuccessfulConnection(rollClient, 3, 4, 5, 6, 7, 8) + .addTopic("topic-A", 3, List.of(3, 4, 5), List.of(3, 4, 5)) + .addTopic("topic-B", 6, List.of(6, 7, 8), List.of(6, 7, 8)) + .addTopic("topic-C", 4, List.of(4, 8, 6), List.of(4, 8, 6)) + .addTopic("topic-D", 7, List.of(7, 3, 5), List.of(7, 3, 5)) + .addTopic("topic-E", 6, List.of(6, 4, 5), List.of(6, 4, 5)) + .mockDescribeConfigs(rollClient, Set.of(), Set.of(), 3, 4, 5, 6, 7, 8) + .mockQuorumLastCaughtUpTimestamps(rollClient, quorumState) + .mockTopics(rollClient) + .mockElectLeaders(rollClient, 3, 4, 5, 6, 7, 8) + .done(); + + var rr = newRollingRestart(platformClient, + rollClient, + agentClient, + nodeRefs.values(), + RackRollingTest::manualRolling, + EMPTY_CONFIG_SUPPLIER, + true, + 3); + + // The expected order to restart nodes individually based on the availability and quorum health and then the broker that is the active controller will be started at last + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 6); + + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 4); + + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 7); + + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 5); + + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 8); + + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 3); // the active controller + + assertEquals(List.of(), rr.loop()); + + for (var nodeRef : nodeRefs.values()) { + Mockito.verify(rollClient, never()).reconfigureNode(eq(nodeRef), any(), any()); + } + } + + @Test + public void shouldRestartInExpectedOrderAndBatchedWithUrp() throws ExecutionException, InterruptedException { + PlatformClient platformClient = mock(PlatformClient.class); + RollClient rollClient = mock(RollClient.class); + AgentClient agentClient = mock(AgentClient.class); + Map quorumState = Map.of(0, 10_000L, 1, 10_000L, 2, 10_000L); + var nodeRefs = new MockBuilder() + .addNodes(platformClient, true, false, 0, 1, 2) // controllers + .addNodes(platformClient, false, true, // brokers + 3, 6, // rack X + 4, 7, // rack Y + 5, 8) // rack Z + .mockLeader(rollClient, 1) + .mockHealthyNodes(platformClient, 0, 1, 2, 3, 4, 5, 6, 7, 8) + .mockSuccessfulConnection(rollClient, 0, 1, 2, 3, 4, 5, 6, 7, 8) + // topic A is at its min ISR, so neither 3 nor 4 should be restarted + .addTopic("topic-A", 3, List.of(3, 4, 5), List.of(3, 4), 2) + .addTopic("topic-B", 6, List.of(6, 7, 8), List.of(6, 7, 8)) + .addTopic("topic-C", 4, List.of(4, 8, 6), List.of(4, 8, 6)) + .addTopic("topic-D", 7, List.of(7, 3, 5), List.of(7, 3, 5)) + .addTopic("topic-E", 6, List.of(6, 4, 5), List.of(6, 4, 5)) + .mockDescribeConfigs(rollClient, Set.of(), Set.of(), 0, 1, 2, 3, 4, 5, 6, 7, 8) + .mockQuorumLastCaughtUpTimestamps(rollClient, quorumState) + .mockTopics(rollClient) + .mockElectLeaders(rollClient, 3, 4, 5, 6, 7, 8) + .done(); + + var rr = newRollingRestart(platformClient, + rollClient, + agentClient, + nodeRefs.values(), + RackRollingTest::manualRolling, + EMPTY_CONFIG_SUPPLIER, + true, + 3); + + // The expected order is non-active controller nodes, the active controller, + // batches of brokers starting with the largest. + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 0); + + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 2); + + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 1); + + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 5, 8); // the largest batch of brokers that do not have partitions in common + + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 7); // 7 doesn't have partitions in common with 3 but 3 will cause under min ISR + + assertNodesRestarted(platformClient, rollClient, nodeRefs, rr, 6); // 6 doesn't have partitions in common with 4 but 4 will cause under min ISR + + var ex = assertThrows(MaxAttemptsExceededException.class, + () -> { + List nodes; + do { + nodes = rr.loop(); + } while (!nodes.isEmpty()); + }); + + assertEquals("Cannot restart nodes [pool-kafka-3/3, pool-kafka-4/4] because they violate quorum health or topic availability. The max attempts (3) to retry the nodes has been reached.", + ex.getMessage()); + + Mockito.verify(platformClient, never()).restartNode(eq(nodeRefs.get(3)), any()); + Mockito.verify(platformClient, never()).restartNode(eq(nodeRefs.get(4)), any()); + } +} diff --git a/cluster-operator/src/test/java/io/strimzi/operator/cluster/operator/resource/rolling/TimeTest.java b/cluster-operator/src/test/java/io/strimzi/operator/cluster/operator/resource/rolling/TimeTest.java new file mode 100644 index 0000000000..19e0460f1d --- /dev/null +++ b/cluster-operator/src/test/java/io/strimzi/operator/cluster/operator/resource/rolling/TimeTest.java @@ -0,0 +1,29 @@ +/* + * Copyright Strimzi authors. + * License: Apache License 2.0 (see the file LICENSE or http://apache.org/licenses/LICENSE-2.0.html). + */ +package io.strimzi.operator.cluster.operator.resource.rolling; + +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; + +import java.util.List; + +import static org.junit.jupiter.api.Assertions.assertTrue; + +class TimeTest { + + public static List