From f43067063d94e9d8331df52e6b32a275c919e58a Mon Sep 17 00:00:00 2001 From: Federico Valeri Date: Thu, 5 Sep 2024 17:01:40 +0200 Subject: [PATCH] Add Topic Operator refactoring: handlers (#10412) Signed-off-by: Federico Valeri --- .../strimzi/operator/topic/BatchingLoop.java | 108 +- .../topic/BatchingTopicController.java | 1628 +++++++---------- .../strimzi/operator/topic/KafkaHandler.java | 438 +++++ .../operator/topic/KubernetesHandler.java | 119 ++ .../operator/topic/ReplicasChangeHandler.java | 203 -- ...entHandler.java => TopicEventHandler.java} | 23 +- .../operator/topic/TopicOperatorConfig.java | 120 +- .../operator/topic/TopicOperatorMain.java | 82 +- .../operator/topic/TopicOperatorUtil.java | 95 +- .../cruisecontrol/CruiseControlClient.java | 7 +- .../CruiseControlClientImpl.java | 28 +- .../cruisecontrol/CruiseControlHandler.java | 216 +++ .../topic/cruisecontrol/UrlBuilder.java | 4 +- .../metrics/TopicOperatorMetricsProvider.java | 16 +- .../strimzi/operator/topic/model/Results.java | 201 ++ .../operator/topic/model/TopicState.java | 9 +- .../topic/BatchingTopicControllerTest.java | 792 +++++--- .../operator/topic/KafkaHandlerTest.java | 274 +++ .../operator/topic/KubernetesHandlerTest.java | 208 +++ .../operator/topic/TopicControllerIT.java | 60 +- .../topic/TopicOperatorMetricsTest.java | 250 ++- .../operator/topic/TopicOperatorTestUtil.java | 21 +- .../CruiseControlHandlerTest.java} | 123 +- .../cruisecontrol/MockCruiseControl.java | 58 +- .../topic/{ => model}/TopicUpsertTest.java | 2 +- 25 files changed, 3210 insertions(+), 1875 deletions(-) create mode 100644 topic-operator/src/main/java/io/strimzi/operator/topic/KafkaHandler.java create mode 100644 topic-operator/src/main/java/io/strimzi/operator/topic/KubernetesHandler.java delete mode 100644 topic-operator/src/main/java/io/strimzi/operator/topic/ReplicasChangeHandler.java rename topic-operator/src/main/java/io/strimzi/operator/topic/{TopicOperatorEventHandler.java => TopicEventHandler.java} (75%) create mode 100644 topic-operator/src/main/java/io/strimzi/operator/topic/cruisecontrol/CruiseControlHandler.java create mode 100644 topic-operator/src/main/java/io/strimzi/operator/topic/model/Results.java create mode 100644 topic-operator/src/test/java/io/strimzi/operator/topic/KafkaHandlerTest.java create mode 100644 topic-operator/src/test/java/io/strimzi/operator/topic/KubernetesHandlerTest.java rename topic-operator/src/test/java/io/strimzi/operator/topic/{ReplicasChangeHandlerTest.java => cruisecontrol/CruiseControlHandlerTest.java} (78%) rename topic-operator/src/test/java/io/strimzi/operator/topic/{ => model}/TopicUpsertTest.java (95%) diff --git a/topic-operator/src/main/java/io/strimzi/operator/topic/BatchingLoop.java b/topic-operator/src/main/java/io/strimzi/operator/topic/BatchingLoop.java index 06e3201be37..15089cef403 100644 --- a/topic-operator/src/main/java/io/strimzi/operator/topic/BatchingLoop.java +++ b/topic-operator/src/main/java/io/strimzi/operator/topic/BatchingLoop.java @@ -30,13 +30,21 @@ * the reconciliation of those events using a {@link BatchingTopicController}. * Any given {@link KafkaTopic} is only being reconciled by a single thread at any one time. */ -class BatchingLoop { - +public class BatchingLoop { static final ReconciliationLogger LOGGER = ReconciliationLogger.create(BatchingLoop.class); private final BatchingTopicController controller; private final BlockingDeque queue; + private final int maxQueueSize; + private final int maxBatchSize; + private final long maxBatchLingerMs; + private final String namespace; + + private final ItemStore itemStore; + private final Runnable stopRunnable; + private final TopicOperatorMetricsHolder metricsHolder; + /** * The set of topics currently being reconciled by a controller. * Guarded by the monitor of the BatchingLoop. @@ -44,41 +52,42 @@ class BatchingLoop { */ private final Set inFlight = new HashSet<>(); // guarded by this private final LoopRunnable[] threads; - private final int maxBatchSize; - private final long maxBatchLingerMs; - private final ItemStore itemStore; - private final Runnable stop; - private final int maxQueueSize; - private final TopicOperatorMetricsHolder metrics; - private final String namespace; - public BatchingLoop( - int maxQueueSize, - BatchingTopicController controller, - int maxThreads, - int maxBatchSize, - long maxBatchLingerMs, - ItemStore itemStore, - Runnable stop, - TopicOperatorMetricsHolder metrics, - String namespace) { - this.maxQueueSize = maxQueueSize; - this.queue = new LinkedBlockingDeque<>(maxQueueSize); + /** + * Create a new instance. + * + * @param config Topic Operator configuration. + * @param controller KafkaTopic controller. + * @param maxThreads Max number of LoopRunnable threads. + * @param itemStore Item store. + * @param stopRunnable Stop runnable. + * @param metricsHolder Metrics holder. + */ + public BatchingLoop(TopicOperatorConfig config, + BatchingTopicController controller, + int maxThreads, + ItemStore itemStore, + Runnable stopRunnable, + TopicOperatorMetricsHolder metricsHolder) { + this.maxQueueSize = config.maxQueueSize(); + this.maxBatchSize = config.maxBatchSize(); + this.maxBatchLingerMs = config.maxBatchLingerMs(); + this.namespace = config.namespace(); + this.controller = controller; + this.itemStore = itemStore; + this.stopRunnable = stopRunnable; + this.metricsHolder = metricsHolder; + + this.queue = new LinkedBlockingDeque<>(maxQueueSize); this.threads = new LoopRunnable[maxThreads]; for (int i = 0; i < maxThreads; i++) { threads[i] = new LoopRunnable("LoopRunnable-" + i); } - this.maxBatchSize = maxBatchSize; - this.maxBatchLingerMs = maxBatchLingerMs; - this.itemStore = itemStore; - this.stop = stop; - this.metrics = metrics; - this.namespace = namespace; } /** - * Starts the threads + * Starts the threads. */ public void start() { for (var thread : threads) { @@ -87,7 +96,7 @@ public void start() { } /** - * Stops the threads + * Stops the threads. * @throws InterruptedException If interrupted while waiting for the threads to stop. */ public void stop() throws InterruptedException { @@ -101,24 +110,24 @@ public void stop() throws InterruptedException { /** * Add an event to be reconciled to the tail of the {@link #queue}. - * @param event The event + * @param event The event. */ public void offer(TopicEvent event) { if (queue.offerFirst(event)) { LOGGER.debugOp("Item {} added to front of queue", event); - metrics.reconciliationsMaxQueueSize(namespace).getAndUpdate(size -> Math.max(size, queue.size())); + metricsHolder.reconciliationsMaxQueueSize(namespace).getAndUpdate(size -> Math.max(size, queue.size())); } else { LOGGER.errorOp("Queue length {} exceeded, stopping operator. Please increase {} environment variable.", - maxQueueSize, - TopicOperatorConfig.MAX_QUEUE_SIZE.key()); - this.stop.run(); + maxQueueSize, + TopicOperatorConfig.MAX_QUEUE_SIZE.key()); + this.stopRunnable.run(); } } /** * The loop is alive if none of the threads have been blocked for more than 2 minutes. * "Blocked" means they're not returned to their outermost loop. - * @return True if the loop is alive.. + * @return True if the loop is alive. */ boolean isAlive() { for (var thread : threads) { @@ -154,7 +163,6 @@ boolean isReady() { * A thread that services the head of the {@link #queue}. */ class LoopRunnable extends Thread { - private volatile boolean stopRequested = false; LoopRunnable(String name) { @@ -163,21 +171,21 @@ class LoopRunnable extends Thread { } static final ReconciliationLogger LOGGER = ReconciliationLogger.create(LoopRunnable.class); - private volatile long lastLoop = System.nanoTime(); + private volatile long lastLoopNs = System.nanoTime(); long msSinceLastLoop() { - return (System.nanoTime() - lastLoop) / 1_000_000; + return (System.nanoTime() - lastLoopNs) / 1_000_000; } @Override public void run() { LOGGER.debugOp("Entering run()"); - Batch batch = new Batch(maxBatchSize); - int batchId = 0; - lastLoop = System.nanoTime(); + var batch = new Batch(maxBatchSize); + var batchId = 0; + lastLoopNs = System.nanoTime(); while (!runOnce(batchId, batch)) { batchId++; - lastLoop = System.nanoTime(); + lastLoopNs = System.nanoTime(); } LOGGER.debugOp("Exiting run()"); } @@ -232,7 +240,7 @@ private ReconcilableTopic lookup(int batchId, TopicUpsert topicUpsert) { var kt = itemStore.get(key); if (kt != null) { LOGGER.traceOp("[Batch #{}] Lookup from item store for {} yielded KafkaTopic with resourceVersion {}", - batchId, topicUpsert, BatchingTopicController.resourceVersion(kt)); + batchId, topicUpsert, TopicOperatorUtil.resourceVersion(kt)); var r = new Reconciliation("upsert", "KafkaTopic", topicUpsert.namespace(), topicUpsert.name()); LOGGER.debugOp("[Batch #{}] Contains {}", batchId, r); return new ReconcilableTopic(r, kt, TopicOperatorUtil.topicName(kt)); @@ -240,7 +248,7 @@ private ReconcilableTopic lookup(int batchId, TopicUpsert topicUpsert) { // Null can happen if the KafkaTopic has been deleted from Kube and we've not yet processed // the corresponding delete event LOGGER.traceOp("[Batch #{}] Lookup from item store for {} yielded nothing", - batchId, topicUpsert); + batchId, topicUpsert); return null; } } @@ -249,20 +257,20 @@ private void fillBatch(int batchId, Batch batch) throws InterruptedException { LOGGER.traceOp("[Batch #{}] Filling", batchId); List rejected = new ArrayList<>(); - final long deadlineNanoTime = System.nanoTime() + maxBatchLingerMs * 1_000_000; + var deadlineNs = System.nanoTime() + maxBatchLingerMs * 1_000_000; while (true) { if (batch.size() >= maxBatchSize) { LOGGER.traceOp("[Batch #{}] Reached maxBatchSize, batch complete", batchId, maxBatchSize); break; } - long timeoutNs = deadlineNanoTime - System.nanoTime(); + var timeoutNs = deadlineNs - System.nanoTime(); if (timeoutNs <= 0) { LOGGER.traceOp("[Batch #{}] {}ms linger expired", batchId, maxBatchLingerMs); break; } LOGGER.traceOp("[Batch #{}] Taking next item from deque head with timeout {}ns", batchId, timeoutNs); - TopicEvent topicEvent = queue.pollFirst(timeoutNs, TimeUnit.NANOSECONDS); + var topicEvent = queue.pollFirst(timeoutNs, TimeUnit.NANOSECONDS); if (topicEvent == null) { LOGGER.traceOp("[Batch #{}] Linger expired, batch complete", batchId); @@ -271,7 +279,7 @@ private void fillBatch(int batchId, Batch batch) throws InterruptedException { addToBatch(batchId, batch, rejected, topicEvent); } LOGGER.traceOp("[Batch #{}] Filled with {} topics", batchId, batch.size()); - metrics.reconciliationsMaxBatchSize(namespace).getAndUpdate(size -> Math.max(size, batch.size())); + metricsHolder.reconciliationsMaxBatchSize(namespace).getAndUpdate(size -> Math.max(size, batch.size())); // here we need a deque and can push `rejected` back on the front of the queue // where they can be taken by the next thread. @@ -287,7 +295,7 @@ private void addToBatch(int batchId, Batch batch, List rejected, Top // E.g. upset then delete is equivalent to just a delete // It's actually a bit tricky since you have to process the events in reverse order to correctly // simplify them, so `Batch` would have to be something like a `Map>`. - KubeRef ref = topicEvent.toRef(); + var ref = topicEvent.toRef(); if (inFlight.add(ref)) { // wasn't already inflight LOGGER.debugOp("[Batch #{}] Adding {}", batchId, topicEvent); @@ -299,7 +307,7 @@ private void addToBatch(int batchId, Batch batch, List rejected, Top } else { LOGGER.debugOp("[Batch #{}] Rejecting item {}, already inflight", batchId, topicEvent); rejected.add(topicEvent); - metrics.lockedReconciliationsCounter(namespace).increment(); + metricsHolder.lockedReconciliationsCounter(namespace).increment(); } } } diff --git a/topic-operator/src/main/java/io/strimzi/operator/topic/BatchingTopicController.java b/topic-operator/src/main/java/io/strimzi/operator/topic/BatchingTopicController.java index 7a49badbc38..d08a86556a6 100644 --- a/topic-operator/src/main/java/io/strimzi/operator/topic/BatchingTopicController.java +++ b/topic-operator/src/main/java/io/strimzi/operator/topic/BatchingTopicController.java @@ -4,50 +4,31 @@ */ package io.strimzi.operator.topic; -import io.fabric8.kubernetes.client.KubernetesClient; import io.fabric8.kubernetes.client.KubernetesClientException; -import io.strimzi.api.kafka.Crds; import io.strimzi.api.kafka.model.common.Condition; import io.strimzi.api.kafka.model.common.ConditionBuilder; import io.strimzi.api.kafka.model.topic.KafkaTopic; -import io.strimzi.api.kafka.model.topic.KafkaTopicBuilder; import io.strimzi.api.kafka.model.topic.KafkaTopicStatusBuilder; +import io.strimzi.api.kafka.model.topic.ReplicasChangeState; import io.strimzi.operator.common.Reconciliation; import io.strimzi.operator.common.ReconciliationLogger; -import io.strimzi.operator.common.model.InvalidResourceException; -import io.strimzi.operator.common.model.StatusDiff; import io.strimzi.operator.common.model.StatusUtils; +import io.strimzi.operator.topic.cruisecontrol.CruiseControlHandler; import io.strimzi.operator.topic.metrics.TopicOperatorMetricsHolder; import io.strimzi.operator.topic.model.Either; import io.strimzi.operator.topic.model.KubeRef; import io.strimzi.operator.topic.model.Pair; import io.strimzi.operator.topic.model.PartitionedByError; import io.strimzi.operator.topic.model.ReconcilableTopic; +import io.strimzi.operator.topic.model.Results; import io.strimzi.operator.topic.model.TopicOperatorException; import io.strimzi.operator.topic.model.TopicState; import io.strimzi.operator.topic.model.UncheckedInterruptedException; -import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.admin.AlterConfigOp; -import org.apache.kafka.clients.admin.AlterConfigsResult; import org.apache.kafka.clients.admin.Config; import org.apache.kafka.clients.admin.ConfigEntry; -import org.apache.kafka.clients.admin.CreatePartitionsResult; -import org.apache.kafka.clients.admin.CreateTopicsResult; -import org.apache.kafka.clients.admin.DeleteTopicsResult; -import org.apache.kafka.clients.admin.DescribeClusterResult; -import org.apache.kafka.clients.admin.DescribeConfigsResult; -import org.apache.kafka.clients.admin.DescribeTopicsResult; import org.apache.kafka.clients.admin.NewPartitions; -import org.apache.kafka.clients.admin.NewTopic; -import org.apache.kafka.clients.admin.PartitionReassignment; -import org.apache.kafka.clients.admin.TopicDescription; -import org.apache.kafka.common.KafkaFuture; -import org.apache.kafka.common.TopicCollection; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.config.ConfigResource; -import org.apache.kafka.common.errors.ApiException; import org.apache.kafka.common.errors.TopicDeletionDisabledException; -import org.apache.kafka.common.errors.TopicExistsException; import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; import java.io.InterruptedIOException; @@ -63,332 +44,204 @@ import java.util.Objects; import java.util.Optional; import java.util.Set; -import java.util.concurrent.ExecutionException; import java.util.function.Function; import java.util.function.Predicate; +import java.util.function.UnaryOperator; import java.util.stream.Collectors; import java.util.stream.Stream; -import static io.strimzi.api.kafka.model.topic.ReplicasChangeState.ONGOING; -import static io.strimzi.api.kafka.model.topic.ReplicasChangeState.PENDING; -import static io.strimzi.operator.topic.TopicOperatorUtil.partitionedByError; -import static io.strimzi.operator.topic.TopicOperatorUtil.partitions; -import static io.strimzi.operator.topic.TopicOperatorUtil.replicas; -import static io.strimzi.operator.topic.TopicOperatorUtil.startReconciliationTimer; -import static io.strimzi.operator.topic.TopicOperatorUtil.stopReconciliationTimer; -import static io.strimzi.operator.topic.TopicOperatorUtil.topicNames; -import static java.util.function.UnaryOperator.identity; - /** * Controller that reconciles batches of {@link KafkaTopic} events. */ @SuppressWarnings({"checkstyle:ClassFanOutComplexity", "checkstyle:ClassDataAbstractionCoupling"}) public class BatchingTopicController { static final ReconciliationLogger LOGGER = ReconciliationLogger.create(BatchingTopicController.class); - - static final String FINALIZER = "strimzi.io/topic-operator"; - static final String AUTO_CREATE_TOPICS_ENABLE = "auto.create.topics.enable"; - static final String MIN_INSYNC_REPLICAS = "min.insync.replicas"; - - private static final int BROKER_DEFAULT = -1; - private final boolean useFinalizer; - private final boolean enableAdditionalMetrics; - - private final Admin admin; + static final List THROTTLING_CONFIG = List.of("follower.replication.throttled.replicas", "leader.replication.throttled.replicas"); + static final String INVALID_CONFIG = "InvalidConfig"; private final TopicOperatorConfig config; private final Map selector; - private final KubernetesClient kubeClient; - - // Key: topic name, Value: The KafkaTopics known to manage that topic - /* test */ final Map> topics = new HashMap<>(); + private final KubernetesHandler kubernetesHandler; + private final KafkaHandler kafkaHandler; + private final TopicOperatorMetricsHolder metricsHolder; + private final CruiseControlHandler cruiseControlHandler; + + /* test */ final Map> topicRefs; // key: topic name, value: the KafkaTopics known to manage that topic + private Map topicIds; // topic id cache updated on every reconciliation + private final Set alterableConfigs; - // topic name id map, which is updated on every reconciliation - private Map topicNameIdMap; - - private final TopicOperatorMetricsHolder metrics; - private final String namespace; - private final ReplicasChangeHandler replicasChangeHandler; - BatchingTopicController(TopicOperatorConfig config, Map selector, - Admin admin, - KubernetesClient kubeClient, - TopicOperatorMetricsHolder metrics, - ReplicasChangeHandler replicasChangeHandler) { + KubernetesHandler kubernetesHandler, + KafkaHandler kafkaHandler, + TopicOperatorMetricsHolder metricsHolder, + CruiseControlHandler cruiseControlHandler) { this.config = config; - this.selector = Objects.requireNonNull(selector); - this.useFinalizer = config.useFinalizer(); - this.admin = admin; + this.selector = selector; + this.kubernetesHandler = kubernetesHandler; + this.kafkaHandler = kafkaHandler; var skipClusterConfigReview = config.skipClusterConfigReview(); if (!skipClusterConfigReview) { // Get the config of some broker and check whether auto topic creation is enabled - Optional autoCreateValue = getClusterConfig(admin, AUTO_CREATE_TOPICS_ENABLE); + Optional autoCreateValue = kafkaHandler.clusterConfig(KafkaHandler.AUTO_CREATE_TOPICS_ENABLE); if (autoCreateValue.filter("true"::equals).isPresent()) { LOGGER.warnOp( - "It is recommended that " + AUTO_CREATE_TOPICS_ENABLE + " is set to 'false' " + - "to avoid races between the operator and Kafka applications auto-creating topics"); + "It is recommended that " + KafkaHandler.AUTO_CREATE_TOPICS_ENABLE + " is set to 'false' " + + "to avoid races between the operator and Kafka applications auto-creating topics"); } } - this.kubeClient = kubeClient; - this.metrics = metrics; - this.namespace = config.namespace(); - this.enableAdditionalMetrics = config.enableAdditionalMetrics(); - this.replicasChangeHandler = replicasChangeHandler; - this.topicNameIdMap = new HashMap<>(); + this.metricsHolder = metricsHolder; + this.cruiseControlHandler = cruiseControlHandler; + this.topicRefs = new HashMap<>(); + this.topicIds = new HashMap<>(); + + if (config.alterableTopicConfig() == null + || config.alterableTopicConfig().equalsIgnoreCase("ALL") + || config.alterableTopicConfig().isEmpty()) { + this.alterableConfigs = null; + } else if (config.alterableTopicConfig().equalsIgnoreCase("NONE")) { + this.alterableConfigs = Set.of(); + } else { + this.alterableConfigs = Arrays.stream(config.alterableTopicConfig().replaceAll("\\s", "") + .split(",")).collect(Collectors.toUnmodifiableSet()); + } } /** - * Retrieves the specified configuration value for a Kafka cluster. - * - * This method queries the Kafka cluster to obtain the configuration value associated with the given name. - * It iterates through all nodes (brokers) in the cluster, requesting their configurations, and returns the - * value of the configuration if found. The search stops at the first occurrence of the configuration name - * across all nodes, assuming uniform configuration across the cluster. + * Handles delete events. * - * @param admin The {@link Admin} client used to interact with the Kafka cluster. - * @param name The name of the configuration to retrieve. - * @return An {@link Optional} {@link String} containing the value of the requested configuration if found, or an empty Optional if not. - * @throws RuntimeException if there is an error during the operation. This exception wraps the underlying exception's message. + * @param reconcilableTopics The topics to reconcile. + * @throws InterruptedException If the thread was interrupted while blocking. */ - private static Optional getClusterConfig(Admin admin, String name) { + void onDelete(List reconcilableTopics) throws InterruptedException { try { - DescribeClusterResult describeClusterResult = admin.describeCluster(); - var nodes = describeClusterResult.nodes().get(); - Map>> futures = new HashMap<>(); - for (var node : nodes) { - ConfigResource nodeResource = new ConfigResource(ConfigResource.Type.BROKER, node.idString()); - futures.put(nodeResource, admin.describeConfigs(Set.of(nodeResource)).all()); - } - for (var entry : futures.entrySet()) { - var nodeConfig = entry.getValue().get().get(entry.getKey()); - var configEntry = nodeConfig.get(name); - return Optional.of(configEntry.value()); + deleteInternal(reconcilableTopics, true); + } catch (UncheckedInterruptedException e) { + throw e.getCause(); + } catch (KubernetesClientException e) { + if (e.getCause() instanceof InterruptedIOException) { + throw new InterruptedException(); + } else { + throw e; } - return Optional.empty(); - } catch (Throwable e) { - throw new RuntimeException("Failed to get cluster configuration: " + e.getMessage()); - } - } - - private static boolean isForDeletion(KafkaTopic kt) { - if (kt.getMetadata().getDeletionTimestamp() != null) { - var deletionTimestamp = StatusUtils.isoUtcDatetime(kt.getMetadata().getDeletionTimestamp()); - var now = Instant.now(); - return !deletionTimestamp.isAfter(now); - } else { - return false; } } - static String resourceVersion(KafkaTopic kt) { - return kt == null || kt.getMetadata() == null ? "null" : kt.getMetadata().getResourceVersion(); - } - - private List addOrRemoveFinalizer(boolean useFinalizer, List reconcilableTopics) { - List collect = reconcilableTopics.stream() - .map(reconcilableTopic -> - new ReconcilableTopic(reconcilableTopic.reconciliation(), useFinalizer - ? addFinalizer(reconcilableTopic) : removeFinalizer(reconcilableTopic), reconcilableTopic.topicName())) - .collect(Collectors.toList()); - LOGGER.traceOp("{} {} topics", useFinalizer ? "Added finalizers to" : "Removed finalizers from", reconcilableTopics.size()); - return collect; - } - - private KafkaTopic addFinalizer(ReconcilableTopic reconcilableTopic) { - if (!reconcilableTopic.kt().getMetadata().getFinalizers().contains(FINALIZER)) { - LOGGER.debugCr(reconcilableTopic.reconciliation(), "Adding finalizer {}", FINALIZER); - var timerSample = TopicOperatorUtil.startExternalRequestTimer(metrics, enableAdditionalMetrics); - KafkaTopic edit = Crds.topicOperation(kubeClient).resource(reconcilableTopic.kt()).edit(old -> - new KafkaTopicBuilder(old).editOrNewMetadata().addToFinalizers(FINALIZER).endMetadata().build()); - TopicOperatorUtil.stopExternalRequestTimer(timerSample, metrics::addFinalizerTimer, enableAdditionalMetrics, namespace); - LOGGER.traceCr(reconcilableTopic.reconciliation(), "Added finalizer {}, resourceVersion now {}", FINALIZER, resourceVersion(edit)); - return edit; - } - return reconcilableTopic.kt(); - } + private void deleteInternal(List reconcilableTopics, boolean onDeletePath) { + metricsHolder.reconciliationsCounter(config.namespace()).increment(reconcilableTopics.size()); + var managedToDelete = reconcilableTopics.stream().filter(reconcilableTopic -> { + if (TopicOperatorUtil.isManaged(reconcilableTopic.kt())) { + var e = validate(reconcilableTopic); + if (e.isRightEqual(true)) { + // adminDelete, removeFinalizer, forgetTopic, updateStatus + return true; + } else if (e.isRightEqual(false)) { + // do nothing + return false; + } else { + updateStatusForException(reconcilableTopic, e.left()); + return false; + } + } else { + deleteUnmanagedTopic(reconcilableTopic); + return false; + } + }); - private KafkaTopic removeFinalizer(ReconcilableTopic reconcilableTopic) { - if (reconcilableTopic.kt().getMetadata().getFinalizers().contains(FINALIZER)) { - LOGGER.debugCr(reconcilableTopic.reconciliation(), "Removing finalizer {}", FINALIZER); - var timerSample = TopicOperatorUtil.startExternalRequestTimer(metrics, enableAdditionalMetrics); - var result = Crds.topicOperation(kubeClient).resource(reconcilableTopic.kt()).edit(old -> - new KafkaTopicBuilder(old).editOrNewMetadata().removeFromFinalizers(FINALIZER).endMetadata().build()); - TopicOperatorUtil.stopExternalRequestTimer(timerSample, metrics::removeFinalizerTimer, enableAdditionalMetrics, namespace); - LOGGER.traceCr(reconcilableTopic.reconciliation(), "Removed finalizer {}, resourceVersion now {}", FINALIZER, resourceVersion(result)); - return result; - } else { - return reconcilableTopic.kt(); - } + deleteManagedTopics(reconcilableTopics, onDeletePath, managedToDelete); } private Either validate(ReconcilableTopic reconcilableTopic) { var doReconcile = Either.ofRight(true); doReconcile = doReconcile.flatMapRight((Boolean x) -> x ? validateUnchangedTopicName(reconcilableTopic) : Either.ofRight(false)); - doReconcile = doReconcile.mapRight((Boolean x) -> x ? rememberTopic(reconcilableTopic) : false); + doReconcile = doReconcile.mapRight((Boolean x) -> x && rememberReconcilableTopic(reconcilableTopic)); return doReconcile; } - private boolean rememberTopic(ReconcilableTopic reconcilableTopic) { - String tn = reconcilableTopic.topicName(); - var existing = topics.computeIfAbsent(tn, k -> new ArrayList<>(1)); - KubeRef thisRef = new KubeRef(reconcilableTopic.kt()); - if (!existing.contains(thisRef)) { - existing.add(thisRef); - } - return true; + private void deleteUnmanagedTopic(ReconcilableTopic reconcilableTopic) { + var timerSample = TopicOperatorUtil.startReconciliationTimer(metricsHolder); + kubernetesHandler.removeFinalizer(reconcilableTopic); + forgetReconcilableTopic(reconcilableTopic); + TopicOperatorUtil.stopReconciliationTimer(metricsHolder, timerSample, config.namespace()); + metricsHolder.successfulReconciliationsCounter(config.namespace()).increment(); } - private Either validateSingleManagingResource(ReconcilableTopic reconcilableTopic) { - String tn = reconcilableTopic.topicName(); - var existing = topics.get(tn); - KubeRef thisRef = new KubeRef(reconcilableTopic.kt()); - if (existing.size() != 1) { - var byCreationTime = existing.stream().sorted(Comparator.comparing(KubeRef::creationTime)).toList(); - - var oldest = byCreationTime.get(0); - var nextOldest = byCreationTime.size() >= 2 ? byCreationTime.get(1) : null; - TopicOperatorException e = new TopicOperatorException.ResourceConflict("Managed by " + oldest); - if (nextOldest == null) { - // This is only resource for that topic => it is the unique oldest - return Either.ofRight(true); - } else if (thisRef.equals(oldest) && nextOldest.creationTime() != oldest.creationTime()) { - // This resource is the unique oldest, so it's OK. - // The others will eventually get reconciled and put into ResourceConflict - return Either.ofRight(true); - } else if (thisRef.equals(oldest) - && reconcilableTopic.kt().getStatus() != null - && reconcilableTopic.kt().getStatus().getConditions() != null - && reconcilableTopic.kt().getStatus().getConditions().stream().anyMatch(c -> "Ready".equals(c.getType()) && "True".equals(c.getStatus()))) { - return Either.ofRight(true); + private void forgetReconcilableTopic(ReconcilableTopic reconcilableTopic) { + topicRefs.compute(reconcilableTopic.topicName(), (k, v) -> { + if (v != null) { + v.remove(new KubeRef(reconcilableTopic.kt())); + if (v.isEmpty()) { + return null; + } else { + return v; + } } else { - // Return an error putting this resource into ResourceConflict - return Either.ofLeft(e); + return null; } - - } - return Either.ofRight(true); + }); } - /* test */ static boolean matchesSelector(Map selector, Map resourceLabels) { - if (!selector.isEmpty()) { - for (var selectorEntry : selector.entrySet()) { - String resourceValue = resourceLabels.get(selectorEntry.getKey()); - if (resourceValue == null - || !resourceValue.equals(selectorEntry.getValue())) { - return false; - } - } - } - return resourceLabels.keySet().containsAll(selector.keySet()); - } + private void deleteManagedTopics(List reconcilableTopics, boolean onDeletePath, Stream managedToDelete) { + var timerSamples = reconcilableTopics.stream().collect( + Collectors.toMap(UnaryOperator.identity(), rt -> TopicOperatorUtil.startReconciliationTimer(metricsHolder))); - private static Either validateUnchangedTopicName(ReconcilableTopic reconcilableTopic) { - if (reconcilableTopic.kt().getStatus() != null - && reconcilableTopic.kt().getStatus().getTopicName() != null - && !TopicOperatorUtil.topicName(reconcilableTopic.kt()).equals(reconcilableTopic.kt().getStatus().getTopicName())) { - return Either.ofLeft(new TopicOperatorException.NotSupported("Changing spec.topicName is not supported" - )); - } - return Either.ofRight(true); - } + var topicNames = managedToDelete.map(ReconcilableTopic::topicName).collect(Collectors.toSet()); + var deleteResult = kafkaHandler.deleteTopics(reconcilableTopics, topicNames); - private PartitionedByError createTopics(List kts) { - Map newTopicsErrors = new HashMap<>(); - var newTopics = kts.stream().map(reconcilableTopic -> { + // remove the finalizer and forget the topic + deleteResult.ok().forEach(pair -> { try { - return buildNewTopic(reconcilableTopic.kt(), reconcilableTopic.topicName()); - } catch (InvalidResourceException e) { - newTopicsErrors.put(reconcilableTopic, new TopicOperatorException.InternalError(e)); - return null; - } - }).filter(Objects::nonNull).collect(Collectors.toSet()); - - LOGGER.debugOp("Admin.createTopics({})", newTopics); - var timerSample = TopicOperatorUtil.startExternalRequestTimer(metrics, enableAdditionalMetrics); - CreateTopicsResult ctr = admin.createTopics(newTopics); - ctr.all().whenComplete((i, e) -> { - TopicOperatorUtil.stopExternalRequestTimer(timerSample, metrics::createTopicsTimer, enableAdditionalMetrics, namespace); - if (e != null) { - LOGGER.traceOp("Admin.createTopics({}) failed with {}", newTopics, String.valueOf(e)); - } else { - LOGGER.traceOp("Admin.createTopics({}) completed", newTopics); + kubernetesHandler.removeFinalizer(pair.getKey()); + } catch (KubernetesClientException e) { + // If this method be being called because the resource was deleted + // then we expect the PATCH will error with Not Found + if (!(onDeletePath && e.getCode() == 404)) { // 404 = Not Found + throw e; + } } + forgetReconcilableTopic(pair.getKey()); + metricsHolder.successfulReconciliationsCounter(config.namespace()).increment(); }); - Map> values = ctr.values(); - return partitionedByError(kts.stream().map(reconcilableTopic -> { - if (newTopicsErrors.containsKey(reconcilableTopic)) { - return new Pair<>(reconcilableTopic, Either.ofLeft(newTopicsErrors.get(reconcilableTopic))); - } - try { - values.get(reconcilableTopic.topicName()).get(); - topicNameIdMap.put(reconcilableTopic.topicName(), ctr.topicId(reconcilableTopic.topicName()).get().toString()); - return new Pair<>(reconcilableTopic, Either.ofRight((null))); - } catch (ExecutionException e) { - if (e.getCause() != null && e.getCause() instanceof TopicExistsException) { - // we treat this as a success, the next reconciliation checks the configuration - return new Pair<>(reconcilableTopic, Either.ofRight((null))); + + // join that to fail + deleteResult.errors().forEach(entry -> { + if (!config.useFinalizer() && onDeletePath) { + // When not using finalizers and a topic is deleted there will be no KafkaTopic to update + // the status of, so we have to log errors here. + if (entry.getValue().getCause() instanceof TopicDeletionDisabledException) { + LOGGER.warnCr(entry.getKey().reconciliation(), + "Unable to delete topic '{}' from Kafka because topic deletion is disabled on the Kafka controller.", + entry.getKey().topicName()); } else { - return new Pair<>(reconcilableTopic, Either.ofLeft(handleAdminException(e))); + LOGGER.warnCr(entry.getKey().reconciliation(), + "Unable to delete topic '{}' from Kafka.", + entry.getKey().topicName(), + entry.getValue()); } - } catch (InterruptedException e) { - throw new UncheckedInterruptedException(e); - } - })); - } - - private static TopicOperatorException handleAdminException(ExecutionException e) { - var cause = e.getCause(); - if (cause instanceof ApiException) { - return new TopicOperatorException.KafkaError((ApiException) cause); - } else { - return new TopicOperatorException.InternalError(cause); - } - } - - private static NewTopic buildNewTopic(KafkaTopic kt, String topicName) { - return new NewTopic(topicName, partitions(kt), replicas(kt)).configs(buildConfigsMap(kt)); - } - - private static Map buildConfigsMap(KafkaTopic kt) { - Map configs = new HashMap<>(); - if (hasConfig(kt)) { - for (var entry : kt.getSpec().getConfig().entrySet()) { - configs.put(entry.getKey(), configValueAsString(entry.getKey(), entry.getValue())); + metricsHolder.failedReconciliationsCounter(config.namespace()).increment(); + } else { + updateStatusForException(entry.getKey(), entry.getValue()); } - } - return configs; - } - - private static String configValueAsString(String key, Object value) { - String valueStr; - if (value == null) { - valueStr = null; - } else if (value instanceof String - || value instanceof Boolean) { - valueStr = value.toString(); - } else if (value instanceof Number) { - valueStr = value.toString(); - } else if (value instanceof List) { - valueStr = ((List) value).stream() - .map(v -> BatchingTopicController.configValueAsString(key, v)) - .collect(Collectors.joining(",")); - } else { - throw new InvalidResourceException("Invalid value for topic config '" + key + "': " + value); - } - return valueStr; + }); + timerSamples.keySet().forEach(rt -> TopicOperatorUtil.stopReconciliationTimer(metricsHolder, timerSamples.get(rt), config.namespace())); } /** - * @param topics The topics to reconcile - * @throws InterruptedException If the thread was interrupted while blocking + * Handles upsert events. + * + * In order to make it easier to reason about the reconciliation logic, all internal operations should be free + * of side effects, such as mutating the {@link KafkaTopic} resources. The {@link Results} class is used to store + * intermediate results and status updates. The {@link KafkaTopic} resources are only updated at the end. + * + * @param reconcilableTopics The topics to reconcile. + * @throws InterruptedException If the thread was interrupted while blocking. */ - void onUpdate(List topics) throws InterruptedException { + void onUpdate(List reconcilableTopics) throws InterruptedException { try { - updateInternal(topics); + updateInternal(reconcilableTopics); } catch (UncheckedInterruptedException e) { throw e.getCause(); } catch (KubernetesClientException e) { @@ -400,16 +253,16 @@ void onUpdate(List topics) throws InterruptedException { } } - private void updateInternal(List batch) { - LOGGER.debugOp("Reconciling batch {}", batch); - Map> results = new HashMap<>(); - topicNameIdMap.clear(); + private void updateInternal(List reconcilableTopics) { + LOGGER.debugOp("Reconciling batch {}", reconcilableTopics); + Results results = new Results(); + topicIds.clear(); // process deletions - var partitionedByDeletion = batch.stream().filter(reconcilableTopic -> { + var partitionedByDeletion = reconcilableTopics.stream().filter(reconcilableTopic -> { var kt = reconcilableTopic.kt(); if (!matchesSelector(selector, kt.getMetadata().getLabels())) { - forgetTopic(reconcilableTopic); + forgetReconcilableTopic(reconcilableTopic); LOGGER.debugCr(reconcilableTopic.reconciliation(), "Ignoring KafkaTopic with labels {} not selected by selector {}", kt.getMetadata().getLabels(), selector); return false; @@ -431,183 +284,68 @@ private void updateInternal(List batch) { // process remaining var remainingAfterDeletions = partitionedByDeletion.get(false); var timerSamples = remainingAfterDeletions.stream().collect( - Collectors.toMap(identity(), rt -> startReconciliationTimer(metrics))); - - var partitionedByManaged = remainingAfterDeletions.stream().collect( - Collectors.partitioningBy(reconcilableTopic -> TopicOperatorUtil.isManaged(reconcilableTopic.kt()))); - + Collectors.toMap(UnaryOperator.identity(), rt -> TopicOperatorUtil.startReconciliationTimer(metricsHolder))); + var partitionedByManaged = remainingAfterDeletions.stream() + .collect(Collectors.partitioningBy(reconcilableTopic -> TopicOperatorUtil.isManaged(reconcilableTopic.kt()))); + // process unmanaged var unmanaged = partitionedByManaged.get(false); - unmanaged.forEach(rt -> putResult(results, rt, Either.ofRight(null))); + results.addRightResults(unmanaged); - // process paused + // process managed paused var managed = partitionedByManaged.get(true); var partitionedByPaused = validateManagedTopics(managed).stream().filter(hasTopicSpec) .collect(Collectors.partitioningBy(reconcilableTopic -> TopicOperatorUtil.isPaused(reconcilableTopic.kt()))); - var paused = partitionedByPaused.get(true); - paused.forEach(rt -> putResult(results, rt, Either.ofRight(null))); - - // process managed not paused - var mayNeedUpdate = partitionedByPaused.get(false); - addOrRemoveFinalizer(useFinalizer, mayNeedUpdate); - - // figure out necessary updates - var currentStatesOrError = describeTopic(mayNeedUpdate); - createMissingTopics(results, currentStatesOrError); - List>> someAlterConfigs = configChanges(results, currentStatesOrError); - List> someCreatePartitions = partitionChanges(results, currentStatesOrError); - - // execute those updates - var alterConfigsResults = alterConfigs(someAlterConfigs); - var createPartitionsResults = createPartitions(someCreatePartitions); - var checkReplicasChangesResults = checkReplicasChanges(batch, currentStatesOrError); - - // update statuses - accumulateResults(results, alterConfigsResults, createPartitionsResults, checkReplicasChangesResults); - updateStatuses(results); - metrics.reconciliationsCounter(namespace).increment(results.size()); - timerSamples.keySet().forEach(rt -> stopReconciliationTimer(metrics, timerSamples.get(rt), namespace)); - LOGGER.traceOp("Reconciled batch of {} KafkaTopics", results.size()); - } - - /** - * Check topic replicas changes. - * - *

If Cruise Control integration is disabled, it simply returns an error for each change.

- * - *

- * If Cruise Control integration is enabled, it runs the following operations in order: - *

    - *
  1. Request new and pending changes
  2. - *
  3. Check the state of ongoing changes
  4. - *
  5. Complete pending but completed changes (*)
  6. - *
- * (*) A pending change needs to be completed by the reconciliation when it is unknown to Cruise Control - * due to a restart, but the task actually completed successfully (i.e. the new replication factor - * was applied), or when the user reverts an invalid replicas change to the previous value. - *

- * - * @param reconcilableTopics Reconcilable topics from Kube - * @param currentStatesOrError Current topic state or error from Kafka - * @return Reconcilable topics partitioned by error - */ - /* test */ PartitionedByError checkReplicasChanges(List reconcilableTopics, - PartitionedByError currentStatesOrError) { - var differentRfResults = findDifferentRf(currentStatesOrError); - Stream>> successStream; - - if (config.cruiseControlEnabled()) { - var results = new HashSet(); - var differentRfMap = differentRfResults.ok().map(Pair::getKey).collect(Collectors.toList()) - .stream().collect(Collectors.toMap(ReconcilableTopic::topicName, Function.identity())); - - var pending = topicsMatching(reconcilableTopics, this::isPendingReplicasChange); - var brandNew = differentRfMap.values().stream() - .filter(rt -> !isPendingReplicasChange(rt.kt()) && !isOngoingReplicasChange(rt.kt())) - .collect(Collectors.toList()); - pending.addAll(brandNew); - warnTooLargeMinIsr(pending); - results.addAll(replicasChangeHandler.requestPendingChanges(pending)); - - var ongoing = topicsMatching(reconcilableTopics, this::isOngoingReplicasChange); - results.addAll(replicasChangeHandler.requestOngoingChanges(ongoing)); - - var completed = pending.stream() - .filter(rt -> !differentRfMap.containsKey(rt.topicName()) && !isFailedReplicasChange(rt.kt())) - .collect(Collectors.toList()); - var reverted = pending.stream() - .filter(rt -> !differentRfMap.containsKey(rt.topicName()) && isFailedReplicasChange(rt.kt())) - .collect(Collectors.toList()); - completed.addAll(reverted); - if (!completed.isEmpty()) { - LOGGER.debugOp("Pending but completed replicas changes, Topics: {}", topicNames(completed)); - } - completed.forEach(reconcilableTopic -> { - reconcilableTopic.kt().getStatus().setReplicasChange(null); - }); - results.addAll(completed); - - successStream = results.stream().map(reconcilableTopic -> new Pair<>(reconcilableTopic, Either.ofRight(null))); - - } else { - successStream = differentRfResults.ok().map(pair -> { - var reconcilableTopic = pair.getKey(); - var specReplicas = replicas(reconcilableTopic.kt()); - var partitions = pair.getValue().partitionsWithDifferentRfThan(specReplicas); - return new Pair<>(reconcilableTopic, Either.ofLeft(new TopicOperatorException.NotSupported( - "Replication factor change not supported, but required for partitions " + partitions))); - }); - } - - Stream>> errorStream = differentRfResults.errors() - .map(pair -> new Pair<>(pair.getKey(), Either.ofLeft(pair.getValue()))); - - return partitionedByError(Stream.concat(successStream, errorStream)); - } + var managedPaused = partitionedByPaused.get(true); + results.addRightResults(managedPaused); - private PartitionedByError findDifferentRf(PartitionedByError currentStatesOrError) { - var apparentlyDifferentRf = currentStatesOrError.ok().filter(pair -> { - var reconcilableTopic = pair.getKey(); - var currentState = pair.getValue(); - return reconcilableTopic.kt().getSpec().getReplicas() != null - && currentState.uniqueReplicationFactor() != reconcilableTopic.kt().getSpec().getReplicas(); - }).toList(); + // process managed non paused + var managedNonPaused = partitionedByPaused.get(false); + results.merge(updateManagedNonPausedTopics(reconcilableTopics, managedNonPaused)); - return partitionedByError(filterByReassignmentTargetReplicas(apparentlyDifferentRf).stream()); + // update status and metrics + updateStatuses(results); + metricsHolder.reconciliationsCounter(config.namespace()).increment(results.size()); + timerSamples.keySet().forEach(rt -> TopicOperatorUtil.stopReconciliationTimer(metricsHolder, timerSamples.get(rt), config.namespace())); + LOGGER.traceOp("Reconciled batch of {} KafkaTopics", results.size()); } - /** - * Cruise Control allows scale down of the replication factor under the min.insync.replicas value, which can cause - * disruption to producers with acks=all. When this happens, the Topic Operator won't block the operation, but will - * just log a warning, because the KafkaRoller ignores topics with RF < minISR, and they don't even show up as under - * replicated in Kafka metrics. - * - * @param reconcilableTopics Reconcilable topic. - */ - private void warnTooLargeMinIsr(List reconcilableTopics) { - if (config.skipClusterConfigReview()) { - // This method is for internal configurations. So skipping. - return; - } - - Optional clusterMinIsr = getClusterConfig(admin, MIN_INSYNC_REPLICAS); - for (ReconcilableTopic reconcilableTopic : reconcilableTopics) { - var topicConfig = reconcilableTopic.kt().getSpec().getConfig(); - if (topicConfig != null) { - Integer topicMinIsr = (Integer) topicConfig.get(MIN_INSYNC_REPLICAS); - int minIsr = topicMinIsr != null ? topicMinIsr : clusterMinIsr.isPresent() ? Integer.parseInt(clusterMinIsr.get()) : 1; - int targetRf = reconcilableTopic.kt().getSpec().getReplicas(); - if (targetRf < minIsr) { - LOGGER.warnCr(reconcilableTopic.reconciliation(), - "The target replication factor ({}) is below the configured {} ({})", targetRf, MIN_INSYNC_REPLICAS, minIsr); + /* test */ static boolean matchesSelector(Map selector, Map resourceLabels) { + if (!selector.isEmpty()) { + for (var selectorEntry : selector.entrySet()) { + String resourceValue = resourceLabels.get(selectorEntry.getKey()); + if (resourceValue == null + || !resourceValue.equals(selectorEntry.getValue())) { + return false; } } } + return resourceLabels.keySet().containsAll(selector.keySet()); } - private List topicsMatching(List reconcilableTopics, Predicate status) { - return reconcilableTopics.stream().filter(rt -> status.test(rt.kt())).collect(Collectors.toList()); - } - - private boolean isPendingReplicasChange(KafkaTopic kafkaTopic) { - return TopicOperatorUtil.hasReplicasChange(kafkaTopic.getStatus()) - && kafkaTopic.getStatus().getReplicasChange().getState() == PENDING - && kafkaTopic.getStatus().getReplicasChange().getSessionId() == null; - } - - private boolean isOngoingReplicasChange(KafkaTopic kafkaTopic) { - return TopicOperatorUtil.hasReplicasChange(kafkaTopic.getStatus()) - && kafkaTopic.getStatus().getReplicasChange().getState() == ONGOING - && kafkaTopic.getStatus().getReplicasChange().getSessionId() != null; + private static boolean isForDeletion(KafkaTopic kt) { + if (kt.getMetadata().getDeletionTimestamp() != null) { + var deletionTimestamp = StatusUtils.isoUtcDatetime(kt.getMetadata().getDeletionTimestamp()); + var now = Instant.now(); + return !deletionTimestamp.isAfter(now); + } else { + return false; + } } - - private boolean isFailedReplicasChange(KafkaTopic kafkaTopic) { - return TopicOperatorUtil.hasReplicasChange(kafkaTopic.getStatus()) - && kafkaTopic.getStatus().getReplicasChange().getState() == PENDING - && kafkaTopic.getStatus().getReplicasChange().getMessage() != null; + + private List addOrRemoveFinalizer(List reconcilableTopics) { + var collect = reconcilableTopics.stream() + .map(reconcilableTopic -> new ReconcilableTopic(reconcilableTopic.reconciliation(), + config.useFinalizer() + ? kubernetesHandler.addFinalizer(reconcilableTopic) + : kubernetesHandler.removeFinalizer(reconcilableTopic), reconcilableTopic.topicName())) + .collect(Collectors.toList()); + LOGGER.traceOp("{} {} topics", config.useFinalizer() ? + "Added finalizers to" : "Removed finalizers from", reconcilableTopics.size()); + return collect; } - - private Predicate hasTopicSpec = reconcilableTopic -> { + + private final Predicate hasTopicSpec = reconcilableTopic -> { var hasSpec = reconcilableTopic.kt().getSpec() != null; if (!hasSpec) { LOGGER.warnCr(reconcilableTopic.reconciliation(), "Topic has no spec."); @@ -616,7 +354,7 @@ private boolean isFailedReplicasChange(KafkaTopic kafkaTopic) { }; private List validateManagedTopics(List reconcilableTopics) { - var mayNeedUpdate = reconcilableTopics.stream().filter(reconcilableTopic -> { + return reconcilableTopics.stream().filter(reconcilableTopic -> { var e = validate(reconcilableTopic); if (e.isRightEqual(false)) { // Do nothing @@ -639,432 +377,318 @@ private List validateManagedTopics(List re return false; } }).toList(); - return mayNeedUpdate; } - private static void putResult(Map> results, ReconcilableTopic key, Either result) { - results.compute(key, (k, v) -> { - if (v == null) { - return result; - } else if (v.isRight()) { - return result; - } else { - return v; - } - }); - } + private Either validateSingleManagingResource(ReconcilableTopic reconcilableTopic) { + var topicName = reconcilableTopic.topicName(); + var existing = topicRefs.get(topicName); + var thisRef = new KubeRef(reconcilableTopic.kt()); + if (existing.size() != 1) { + var byCreationTime = existing.stream().sorted(Comparator.comparing(KubeRef::creationTime)).toList(); - private void createMissingTopics(Map> results, PartitionedByError currentStatesOrError) { - var partitionedByUnknownTopic = currentStatesOrError.errors().collect(Collectors.partitioningBy(pair -> { - var ex = pair.getValue(); - return ex instanceof TopicOperatorException.KafkaError - && ex.getCause() instanceof UnknownTopicOrPartitionException; - })); - partitionedByUnknownTopic.get(false).forEach(pair -> putResult(results, pair.getKey(), Either.ofLeft(pair.getValue()))); + var oldest = byCreationTime.get(0); + var nextOldest = byCreationTime.size() >= 2 ? byCreationTime.get(1) : null; + TopicOperatorException e = new TopicOperatorException.ResourceConflict("Managed by " + oldest); + if (nextOldest == null) { + // This is only resource for that topic => it is the unique oldest + return Either.ofRight(true); + } else if (thisRef.equals(oldest) && nextOldest.creationTime() != oldest.creationTime()) { + // This resource is the unique oldest, so it's OK. + // The others will eventually get reconciled and put into ResourceConflict + return Either.ofRight(true); + } else if (thisRef.equals(oldest) + && reconcilableTopic.kt().getStatus() != null + && reconcilableTopic.kt().getStatus().getConditions() != null + && reconcilableTopic.kt().getStatus().getConditions().stream() + .anyMatch(c -> "Ready".equals(c.getType()) && "True".equals(c.getStatus()))) { + return Either.ofRight(true); + } else { + // Return an error putting this resource into ResourceConflict + return Either.ofLeft(e); + } - if (!partitionedByUnknownTopic.get(true).isEmpty()) { - var createResults = createTopics(partitionedByUnknownTopic.get(true).stream().map(Pair::getKey).toList()); - createResults.ok().forEach(pair -> putResult(results, pair.getKey(), Either.ofRight(null))); - createResults.errors().forEach(pair -> putResult(results, pair.getKey(), Either.ofLeft(pair.getValue()))); } + return Either.ofRight(true); } - private void updateStatuses(Map> results) { - // Update statues with the overall results. - results.entrySet().stream().forEach(entry -> { - var reconcilableTopic = entry.getKey(); - var either = entry.getValue(); - if (either.isRight()) { - updateStatusForSuccess(reconcilableTopic); - } else { - updateStatusForException(reconcilableTopic, either.left()); + private Results updateManagedNonPausedTopics(List inputBatch, List managedNonPaused) { + var results = new Results(); + + var addedFinalizer = addOrRemoveFinalizer(managedNonPaused); + var describedTopics = kafkaHandler.describeTopics(addedFinalizer); + describedTopics.ok().forEach(pair -> { + if (pair.getValue() != null && pair.getValue().topicId() != null) { + topicIds.put(pair.getKey().topicName(), pair.getValue().topicId()); } }); - LOGGER.traceOp("Updated status of {} KafkaTopics", results.size()); - } - - private void accumulateResults(Map> results, - PartitionedByError alterConfigsResults, - PartitionedByError createPartitionsResults, - PartitionedByError replicasChangeResults) { - // add the successes to the results - alterConfigsResults.ok().forEach(pair -> putResult(results, pair.getKey(), Either.ofRight(null))); - createPartitionsResults.ok().forEach(pair -> putResult(results, pair.getKey(), Either.ofRight(null))); - replicasChangeResults.ok().forEach(pair -> putResult(results, pair.getKey(), Either.ofRight(null))); - - // add to errors (potentially overwriting some successes, e.g. if configs succeeded but partitions failed) - alterConfigsResults.errors().forEach(pair -> putResult(results, pair.getKey(), Either.ofLeft(pair.getValue()))); - createPartitionsResults.errors().forEach(pair -> putResult(results, pair.getKey(), Either.ofLeft(pair.getValue()))); - replicasChangeResults.errors().forEach(pair -> putResult(results, pair.getKey(), Either.ofLeft(pair.getValue()))); - } - - private List>> configChanges(Map> results, PartitionedByError currentStatesOrError) { - // Determine config changes - Map>>> alterConfigs = currentStatesOrError.ok().map(pair -> { - var reconcilableTopic = pair.getKey(); - var currentState = pair.getValue(); - // determine config changes - return new Pair<>(reconcilableTopic, buildAlterConfigOps(reconcilableTopic.reconciliation(), reconcilableTopic.kt(), currentState.configs())); - }).collect(Collectors.partitioningBy(pair -> pair.getValue().isEmpty())); - - // add topics which don't require configs changes to the results (may be overwritten later) - alterConfigs.get(true).forEach(pair -> putResult(results, pair.getKey(), Either.ofRight(null))); - var someAlterConfigs = alterConfigs.get(false); - return someAlterConfigs; - } - - private static List> partitionChanges(Map> results, PartitionedByError currentStatesOrError) { - // Determine partition changes - PartitionedByError newPartitionsOrError = partitionedByError(currentStatesOrError.ok().map(pair -> { - var reconcilableTopic = pair.getKey(); - var currentState = pair.getValue(); - // determine config changes - return new Pair<>(reconcilableTopic, buildNewPartitions(reconcilableTopic.reconciliation(), reconcilableTopic.kt(), currentState.numPartitions())); - })); - newPartitionsOrError.errors().forEach(pair -> putResult(results, pair.getKey(), Either.ofLeft(pair.getValue()))); - - var createPartitions = newPartitionsOrError.ok().collect( - Collectors.partitioningBy(pair -> pair.getValue() == null)); - // add topics which don't require partitions changes to the results (may be overwritten later) - createPartitions.get(true).forEach(pair -> putResult(results, pair.getKey(), Either.ofRight(null))); - var someCreatePartitions = createPartitions.get(false); - return someCreatePartitions; + + results.merge(createMissingTopics(describedTopics)); + + var partitionedByDifferentConfigs = partitionByHavingDifferentConfigs(describedTopics.ok()); + // record topics which don't require configs changes (may be overwritten later) + results.addRightResults(partitionedByDifferentConfigs.get(false).stream()); + // filter out topics whose configs can't be changed (e.g. throttling managed by CC, or ones prohibited by this operator's config) + var filteredChanges = filterOutNonAlterableConfigChanges(partitionedByDifferentConfigs.get(true), inputBatch); + results.merge(filteredChanges); + + var newPartitionsOrError = partitionByRequiresNewPartitions(describedTopics.ok()); + results.addLeftResults(newPartitionsOrError.errors()); + var partitionedByRequiredNewPartitions = newPartitionsOrError.ok() + .collect(Collectors.partitioningBy(pair -> pair.getValue() == null)); + // record topics which don't require partitions changes to the results (may be overwritten later) + results.addRightResults(partitionedByRequiredNewPartitions.get(true).stream()); + var partitionsToCreate = partitionedByRequiredNewPartitions.get(false); + + results.addResults(kafkaHandler.alterConfigs(filteredChanges.getConfigChanges())); + results.addResults(kafkaHandler.createPartitions(partitionsToCreate)); + results.merge(checkReplicasChanges(describedTopics.ok(), inputBatch)); + + return results; } - private List>> filterByReassignmentTargetReplicas( - List> apparentlyDifferentRfTopics) { - if (apparentlyDifferentRfTopics.isEmpty()) { - return List.of(); - } - Set apparentDifferentRfPartitions = apparentlyDifferentRfTopics.stream() - .flatMap(pair -> pair.getValue().description().partitions().stream() - .filter(pi -> { - // includes only the partitions of the topic with a RF that mismatches the desired RF - var desiredRf = pair.getKey().kt().getSpec().getReplicas(); - return desiredRf != pi.replicas().size(); - }) - .map(pi -> new TopicPartition(pair.getKey().topicName(), pi.partition()))).collect(Collectors.toSet()); - - Map reassignments; - LOGGER.traceOp("Admin.listPartitionReassignments({})", apparentDifferentRfPartitions); - var timerSample = TopicOperatorUtil.startExternalRequestTimer(metrics, enableAdditionalMetrics); - try { - reassignments = admin.listPartitionReassignments(apparentDifferentRfPartitions).reassignments().get(); - TopicOperatorUtil.stopExternalRequestTimer(timerSample, metrics::listReassignmentsTimer, enableAdditionalMetrics, namespace); - LOGGER.traceOp("Admin.listPartitionReassignments({}) completed", apparentDifferentRfPartitions); - } catch (ExecutionException e) { - TopicOperatorUtil.stopExternalRequestTimer(timerSample, metrics::listReassignmentsTimer, enableAdditionalMetrics, namespace); - LOGGER.traceOp("Admin.listPartitionReassignments({}) failed with {}", apparentDifferentRfPartitions, e); - return apparentlyDifferentRfTopics.stream().map(pair -> - new Pair<>(pair.getKey(), Either.ofLeft(handleAdminException(e)))).toList(); - } catch (InterruptedException e) { - throw new UncheckedInterruptedException(e); - } + private Results createMissingTopics(PartitionedByError currentStatesOrError) { + var results = new Results(); - var partitionToTargetRf = reassignments.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, entry -> { - var tp = entry.getKey(); - var partitionReassignment = entry.getValue(); - // See https://cwiki.apache.org/confluence/display/KAFKA/KIP-455%3A+Create+an+Administrative+API+for+Replica+Reassignment#KIP455:CreateanAdministrativeAPIforReplicaReassignment-Algorithm - // for a full description of the algorithm - // but in essence replicas() will include addingReplicas() from the beginning - // so the target rf will be the replicas minus the removing - var target = new HashSet<>(partitionReassignment.replicas()); - target.removeAll(partitionReassignment.removingReplicas()); - return target.size(); + // partition the topics with errors by whether they're due to UnknownTopicOrPartitionException + var partitionedByUnknownTopic = currentStatesOrError.errors().collect(Collectors.partitioningBy(pair -> { + var ex = pair.getValue(); + return ex instanceof TopicOperatorException.KafkaError + && ex.getCause() instanceof UnknownTopicOrPartitionException; })); - return apparentlyDifferentRfTopics.stream().filter(pair -> { - boolean b = pair.getValue().description().partitions().stream().anyMatch(pi -> { - TopicPartition tp = new TopicPartition(pair.getKey().topicName(), pi.partition()); - Integer targetRf = partitionToTargetRf.get(tp); - Integer desiredRf = pair.getKey().kt().getSpec().getReplicas(); - return !Objects.equals(targetRf, desiredRf); - }); - return b; - }).map(pair -> new Pair<>(pair.getKey(), Either.ofRight(pair.getValue()))).toList(); - } + // record the errors for those which are NOT due to UnknownTopicOrPartitionException + results.addLeftResults(partitionedByUnknownTopic.get(false).stream()); - private PartitionedByError alterConfigs(List>> someAlterConfigs) { - if (someAlterConfigs.isEmpty()) { - return new PartitionedByError<>(List.of(), List.of()); - } - Map> alteredConfigs = someAlterConfigs.stream().collect(Collectors.toMap(entry -> topicConfigResource(entry.getKey().topicName()), Pair::getValue)); - LOGGER.debugOp("Admin.incrementalAlterConfigs({})", alteredConfigs); - var timerSample = TopicOperatorUtil.startExternalRequestTimer(metrics, enableAdditionalMetrics); - AlterConfigsResult acr = admin.incrementalAlterConfigs(alteredConfigs); - TopicOperatorUtil.stopExternalRequestTimer(timerSample, metrics::alterConfigsTimer, enableAdditionalMetrics, namespace); - acr.all().whenComplete((i, e) -> { - TopicOperatorUtil.stopExternalRequestTimer(timerSample, metrics::alterConfigsTimer, enableAdditionalMetrics, namespace); - if (e != null) { - LOGGER.traceOp("Admin.incrementalAlterConfigs({}) failed with {}", alteredConfigs, String.valueOf(e)); - } else { - LOGGER.traceOp("Admin.incrementalAlterConfigs({}) completed", alteredConfigs); - } - }); - var alterConfigsResult = acr.values(); - Stream>> entryStream = someAlterConfigs.stream().map(entry -> { - try { - return new Pair<>(entry.getKey(), Either.ofRight(alterConfigsResult.get(topicConfigResource(entry.getKey().topicName())).get())); - } catch (ExecutionException e) { - return new Pair<>(entry.getKey(), Either.ofLeft(handleAdminException(e))); - } catch (InterruptedException e) { - throw new UncheckedInterruptedException(e); - } - }); - return partitionedByError(entryStream); - } + // create topics in Kafka for those which ARE due to UnknownTopicOrPartitionException... + var unknownTopics = partitionedByUnknownTopic.get(true).stream().map(Pair::getKey).toList(); + // ... and record those results + var createTopicsReply = kafkaHandler.createTopics(unknownTopics); - private PartitionedByError createPartitions(List> someCreatePartitions) { - if (someCreatePartitions.isEmpty()) { - return new PartitionedByError<>(List.of(), List.of()); - } - Map newPartitions = someCreatePartitions.stream().collect(Collectors.toMap(pair -> pair.getKey().topicName(), Pair::getValue)); - LOGGER.debugOp("Admin.createPartitions({})", newPartitions); - var timerSample = TopicOperatorUtil.startExternalRequestTimer(metrics, enableAdditionalMetrics); - CreatePartitionsResult cpr = admin.createPartitions(newPartitions); - cpr.all().whenComplete((i, e) -> { - TopicOperatorUtil.stopExternalRequestTimer(timerSample, metrics::createPartitionsTimer, enableAdditionalMetrics, namespace); - if (e != null) { - LOGGER.traceOp("Admin.createPartitions({}) failed with {}", newPartitions, String.valueOf(e)); - } else { - LOGGER.traceOp("Admin.createPartitions({}) completed", newPartitions); - } - }); - var createPartitionsResult = cpr.values(); - var entryStream = someCreatePartitions.stream().map(entry -> { - try { - createPartitionsResult.get(entry.getKey().topicName()).get(); - return new Pair<>(entry.getKey(), Either.ofRight(null)); - } catch (ExecutionException e) { - return new Pair<>(entry.getKey(), Either.ofLeft(handleAdminException(e))); - } catch (InterruptedException e) { - throw new UncheckedInterruptedException(e); + results.addResults(createTopicsReply); + createTopicsReply.ok().forEach(pair -> { + if (pair.getValue() != null && pair.getValue().topicId() != null) { + topicIds.put(pair.getKey().topicName(), pair.getValue().topicId()); } }); - return partitionedByError(entryStream); + + return results; } - private static ConfigResource topicConfigResource(String tn) { - return new ConfigResource(ConfigResource.Type.TOPIC, tn); + private Map>>> partitionByHavingDifferentConfigs( + Stream> currentStates + ) { + // determine config changes + return currentStates.map(pair -> { + var reconcilableTopic = pair.getKey(); + var currentState = pair.getValue(); + return new Pair<>(reconcilableTopic, buildAlterConfigOps(reconcilableTopic, currentState.configs())); + }).collect(Collectors.partitioningBy(pair -> !pair.getValue().isEmpty())); } - private PartitionedByError describeTopic(List batch) { - if (batch.isEmpty()) { - return new PartitionedByError<>(List.of(), List.of()); - } - Set configResources = batch.stream() - .map(reconcilableTopic -> topicConfigResource(reconcilableTopic.topicName())) - .collect(Collectors.toSet()); - Set tns = batch.stream().map(ReconcilableTopic::topicName).collect(Collectors.toSet()); - - DescribeTopicsResult describeTopicsResult; - { - LOGGER.debugOp("Admin.describeTopics({})", tns); - var timerSample = TopicOperatorUtil.startExternalRequestTimer(metrics, enableAdditionalMetrics); - describeTopicsResult = admin.describeTopics(tns); - describeTopicsResult.allTopicNames().whenComplete((i, e) -> { - TopicOperatorUtil.stopExternalRequestTimer(timerSample, metrics::describeTopicsTimer, enableAdditionalMetrics, namespace); - if (e != null) { - LOGGER.traceOp("Admin.describeTopics({}) failed with {}", tns, String.valueOf(e)); - } else { - LOGGER.traceOp("Admin.describeTopics({}) completed", tns); + private Collection buildAlterConfigOps(ReconcilableTopic reconcilableTopic, Config configs) { + Set alterConfigOps = new HashSet<>(); + if (TopicOperatorUtil.hasConfig(reconcilableTopic.kt())) { + for (var specConfigEntry : reconcilableTopic.kt().getSpec().getConfig().entrySet()) { + String key = specConfigEntry.getKey(); + var specValueStr = TopicOperatorUtil.configValueAsString(specConfigEntry.getKey(), specConfigEntry.getValue()); + var kafkaConfigEntry = configs.get(key); + if (kafkaConfigEntry == null + || !Objects.equals(specValueStr, kafkaConfigEntry.value())) { + alterConfigOps.add(new AlterConfigOp( + new ConfigEntry(key, specValueStr), + AlterConfigOp.OpType.SET)); } - }); + } } - DescribeConfigsResult describeConfigsResult; - { - LOGGER.debugOp("Admin.describeConfigs({})", configResources); - var timerSample = TopicOperatorUtil.startExternalRequestTimer(metrics, enableAdditionalMetrics); - describeConfigsResult = admin.describeConfigs(configResources); - describeConfigsResult.all().whenComplete((i, e) -> { - TopicOperatorUtil.stopExternalRequestTimer(timerSample, metrics::describeConfigsTimer, enableAdditionalMetrics, namespace); - if (e != null) { - LOGGER.traceOp("Admin.describeConfigs({}) failed with {}", configResources, String.valueOf(e)); - } else { - LOGGER.traceOp("Admin.describeConfigs({}) completed", configResources); - } - }); + HashSet keysToRemove = configs.entries().stream() + .filter(configEntry -> configEntry.source() == ConfigEntry.ConfigSource.DYNAMIC_TOPIC_CONFIG) + .map(ConfigEntry::name).collect(Collectors.toCollection(HashSet::new)); + if (TopicOperatorUtil.hasConfig(reconcilableTopic.kt())) { + keysToRemove.removeAll(reconcilableTopic.kt().getSpec().getConfig().keySet()); } - - var cs1 = describeTopicsResult.topicNameValues(); - var cs2 = describeConfigsResult.values(); - return partitionedByError(batch.stream().map(reconcilableTopic -> { - Config configs = null; - TopicDescription description = null; - ExecutionException exception = null; - try { - description = cs1.get(reconcilableTopic.topicName()).get(); - topicNameIdMap.put(reconcilableTopic.topicName(), description.topicId().toString()); - } catch (ExecutionException e) { - exception = e; - } catch (InterruptedException e) { - throw new UncheckedInterruptedException(e); - } - - try { - configs = cs2.get(topicConfigResource(reconcilableTopic.topicName())).get(); - } catch (ExecutionException e) { - exception = e; - } catch (InterruptedException e) { - throw new UncheckedInterruptedException(e); - } - if (exception != null) { - return new Pair<>(reconcilableTopic, Either.ofLeft(handleAdminException(exception))); - } else { - return new Pair<>(reconcilableTopic, Either.ofRight(new TopicState(description, configs))); - } - })); - } - - void onDelete(List batch) throws InterruptedException { - try { - deleteInternal(batch, true); - } catch (UncheckedInterruptedException e) { - throw e.getCause(); - } catch (KubernetesClientException e) { - if (e.getCause() instanceof InterruptedIOException) { - throw new InterruptedException(); - } else { - throw e; - } + for (var key : keysToRemove) { + alterConfigOps.add(new AlterConfigOp( + new ConfigEntry(key, null), + AlterConfigOp.OpType.DELETE)); } + return alterConfigOps; } - private void deleteInternal(List batch, boolean onDeletePath) { - metrics.reconciliationsCounter(namespace).increment(batch.size()); - var managedToDelete = batch.stream().filter(reconcilableTopic -> { - if (TopicOperatorUtil.isManaged(reconcilableTopic.kt())) { - var e = validate(reconcilableTopic); - if (e.isRightEqual(true)) { - // adminDelete, removeFinalizer, forgetTopic, updateStatus - return true; - } else if (e.isRightEqual(false)) { - // do nothing - return false; - } else { - updateStatusForException(reconcilableTopic, e.left()); - return false; - } + /** + * Filters out non alterable config changes. + * + * @param alterConfigPairs Detected alter config ops. + * @param reconcilableTopics Batch of reconcilable topics. + * @return Filtered alter config ops. + */ + private Results filterOutNonAlterableConfigChanges( + List>> alterConfigPairs, + List reconcilableTopics + ) { + var results = new Results(); + + var nonThrottlingConfigResults = filterOutThrottlingConfig(alterConfigPairs, reconcilableTopics); + results.merge(nonThrottlingConfigResults); + var alterableConfigResults = filterOutNonAlterableConfig(nonThrottlingConfigResults, reconcilableTopics); + results.merge(alterableConfigResults); + + reconcilableTopics.forEach(reconcilableTopic -> { + var configChanges = results.getConfigChanges().stream() + .filter(pair -> pair.getKey().equals(reconcilableTopic)).findFirst(); + if (configChanges != null && configChanges.isEmpty()) { + LOGGER.debugCr(reconcilableTopic.reconciliation(), "Config changes {}", configChanges); } else { - deleteUnmanagedTopic(reconcilableTopic); - return false; + LOGGER.debugCr(reconcilableTopic.reconciliation(), "No config change"); } }); - - deleteManagedTopics(batch, onDeletePath, managedToDelete); - } - private void deleteUnmanagedTopic(ReconcilableTopic reconcilableTopic) { - var timerSample = startReconciliationTimer(metrics); - removeFinalizer(reconcilableTopic); - forgetTopic(reconcilableTopic); - TopicOperatorUtil.stopReconciliationTimer(metrics, timerSample, namespace); - metrics.successfulReconciliationsCounter(namespace).increment(); + return results; } - private void deleteManagedTopics(List batch, boolean onDeletePath, Stream managedToDelete) { - var timerSamples = batch.stream().collect( - Collectors.toMap(identity(), rt -> startReconciliationTimer(metrics))); - - Set topicNames = managedToDelete.map(ReconcilableTopic::topicName).collect(Collectors.toSet()); - PartitionedByError deleteResult = deleteTopics(batch, topicNames); + /** + *

If Cruise Control integration is enabled, we ignore throttle configurations unless the user + * explicitly set them in {@code .spec.config}, but we always give a warning.

+ * + *

This avoids the issue caused by the race condition between Cruise Control that dynamically sets + * them during throttled rebalances, and the operator that reverts them on periodic reconciliations.

+ * + * @param alterConfigPairs Detected alter config ops. + * @param reconcilableTopics Batch of reconcilable topics. + * @return Filtered alter config ops. + */ + private Results filterOutThrottlingConfig( + List>> alterConfigPairs, + List reconcilableTopics + ) { + var results = new Results(); - // remove the finalizer and forget the topic - deleteResult.ok().forEach(pair -> { - try { - removeFinalizer(pair.getKey()); - } catch (KubernetesClientException e) { - // If this method be being called because the resource was deleted - // then we expect the PATCH will error with Not Found - if (!(onDeletePath && e.getCode() == 404)) { // 404 = Not Found - throw e; - } - } - forgetTopic(pair.getKey()); - metrics.successfulReconciliationsCounter(namespace).increment(); - }); + // filter + if (config.cruiseControlEnabled()) { + results.replaceConfigChanges( + alterConfigPairs.stream().map(pair -> { + var reconcilableTopic = pair.getKey(); + Collection filteredOps = pair.getValue().stream() + .filter(op -> !(THROTTLING_CONFIG.contains(op.configEntry().name()) + && !hasConfigProperty(reconcilableTopic.kt(), op.configEntry().name()))) + .toList(); + return new Pair<>(pair.getKey(), filteredOps); + }) + .filter(pair -> !pair.getValue().isEmpty()) + .toList() + ); + } else { + results.replaceConfigChanges(alterConfigPairs); + } - // join that to fail - deleteResult.errors().forEach(entry -> { - if (!this.useFinalizer - && onDeletePath) { - // When not using finalizers and a topic is deleted there will be no KafkaTopic to update - // the status of, so we have to log errors here. - if (entry.getValue().getCause() instanceof TopicDeletionDisabledException) { - LOGGER.warnCr(entry.getKey().reconciliation(), - "Unable to delete topic '{}' from Kafka because topic deletion is disabled on the Kafka controller.", - entry.getKey().topicName()); - } else { - LOGGER.warnCr(entry.getKey().reconciliation(), - "Unable to delete topic '{}' from Kafka.", - entry.getKey().topicName(), - entry.getValue()); - } - metrics.failedReconciliationsCounter(namespace).increment(); - } else { - updateStatusForException(entry.getKey(), entry.getValue()); - } - }); - timerSamples.keySet().forEach(rt -> stopReconciliationTimer(metrics, timerSamples.get(rt), namespace)); + // add warnings + reconcilableTopics.stream().forEach(reconcilableTopic -> + THROTTLING_CONFIG.stream() + .filter(prop -> hasConfigProperty(reconcilableTopic.kt(), prop)) + .forEach(prop -> results.addCondition(reconcilableTopic, + new ConditionBuilder() + .withReason(INVALID_CONFIG) + .withMessage(String.format("Property %s may conflict with throttled rebalances", prop)) + .withStatus("True") + .withType("Warning") + .withLastTransitionTime(StatusUtils.iso8601Now()) + .build()))); + + return results; + } + + private static boolean hasConfigProperty(KafkaTopic kt, String prop) { + return TopicOperatorUtil.hasConfig(kt) && kt.getSpec().getConfig().containsKey(prop); } - private PartitionedByError deleteTopics(List batch, Set topicNames) { - if (topicNames.isEmpty()) { - return new PartitionedByError<>(List.of(), List.of()); + /** + *

The {@code STRIMZI_ALTERABLE_TOPIC_CONFIG} env var can be used to specify a comma separated list (allow list) + * of Kafka topic configurations that can be altered by users through {@code .spec.config}. Keep in mind that + * when changes are applied directly in Kafka, the operator will try to revert them with a warning.

+ * + *

The default value is "ALL", which means no restrictions in changing {@code .spec.config}. + * The opposite is "NONE", which can be set to explicitly disable any change.

+ * + *

This is useful in standalone mode when you have a Kafka service that restricts + * alter operations to a subset of all the Kafka topic configurations.

+ * + * @param nonThrottlingConfigResults Detected alter config ops. + * @param reconcilableTopics Batch of reconcilable topics. + * @return Filtered alter config ops for this topic. + */ + private Results filterOutNonAlterableConfig( + Results nonThrottlingConfigResults, + List reconcilableTopics + ) { + var results = new Results(); + + // filter + if (alterableConfigs == null) { + results.replaceConfigChanges(nonThrottlingConfigResults.getConfigChanges()); + } else if (alterableConfigs.isEmpty()) { + results.replaceConfigChanges(List.of()); + } else { + results.replaceConfigChanges( + nonThrottlingConfigResults.getConfigChanges().stream().map(pair -> { + Collection filteredOps = pair.getValue().stream() + .filter(op -> { + var propName = op.configEntry().name(); + return alterableConfigs.contains(propName) || THROTTLING_CONFIG.contains(propName); + }) + .toList(); + return new Pair<>(pair.getKey(), filteredOps); + }) + .filter(pair -> !pair.getValue().isEmpty()) + .toList() + ); + } + + // add warnings + if (alterableConfigs != null) { + reconcilableTopics.stream().forEach(reconcilableTopic -> { + var specConfig = TopicOperatorUtil.hasConfig(reconcilableTopic.kt()) + ? reconcilableTopic.kt().getSpec().getConfig().keySet().stream().sorted().collect(Collectors.toList()) + : List.of(); + specConfig.forEach(prop -> { + if (!alterableConfigs.contains(prop) && !THROTTLING_CONFIG.contains(prop)) { + results.addCondition(reconcilableTopic, + new ConditionBuilder() + .withReason(INVALID_CONFIG) + .withMessage(String.format("Property %s is ignored according to alterable config", prop)) + .withStatus("True") + .withType("Warning") + .withLastTransitionTime(StatusUtils.iso8601Now()) + .build()); + } + }); + }); } - var someDeleteTopics = TopicCollection.ofTopicNames(topicNames); - LOGGER.debugOp("Admin.deleteTopics({})", someDeleteTopics.topicNames()); - - // Admin delete - var timerSample = TopicOperatorUtil.startExternalRequestTimer(metrics, enableAdditionalMetrics); - DeleteTopicsResult dtr = admin.deleteTopics(someDeleteTopics); - dtr.all().whenComplete((i, e) -> { - TopicOperatorUtil.stopExternalRequestTimer(timerSample, metrics::deleteTopicsTimer, enableAdditionalMetrics, namespace); - if (e != null) { - LOGGER.traceOp("Admin.deleteTopics({}) failed with {}", someDeleteTopics.topicNames(), String.valueOf(e)); - } else { - LOGGER.traceOp("Admin.deleteTopics({}) completed", someDeleteTopics.topicNames()); - } - }); - var futuresMap = dtr.topicNameValues(); - var deleteResult = partitionedByError(batch.stream().map(reconcilableTopic -> { - try { - futuresMap.get(reconcilableTopic.topicName()).get(); - return new Pair<>(reconcilableTopic, Either.ofRight(null)); - } catch (ExecutionException e) { - if (e.getCause() instanceof UnknownTopicOrPartitionException) { - return new Pair<>(reconcilableTopic, Either.ofRight(null)); - } else { - return new Pair<>(reconcilableTopic, Either.ofLeft(handleAdminException(e))); - } - } catch (InterruptedException e) { - throw new UncheckedInterruptedException(e); - } - })); - return deleteResult; + + return results; } - private void forgetTopic(ReconcilableTopic reconcilableTopic) { - topics.compute(reconcilableTopic.topicName(), (k, v) -> { - if (v != null) { - v.remove(new KubeRef(reconcilableTopic.kt())); - if (v.isEmpty()) { - return null; - } else { - return v; - } - } else { - return null; - } + private PartitionedByError partitionByRequiresNewPartitions( + Stream> currentStates + ) { + // determine partition changes + Stream>> partitionChanges = currentStates.map(pair -> { + var reconcilableTopic = pair.getKey(); + var currentState = pair.getValue(); + // determine config changes + return new Pair<>(reconcilableTopic, partitionChanges(reconcilableTopic.reconciliation(), + reconcilableTopic.kt(), currentState.numPartitions())); }); + return TopicOperatorUtil.partitionedByError(partitionChanges); } - private static Either buildNewPartitions(Reconciliation reconciliation, KafkaTopic kt, int currentNumPartitions) { - int requested = kt.getSpec() == null || kt.getSpec().getPartitions() == null ? BROKER_DEFAULT : kt.getSpec().getPartitions(); + private static Either partitionChanges( + Reconciliation reconciliation, KafkaTopic kafkaTopic, int currentNumPartitions + ) { + var requested = kafkaTopic.getSpec() == null || kafkaTopic.getSpec().getPartitions() == null + ? KafkaHandler.BROKER_DEFAULT : kafkaTopic.getSpec().getPartitions(); if (requested > currentNumPartitions) { LOGGER.debugCr(reconciliation, "Partition increase from {} to {}", currentNumPartitions, requested); return Either.ofRight(NewPartitions.increaseTo(requested)); - } else if (requested != BROKER_DEFAULT && requested < currentNumPartitions) { + } else if (requested != KafkaHandler.BROKER_DEFAULT && requested < currentNumPartitions) { LOGGER.debugCr(reconciliation, "Partition decrease from {} to {}", currentNumPartitions, requested); return Either.ofLeft(new TopicOperatorException.NotSupported("Decreasing partitions not supported")); } else { @@ -1073,75 +697,173 @@ private static Either buildNewPartitions( } } - private Collection buildAlterConfigOps(Reconciliation reconciliation, KafkaTopic kt, Config configs) { - Set alterConfigOps = new HashSet<>(); - if (hasConfig(kt)) { - for (var specConfigEntry : kt.getSpec().getConfig().entrySet()) { - String key = specConfigEntry.getKey(); - var specValueStr = configValueAsString(specConfigEntry.getKey(), specConfigEntry.getValue()); - var kafkaConfigEntry = configs.get(key); - if (kafkaConfigEntry == null - || !Objects.equals(specValueStr, kafkaConfigEntry.value())) { - alterConfigOps.add(new AlterConfigOp( - new ConfigEntry(key, specValueStr), - AlterConfigOp.OpType.SET)); - } - } - } - HashSet keysToRemove = configs.entries().stream() - .filter(configEntry -> configEntry.source() == ConfigEntry.ConfigSource.DYNAMIC_TOPIC_CONFIG) - .map(ConfigEntry::name).collect(Collectors.toCollection(HashSet::new)); - if (hasConfig(kt)) { - keysToRemove.removeAll(kt.getSpec().getConfig().keySet()); + private static Either validateUnchangedTopicName(ReconcilableTopic reconcilableTopic) { + if (reconcilableTopic.kt().getStatus() != null + && reconcilableTopic.kt().getStatus().getTopicName() != null + && !TopicOperatorUtil.topicName(reconcilableTopic.kt()).equals(reconcilableTopic.kt().getStatus().getTopicName())) { + return Either.ofLeft(new TopicOperatorException.NotSupported("Changing spec.topicName is not supported" + )); } - for (var key : keysToRemove) { - alterConfigOps.add(new AlterConfigOp( - new ConfigEntry(key, null), - AlterConfigOp.OpType.DELETE)); + return Either.ofRight(true); + } + + private boolean rememberReconcilableTopic(ReconcilableTopic reconcilableTopic) { + var topicName = reconcilableTopic.topicName(); + var existing = topicRefs.computeIfAbsent(topicName, k -> new ArrayList<>(1)); + var thisRef = new KubeRef(reconcilableTopic.kt()); + if (!existing.contains(thisRef)) { + existing.add(thisRef); } + return true; + } - skipNonAlterableConfigs(alterConfigOps); + /** + * Check replicas changes. + * + *

If Cruise Control integration is disabled, it returns an error for each change.

+ * + *

+ * If Cruise Control integration is enabled, it runs the following operations in order: + *

    + *
  1. Request new and pending changes
  2. + *
  3. Check the state of ongoing changes
  4. + *
  5. Complete pending but completed changes (*)
  6. + *
+ * (*) A pending change needs to be completed by the reconciliation when it is unknown to Cruise Control + * due to a restart, but the task actually completed successfully (i.e. the new replication factor + * was applied), or when the user reverts an invalid replicas change to the previous value. + *

+ * + * @param currentStates State of topics in Kafka. + * @param reconcilableTopics Batch of reconcilable topics. + * @return Check results. + */ + /* test */ Results checkReplicasChanges( + Stream> currentStates, + List reconcilableTopics + ) { + var results = new Results(); + var differentRfResults = findDifferentRf(currentStates); + results.addLeftResults(differentRfResults.errors()); + + if (config.cruiseControlEnabled()) { + var differentRfMap = differentRfResults.ok().map(Pair::getKey).toList() + .stream().collect(Collectors.toMap(ReconcilableTopic::topicName, Function.identity())); + + // process new and pending changes + var pending = topicsMatching(reconcilableTopics, kt -> hasPendingReplicasChange(kt)); + var brandNew = differentRfMap.values().stream() + .filter(rt -> !TopicOperatorUtil.hasReplicasChangeStatus(rt.kt())) + .toList(); + pending.addAll(brandNew); + warnTooLargeMinIsr(pending); + results.merge(cruiseControlHandler.requestPendingChanges(pending)); + + // process ongoing changes + var ongoing = topicsMatching(reconcilableTopics, kt -> hasOngoingReplicasChange(kt)); + results.merge(cruiseControlHandler.requestOngoingChanges(ongoing)); + + // complete zombie changes + var completed = reconcilableTopics.stream() + .filter(rt -> !differentRfMap.containsKey(rt.topicName()) + && pending.contains(rt) && !hasFailedReplicasChange(rt.kt())) + .collect(Collectors.toList()); + var reverted = reconcilableTopics.stream() + .filter(rt -> !differentRfMap.containsKey(rt.topicName()) + && pending.contains(rt) && hasFailedReplicasChange(rt.kt())) + .toList(); + completed.addAll(reverted); + if (!completed.isEmpty()) { + results.merge(cruiseControlHandler.completeZombieChanges(completed)); + } - if (alterConfigOps.isEmpty()) { - LOGGER.debugCr(reconciliation, "No config change"); } else { - LOGGER.debugCr(reconciliation, "Config changes {}", alterConfigOps); + results.addLeftResults(differentRfResults.ok().map(pair -> { + var reconcilableTopic = pair.getKey(); + var specReplicas = TopicOperatorUtil.replicas(reconcilableTopic.kt()); + var partitions = pair.getValue().partitionsWithDifferentRfThan(specReplicas); + return new Pair<>(reconcilableTopic, new TopicOperatorException.NotSupported( + String.format("Replication factor change not supported, but required for partitions %s", partitions))); + })); } - return alterConfigOps; + + return results; + } + + private PartitionedByError findDifferentRf( + Stream> currentStates + ) { + var apparentlyDifferentRf = currentStates.filter(pair -> { + var reconcilableTopic = pair.getKey(); + var currentState = pair.getValue(); + return reconcilableTopic.kt().getSpec().getReplicas() != null + && currentState.uniqueReplicationFactor() != reconcilableTopic.kt().getSpec().getReplicas(); + }).toList(); + return TopicOperatorUtil.partitionedByError(kafkaHandler.filterByReassignmentTargetReplicas(apparentlyDifferentRf).stream()); } /** - *

The Topic Operator {@code alterableTopicConfig} can be used to specify a comma separated list of Kafka - * topic configurations that can be altered by users through {@code .spec.config}. Keep in mind that if changes - * are applied directly in Kafka, the operator will try to revert them producing a warning.

- * - *

This is useful in standalone mode when you have a Kafka service that restricts alter operations - * to a subset of all the Kafka topic configurations.

- * - *

The default value is "ALL", which means no restrictions in changing {@code .spec.config}. - * The opposite is "NONE", which can be set to explicitly disable any change.

+ * Cruise Control allows scale down of the replication factor under the min.insync.replicas value, which can cause + * disruption to producers with acks=all. When this happens, the Topic Operator won't block the operation, but will + * just log a warning, because the KafkaRoller ignores topics with RF < minISR, and they don't even show up as under + * replicated in Kafka metrics. * - * @param alterConfigOps Requested alter config operations. + * @param reconcilableTopics Reconcilable topic. */ - private void skipNonAlterableConfigs(Set alterConfigOps) { - var alterableConfigs = config.alterableTopicConfig(); - if (alterableConfigs != null && alterConfigOps != null && !alterableConfigs.isEmpty()) { - if (alterableConfigs.equalsIgnoreCase("NONE")) { - alterConfigOps.clear(); - } else if (!alterableConfigs.equalsIgnoreCase("ALL")) { - var alterablePropertySet = Arrays.stream(alterableConfigs.replaceAll("\\s", "").split(",")) - .collect(Collectors.toSet()); - alterConfigOps.removeIf(op -> !alterablePropertySet.contains(op.configEntry().name())); + private void warnTooLargeMinIsr(List reconcilableTopics) { + if (config.skipClusterConfigReview()) { + // This method is for internal configurations. So skipping. + return; + } + + var clusterMinIsr = kafkaHandler.clusterConfig(KafkaHandler.MIN_INSYNC_REPLICAS); + for (ReconcilableTopic reconcilableTopic : reconcilableTopics) { + var topicConfig = reconcilableTopic.kt().getSpec().getConfig(); + if (topicConfig != null) { + Integer topicMinIsr = (Integer) topicConfig.get(KafkaHandler.MIN_INSYNC_REPLICAS); + var minIsr = topicMinIsr != null ? topicMinIsr : clusterMinIsr.map(Integer::parseInt).orElse(1); + var targetRf = reconcilableTopic.kt().getSpec().getReplicas(); + if (targetRf < minIsr) { + LOGGER.warnCr(reconcilableTopic.reconciliation(), + "The target replication factor ({}) is below the configured {} ({})", + targetRf, KafkaHandler.MIN_INSYNC_REPLICAS, minIsr); + } } } } - private static boolean hasConfig(KafkaTopic kt) { - return kt.getSpec() != null - && kt.getSpec().getConfig() != null; + private List topicsMatching(List reconcilableTopics, Predicate status) { + return reconcilableTopics.stream().filter(rt -> status.test(rt.kt())).collect(Collectors.toList()); + } + + private static boolean hasPendingReplicasChange(KafkaTopic kafkaTopic) { + return TopicOperatorUtil.hasReplicasChangeStatus(kafkaTopic) + && kafkaTopic.getStatus().getReplicasChange().getState() == ReplicasChangeState.PENDING + && kafkaTopic.getStatus().getReplicasChange().getSessionId() == null; } - - private void updateStatusForSuccess(ReconcilableTopic reconcilableTopic) { + + private static boolean hasOngoingReplicasChange(KafkaTopic kafkaTopic) { + return TopicOperatorUtil.hasReplicasChangeStatus(kafkaTopic) + && kafkaTopic.getStatus().getReplicasChange().getState() == ReplicasChangeState.ONGOING + && kafkaTopic.getStatus().getReplicasChange().getSessionId() != null; + } + + private static boolean hasFailedReplicasChange(KafkaTopic kafkaTopic) { + return TopicOperatorUtil.hasReplicasChangeStatus(kafkaTopic) + && kafkaTopic.getStatus().getReplicasChange().getState() == ReplicasChangeState.PENDING + && kafkaTopic.getStatus().getReplicasChange().getMessage() != null; + } + + private void updateStatuses(Results results) { + // update statues with the overall results + results.forEachRightResult((reconcilableTopic, ignored) -> + updateStatusForSuccess(reconcilableTopic, results) + ); + results.forEachLeftResult(this::updateStatusForException); + LOGGER.traceOp("Updated status of {} KafkaTopics", results.size()); + } + + private void updateStatusForSuccess(ReconcilableTopic reconcilableTopic, Results results) { List conditions = new ArrayList<>(); var conditionType = "Ready"; if (!TopicOperatorUtil.isManaged(reconcilableTopic.kt())) { @@ -1149,58 +871,35 @@ private void updateStatusForSuccess(ReconcilableTopic reconcilableTopic) { } else if (TopicOperatorUtil.isPaused(reconcilableTopic.kt())) { conditionType = "ReconciliationPaused"; } - + conditions.add(new ConditionBuilder() - .withType(conditionType) - .withStatus("True") - .withLastTransitionTime(StatusUtils.iso8601Now()) - .build()); - - addNonAlterableConfigsWarning(reconcilableTopic, conditions); - + .withType(conditionType) + .withStatus("True") + .withLastTransitionTime(StatusUtils.iso8601Now()) + .build()); + + conditions.addAll(results.getConditions(reconcilableTopic)); + reconcilableTopic.kt().setStatus( new KafkaTopicStatusBuilder(reconcilableTopic.kt().getStatus()) + .withTopicName( + !TopicOperatorUtil.isManaged(reconcilableTopic.kt()) + ? null : TopicOperatorUtil.topicName(reconcilableTopic.kt()) + ) + .withTopicId( + (!TopicOperatorUtil.isManaged(reconcilableTopic.kt()) || TopicOperatorUtil.isPaused(reconcilableTopic.kt())) + ? null : topicIds.get(TopicOperatorUtil.topicName(reconcilableTopic.kt())) + ) .withConditions(conditions) - .build()); - updateStatus(reconcilableTopic); - metrics.successfulReconciliationsCounter(namespace).increment(); - } - - private void addNonAlterableConfigsWarning(ReconcilableTopic reconcilableTopic, - List conditions) { - var readOnlyConfigs = new ArrayList<>(); - var alterableConfigs = config.alterableTopicConfig(); - - if (reconcilableTopic != null && reconcilableTopic.kt() != null - && hasConfig(reconcilableTopic.kt()) && alterableConfigs != null) { - if (alterableConfigs.equalsIgnoreCase("NONE")) { - reconcilableTopic.kt().getSpec().getConfig().forEach((key, value) -> readOnlyConfigs.add(key)); - } else if (!alterableConfigs.equalsIgnoreCase("ALL") && !alterableConfigs.isBlank()) { - var alterablePropertySet = Arrays.stream(alterableConfigs.replaceAll("\\s", "").split(",")) - .collect(Collectors.toSet()); - reconcilableTopic.kt().getSpec().getConfig().forEach((key, value) -> { - if (!alterablePropertySet.contains(key)) { - readOnlyConfigs.add(key); - } - }); - } - } - - if (!readOnlyConfigs.isEmpty()) { - var properties = String.join(", ", readOnlyConfigs.toArray(new String[0])); - var message = "These .spec.config properties are not configurable: [" + properties + "]"; - LOGGER.warnCr(reconcilableTopic.reconciliation(), message); - conditions.add(new ConditionBuilder() - .withMessage(message) - .withReason("NotConfigurable") - .withStatus("True") - .withType("Warning") - .withLastTransitionTime(StatusUtils.iso8601Now()) - .build()); - } + .withReplicasChange(results.getReplicasChange(reconcilableTopic)) + .build()); + + kubernetesHandler.updateStatus(reconcilableTopic); + metricsHolder.successfulReconciliationsCounter(config.namespace()).increment(); } private void updateStatusForException(ReconcilableTopic reconcilableTopic, Exception e) { + List conditions = new ArrayList<>(); String reason; if (e instanceof TopicOperatorException) { LOGGER.warnCr(reconcilableTopic.reconciliation(), "Reconciliation failed: {}", e.getMessage()); @@ -1209,61 +908,32 @@ private void updateStatusForException(ReconcilableTopic reconcilableTopic, Excep LOGGER.errorCr(reconcilableTopic.reconciliation(), "Reconciliation failed with unexpected exception", e); reason = e.getClass().getSimpleName(); } + + conditions.add(new ConditionBuilder() + .withType("Ready") + .withStatus("False") + .withReason(reason) + .withMessage(e.getMessage()) + .withLastTransitionTime(StatusUtils.iso8601Now()) + .build()); + reconcilableTopic.kt().setStatus( new KafkaTopicStatusBuilder(reconcilableTopic.kt().getStatus()) - .withConditions(List.of(new ConditionBuilder() - .withType("Ready") - .withStatus("False") - .withReason(reason) - .withMessage(e.getMessage()) - .withLastTransitionTime(StatusUtils.iso8601Now()) - .build())) + .withTopicName( + !TopicOperatorUtil.isManaged(reconcilableTopic.kt()) + ? null + // we keep the old topicName if the topic is managed and the error is caused by a spec.topicName change + : reconcilableTopic.kt().getStatus() != null && reconcilableTopic.kt().getStatus().getTopicName() != null + ? reconcilableTopic.kt().getStatus().getTopicName() : TopicOperatorUtil.topicName(reconcilableTopic.kt()) + ) + .withTopicId( + (!TopicOperatorUtil.isManaged(reconcilableTopic.kt()) || TopicOperatorUtil.isPaused(reconcilableTopic.kt())) + ? null : topicIds.get(TopicOperatorUtil.topicName(reconcilableTopic.kt())) + ) + .withConditions(conditions) .build()); - updateStatus(reconcilableTopic); - metrics.failedReconciliationsCounter(namespace).increment(); - } - - private void updateStatus(ReconcilableTopic reconcilableTopic) { - var oldStatus = Crds.topicOperation(kubeClient) - .inNamespace(reconcilableTopic.kt().getMetadata().getNamespace()) - .withName(reconcilableTopic.kt().getMetadata().getName()).get().getStatus(); - - // the observedGeneration is a marker that shows that the operator works and that it saw the last update to the resource - reconcilableTopic.kt().getStatus().setObservedGeneration(reconcilableTopic.kt().getMetadata().getGeneration()); - - // add or remove topicName - reconcilableTopic.kt().getStatus().setTopicName( - !TopicOperatorUtil.isManaged(reconcilableTopic.kt()) - ? null - : oldStatus != null && oldStatus.getTopicName() != null - ? oldStatus.getTopicName() - : TopicOperatorUtil.topicName(reconcilableTopic.kt()) - ); - - // add or remove topicId - reconcilableTopic.kt().getStatus().setTopicId( - (!TopicOperatorUtil.isManaged(reconcilableTopic.kt()) || TopicOperatorUtil.isPaused(reconcilableTopic.kt())) - ? null : topicNameIdMap.get(reconcilableTopic.topicName()) - ); - - StatusDiff statusDiff = new StatusDiff(oldStatus, reconcilableTopic.kt().getStatus()); - if (!statusDiff.isEmpty()) { - var updatedTopic = new KafkaTopicBuilder(reconcilableTopic.kt()) - .editOrNewMetadata() - .withResourceVersion(null) - .endMetadata() - .withStatus(reconcilableTopic.kt().getStatus()) - .build(); - LOGGER.debugCr(reconcilableTopic.reconciliation(), "Updating status with {}", updatedTopic.getStatus()); - var timerSample = TopicOperatorUtil.startExternalRequestTimer(metrics, enableAdditionalMetrics); - try { - var got = Crds.topicOperation(kubeClient).resource(updatedTopic).updateStatus(); - LOGGER.traceCr(reconcilableTopic.reconciliation(), "Updated status to observedGeneration {}, resourceVersion {}", - got.getStatus().getObservedGeneration(), got.getMetadata().getResourceVersion()); - } catch (Throwable e) { - LOGGER.errorOp("Status update failed: {}", e.getMessage()); - } - TopicOperatorUtil.stopExternalRequestTimer(timerSample, metrics::updateStatusTimer, enableAdditionalMetrics, namespace); - } + + kubernetesHandler.updateStatus(reconcilableTopic); + metricsHolder.failedReconciliationsCounter(config.namespace()).increment(); } } diff --git a/topic-operator/src/main/java/io/strimzi/operator/topic/KafkaHandler.java b/topic-operator/src/main/java/io/strimzi/operator/topic/KafkaHandler.java new file mode 100644 index 00000000000..74baccbe5e5 --- /dev/null +++ b/topic-operator/src/main/java/io/strimzi/operator/topic/KafkaHandler.java @@ -0,0 +1,438 @@ +/* + * 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.topic; + +import io.strimzi.api.kafka.model.topic.KafkaTopic; +import io.strimzi.operator.common.ReconciliationLogger; +import io.strimzi.operator.common.model.InvalidResourceException; +import io.strimzi.operator.topic.metrics.TopicOperatorMetricsHolder; +import io.strimzi.operator.topic.model.Either; +import io.strimzi.operator.topic.model.Pair; +import io.strimzi.operator.topic.model.PartitionedByError; +import io.strimzi.operator.topic.model.ReconcilableTopic; +import io.strimzi.operator.topic.model.TopicOperatorException; +import io.strimzi.operator.topic.model.TopicState; +import io.strimzi.operator.topic.model.UncheckedInterruptedException; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.AlterConfigOp; +import org.apache.kafka.clients.admin.Config; +import org.apache.kafka.clients.admin.CreatePartitionsResult; +import org.apache.kafka.clients.admin.DescribeConfigsResult; +import org.apache.kafka.clients.admin.DescribeTopicsResult; +import org.apache.kafka.clients.admin.NewPartitions; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.admin.PartitionReassignment; +import org.apache.kafka.clients.admin.TopicDescription; +import org.apache.kafka.common.KafkaFuture; +import org.apache.kafka.common.TopicCollection; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.config.ConfigResource; +import org.apache.kafka.common.errors.ApiException; +import org.apache.kafka.common.errors.TopicExistsException; +import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; + +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * Handler for Kafka requests. + */ +public class KafkaHandler { + static final ReconciliationLogger LOGGER = ReconciliationLogger.create(KafkaHandler.class); + + /** Default value for partitions and replicas. */ + public static final int BROKER_DEFAULT = -1; + /** Kafka configuration for auto create topic. */ + public static final String AUTO_CREATE_TOPICS_ENABLE = "auto.create.topics.enable"; + /** Kafka configuration for min insync replicas. */ + public static final String MIN_INSYNC_REPLICAS = "min.insync.replicas"; + + private final TopicOperatorConfig config; + private final TopicOperatorMetricsHolder metricsHolder; + private final Admin kafkaAdminClient; + + /** + * Create a new instance. + * + * @param config Topic Operator configuration. + * @param metricsHolder Metrics holder. + * @param kafkaAdminClient Kafka admin client. + */ + KafkaHandler(TopicOperatorConfig config, TopicOperatorMetricsHolder metricsHolder, Admin kafkaAdminClient) { + this.config = config; + this.metricsHolder = metricsHolder; + this.kafkaAdminClient = kafkaAdminClient; + } + + /** + * Retrieve the specified configuration value for a Kafka cluster. + *

+ * This method queries the Kafka cluster to obtain the configuration value associated with the given name. + * It iterates through all nodes (brokers) in the cluster, requesting their configurations, and returns the + * value of the configuration if found. The search stops at the first occurrence of the configuration name + * across all nodes, assuming uniform configuration across the cluster. + * + * @param configName The name of the configuration to retrieve. + * @return A string containing the value of the requested configuration if found. + * @throws RuntimeException if there is an error during the operation. + * This exception wraps the underlying exception's message. + */ + public Optional clusterConfig(String configName) { + try { + var describeClusterResult = kafkaAdminClient.describeCluster(); + var nodes = describeClusterResult.nodes().get(); + Map>> futures = new HashMap<>(); + for (var node : nodes) { + ConfigResource nodeResource = new ConfigResource(ConfigResource.Type.BROKER, node.idString()); + futures.put(nodeResource, kafkaAdminClient.describeConfigs(Set.of(nodeResource)).all()); + } + for (var entry : futures.entrySet()) { + var nodeConfig = entry.getValue().get().get(entry.getKey()); + var configEntry = nodeConfig.get(configName); + return Optional.of(configEntry.value()); + } + return Optional.empty(); + } catch (Throwable e) { + throw new RuntimeException("Failed to get cluster configuration: " + e.getMessage()); + } + } + + /** + * Create topics. + * + * @param reconcilableTopics Reconcilable topics. + * @return Result partitioned by error. + */ + public PartitionedByError createTopics(List reconcilableTopics) { + Map newTopicsErrors = new HashMap<>(); + var newTopics = reconcilableTopics.stream().map(reconcilableTopic -> { + try { + return buildNewTopic(reconcilableTopic.kt(), reconcilableTopic.topicName()); + } catch (InvalidResourceException e) { + newTopicsErrors.put(reconcilableTopic, new TopicOperatorException.InternalError(e)); + return null; + } + }).filter(Objects::nonNull).collect(Collectors.toSet()); + + LOGGER.debugOp("Admin.createTopics({})", newTopics); + var timerSample = TopicOperatorUtil.startExternalRequestTimer(metricsHolder, config.enableAdditionalMetrics()); + var ctr = kafkaAdminClient.createTopics(newTopics); + ctr.all().whenComplete((i, e) -> { + TopicOperatorUtil.stopExternalRequestTimer(timerSample, metricsHolder::createTopicsTimer, config.enableAdditionalMetrics(), config.namespace()); + if (e != null) { + LOGGER.traceOp("Admin.createTopics({}) failed with {}", newTopics, String.valueOf(e)); + } else { + LOGGER.traceOp("Admin.createTopics({}) completed", newTopics); + } + }); + var values = ctr.values(); + return TopicOperatorUtil.partitionedByError(reconcilableTopics.stream().map(reconcilableTopic -> { + if (newTopicsErrors.containsKey(reconcilableTopic)) { + return new Pair<>(reconcilableTopic, Either.ofLeft(newTopicsErrors.get(reconcilableTopic))); + } + try { + values.get(reconcilableTopic.topicName()).get(); + return new Pair<>(reconcilableTopic, Either.ofRight( + new TopicState(new TopicDescription(reconcilableTopic.topicName(), + false, List.of(), Set.of(), ctr.topicId(reconcilableTopic.topicName()).get()), null) + )); + } catch (ExecutionException e) { + if (e.getCause() != null && e.getCause() instanceof TopicExistsException) { + // we treat this as a success, the next reconciliation checks the configuration + return new Pair<>(reconcilableTopic, Either.ofRight(null)); + } else { + return new Pair<>(reconcilableTopic, Either.ofLeft(handleAdminException(e))); + } + } catch (InterruptedException e) { + throw new UncheckedInterruptedException(e); + } + })); + } + + /** + * Filter topics with RF change. + * + * @param apparentlyDifferentRfTopics Topics with possible RF change. + * @return Filtered list of topics with RF change. + */ + public List>> filterByReassignmentTargetReplicas( + List> apparentlyDifferentRfTopics) { + if (apparentlyDifferentRfTopics.isEmpty()) { + return List.of(); + } + var apparentDifferentRfPartitions = apparentlyDifferentRfTopics.stream() + .flatMap(pair -> pair.getValue().description().partitions().stream() + .filter(pi -> { + // includes only the partitions of the topic with a RF that mismatches the desired RF + var desiredRf = pair.getKey().kt().getSpec().getReplicas(); + return desiredRf != pi.replicas().size(); + }) + .map(pi -> new TopicPartition(pair.getKey().topicName(), pi.partition()))).collect(Collectors.toSet()); + + Map reassignments; + LOGGER.traceOp("Admin.listPartitionReassignments({})", apparentDifferentRfPartitions); + var timerSample = TopicOperatorUtil.startExternalRequestTimer(metricsHolder, config.enableAdditionalMetrics()); + try { + reassignments = kafkaAdminClient.listPartitionReassignments(apparentDifferentRfPartitions).reassignments().get(); + LOGGER.traceOp("Admin.listPartitionReassignments({}) completed", apparentDifferentRfPartitions); + } catch (ExecutionException e) { + LOGGER.traceOp("Admin.listPartitionReassignments({}) failed with {}", apparentDifferentRfPartitions, e); + return apparentlyDifferentRfTopics.stream().map(pair -> + new Pair<>(pair.getKey(), Either.ofLeft(handleAdminException(e)))).toList(); + } catch (InterruptedException e) { + throw new UncheckedInterruptedException(e); + } + TopicOperatorUtil.stopExternalRequestTimer(timerSample, metricsHolder::listReassignmentsTimer, config.enableAdditionalMetrics(), config.namespace()); + + var partitionToTargetRf = reassignments.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, entry -> { + var partitionReassignment = entry.getValue(); + // See https://cwiki.apache.org/confluence/display/KAFKA/KIP-455%3A+Create+an+Administrative+API+for+Replica+Reassignment#KIP455:CreateanAdministrativeAPIforReplicaReassignment-Algorithm + // for a full description of the algorithm + // but in essence replicas() will include addingReplicas() from the beginning + // so the target rf will be the replicas minus the removing + var target = new HashSet<>(partitionReassignment.replicas()); + partitionReassignment.removingReplicas().forEach(target::remove); + return target.size(); + })); + + return apparentlyDifferentRfTopics.stream() + .filter(pair -> pair.getValue().description().partitions().stream() + .anyMatch(pi -> { + TopicPartition tp = new TopicPartition(pair.getKey().topicName(), pi.partition()); + Integer targetRf = partitionToTargetRf.get(tp); + Integer desiredRf = pair.getKey().kt().getSpec().getReplicas(); + return !Objects.equals(targetRf, desiredRf); + }) + ).map(pair -> new Pair<>(pair.getKey(), Either.ofRight(pair.getValue()))) + .toList(); + } + + /** + * Alter topic configuration. + * + * @param someAlterConfigs Alter configurations. + * @return Result partitioned by error. + */ + public PartitionedByError alterConfigs(List>> someAlterConfigs) { + if (someAlterConfigs.isEmpty()) { + return new PartitionedByError<>(List.of(), List.of()); + } + var alteredConfigs = someAlterConfigs.stream().collect(Collectors.toMap(entry -> buildTopicConfigResource(entry.getKey().topicName()), Pair::getValue)); + LOGGER.debugOp("Admin.incrementalAlterConfigs({})", alteredConfigs); + var timerSample = TopicOperatorUtil.startExternalRequestTimer(metricsHolder, config.enableAdditionalMetrics()); + var acr = kafkaAdminClient.incrementalAlterConfigs(alteredConfigs); + TopicOperatorUtil.stopExternalRequestTimer(timerSample, metricsHolder::alterConfigsTimer, config.enableAdditionalMetrics(), config.namespace()); + acr.all().whenComplete((i, e) -> { + TopicOperatorUtil.stopExternalRequestTimer(timerSample, metricsHolder::alterConfigsTimer, config.enableAdditionalMetrics(), config.namespace()); + if (e != null) { + LOGGER.traceOp("Admin.incrementalAlterConfigs({}) failed with {}", alteredConfigs, String.valueOf(e)); + } else { + LOGGER.traceOp("Admin.incrementalAlterConfigs({}) completed", alteredConfigs); + } + }); + var alterConfigsResult = acr.values(); + Stream>> entryStream = someAlterConfigs.stream().map(entry -> { + try { + return new Pair<>(entry.getKey(), Either.ofRight(alterConfigsResult.get(buildTopicConfigResource(entry.getKey().topicName())).get())); + } catch (ExecutionException e) { + return new Pair<>(entry.getKey(), Either.ofLeft(handleAdminException(e))); + } catch (InterruptedException e) { + throw new UncheckedInterruptedException(e); + } + }); + return TopicOperatorUtil.partitionedByError(entryStream); + } + + /** + * Create topic partitions. + * + * @param someCreatePartitions Partitions to be created. + * @return Result partitioned by error. + */ + public PartitionedByError createPartitions(List> someCreatePartitions) { + if (someCreatePartitions.isEmpty()) { + return new PartitionedByError<>(List.of(), List.of()); + } + var newPartitions = someCreatePartitions.stream().collect(Collectors.toMap(pair -> pair.getKey().topicName(), Pair::getValue)); + LOGGER.debugOp("Admin.createPartitions({})", newPartitions); + var timerSample = TopicOperatorUtil.startExternalRequestTimer(metricsHolder, config.enableAdditionalMetrics()); + CreatePartitionsResult cpr = kafkaAdminClient.createPartitions(newPartitions); + cpr.all().whenComplete((i, e) -> { + TopicOperatorUtil.stopExternalRequestTimer(timerSample, metricsHolder::createPartitionsTimer, config.enableAdditionalMetrics(), config.namespace()); + if (e != null) { + LOGGER.traceOp("Admin.createPartitions({}) failed with {}", newPartitions, String.valueOf(e)); + } else { + LOGGER.traceOp("Admin.createPartitions({}) completed", newPartitions); + } + }); + var createPartitionsResult = cpr.values(); + var entryStream = someCreatePartitions.stream().map(entry -> { + try { + createPartitionsResult.get(entry.getKey().topicName()).get(); + return new Pair<>(entry.getKey(), Either.ofRight(null)); + } catch (ExecutionException e) { + return new Pair<>(entry.getKey(), Either.ofLeft(handleAdminException(e))); + } catch (InterruptedException e) { + throw new UncheckedInterruptedException(e); + } + }); + return TopicOperatorUtil.partitionedByError(entryStream); + } + + /** + * Describe topics. + * + * @param reconcilableTopics Topics to describe. + * @return Result partitioned by error. + */ + public PartitionedByError describeTopics(List reconcilableTopics) { + if (reconcilableTopics.isEmpty()) { + return new PartitionedByError<>(List.of(), List.of()); + } + Set configResources = reconcilableTopics.stream() + .map(reconcilableTopic -> buildTopicConfigResource(reconcilableTopic.topicName())) + .collect(Collectors.toSet()); + Set tns = reconcilableTopics.stream().map(ReconcilableTopic::topicName).collect(Collectors.toSet()); + + DescribeTopicsResult describeTopicsResult; + { + LOGGER.debugOp("Admin.describeTopics({})", tns); + var timerSample = TopicOperatorUtil.startExternalRequestTimer(metricsHolder, config.enableAdditionalMetrics()); + describeTopicsResult = kafkaAdminClient.describeTopics(tns); + describeTopicsResult.allTopicNames().whenComplete((i, e) -> { + TopicOperatorUtil.stopExternalRequestTimer(timerSample, metricsHolder::describeTopicsTimer, config.enableAdditionalMetrics(), config.namespace()); + if (e != null) { + LOGGER.traceOp("Admin.describeTopics({}) failed with {}", tns, String.valueOf(e)); + } else { + LOGGER.traceOp("Admin.describeTopics({}) completed", tns); + } + }); + } + DescribeConfigsResult describeConfigsResult; + { + LOGGER.debugOp("Admin.describeConfigs({})", configResources); + var timerSample = TopicOperatorUtil.startExternalRequestTimer(metricsHolder, config.enableAdditionalMetrics()); + describeConfigsResult = kafkaAdminClient.describeConfigs(configResources); + describeConfigsResult.all().whenComplete((i, e) -> { + TopicOperatorUtil.stopExternalRequestTimer(timerSample, metricsHolder::describeConfigsTimer, config.enableAdditionalMetrics(), config.namespace()); + if (e != null) { + LOGGER.traceOp("Admin.describeConfigs({}) failed with {}", configResources, String.valueOf(e)); + } else { + LOGGER.traceOp("Admin.describeConfigs({}) completed", configResources); + } + }); + } + + var cs1 = describeTopicsResult.topicNameValues(); + var cs2 = describeConfigsResult.values(); + return TopicOperatorUtil.partitionedByError(reconcilableTopics.stream().map(reconcilableTopic -> { + Config configs = null; + TopicDescription description = null; + ExecutionException exception = null; + try { + description = cs1.get(reconcilableTopic.topicName()).get(); + } catch (ExecutionException e) { + exception = e; + } catch (InterruptedException e) { + throw new UncheckedInterruptedException(e); + } + + try { + configs = cs2.get(buildTopicConfigResource(reconcilableTopic.topicName())).get(); + } catch (ExecutionException e) { + exception = e; + } catch (InterruptedException e) { + throw new UncheckedInterruptedException(e); + } + if (exception != null) { + return new Pair<>(reconcilableTopic, Either.ofLeft(handleAdminException(exception))); + } else { + return new Pair<>(reconcilableTopic, Either.ofRight(new TopicState(description, configs))); + } + })); + } + + /** + * Delete topics. + * + * @param reconcilableTopics List of topics. + * @param topicNames Topic names to delete. + * @return Result partitioned by error. + */ + public PartitionedByError deleteTopics(List reconcilableTopics, Set topicNames) { + if (topicNames.isEmpty()) { + return new PartitionedByError<>(List.of(), List.of()); + } + var someDeleteTopics = TopicCollection.ofTopicNames(topicNames); + LOGGER.debugOp("Admin.deleteTopics({})", someDeleteTopics.topicNames()); + + // Admin delete + var timerSample = TopicOperatorUtil.startExternalRequestTimer(metricsHolder, config.enableAdditionalMetrics()); + var dtr = kafkaAdminClient.deleteTopics(someDeleteTopics); + dtr.all().whenComplete((i, e) -> { + TopicOperatorUtil.stopExternalRequestTimer(timerSample, metricsHolder::deleteTopicsTimer, config.enableAdditionalMetrics(), config.namespace()); + if (e != null) { + LOGGER.traceOp("Admin.deleteTopics({}) failed with {}", someDeleteTopics.topicNames(), String.valueOf(e)); + } else { + LOGGER.traceOp("Admin.deleteTopics({}) completed", someDeleteTopics.topicNames()); + } + }); + var futuresMap = dtr.topicNameValues(); + + return TopicOperatorUtil.partitionedByError(reconcilableTopics.stream() + .filter(reconcilableTopic -> futuresMap.get(reconcilableTopic.topicName()) != null) + .map(reconcilableTopic -> { + try { + futuresMap.get(reconcilableTopic.topicName()).get(); + return new Pair<>(reconcilableTopic, Either.ofRight(null)); + } catch (ExecutionException e) { + if (e.getCause() instanceof UnknownTopicOrPartitionException) { + return new Pair<>(reconcilableTopic, Either.ofRight(null)); + } else { + return new Pair<>(reconcilableTopic, Either.ofLeft(handleAdminException(e))); + } + } catch (InterruptedException e) { + throw new UncheckedInterruptedException(e); + } + })); + } + + private static NewTopic buildNewTopic(KafkaTopic kafkaTopic, String topicName) { + return new NewTopic(topicName, TopicOperatorUtil.partitions(kafkaTopic), + TopicOperatorUtil.replicas(kafkaTopic)).configs(buildConfigsMap(kafkaTopic)); + } + + private static Map buildConfigsMap(KafkaTopic kafkaTopic) { + Map configs = new HashMap<>(); + if (TopicOperatorUtil.hasConfig(kafkaTopic)) { + for (var entry : kafkaTopic.getSpec().getConfig().entrySet()) { + configs.put(entry.getKey(), TopicOperatorUtil.configValueAsString(entry.getKey(), entry.getValue())); + } + } + return configs; + } + + private static ConfigResource buildTopicConfigResource(String topicName) { + return new ConfigResource(ConfigResource.Type.TOPIC, topicName); + } + + private static TopicOperatorException handleAdminException(ExecutionException e) { + var cause = e.getCause(); + if (cause instanceof ApiException) { + return new TopicOperatorException.KafkaError((ApiException) cause); + } else { + return new TopicOperatorException.InternalError(cause); + } + } +} diff --git a/topic-operator/src/main/java/io/strimzi/operator/topic/KubernetesHandler.java b/topic-operator/src/main/java/io/strimzi/operator/topic/KubernetesHandler.java new file mode 100644 index 00000000000..8b820bd6abc --- /dev/null +++ b/topic-operator/src/main/java/io/strimzi/operator/topic/KubernetesHandler.java @@ -0,0 +1,119 @@ +/* + * 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.topic; + +import io.fabric8.kubernetes.client.KubernetesClient; +import io.strimzi.api.kafka.Crds; +import io.strimzi.api.kafka.model.topic.KafkaTopic; +import io.strimzi.api.kafka.model.topic.KafkaTopicBuilder; +import io.strimzi.operator.common.ReconciliationLogger; +import io.strimzi.operator.common.model.StatusDiff; +import io.strimzi.operator.topic.metrics.TopicOperatorMetricsHolder; +import io.strimzi.operator.topic.model.ReconcilableTopic; + +/** + * Handler for Kubernetes requests. + */ +public class KubernetesHandler { + static final ReconciliationLogger LOGGER = ReconciliationLogger.create(KubernetesHandler.class); + + /** Annotation for managing and unmanaging a KafkaTopic. */ + public static final String ANNO_STRIMZI_IO_MANAGED = "strimzi.io/managed"; + static final String FINALIZER_STRIMZI_IO_TO = "strimzi.io/topic-operator"; + + private final TopicOperatorConfig config; + private final TopicOperatorMetricsHolder metricsHolder; + private final KubernetesClient kubernetesClient; + + /** + * Create a new instance. + * + * @param config Topic Operator configuration. + * @param metricsHolder Metrics holder. + * @param kubernetesClient Kubernetes client. + */ + KubernetesHandler(TopicOperatorConfig config, TopicOperatorMetricsHolder metricsHolder, KubernetesClient kubernetesClient) { + this.config = config; + this.metricsHolder = metricsHolder; + this.kubernetesClient = kubernetesClient; + } + + /** + * Add finalizer to a KafkaTopic resource. + * + * @param reconcilableTopic Reconcilable topic. + * @return KafkaTopic resource. + */ + public KafkaTopic addFinalizer(ReconcilableTopic reconcilableTopic) { + if (!reconcilableTopic.kt().getMetadata().getFinalizers().contains(FINALIZER_STRIMZI_IO_TO)) { + LOGGER.traceCr(reconcilableTopic.reconciliation(), "Adding finalizer {}", FINALIZER_STRIMZI_IO_TO); + var timerSample = TopicOperatorUtil.startExternalRequestTimer(metricsHolder, config.enableAdditionalMetrics()); + var withFinalizer = Crds.topicOperation(kubernetesClient).resource(reconcilableTopic.kt()).edit(old -> + new KafkaTopicBuilder(old).editOrNewMetadata().addToFinalizers(FINALIZER_STRIMZI_IO_TO).endMetadata().build()); + TopicOperatorUtil.stopExternalRequestTimer(timerSample, metricsHolder::addFinalizerTimer, config.enableAdditionalMetrics(), config.namespace()); + LOGGER.traceCr(reconcilableTopic.reconciliation(), "Added finalizer {}, resourceVersion now {}", FINALIZER_STRIMZI_IO_TO, TopicOperatorUtil.resourceVersion(withFinalizer)); + return withFinalizer; + } + return reconcilableTopic.kt(); + } + + /** + * Remove finalizer from a KafkaTopic resource. + * + * @param reconcilableTopic Reconcilable topic. + * @return KafkaTopic resource. + */ + public KafkaTopic removeFinalizer(ReconcilableTopic reconcilableTopic) { + if (reconcilableTopic.kt().getMetadata().getFinalizers().contains(FINALIZER_STRIMZI_IO_TO)) { + LOGGER.traceCr(reconcilableTopic.reconciliation(), "Removing finalizer {}", FINALIZER_STRIMZI_IO_TO); + var timerSample = TopicOperatorUtil.startExternalRequestTimer(metricsHolder, config.enableAdditionalMetrics()); + var withoutFinalizer = Crds.topicOperation(kubernetesClient).resource(reconcilableTopic.kt()).edit(old -> + new KafkaTopicBuilder(old).editOrNewMetadata().removeFromFinalizers(FINALIZER_STRIMZI_IO_TO).endMetadata().build()); + TopicOperatorUtil.stopExternalRequestTimer(timerSample, metricsHolder::removeFinalizerTimer, config.enableAdditionalMetrics(), config.namespace()); + LOGGER.traceCr(reconcilableTopic.reconciliation(), "Removed finalizer {}, resourceVersion now {}", FINALIZER_STRIMZI_IO_TO, TopicOperatorUtil.resourceVersion(withoutFinalizer)); + return withoutFinalizer; + } else { + return reconcilableTopic.kt(); + } + } + + /** + * Update the KafkaTopic status. + * + * @param reconcilableTopic Reconcilable topic. + * @return KafkaTopic resource. + */ + public KafkaTopic updateStatus(ReconcilableTopic reconcilableTopic) { + var oldStatus = Crds.topicOperation(kubernetesClient) + .inNamespace(reconcilableTopic.kt().getMetadata().getNamespace()) + .withName(reconcilableTopic.kt().getMetadata().getName()).get().getStatus(); + + // we always set the observedGeneration because it shows that the operator works, + // and that it saw the last update to the resource + reconcilableTopic.kt().getStatus().setObservedGeneration(reconcilableTopic.kt().getMetadata().getGeneration()); + + StatusDiff statusDiff = new StatusDiff(oldStatus, reconcilableTopic.kt().getStatus()); + if (!statusDiff.isEmpty()) { + var updatedTopic = new KafkaTopicBuilder(reconcilableTopic.kt()) + .editOrNewMetadata() + .withResourceVersion(null) + .endMetadata() + .withStatus(reconcilableTopic.kt().getStatus()) + .build(); + LOGGER.debugCr(reconcilableTopic.reconciliation(), "Updating status with {}", updatedTopic.getStatus()); + var timerSample = TopicOperatorUtil.startExternalRequestTimer(metricsHolder, config.enableAdditionalMetrics()); + try { + var got = Crds.topicOperation(kubernetesClient).resource(updatedTopic).updateStatus(); + LOGGER.traceCr(reconcilableTopic.reconciliation(), "Updated status to observedGeneration {}, resourceVersion {}", + got.getStatus().getObservedGeneration(), got.getMetadata().getResourceVersion()); + } catch (Throwable e) { + TopicOperatorUtil.stopExternalRequestTimer(timerSample, metricsHolder::updateStatusTimer, config.enableAdditionalMetrics(), config.namespace()); + LOGGER.errorOp("Status update failed: {}", e.getMessage()); + } + TopicOperatorUtil.stopExternalRequestTimer(timerSample, metricsHolder::updateStatusTimer, config.enableAdditionalMetrics(), config.namespace()); + } + return reconcilableTopic.kt(); + } +} diff --git a/topic-operator/src/main/java/io/strimzi/operator/topic/ReplicasChangeHandler.java b/topic-operator/src/main/java/io/strimzi/operator/topic/ReplicasChangeHandler.java deleted file mode 100644 index 999d9ea3487..00000000000 --- a/topic-operator/src/main/java/io/strimzi/operator/topic/ReplicasChangeHandler.java +++ /dev/null @@ -1,203 +0,0 @@ -/* - * 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.topic; - -import io.micrometer.core.instrument.Timer; -import io.strimzi.api.kafka.model.topic.KafkaTopic; -import io.strimzi.api.kafka.model.topic.KafkaTopicStatusBuilder; -import io.strimzi.api.kafka.model.topic.ReplicasChangeStatusBuilder; -import io.strimzi.operator.common.Reconciliation; -import io.strimzi.operator.common.ReconciliationLogger; -import io.strimzi.operator.topic.cruisecontrol.CruiseControlClient; -import io.strimzi.operator.topic.cruisecontrol.CruiseControlClient.TaskState; -import io.strimzi.operator.topic.cruisecontrol.CruiseControlClient.UserTasksResponse; -import io.strimzi.operator.topic.metrics.TopicOperatorMetricsHolder; -import io.strimzi.operator.topic.model.ReconcilableTopic; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -import static io.strimzi.api.kafka.model.topic.ReplicasChangeState.ONGOING; -import static io.strimzi.api.kafka.model.topic.ReplicasChangeState.PENDING; -import static io.strimzi.operator.topic.TopicOperatorUtil.hasReplicasChange; -import static io.strimzi.operator.topic.TopicOperatorUtil.topicNames; -import static java.lang.String.format; -import static java.nio.charset.StandardCharsets.UTF_8; -import static org.apache.logging.log4j.core.util.Throwables.getRootCause; - -/** - * Replicas change handler that interacts with Cruise Control REST API. - *

- * At any given time, a {@code .spec.replicas} change can be in one of the following states: - *
  • Pending: Not in Cruise Control's task queue (not yet sent or request error).
  • - *
  • Ongoing: In Cruise Control's task queue, but execution not started, or not completed.
  • - *
  • Completed: Cruise Control's task execution completed (target replication factor reconciled).
- */ -public class ReplicasChangeHandler { - private static final ReconciliationLogger LOGGER = ReconciliationLogger.create(ReplicasChangeHandler.class); - - private final TopicOperatorConfig config; - private final TopicOperatorMetricsHolder metrics; - private final CruiseControlClient cruiseControlClient; - - ReplicasChangeHandler(TopicOperatorConfig config, - TopicOperatorMetricsHolder metrics) { - this(config, metrics, CruiseControlClient.create( - config.cruiseControlHostname(), - config.cruiseControlPort(), - config.cruiseControlRackEnabled(), - config.cruiseControlSslEnabled(), - config.cruiseControlSslEnabled() ? getFileContent(config.cruiseControlCrtFilePath()) : null, - config.cruiseControlAuthEnabled(), - config.cruiseControlAuthEnabled() ? new String(getFileContent(config.cruiseControlApiUserPath()), UTF_8) : null, - config.cruiseControlAuthEnabled() ? new String(getFileContent(config.cruiseControlApiPassPath()), UTF_8) : null - )); - } - - ReplicasChangeHandler(TopicOperatorConfig config, - TopicOperatorMetricsHolder metrics, - CruiseControlClient cruiseControlClient) { - this.config = config; - this.metrics = metrics; - this.cruiseControlClient = cruiseControlClient; - } - - /** - * Stop the replicas change handler. - */ - public void stop() { - try { - cruiseControlClient.close(); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - /** - * Send a topic_configuration request to create a task for replication factor change of one or more topics. - * This should be called when one or more .spec.replicas changes are detected. - * Note that this method also updates the KafkaTopic status. - * - * @param reconcilableTopics Pending replicas changes. - * @return Replicas changes with status update. - */ - public List requestPendingChanges(List reconcilableTopics) { - List result = new ArrayList<>(); - if (reconcilableTopics.isEmpty()) { - return result; - } - updateToPending(reconcilableTopics, "Replicas change pending"); - result.addAll(reconcilableTopics); - - Timer.Sample timerSample = TopicOperatorUtil.startExternalRequestTimer(metrics, config.enableAdditionalMetrics()); - try { - LOGGER.debugOp("Sending topic configuration request, topics {}", topicNames(reconcilableTopics)); - List kafkaTopics = reconcilableTopics.stream().map(rt -> rt.kt()).collect(Collectors.toList()); - String userTaskId = cruiseControlClient.topicConfiguration(kafkaTopics); - updateToOngoing(result, "Replicas change ongoing", userTaskId); - } catch (Throwable t) { - updateToFailed(result, format("Replicas change failed, %s", getRootCause(t).getMessage())); - } - TopicOperatorUtil.stopExternalRequestTimer(timerSample, metrics::cruiseControlTopicConfig, config.enableAdditionalMetrics(), config.namespace()); - return result; - } - - /** - * Send a user_tasks request to check the state of ongoing replication factor changes. - * This should be called periodically to update the active tasks cache and KafkaTopic status. - * Note that this method also updates the KafkaTopic status. - * - * @param reconcilableTopics Ongoing replicas changes. - * @return Replicas changes with status update. - */ - public List requestOngoingChanges(List reconcilableTopics) { - List result = new ArrayList<>(); - if (reconcilableTopics.isEmpty()) { - return result; - } - result.addAll(reconcilableTopics); - - Map> groupByUserTaskId = reconcilableTopics.stream() - .filter(rt -> hasReplicasChange(rt.kt().getStatus()) && rt.kt().getStatus().getReplicasChange().getSessionId() != null) - .map(rt -> new ReconcilableTopic(new Reconciliation("", KafkaTopic.RESOURCE_KIND, "", ""), rt.kt(), rt.topicName())) - .collect(Collectors.groupingBy(rt -> rt.kt().getStatus().getReplicasChange().getSessionId(), HashMap::new, Collectors.toList())); - - Timer.Sample timerSample = TopicOperatorUtil.startExternalRequestTimer(metrics, config.enableAdditionalMetrics()); - try { - LOGGER.debugOp("Sending user tasks request, Tasks {}", groupByUserTaskId.keySet()); - UserTasksResponse utr = cruiseControlClient.userTasks(groupByUserTaskId.keySet()); - if (utr.userTasks().isEmpty()) { - // Cruise Control restarted: reset the state because the tasks queue is not persisted - // this may also happen when the tasks' retention time expires, or the cache becomes full - updateToPending(result, "Task not found, Resetting the state"); - } else { - for (var userTask : utr.userTasks()) { - String userTaskId = userTask.userTaskId(); - TaskState state = TaskState.get(userTask.status()); - switch (state) { - case COMPLETED: - updateToCompleted(groupByUserTaskId.get(userTaskId), "Replicas change completed"); - break; - case COMPLETED_WITH_ERROR: - updateToFailed(groupByUserTaskId.get(userTaskId), "Replicas change completed with error"); - break; - case ACTIVE: - case IN_EXECUTION: - // do nothing - break; - } - } - } - } catch (Throwable t) { - updateToFailed(result, format("Replicas change failed, %s", getRootCause(t).getMessage())); - } - TopicOperatorUtil.stopExternalRequestTimer(timerSample, metrics::cruiseControlUserTasks, config.enableAdditionalMetrics(), config.namespace()); - return result; - } - - private static byte[] getFileContent(String filePath) { - try { - return Files.readAllBytes(Path.of(filePath)); - } catch (IOException ioe) { - throw new UncheckedIOException(format("File not found: %s", filePath), ioe); - } - } - - private void updateToPending(List reconcilableTopics, String message) { - LOGGER.infoOp("{}, Topics: {}", message, topicNames(reconcilableTopics)); - reconcilableTopics.forEach(reconcilableTopic -> - reconcilableTopic.kt().setStatus(new KafkaTopicStatusBuilder(reconcilableTopic.kt().getStatus()) - .withReplicasChange(new ReplicasChangeStatusBuilder() - .withState(PENDING).withTargetReplicas(reconcilableTopic.kt().getSpec().getReplicas()).build()).build())); - } - - private void updateToOngoing(List reconcilableTopics, String message, String userTaskId) { - LOGGER.infoOp("{}, Topics: {}", message, topicNames(reconcilableTopics)); - reconcilableTopics.forEach(reconcilableTopic -> - reconcilableTopic.kt().setStatus(new KafkaTopicStatusBuilder(reconcilableTopic.kt().getStatus()) - .editOrNewReplicasChange().withState(ONGOING).withSessionId(userTaskId).endReplicasChange().build())); - } - - private void updateToCompleted(List reconcilableTopics, String message) { - LOGGER.infoOp("{}, Topics: {}", message, topicNames(reconcilableTopics)); - reconcilableTopics.forEach(reconcilableTopic -> - reconcilableTopic.kt().setStatus(new KafkaTopicStatusBuilder(reconcilableTopic.kt().getStatus()) - .withReplicasChange(null).build())); - } - - private void updateToFailed(List reconcilableTopics, String message) { - LOGGER.errorOp("{}, Topics: {}", message, topicNames(reconcilableTopics)); - reconcilableTopics.forEach(reconcilableTopic -> - reconcilableTopic.kt().setStatus(new KafkaTopicStatusBuilder(reconcilableTopic.kt().getStatus()) - .editOrNewReplicasChange().withMessage(message).endReplicasChange().build())); - } -} diff --git a/topic-operator/src/main/java/io/strimzi/operator/topic/TopicOperatorEventHandler.java b/topic-operator/src/main/java/io/strimzi/operator/topic/TopicEventHandler.java similarity index 75% rename from topic-operator/src/main/java/io/strimzi/operator/topic/TopicOperatorEventHandler.java rename to topic-operator/src/main/java/io/strimzi/operator/topic/TopicEventHandler.java index 7d599ce1618..278e9159bc5 100644 --- a/topic-operator/src/main/java/io/strimzi/operator/topic/TopicOperatorEventHandler.java +++ b/topic-operator/src/main/java/io/strimzi/operator/topic/TopicEventHandler.java @@ -6,21 +6,20 @@ import io.fabric8.kubernetes.client.informers.ResourceEventHandler; import io.strimzi.api.kafka.model.topic.KafkaTopic; +import io.strimzi.operator.common.Annotations; import io.strimzi.operator.common.ReconciliationLogger; import io.strimzi.operator.common.metrics.MetricsHolder; import io.strimzi.operator.topic.model.TopicEvent.TopicDelete; import io.strimzi.operator.topic.model.TopicEvent.TopicUpsert; import java.util.Objects; - -import static io.strimzi.operator.common.Annotations.isReconciliationPausedWithAnnotation; -import static java.util.concurrent.TimeUnit.NANOSECONDS; +import java.util.concurrent.TimeUnit; /** * Handler for {@link KafkaTopic} events. */ -class TopicOperatorEventHandler implements ResourceEventHandler { - private final static ReconciliationLogger LOGGER = ReconciliationLogger.create(TopicOperatorEventHandler.class); +class TopicEventHandler implements ResourceEventHandler { + private final static ReconciliationLogger LOGGER = ReconciliationLogger.create(TopicEventHandler.class); private final TopicOperatorConfig config; private final BatchingLoop queue; @@ -28,7 +27,7 @@ class TopicOperatorEventHandler implements ResourceEventHandler { private long lastPeriodicTimestampMs; - public TopicOperatorEventHandler(TopicOperatorConfig config, BatchingLoop queue, MetricsHolder metrics) { + public TopicEventHandler(TopicOperatorConfig config, BatchingLoop queue, MetricsHolder metrics) { this.config = config; this.queue = queue; this.metrics = metrics; @@ -38,7 +37,7 @@ public TopicOperatorEventHandler(TopicOperatorConfig config, BatchingLoop queue, public void onAdd(KafkaTopic obj) { LOGGER.debugOp("Informed about add event for topic {}", TopicOperatorUtil.topicName(obj)); metrics.resourceCounter(config.namespace()).incrementAndGet(); - if (isReconciliationPausedWithAnnotation(obj)) { + if (Annotations.isReconciliationPausedWithAnnotation(obj)) { metrics.pausedResourceCounter(config.namespace()).incrementAndGet(); } queue.offer(new TopicUpsert(System.nanoTime(), obj.getMetadata().getNamespace(), @@ -49,16 +48,16 @@ public void onAdd(KafkaTopic obj) { @Override public void onUpdate(KafkaTopic oldObj, KafkaTopic newObj) { String trigger = Objects.equals(oldObj, newObj) ? "resync" : "update"; - if (trigger.equals("resync") && (NANOSECONDS.toMillis(System.nanoTime()) - lastPeriodicTimestampMs) > config.fullReconciliationIntervalMs()) { + if (trigger.equals("resync") && (TimeUnit.NANOSECONDS.toMillis(System.nanoTime()) - lastPeriodicTimestampMs) > config.fullReconciliationIntervalMs()) { LOGGER.infoOp("Triggering periodic reconciliation of {} resources for namespace {}", KafkaTopic.RESOURCE_KIND, config.namespace()); - this.lastPeriodicTimestampMs = NANOSECONDS.toMillis(System.nanoTime()); + this.lastPeriodicTimestampMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime()); } if (trigger.equals("update")) { LOGGER.debugOp("Informed about update event for topic {}", TopicOperatorUtil.topicName(newObj)); } - if (isReconciliationPausedWithAnnotation(oldObj) && !isReconciliationPausedWithAnnotation(newObj)) { + if (Annotations.isReconciliationPausedWithAnnotation(oldObj) && !Annotations.isReconciliationPausedWithAnnotation(newObj)) { metrics.pausedResourceCounter(config.namespace()).decrementAndGet(); - } else if (!isReconciliationPausedWithAnnotation(oldObj) && isReconciliationPausedWithAnnotation(newObj)) { + } else if (!Annotations.isReconciliationPausedWithAnnotation(oldObj) && Annotations.isReconciliationPausedWithAnnotation(newObj)) { metrics.pausedResourceCounter(config.namespace()).incrementAndGet(); } queue.offer(new TopicUpsert(System.nanoTime(), newObj.getMetadata().getNamespace(), @@ -70,7 +69,7 @@ public void onUpdate(KafkaTopic oldObj, KafkaTopic newObj) { public void onDelete(KafkaTopic obj, boolean deletedFinalStateUnknown) { LOGGER.debugOp("Informed about delete event for topic {}", TopicOperatorUtil.topicName(obj)); metrics.resourceCounter(config.namespace()).decrementAndGet(); - if (isReconciliationPausedWithAnnotation(obj)) { + if (Annotations.isReconciliationPausedWithAnnotation(obj)) { metrics.pausedResourceCounter(config.namespace()).decrementAndGet(); } if (config.useFinalizer()) { diff --git a/topic-operator/src/main/java/io/strimzi/operator/topic/TopicOperatorConfig.java b/topic-operator/src/main/java/io/strimzi/operator/topic/TopicOperatorConfig.java index 69eef502ffe..14c2dd49b6d 100644 --- a/topic-operator/src/main/java/io/strimzi/operator/topic/TopicOperatorConfig.java +++ b/topic-operator/src/main/java/io/strimzi/operator/topic/TopicOperatorConfig.java @@ -4,12 +4,14 @@ */ package io.strimzi.operator.topic; +import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import io.strimzi.operator.common.InvalidConfigurationException; import io.strimzi.operator.common.ReconciliationLogger; import io.strimzi.operator.common.config.ConfigParameter; +import io.strimzi.operator.common.config.ConfigParameterParser; import io.strimzi.operator.common.featuregates.FeatureGates; import io.strimzi.operator.common.model.Labels; import io.strimzi.operator.common.model.cruisecontrol.CruiseControlApiProperties; @@ -23,16 +25,6 @@ import java.util.Set; import java.util.UUID; -import static com.fasterxml.jackson.core.JsonParser.Feature.ALLOW_COMMENTS; -import static io.strimzi.operator.common.config.ConfigParameterParser.BOOLEAN; -import static io.strimzi.operator.common.config.ConfigParameterParser.INTEGER; -import static io.strimzi.operator.common.config.ConfigParameterParser.LABEL_PREDICATE; -import static io.strimzi.operator.common.config.ConfigParameterParser.LONG; -import static io.strimzi.operator.common.config.ConfigParameterParser.NON_EMPTY_STRING; -import static io.strimzi.operator.common.config.ConfigParameterParser.STRING; -import static io.strimzi.operator.common.config.ConfigParameterParser.parseFeatureGates; -import static io.strimzi.operator.common.config.ConfigParameterParser.strictlyPositive; - /** * Topic Operator configuration. * @@ -108,45 +100,77 @@ public record TopicOperatorConfig( boolean skipClusterConfigReview ) { private final static ReconciliationLogger LOGGER = ReconciliationLogger.create(TopicOperatorConfig.class); - private static final Map> CONFIG_VALUES = new HashMap<>(); - private static final TypeReference> STRING_HASH_MAP_TYPE_REFERENCE = new TypeReference<>() { }; - - static final ConfigParameter NAMESPACE = new ConfigParameter<>("STRIMZI_NAMESPACE", NON_EMPTY_STRING, CONFIG_VALUES); - static final ConfigParameter RESOURCE_LABELS = new ConfigParameter<>("STRIMZI_RESOURCE_LABELS", LABEL_PREDICATE, "", CONFIG_VALUES); - static final ConfigParameter BOOTSTRAP_SERVERS = new ConfigParameter<>("STRIMZI_KAFKA_BOOTSTRAP_SERVERS", NON_EMPTY_STRING, CONFIG_VALUES); - static final ConfigParameter CLIENT_ID = new ConfigParameter<>("STRIMZI_CLIENT_ID", NON_EMPTY_STRING, "strimzi-topic-operator-" + UUID.randomUUID(), CONFIG_VALUES); - static final ConfigParameter FULL_RECONCILIATION_INTERVAL_MS = new ConfigParameter<>("STRIMZI_FULL_RECONCILIATION_INTERVAL_MS", strictlyPositive(LONG), "120000", CONFIG_VALUES); - static final ConfigParameter TLS_ENABLED = new ConfigParameter<>("STRIMZI_TLS_ENABLED", BOOLEAN, "false", CONFIG_VALUES); - static final ConfigParameter TRUSTSTORE_LOCATION = new ConfigParameter<>("STRIMZI_TRUSTSTORE_LOCATION", STRING, "", CONFIG_VALUES); - static final ConfigParameter TRUSTSTORE_PASSWORD = new ConfigParameter<>("STRIMZI_TRUSTSTORE_PASSWORD", STRING, "", CONFIG_VALUES); - static final ConfigParameter KEYSTORE_LOCATION = new ConfigParameter<>("STRIMZI_KEYSTORE_LOCATION", STRING, "", CONFIG_VALUES); - static final ConfigParameter KEYSTORE_PASSWORD = new ConfigParameter<>("STRIMZI_KEYSTORE_PASSWORD", STRING, "", CONFIG_VALUES); - static final ConfigParameter SSL_ENDPOINT_IDENTIFICATION_ALGORITHM = new ConfigParameter<>("STRIMZI_SSL_ENDPOINT_IDENTIFICATION_ALGORITHM", STRING, "HTTPS", CONFIG_VALUES); - static final ConfigParameter SASL_ENABLED = new ConfigParameter<>("STRIMZI_SASL_ENABLED", BOOLEAN, "false", CONFIG_VALUES); - static final ConfigParameter SASL_MECHANISM = new ConfigParameter<>("STRIMZI_SASL_MECHANISM", STRING, "", CONFIG_VALUES); - static final ConfigParameter SASL_CUSTOM_CONFIG_JSON = new ConfigParameter<>("STRIMZI_SASL_CUSTOM_CONFIG_JSON", STRING, "", CONFIG_VALUES); - static final ConfigParameter SASL_USERNAME = new ConfigParameter<>("STRIMZI_SASL_USERNAME", STRING, "", CONFIG_VALUES); - static final ConfigParameter SASL_PASSWORD = new ConfigParameter<>("STRIMZI_SASL_PASSWORD", STRING, "", CONFIG_VALUES); - static final ConfigParameter SECURITY_PROTOCOL = new ConfigParameter<>("STRIMZI_SECURITY_PROTOCOL", STRING, "", CONFIG_VALUES); - static final ConfigParameter USE_FINALIZERS = new ConfigParameter<>("STRIMZI_USE_FINALIZERS", BOOLEAN, "true", CONFIG_VALUES); - static final ConfigParameter MAX_QUEUE_SIZE = new ConfigParameter<>("STRIMZI_MAX_QUEUE_SIZE", strictlyPositive(INTEGER), "1024", CONFIG_VALUES); - static final ConfigParameter MAX_BATCH_SIZE = new ConfigParameter<>("STRIMZI_MAX_BATCH_SIZE", strictlyPositive(INTEGER), "100", CONFIG_VALUES); - static final ConfigParameter MAX_BATCH_LINGER_MS = new ConfigParameter<>("STRIMZI_MAX_BATCH_LINGER_MS", strictlyPositive(LONG), "100", CONFIG_VALUES); - static final ConfigParameter ENABLE_ADDITIONAL_METRICS = new ConfigParameter<>("STRIMZI_ENABLE_ADDITIONAL_METRICS", BOOLEAN, "false", CONFIG_VALUES); - static final ConfigParameter ALTERABLE_TOPIC_CONFIG = new ConfigParameter<>("STRIMZI_ALTERABLE_TOPIC_CONFIG", STRING, "ALL", CONFIG_VALUES); - static final ConfigParameter SKIP_CLUSTER_CONFIG_REVIEW = new ConfigParameter<>("STRIMZI_SKIP_CLUSTER_CONFIG_REVIEW", BOOLEAN, "false", CONFIG_VALUES); - static final ConfigParameter FEATURE_GATES = new ConfigParameter<>("STRIMZI_FEATURE_GATES", parseFeatureGates(), "", CONFIG_VALUES); - static final ConfigParameter CRUISE_CONTROL_ENABLED = new ConfigParameter<>("STRIMZI_CRUISE_CONTROL_ENABLED", BOOLEAN, "false", CONFIG_VALUES); - static final ConfigParameter CRUISE_CONTROL_RACK_ENABLED = new ConfigParameter<>("STRIMZI_CRUISE_CONTROL_RACK_ENABLED", BOOLEAN, "false", CONFIG_VALUES); - static final ConfigParameter CRUISE_CONTROL_HOSTNAME = new ConfigParameter<>("STRIMZI_CRUISE_CONTROL_HOSTNAME", STRING, "127.0.0.1", CONFIG_VALUES); - static final ConfigParameter CRUISE_CONTROL_PORT = new ConfigParameter<>("STRIMZI_CRUISE_CONTROL_PORT", strictlyPositive(INTEGER), "9090", CONFIG_VALUES); - static final ConfigParameter CRUISE_CONTROL_SSL_ENABLED = new ConfigParameter<>("STRIMZI_CRUISE_CONTROL_SSL_ENABLED", BOOLEAN, "false", CONFIG_VALUES); - static final ConfigParameter CRUISE_CONTROL_AUTH_ENABLED = new ConfigParameter<>("STRIMZI_CRUISE_CONTROL_AUTH_ENABLED", BOOLEAN, "false", CONFIG_VALUES); - static final ConfigParameter CRUISE_CONTROL_CRT_FILE_PATH = new ConfigParameter<>("STRIMZI_CRUISE_CONTROL_CRT_FILE_PATH", STRING, "/etc/tls-sidecar/cluster-ca-certs/ca.crt", CONFIG_VALUES); - static final ConfigParameter CRUISE_CONTROL_API_USER_PATH = new ConfigParameter<>("STRIMZI_CRUISE_CONTROL_API_USER_PATH", STRING, "/etc/eto-cc-api/" + CruiseControlApiProperties.TOPIC_OPERATOR_USERNAME_KEY, CONFIG_VALUES); - static final ConfigParameter CRUISE_CONTROL_API_PASS_PATH = new ConfigParameter<>("STRIMZI_CRUISE_CONTROL_API_PASS_PATH", STRING, "/etc/eto-cc-api/" + CruiseControlApiProperties.TOPIC_OPERATOR_PASSWORD_KEY, CONFIG_VALUES); + + /** Namespace in which the operator will run and create resources. */ + public static final ConfigParameter NAMESPACE = new ConfigParameter<>("STRIMZI_NAMESPACE", ConfigParameterParser.NON_EMPTY_STRING, CONFIG_VALUES); + /** Labels used to filter the custom resources seen by the cluster operator. */ + public static final ConfigParameter RESOURCE_LABELS = new ConfigParameter<>("STRIMZI_RESOURCE_LABELS", ConfigParameterParser.LABEL_PREDICATE, "", CONFIG_VALUES); + /** Kafka bootstrap address for the target Kafka cluster used by the internal admin client. */ + public static final ConfigParameter BOOTSTRAP_SERVERS = new ConfigParameter<>("STRIMZI_KAFKA_BOOTSTRAP_SERVERS", ConfigParameterParser.NON_EMPTY_STRING, CONFIG_VALUES); + /** Kafka client ID used by the internal admin client. */ + public static final ConfigParameter CLIENT_ID = new ConfigParameter<>("STRIMZI_CLIENT_ID", ConfigParameterParser.NON_EMPTY_STRING, "strimzi-topic-operator-" + UUID.randomUUID(), CONFIG_VALUES); + /** Periodic reconciliation interval in milliseconds. */ + public static final ConfigParameter FULL_RECONCILIATION_INTERVAL_MS = new ConfigParameter<>("STRIMZI_FULL_RECONCILIATION_INTERVAL_MS", ConfigParameterParser.strictlyPositive(ConfigParameterParser.LONG), "120000", CONFIG_VALUES); + /** TLS: whether to enable configuration. */ + public static final ConfigParameter TLS_ENABLED = new ConfigParameter<>("STRIMZI_TLS_ENABLED", ConfigParameterParser.BOOLEAN, "false", CONFIG_VALUES); + /** TLS: truststore location. */ + public static final ConfigParameter TRUSTSTORE_LOCATION = new ConfigParameter<>("STRIMZI_TRUSTSTORE_LOCATION", ConfigParameterParser.STRING, "", CONFIG_VALUES); + /** TLS: truststore password. */ + public static final ConfigParameter TRUSTSTORE_PASSWORD = new ConfigParameter<>("STRIMZI_TRUSTSTORE_PASSWORD", ConfigParameterParser.STRING, "", CONFIG_VALUES); + /** TLS: keystore location. */ + public static final ConfigParameter KEYSTORE_LOCATION = new ConfigParameter<>("STRIMZI_KEYSTORE_LOCATION", ConfigParameterParser.STRING, "", CONFIG_VALUES); + /** TLS: keystore location. */ + public static final ConfigParameter KEYSTORE_PASSWORD = new ConfigParameter<>("STRIMZI_KEYSTORE_PASSWORD", ConfigParameterParser.STRING, "", CONFIG_VALUES); + /** TLS: endpoint identification algorithm. */ + public static final ConfigParameter SSL_ENDPOINT_IDENTIFICATION_ALGORITHM = new ConfigParameter<>("STRIMZI_SSL_ENDPOINT_IDENTIFICATION_ALGORITHM", ConfigParameterParser.STRING, "HTTPS", CONFIG_VALUES); + /** SASL: whether to enable configuration. */ + public static final ConfigParameter SASL_ENABLED = new ConfigParameter<>("STRIMZI_SASL_ENABLED", ConfigParameterParser.BOOLEAN, "false", CONFIG_VALUES); + /** SASL: mechanism. */ + public static final ConfigParameter SASL_MECHANISM = new ConfigParameter<>("STRIMZI_SASL_MECHANISM", ConfigParameterParser.STRING, "", CONFIG_VALUES); + /** SASL: custom configuration. */ + public static final ConfigParameter SASL_CUSTOM_CONFIG_JSON = new ConfigParameter<>("STRIMZI_SASL_CUSTOM_CONFIG_JSON", ConfigParameterParser.STRING, "", CONFIG_VALUES); + /** SASL: username. */ + public static final ConfigParameter SASL_USERNAME = new ConfigParameter<>("STRIMZI_SASL_USERNAME", ConfigParameterParser.STRING, "", CONFIG_VALUES); + /** SASL: password. */ + public static final ConfigParameter SASL_PASSWORD = new ConfigParameter<>("STRIMZI_SASL_PASSWORD", ConfigParameterParser.STRING, "", CONFIG_VALUES); + /** SASL: security protocol. */ + public static final ConfigParameter SECURITY_PROTOCOL = new ConfigParameter<>("STRIMZI_SECURITY_PROTOCOL", ConfigParameterParser.STRING, "", CONFIG_VALUES); + /** Whether to use finalizers for KafkaTopic resources. */ + public static final ConfigParameter USE_FINALIZERS = new ConfigParameter<>("STRIMZI_USE_FINALIZERS", ConfigParameterParser.BOOLEAN, "true", CONFIG_VALUES); + /** Max topic event queue size. */ + public static final ConfigParameter MAX_QUEUE_SIZE = new ConfigParameter<>("STRIMZI_MAX_QUEUE_SIZE", ConfigParameterParser.strictlyPositive(ConfigParameterParser.INTEGER), "1024", CONFIG_VALUES); + /** Max size of a topic event batch. */ + public static final ConfigParameter MAX_BATCH_SIZE = new ConfigParameter<>("STRIMZI_MAX_BATCH_SIZE", ConfigParameterParser.strictlyPositive(ConfigParameterParser.INTEGER), "100", CONFIG_VALUES); + /** Max linger time in milliseconds before creating a new topic event batch. */ + public static final ConfigParameter MAX_BATCH_LINGER_MS = new ConfigParameter<>("STRIMZI_MAX_BATCH_LINGER_MS", ConfigParameterParser.strictlyPositive(ConfigParameterParser.LONG), "100", CONFIG_VALUES); + /** Whether to enable additional metrics related to requests to external services (Kafka, Kubernetes, Cruise Control). */ + public static final ConfigParameter ENABLE_ADDITIONAL_METRICS = new ConfigParameter<>("STRIMZI_ENABLE_ADDITIONAL_METRICS", ConfigParameterParser.BOOLEAN, "false", CONFIG_VALUES); + /** An allow list of topic configurations that are reconciles, everything else is ignored. */ + public static final ConfigParameter ALTERABLE_TOPIC_CONFIG = new ConfigParameter<>("STRIMZI_ALTERABLE_TOPIC_CONFIG", ConfigParameterParser.STRING, "ALL", CONFIG_VALUES); + /** Skip cluster level configuration checks. */ + public static final ConfigParameter SKIP_CLUSTER_CONFIG_REVIEW = new ConfigParameter<>("STRIMZI_SKIP_CLUSTER_CONFIG_REVIEW", ConfigParameterParser.BOOLEAN, "false", CONFIG_VALUES); + /** List of enabled and disabled feature gates. */ + public static final ConfigParameter FEATURE_GATES = new ConfigParameter<>("STRIMZI_FEATURE_GATES", ConfigParameterParser.parseFeatureGates(), "", CONFIG_VALUES); + /** Cruise Control: whether to enable configuration. */ + public static final ConfigParameter CRUISE_CONTROL_ENABLED = new ConfigParameter<>("STRIMZI_CRUISE_CONTROL_ENABLED", ConfigParameterParser.BOOLEAN, "false", CONFIG_VALUES); + /** Cruise Control: whether rack awareness is enabled. */ + public static final ConfigParameter CRUISE_CONTROL_RACK_ENABLED = new ConfigParameter<>("STRIMZI_CRUISE_CONTROL_RACK_ENABLED", ConfigParameterParser.BOOLEAN, "false", CONFIG_VALUES); + /** Cruise Control: server hostname. */ + public static final ConfigParameter CRUISE_CONTROL_HOSTNAME = new ConfigParameter<>("STRIMZI_CRUISE_CONTROL_HOSTNAME", ConfigParameterParser.STRING, "127.0.0.1", CONFIG_VALUES); + /** Cruise Control: server port. */ + public static final ConfigParameter CRUISE_CONTROL_PORT = new ConfigParameter<>("STRIMZI_CRUISE_CONTROL_PORT", ConfigParameterParser.strictlyPositive(ConfigParameterParser.INTEGER), "9090", CONFIG_VALUES); + /** Cruise Control: whether rack awareness is enabled. */ + public static final ConfigParameter CRUISE_CONTROL_SSL_ENABLED = new ConfigParameter<>("STRIMZI_CRUISE_CONTROL_SSL_ENABLED", ConfigParameterParser.BOOLEAN, "false", CONFIG_VALUES); + /** Cruise Control: whether authentication is enabled. */ + public static final ConfigParameter CRUISE_CONTROL_AUTH_ENABLED = new ConfigParameter<>("STRIMZI_CRUISE_CONTROL_AUTH_ENABLED", ConfigParameterParser.BOOLEAN, "false", CONFIG_VALUES); + /** Cruise Control: CA certificate file location. */ + public static final ConfigParameter CRUISE_CONTROL_CRT_FILE_PATH = new ConfigParameter<>("STRIMZI_CRUISE_CONTROL_CRT_FILE_PATH", ConfigParameterParser.STRING, "/etc/tls-sidecar/cluster-ca-certs/ca.crt", CONFIG_VALUES); + /** Cruise Control: username file location. */ + public static final ConfigParameter CRUISE_CONTROL_API_USER_PATH = new ConfigParameter<>("STRIMZI_CRUISE_CONTROL_API_USER_PATH", ConfigParameterParser.STRING, "/etc/eto-cc-api/" + CruiseControlApiProperties.TOPIC_OPERATOR_USERNAME_KEY, CONFIG_VALUES); + /** Cruise Control: password file location. */ + public static final ConfigParameter CRUISE_CONTROL_API_PASS_PATH = new ConfigParameter<>("STRIMZI_CRUISE_CONTROL_API_PASS_PATH", ConfigParameterParser.STRING, "/etc/eto-cc-api/" + CruiseControlApiProperties.TOPIC_OPERATOR_PASSWORD_KEY, CONFIG_VALUES); @SuppressWarnings("unchecked") private static T get(Map map, ConfigParameter value) { @@ -284,7 +308,7 @@ private void setCustomSaslConfigs(Map kafkaClientProps) { } ObjectMapper objectMapper = new ObjectMapper(); - objectMapper.configure(ALLOW_COMMENTS, true); + objectMapper.configure(JsonParser.Feature.ALLOW_COMMENTS, true); try { Map customProperties = objectMapper.readValue(customPropsString, STRING_HASH_MAP_TYPE_REFERENCE); diff --git a/topic-operator/src/main/java/io/strimzi/operator/topic/TopicOperatorMain.java b/topic-operator/src/main/java/io/strimzi/operator/topic/TopicOperatorMain.java index 2ada1742d46..3883aeb4cc7 100644 --- a/topic-operator/src/main/java/io/strimzi/operator/topic/TopicOperatorMain.java +++ b/topic-operator/src/main/java/io/strimzi/operator/topic/TopicOperatorMain.java @@ -25,11 +25,12 @@ import io.strimzi.operator.common.http.Liveness; import io.strimzi.operator.common.http.Readiness; import io.strimzi.operator.common.model.Labels; +import io.strimzi.operator.topic.cruisecontrol.CruiseControlClient; +import io.strimzi.operator.topic.cruisecontrol.CruiseControlHandler; import io.strimzi.operator.topic.metrics.TopicOperatorMetricsHolder; import io.strimzi.operator.topic.metrics.TopicOperatorMetricsProvider; import org.apache.kafka.clients.admin.Admin; -import java.util.Map; import java.util.Objects; import java.util.concurrent.ExecutionException; @@ -39,39 +40,41 @@ public class TopicOperatorMain implements Liveness, Readiness { private final static ReconciliationLogger LOGGER = ReconciliationLogger.create(TopicOperatorMain.class); private final static long INFORMER_PERIOD_MS = 2_000; - - private final String namespace; - private final KubernetesClient kubeClient; + + private final TopicOperatorConfig config; + private final KubernetesClient kubernetesClient; + private final Admin kafkaAdminClient; + private final CruiseControlClient cruiseControlClient; + /* test */ final BatchingLoop queue; - private final long resyncIntervalMs; private final BasicItemStore itemStore; - private final ReplicasChangeHandler replicasChangeHandler; /* test */ final BatchingTopicController controller; - private final Admin admin; + private SharedIndexInformer informer; // guarded by this Thread shutdownHook; // guarded by this private final ResourceEventHandler resourceEventHandler; private final HealthCheckAndMetricsServer healthAndMetricsServer; - TopicOperatorMain(String namespace, - Map selector, - Admin admin, - KubernetesClient kubeClient, - TopicOperatorConfig config) { - Objects.requireNonNull(namespace); - Objects.requireNonNull(selector); - this.namespace = namespace; - this.kubeClient = kubeClient; - this.resyncIntervalMs = config.fullReconciliationIntervalMs(); - this.admin = admin; - TopicOperatorMetricsProvider metricsProvider = createMetricsProvider(); - TopicOperatorMetricsHolder metrics = new TopicOperatorMetricsHolder(KafkaTopic.RESOURCE_KIND, Labels.fromMap(selector), metricsProvider); - this.replicasChangeHandler = new ReplicasChangeHandler(config, metrics); - this.controller = new BatchingTopicController(config, selector, admin, kubeClient, metrics, replicasChangeHandler); + TopicOperatorMain(TopicOperatorConfig config, Admin kafkaAdminClient) { + Objects.requireNonNull(config.namespace()); + Objects.requireNonNull(config.labelSelector()); + this.config = config; + var selector = config.labelSelector().toMap(); + this.kubernetesClient = TopicOperatorUtil.createKubernetesClient("main"); + this.kafkaAdminClient = kafkaAdminClient; + this.cruiseControlClient = TopicOperatorUtil.createCruiseControlClient(config); + + var metricsProvider = createMetricsProvider(); + var metricsHolder = new TopicOperatorMetricsHolder(KafkaTopic.RESOURCE_KIND, Labels.fromMap(selector), metricsProvider); + var kubeHandler = new KubernetesHandler(config, metricsHolder, kubernetesClient); + var kafkaHandler = new KafkaHandler(config, metricsHolder, kafkaAdminClient); + var cruiseControlHandler = new CruiseControlHandler(config, metricsHolder, cruiseControlClient); + + this.controller = new BatchingTopicController(config, selector, kubeHandler, kafkaHandler, metricsHolder, cruiseControlHandler); this.itemStore = new BasicItemStore<>(Cache::metaNamespaceKeyFunc); - this.queue = new BatchingLoop(config.maxQueueSize(), controller, 1, config.maxBatchSize(), config.maxBatchLingerMs(), itemStore, this::stop, metrics, namespace); - this.resourceEventHandler = new TopicOperatorEventHandler(config, queue, metrics); + this.queue = new BatchingLoop(config, controller, 1, itemStore, this::stop, metricsHolder); + this.resourceEventHandler = new TopicEventHandler(config, queue, metricsHolder); this.healthAndMetricsServer = new HealthCheckAndMetricsServer(8080, this, this, metricsProvider); } @@ -88,8 +91,8 @@ synchronized void start() { healthAndMetricsServer.start(); LOGGER.infoOp("Starting queue"); queue.start(); - informer = Crds.topicOperation(kubeClient) - .inNamespace(namespace) + informer = Crds.topicOperation(kubernetesClient) + .inNamespace(config.namespace()) // Do NOT use withLabels to filter the informer, since the controller is stateful // (topics need to be added to removed from TopicController.topics if KafkaTopics transition between // selected and unselected). @@ -99,7 +102,7 @@ synchronized void start() { // is that the handler skips one informer intervals. Setting both intervals to the same value generates // just enough skew that when the informer checks if the handler is ready for resync it sees that // it still needs another couple of micro-seconds and skips to the next informer level resync. - .addEventHandlerWithResyncPeriod(resourceEventHandler, resyncIntervalMs + INFORMER_PERIOD_MS) + .addEventHandlerWithResyncPeriod(resourceEventHandler, config.fullReconciliationIntervalMs() + INFORMER_PERIOD_MS) .itemStore(itemStore); LOGGER.infoOp("Starting informer"); informer.run(); @@ -111,16 +114,16 @@ synchronized void stop() { throw new IllegalStateException(); } // shutdown(), will be be invoked indirectly by calling - // h.run() has the side effect of nullifying this.shutdownHook + // hook.run() has the side effect of nullifying this.shutdownHook // so retain a reference now so we have something to call // removeShutdownHook() with. - var h = shutdownHook; + var hook = shutdownHook; // Call run (not start()) on the thread so that shutdown() is executed // on this thread. shutdown(); // stop() is _not_ called from the shutdown hook, so calling // removeShutdownHook() should not cause IAE. - Runtime.getRuntime().removeShutdownHook(h); + Runtime.getRuntime().removeShutdownHook(hook); } private synchronized void shutdown() { @@ -133,11 +136,10 @@ private synchronized void shutdown() { informer.stop(); informer = null; } - if (replicasChangeHandler != null) { - replicasChangeHandler.stop(); - } this.queue.stop(); - this.admin.close(); + this.kafkaAdminClient.close(); + this.kubernetesClient.close(); + this.cruiseControlClient.close(); this.healthAndMetricsServer.stop(); LOGGER.infoOp("Shutdown completed normally"); } catch (InterruptedException e) { @@ -147,19 +149,17 @@ private synchronized void shutdown() { } /** - * Entrypoint. - * * @param args Command line args. * @throws Exception If bad things happen. */ public static void main(String[] args) throws Exception { - TopicOperatorConfig topicOperatorConfig = TopicOperatorConfig.buildFromMap(System.getenv()); - TopicOperatorMain operator = operator(topicOperatorConfig, kubeClient(), Admin.create(topicOperatorConfig.adminClientConfig())); + var config = TopicOperatorConfig.buildFromMap(System.getenv()); + var operator = operator(config, TopicOperatorUtil.createKafkaAdminClient(config)); operator.start(); } - static TopicOperatorMain operator(TopicOperatorConfig topicOperatorConfig, KubernetesClient client, Admin admin) throws ExecutionException, InterruptedException { - return new TopicOperatorMain(topicOperatorConfig.namespace(), topicOperatorConfig.labelSelector().toMap(), admin, client, topicOperatorConfig); + static TopicOperatorMain operator(TopicOperatorConfig config, Admin kafkaAdmin) throws ExecutionException, InterruptedException { + return new TopicOperatorMain(config, kafkaAdmin); } static KubernetesClient kubeClient() { @@ -201,7 +201,7 @@ public boolean isReady() { * Creates the MetricsProvider instance based on a PrometheusMeterRegistry * and binds the JVM metrics to it. * - * @return MetricsProvider instance + * @return MetricsProvider instance. */ private static TopicOperatorMetricsProvider createMetricsProvider() { MeterRegistry registry = new PrometheusMeterRegistry(PrometheusConfig.DEFAULT); diff --git a/topic-operator/src/main/java/io/strimzi/operator/topic/TopicOperatorUtil.java b/topic-operator/src/main/java/io/strimzi/operator/topic/TopicOperatorUtil.java index f7b66bcdcd1..254d53d2ad2 100644 --- a/topic-operator/src/main/java/io/strimzi/operator/topic/TopicOperatorUtil.java +++ b/topic-operator/src/main/java/io/strimzi/operator/topic/TopicOperatorUtil.java @@ -7,16 +7,23 @@ import io.fabric8.kubernetes.client.KubernetesClient; import io.micrometer.core.instrument.Timer; import io.strimzi.api.kafka.model.topic.KafkaTopic; -import io.strimzi.api.kafka.model.topic.KafkaTopicStatus; import io.strimzi.operator.common.Annotations; import io.strimzi.operator.common.OperatorKubernetesClientBuilder; +import io.strimzi.operator.common.model.InvalidResourceException; +import io.strimzi.operator.topic.cruisecontrol.CruiseControlClient; import io.strimzi.operator.topic.metrics.TopicOperatorMetricsHolder; import io.strimzi.operator.topic.model.Either; import io.strimzi.operator.topic.model.Pair; import io.strimzi.operator.topic.model.PartitionedByError; import io.strimzi.operator.topic.model.ReconcilableTopic; import io.strimzi.operator.topic.model.TopicOperatorException; +import org.apache.kafka.clients.admin.Admin; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; import java.util.List; import java.util.function.Function; import java.util.stream.Collectors; @@ -45,6 +52,45 @@ public static KubernetesClient createKubernetesClient(String id) { .build(); } + /** + * Create a new Kafka admin client instance. + * + * @param config Topic Operator configuration. + * @return Kafka admin client. + */ + public static Admin createKafkaAdminClient(TopicOperatorConfig config) { + return Admin.create(config.adminClientConfig()); + } + + /** + * Create a new Cruise Control client instance. + * + * @param config Topic Operator configuration. + * @return Cruise Control client. + */ + public static CruiseControlClient createCruiseControlClient(TopicOperatorConfig config) { + return CruiseControlClient.create( + config.cruiseControlHostname(), + config.cruiseControlPort(), + config.cruiseControlRackEnabled(), + config.cruiseControlSslEnabled(), + config.cruiseControlSslEnabled() ? getFileContent(config.cruiseControlCrtFilePath()) : null, + config.cruiseControlAuthEnabled(), + config.cruiseControlAuthEnabled() + ? new String(getFileContent(config.cruiseControlApiUserPath()), StandardCharsets.UTF_8) : null, + config.cruiseControlAuthEnabled() + ? new String(getFileContent(config.cruiseControlApiPassPath()), StandardCharsets.UTF_8) : null + ); + } + + private static byte[] getFileContent(String filePath) { + try { + return Files.readAllBytes(Path.of(filePath)); + } catch (IOException ioe) { + throw new UncheckedIOException(String.format("File not found: %s", filePath), ioe); + } + } + /** * Get the topic name from a {@link KafkaTopic} resource. * @@ -151,13 +197,23 @@ public static boolean isPaused(KafkaTopic kt) { } /** - * Whether the {@link KafkaTopic} status has replicas change. - * - * @param status Topic status. + * Whether the {@link KafkaTopic} status has config. + * + * @param kafkaTopic Kafka topic. + * @return True if there is config. + */ + public static boolean hasConfig(KafkaTopic kafkaTopic) { + return kafkaTopic.getSpec() != null && kafkaTopic.getSpec().getConfig() != null; + } + + /** + * Whether the {@link KafkaTopic} status has replicas change status. + * + * @param kafkaTopic Kafka topic. * @return True if there is replicas change status. */ - public static boolean hasReplicasChange(KafkaTopicStatus status) { - return status != null && status.getReplicasChange() != null; + public static boolean hasReplicasChangeStatus(KafkaTopic kafkaTopic) { + return kafkaTopic.getStatus() != null && kafkaTopic.getStatus().getReplicasChange() != null; } /** @@ -202,4 +258,31 @@ public static int partitions(KafkaTopic kt) { public static short replicas(KafkaTopic kt) { return kt.getSpec().getReplicas() != null ? kt.getSpec().getReplicas().shortValue() : BROKER_DEFAULT; } + + /** + * Get Kafka configuration value as string. + * + * @param key Configuration key. + * @param value Configuration value. + * @return Value as string. + */ + public static String configValueAsString(String key, Object value) { + String valueStr; + if (value == null) { + valueStr = null; + } else if (value instanceof String + || value instanceof Boolean) { + valueStr = value.toString(); + } else if (value instanceof Number) { + valueStr = value.toString(); + } else if (value instanceof List) { + valueStr = ((List) value).stream() + .map(v -> configValueAsString(key, v)) + .collect(Collectors.joining(",")); + } else { + throw new InvalidResourceException( + String.format("Invalid value for topic config '%s': %s", key, value)); + } + return valueStr; + } } diff --git a/topic-operator/src/main/java/io/strimzi/operator/topic/cruisecontrol/CruiseControlClient.java b/topic-operator/src/main/java/io/strimzi/operator/topic/cruisecontrol/CruiseControlClient.java index 3b4fb4ec315..0efc4b2583d 100644 --- a/topic-operator/src/main/java/io/strimzi/operator/topic/cruisecontrol/CruiseControlClient.java +++ b/topic-operator/src/main/java/io/strimzi/operator/topic/cruisecontrol/CruiseControlClient.java @@ -19,7 +19,7 @@ * The server runs one task execution at a time, additional * requests are queued up to {@code max.active.user.tasks}. */ -public interface CruiseControlClient extends AutoCloseable { +public interface CruiseControlClient { /** * HTTP request timeout in seconds. */ @@ -58,6 +58,11 @@ static CruiseControlClient create(String serverHostname, ); } + /** + * Close the client. + */ + void close(); + /** * Send a POST request to {@code topic_configuration} endpoint. * This can be used to request replication factor changes (async operation). diff --git a/topic-operator/src/main/java/io/strimzi/operator/topic/cruisecontrol/CruiseControlClientImpl.java b/topic-operator/src/main/java/io/strimzi/operator/topic/cruisecontrol/CruiseControlClientImpl.java index e96cee52468..081b2036174 100644 --- a/topic-operator/src/main/java/io/strimzi/operator/topic/cruisecontrol/CruiseControlClientImpl.java +++ b/topic-operator/src/main/java/io/strimzi/operator/topic/cruisecontrol/CruiseControlClientImpl.java @@ -85,9 +85,19 @@ public class CruiseControlClientImpl implements CruiseControlClient { } @Override - public void close() throws Exception { - stopExecutor(httpClientExecutor, 10_000); + public void close() { httpClient = null; + if (httpClientExecutor == null) { + return; + } + try { + httpClientExecutor.shutdown(); + httpClientExecutor.awaitTermination(10_000, TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + if (!httpClientExecutor.isTerminated()) { + httpClientExecutor.shutdownNow(); + } + } } @Override @@ -250,18 +260,4 @@ private HttpClient buildHttpClient() { throw new RuntimeException(format("HTTP client build failed: %s", t.getMessage())); } } - - private static void stopExecutor(ExecutorService executor, long timeoutMs) { - if (executor == null || timeoutMs < 0) { - return; - } - try { - executor.shutdown(); - executor.awaitTermination(timeoutMs, TimeUnit.MILLISECONDS); - } catch (InterruptedException e) { - if (!executor.isTerminated()) { - executor.shutdownNow(); - } - } - } } diff --git a/topic-operator/src/main/java/io/strimzi/operator/topic/cruisecontrol/CruiseControlHandler.java b/topic-operator/src/main/java/io/strimzi/operator/topic/cruisecontrol/CruiseControlHandler.java new file mode 100644 index 00000000000..24609cf8964 --- /dev/null +++ b/topic-operator/src/main/java/io/strimzi/operator/topic/cruisecontrol/CruiseControlHandler.java @@ -0,0 +1,216 @@ +/* + * 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.topic.cruisecontrol; + +import io.strimzi.api.kafka.model.topic.ReplicasChangeState; +import io.strimzi.api.kafka.model.topic.ReplicasChangeStatusBuilder; +import io.strimzi.operator.common.ReconciliationLogger; +import io.strimzi.operator.topic.TopicOperatorConfig; +import io.strimzi.operator.topic.TopicOperatorUtil; +import io.strimzi.operator.topic.cruisecontrol.CruiseControlClient.TaskState; +import io.strimzi.operator.topic.metrics.TopicOperatorMetricsHolder; +import io.strimzi.operator.topic.model.ReconcilableTopic; +import io.strimzi.operator.topic.model.Results; +import org.apache.logging.log4j.core.util.Throwables; + +import java.util.HashMap; +import java.util.List; +import java.util.stream.Collectors; + +/** + * Handler for Cruise Control requests. + * + *

When new or pending replicas changes are detected (status.replicasChange.state=pending) the + * {@link #requestPendingChanges(List)} method is called to send a {@code topic_configuration} + * request to Cruise Control. Pending changes are retried in the following reconciliations until + * Cruise Control accept the request. Once the request is accepted, the state moves to ongoing.

+ * + *

When ongoing replicas changes are detected (status.replicasChange.state=ongoing) the + * {@link #requestOngoingChanges(List)} method is called to send a {@code user_tasks} + * request to Cruise Control. Ongoing changes are retried in the following reconciliations until + * Cruise Control replies with completed or completed with error.

+ * + *

Empty state (status.replicasChange == null) means no replicas change. + * In case of error the message is reflected in (status.replicasChange.message).

+ * + *
+ *          /---------------------------------\
+ *         V                                   \
+ *     [empty] ---> [pending] ------------> [ongoing]
+ *                      \                      /
+ *                       \----> [error] <-----/
+ * 
+ */ +public class CruiseControlHandler { + private static final ReconciliationLogger LOGGER = ReconciliationLogger.create(CruiseControlHandler.class); + + private final TopicOperatorConfig config; + private final TopicOperatorMetricsHolder metricsHolder; + private final CruiseControlClient cruiseControlClient; + + /** + * Create a new instance. + * + * @param config Topic Operator configuration. + * @param metricsHolder Metrics holder. + * @param cruiseControlClient Cruise Control client. + */ + public CruiseControlHandler(TopicOperatorConfig config, + TopicOperatorMetricsHolder metricsHolder, + CruiseControlClient cruiseControlClient) { + this.config = config; + this.metricsHolder = metricsHolder; + this.cruiseControlClient = cruiseControlClient; + } + + /** + * Send a topic_configuration request to create a task for replication factor change of one or more topics. + * This should be called when one or more .spec.replicas changes are detected. + * + * @param reconcilableTopics Pending replicas changes. + * @return Results with status updates. + */ + public Results requestPendingChanges(List reconcilableTopics) { + var results = new Results(); + if (reconcilableTopics.isEmpty()) { + return results; + } + results.merge(updateToPending(reconcilableTopics, "Replicas change pending")); + + var timerSample = TopicOperatorUtil.startExternalRequestTimer(metricsHolder, config.enableAdditionalMetrics()); + try { + LOGGER.debugOp("Sending topic configuration request, topics {}", TopicOperatorUtil.topicNames(reconcilableTopics)); + var kafkaTopics = reconcilableTopics.stream().map(ReconcilableTopic::kt).collect(Collectors.toList()); + var userTaskId = cruiseControlClient.topicConfiguration(kafkaTopics); + results.merge(updateToOngoing(reconcilableTopics, "Replicas change ongoing", userTaskId)); + } catch (Throwable t) { + results.merge(updateToFailed(reconcilableTopics, String.format("Replicas change failed, %s", Throwables.getRootCause(t).getMessage()))); + } + TopicOperatorUtil.stopExternalRequestTimer(timerSample, metricsHolder::cruiseControlTopicConfig, config.enableAdditionalMetrics(), config.namespace()); + + return results; + } + + /** + * Send a user_tasks request to check the state of ongoing replication factor changes. + * This should be called periodically to update the active tasks cache and KafkaTopic status. + * + * @param reconcilableTopics Ongoing replicas changes. + * @return Results with status updates. + */ + public Results requestOngoingChanges(List reconcilableTopics) { + var results = new Results(); + if (reconcilableTopics.isEmpty()) { + return results; + } + + var groupByUserTaskId = reconcilableTopics.stream() + .filter(rt -> TopicOperatorUtil.hasReplicasChangeStatus(rt.kt()) && rt.kt().getStatus().getReplicasChange().getSessionId() != null) + .collect(Collectors.groupingBy(rt -> rt.kt().getStatus().getReplicasChange().getSessionId(), HashMap::new, Collectors.toList())); + + var timerSample = TopicOperatorUtil.startExternalRequestTimer(metricsHolder, config.enableAdditionalMetrics()); + try { + LOGGER.debugOp("Sending user tasks request, Tasks {}", groupByUserTaskId.keySet()); + var userTasksResponse = cruiseControlClient.userTasks(groupByUserTaskId.keySet()); + if (userTasksResponse.userTasks().isEmpty()) { + // Cruise Control restarted: reset the state because the tasks queue is not persisted + // this may also happen when the tasks' retention time expires, or the cache becomes full + results.merge(updateToPending(reconcilableTopics, "Task not found, Resetting the state")); + } else { + for (var userTask : userTasksResponse.userTasks()) { + String userTaskId = userTask.userTaskId(); + TaskState state = TaskState.get(userTask.status()); + switch (state) { + case COMPLETED: + results.merge(updateToCompleted(groupByUserTaskId.get(userTaskId), "Replicas change completed")); + break; + case COMPLETED_WITH_ERROR: + results.merge(updateToFailed(groupByUserTaskId.get(userTaskId), "Replicas change completed with error")); + break; + case ACTIVE: + case IN_EXECUTION: + results.merge(updateToOngoing(reconcilableTopics, "Replicas change ongoing", userTaskId)); + break; + } + } + } + } catch (Throwable t) { + results.merge(updateToFailed(reconcilableTopics, String.format("Replicas change failed, %s", Throwables.getRootCause(t).getMessage()))); + } + TopicOperatorUtil.stopExternalRequestTimer(timerSample, metricsHolder::cruiseControlUserTasks, config.enableAdditionalMetrics(), config.namespace()); + + return results; + } + + /** + * Complete zombie changes. + * These are pending but completed replication factor changes caused by Cruise Control restart or user revert. + * + * @param reconcilableTopics Pending but completed replicas changes. + * @return Results with status updates. + */ + public Results completeZombieChanges(List reconcilableTopics) { + return updateToCompleted(reconcilableTopics, "Replicas change completed or reverted"); + } + + private Results updateToPending(List reconcilableTopics, String message) { + var results = new Results(); + LOGGER.infoOp("{}, Topics: {}", message, TopicOperatorUtil.topicNames(reconcilableTopics)); + reconcilableTopics.forEach(reconcilableTopic -> + results.addReplicasChange(reconcilableTopic, + new ReplicasChangeStatusBuilder() + .withState(ReplicasChangeState.PENDING) + .withTargetReplicas(reconcilableTopic.kt().getSpec().getReplicas()) + .build()) + ); + return results; + } + + private Results updateToOngoing(List reconcilableTopics, String message, String userTaskId) { + var results = new Results(); + LOGGER.infoOp("{}, Topics: {}", message, TopicOperatorUtil.topicNames(reconcilableTopics)); + reconcilableTopics.forEach(reconcilableTopic -> { + var targetReplicas = TopicOperatorUtil.hasReplicasChangeStatus(reconcilableTopic.kt()) + ? reconcilableTopic.kt().getStatus().getReplicasChange().getTargetReplicas() + : reconcilableTopic.kt().getSpec().getReplicas(); + results.addReplicasChange(reconcilableTopic, + new ReplicasChangeStatusBuilder() + .withState(ReplicasChangeState.ONGOING) + .withTargetReplicas(targetReplicas) + .withSessionId(userTaskId) + .build()); + }); + return results; + } + + private Results updateToCompleted(List reconcilableTopics, String message) { + var results = new Results(); + LOGGER.infoOp("{}, Topics: {}", message, TopicOperatorUtil.topicNames(reconcilableTopics)); + reconcilableTopics.forEach(reconcilableTopic -> + results.addReplicasChange(reconcilableTopic, null) + ); + return results; + } + + private Results updateToFailed(List reconcilableTopics, String message) { + var results = new Results(); + LOGGER.errorOp("{}, Topics: {}", message, TopicOperatorUtil.topicNames(reconcilableTopics)); + reconcilableTopics.forEach(reconcilableTopic -> { + var state = TopicOperatorUtil.hasReplicasChangeStatus(reconcilableTopic.kt()) + ? reconcilableTopic.kt().getStatus().getReplicasChange().getState() + : ReplicasChangeState.PENDING; + var targetReplicas = TopicOperatorUtil.hasReplicasChangeStatus(reconcilableTopic.kt()) + ? reconcilableTopic.kt().getStatus().getReplicasChange().getTargetReplicas() + : reconcilableTopic.kt().getSpec().getReplicas(); + results.addReplicasChange(reconcilableTopic, + new ReplicasChangeStatusBuilder() + .withState(state) + .withTargetReplicas(targetReplicas) + .withMessage(message) + .build()); + }); + return results; + } +} diff --git a/topic-operator/src/main/java/io/strimzi/operator/topic/cruisecontrol/UrlBuilder.java b/topic-operator/src/main/java/io/strimzi/operator/topic/cruisecontrol/UrlBuilder.java index 1894796bb98..db845703ec2 100644 --- a/topic-operator/src/main/java/io/strimzi/operator/topic/cruisecontrol/UrlBuilder.java +++ b/topic-operator/src/main/java/io/strimzi/operator/topic/cruisecontrol/UrlBuilder.java @@ -11,8 +11,6 @@ import java.net.URI; import java.util.List; -import static java.lang.String.format; - /** * Cruise Control URL builder. */ @@ -29,7 +27,7 @@ class UrlBuilder { * @param ssl Whether SSL should be used. */ public UrlBuilder(String hostname, int port, CruiseControlEndpoints endpoint, boolean ssl) { - uri = format("%s://%s:%d%s?", ssl ? "https" : "http", hostname, port, endpoint); + uri = String.format("%s://%s:%d%s?", ssl ? "https" : "http", hostname, port, endpoint); firstParam = true; } diff --git a/topic-operator/src/main/java/io/strimzi/operator/topic/metrics/TopicOperatorMetricsProvider.java b/topic-operator/src/main/java/io/strimzi/operator/topic/metrics/TopicOperatorMetricsProvider.java index 7682a8341f2..bfc9afdb0f3 100644 --- a/topic-operator/src/main/java/io/strimzi/operator/topic/metrics/TopicOperatorMetricsProvider.java +++ b/topic-operator/src/main/java/io/strimzi/operator/topic/metrics/TopicOperatorMetricsProvider.java @@ -9,7 +9,7 @@ import io.micrometer.core.instrument.Timer; import io.strimzi.operator.common.MicrometerMetricsProvider; -import static java.time.Duration.ofMillis; +import java.time.Duration; /** * Adds fine grained timer to the MicrometerMetricsProvider. @@ -35,13 +35,13 @@ public TopicOperatorMetricsProvider(MeterRegistry metrics) { */ public Timer fineGrainedTimer(String name, String description, Tags tags) { metrics.config().meterFilter(new CustomTimerFilter(name, new double[]{ - ofMillis(10).toNanos(), - ofMillis(20).toNanos(), - ofMillis(50).toNanos(), - ofMillis(100).toNanos(), - ofMillis(500).toNanos(), - ofMillis(1000).toNanos(), - ofMillis(5000).toNanos() + Duration.ofMillis(10).toNanos(), + Duration.ofMillis(20).toNanos(), + Duration.ofMillis(50).toNanos(), + Duration.ofMillis(100).toNanos(), + Duration.ofMillis(500).toNanos(), + Duration.ofMillis(1000).toNanos(), + Duration.ofMillis(5000).toNanos() })); return buildTimer(name, description, tags); } diff --git a/topic-operator/src/main/java/io/strimzi/operator/topic/model/Results.java b/topic-operator/src/main/java/io/strimzi/operator/topic/model/Results.java new file mode 100644 index 00000000000..93094d36b9c --- /dev/null +++ b/topic-operator/src/main/java/io/strimzi/operator/topic/model/Results.java @@ -0,0 +1,201 @@ +/* + * 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.topic.model; + +import io.strimzi.api.kafka.model.common.Condition; +import io.strimzi.api.kafka.model.topic.ReplicasChangeStatus; +import org.apache.kafka.clients.admin.AlterConfigOp; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.BiConsumer; +import java.util.stream.Stream; + +/** + * This class is used to accumulate reconciliation results and associated state. + * It allows methods to return intermediate results, rather than mutating collections passed to them as arguments. + * Results can be merged with other results coming from a different method. + */ +public class Results { + private final Map> results = new HashMap<>(); + private final Map> conditions = new HashMap<>(); + private final Map replicasChanges = new HashMap<>(); + private final List>> configChanges = new ArrayList<>(); + + /** + * Adds given results, except for topics which already have an error recorded. + * + * @param partitionedByError Results partitioned by error. + */ + public void addResults(PartitionedByError partitionedByError) { + addRightResults(partitionedByError.ok()); + addLeftResults(partitionedByError.errors()); + } + + /** + * @param ok Success stream. + */ + public void addRightResults(Stream> ok) { + ok.forEach(pair -> addResult(pair.getKey(), Either.ofRight(null))); + } + + /** + * @param ok Success stream. + */ + public void addRightResults(Collection ok) { + ok.forEach(rt -> addResult(rt, Either.ofRight(null))); + } + + /** + * @param errors Error stream. + */ + public void addLeftResults(Stream> errors) { + errors.forEach(pair -> addResult(pair.getKey(), Either.ofLeft(pair.getValue()))); + } + + private void addResult(ReconcilableTopic key, Either result) { + results.compute(key, (k, v) -> { + if (v == null) { + // use given result if there is no existing result + return result; + } else if (v.isRight()) { + // if the existing result was success use the given result (errors beat successes) + return result; + } else { + // otherwise the existing result must be an error, + // the given result might also be an error, but "first error wins" + return v; + } + }); + } + + /** + * @return Number of reconciliations. + */ + public int size() { + return results.size(); + } + + /** + * @param action Success action. + */ + public void forEachRightResult(BiConsumer action) { + results.forEach((reconcilableTopic, either) -> { + if (either.isRight()) { + action.accept(reconcilableTopic, either.right()); + } + }); + } + + /** + * @param action Error action. + */ + public void forEachLeftResult(BiConsumer action) { + results.forEach((reconcilableTopic, either) -> { + if (!either.isRight()) { + action.accept(reconcilableTopic, either.left()); + } + }); + } + + /** + * Merge two results into a single instance. + * + * @param other Results. + */ + public void merge(Results other) { + for (var entry : other.results.entrySet()) { + addResult(entry.getKey(), entry.getValue()); + } + addConditions(other.getConditions()); + if (other.getReplicasChanges() != null) { + addRightResults(other.getReplicasChanges().keySet()); + addReplicasChanges(other.getReplicasChanges()); + } + if (other.getConfigChanges() != null) { + this.replaceConfigChanges(other.getConfigChanges()); + } + } + + private Map> getConditions() { + return this.conditions; + } + + /** + * @param reconcilableTopic Reconcilable topic. + * @return Status conditions for this reconcilable topic. + */ + public Collection getConditions(ReconcilableTopic reconcilableTopic) { + return this.conditions.getOrDefault(reconcilableTopic, List.of()); + } + + private void addConditions(Map> conditions) { + conditions.forEach(this::addConditions); + } + + /** + * @param reconcilableTopic Reconcilable topic. + * @param conditions Conditions. + */ + public void addConditions(ReconcilableTopic reconcilableTopic, Collection conditions) { + this.conditions.computeIfAbsent(reconcilableTopic, k -> new ArrayList<>()).addAll(conditions); + } + + /** + * @param reconcilableTopic Reconcilable topic. + * @param condition Condition. + */ + public void addCondition(ReconcilableTopic reconcilableTopic, Condition condition) { + this.conditions.computeIfAbsent(reconcilableTopic, k -> new ArrayList<>()).add(condition); + } + + /** + * @return Replicas change statuses. + */ + public Map getReplicasChanges() { + return replicasChanges; + } + + /** + * @param reconcilableTopic Reconcilable topic. + * @return Replicas change status. + */ + public ReplicasChangeStatus getReplicasChange(ReconcilableTopic reconcilableTopic) { + return replicasChanges.get(reconcilableTopic); + } + + /** + * @param replicasChangeStatus Replicas change status. + */ + public void addReplicasChanges(Map replicasChangeStatus) { + replicasChangeStatus.forEach(this::addReplicasChange); + } + + /** + * @param reconcilableTopic Reconcilable topic. + * @param replicasChangeStatus Replicas change status. + */ + public void addReplicasChange(ReconcilableTopic reconcilableTopic, ReplicasChangeStatus replicasChangeStatus) { + this.replicasChanges.put(reconcilableTopic, replicasChangeStatus); + } + + /** + * @return All alter config ops. + */ + public List>> getConfigChanges() { + return this.configChanges; + } + + /** + * @param configChanges Alter config ops. + */ + public void replaceConfigChanges(List>> configChanges) { + this.configChanges.clear(); + this.configChanges.addAll(configChanges); + } +} diff --git a/topic-operator/src/main/java/io/strimzi/operator/topic/model/TopicState.java b/topic-operator/src/main/java/io/strimzi/operator/topic/model/TopicState.java index acd9922e4cc..30593815292 100644 --- a/topic-operator/src/main/java/io/strimzi/operator/topic/model/TopicState.java +++ b/topic-operator/src/main/java/io/strimzi/operator/topic/model/TopicState.java @@ -18,6 +18,13 @@ * @param configs Configurations. */ public record TopicState(TopicDescription description, Config configs) { + /** + * @return The topic id. + */ + public String topicId() { + return description.topicId() != null ? description.topicId().toString() : null; + } + /** * @return The number of partitions. */ @@ -26,7 +33,7 @@ public int numPartitions() { } /** - * @return the unique replication factor for all partitions of this topic, or + * @return The unique replication factor for all partitions of this topic, or * {@link Integer#MIN_VALUE} if there is no unique replication factor. */ public int uniqueReplicationFactor() { diff --git a/topic-operator/src/test/java/io/strimzi/operator/topic/BatchingTopicControllerTest.java b/topic-operator/src/test/java/io/strimzi/operator/topic/BatchingTopicControllerTest.java index 66feec7812f..b3d45c4584e 100644 --- a/topic-operator/src/test/java/io/strimzi/operator/topic/BatchingTopicControllerTest.java +++ b/topic-operator/src/test/java/io/strimzi/operator/topic/BatchingTopicControllerTest.java @@ -13,14 +13,17 @@ import io.strimzi.api.kafka.model.topic.KafkaTopic; import io.strimzi.api.kafka.model.topic.KafkaTopicBuilder; import io.strimzi.api.kafka.model.topic.KafkaTopicStatusBuilder; +import io.strimzi.api.kafka.model.topic.ReplicasChangeState; import io.strimzi.api.kafka.model.topic.ReplicasChangeStatusBuilder; import io.strimzi.operator.common.Reconciliation; +import io.strimzi.operator.topic.cruisecontrol.CruiseControlHandler; import io.strimzi.operator.topic.metrics.TopicOperatorMetricsHolder; import io.strimzi.operator.topic.metrics.TopicOperatorMetricsProvider; import io.strimzi.operator.topic.model.Either; import io.strimzi.operator.topic.model.Pair; import io.strimzi.operator.topic.model.PartitionedByError; import io.strimzi.operator.topic.model.ReconcilableTopic; +import io.strimzi.operator.topic.model.Results; import io.strimzi.operator.topic.model.TopicOperatorException; import io.strimzi.operator.topic.model.TopicState; import io.strimzi.test.mockkube3.MockKube3; @@ -49,7 +52,6 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.NullSource; import org.junit.jupiter.params.provider.ValueSource; import org.mockito.Mockito; @@ -59,9 +61,6 @@ import java.util.Set; import java.util.concurrent.ExecutionException; -import static io.strimzi.api.kafka.model.topic.KafkaTopic.RESOURCE_KIND; -import static io.strimzi.api.kafka.model.topic.ReplicasChangeState.PENDING; -import static io.strimzi.operator.topic.TopicOperatorUtil.topicName; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.is; @@ -74,11 +73,8 @@ import static org.mockito.Mockito.verifyNoInteractions; /** - * This unit test suite is not intended to provide lots of coverage of the {@link BatchingTopicController}, + * This test is not intended to provide lots of coverage of the {@link BatchingTopicController}, * rather it aims to cover some parts that a difficult to test via {@link TopicControllerIT}. - * - * Some examples are internal operations and behaviors, or when you need to test a system - * behavior without starting a slow external service (e.g. Cruise Control). */ @ExtendWith(KafkaClusterExtension.class) class BatchingTopicControllerTest { @@ -123,29 +119,33 @@ private KafkaTopic createKafkaTopic(String name) { } private KafkaTopic createKafkaTopic(String topicName, int replicas) { - var kt = Crds.topicOperation(kubernetesClient).resource(new KafkaTopicBuilder() - .withNewMetadata() - .withName(topicName) - .withNamespace(NAMESPACE) - .addToLabels("key", "VALUE") - .endMetadata() - .withNewSpec() - .withPartitions(2) - .withReplicas(replicas) - .endSpec().build()).create(); - return kt; + return Crds.topicOperation(kubernetesClient).resource(new KafkaTopicBuilder() + .withNewMetadata() + .withName(topicName) + .withNamespace(NAMESPACE) + .addToLabels("key", "VALUE") + .endMetadata() + .withNewSpec() + .withPartitions(2) + .withReplicas(replicas) + .endSpec() + .build()).create(); } - private void assertOnUpdateThrowsInterruptedException(Admin kafkaAdminClient, KafkaTopic kafkaTopic) { + private void assertOnUpdateThrowsInterruptedException(Admin kafkaAdmin, KafkaTopic kafkaTopic) { var config = Mockito.mock(TopicOperatorConfig.class); Mockito.doReturn(NAMESPACE).when(config).namespace(); Mockito.doReturn(true).when(config).useFinalizer(); Mockito.doReturn(false).when(config).enableAdditionalMetrics(); - var metricsHolder = new TopicOperatorMetricsHolder(RESOURCE_KIND, null, new TopicOperatorMetricsProvider(new SimpleMeterRegistry())); - var controller = new BatchingTopicController(config, Map.of("key", "VALUE"), kafkaAdminClient, kubernetesClient, metricsHolder, new ReplicasChangeHandler(config, metricsHolder)); - - List batch = List.of(new ReconcilableTopic(new Reconciliation("test", "KafkaTopic", NAMESPACE, "my-topic"), kafkaTopic, topicName(kafkaTopic))); + var metricsHolder = new TopicOperatorMetricsHolder(KafkaTopic.RESOURCE_KIND, null, new TopicOperatorMetricsProvider(new SimpleMeterRegistry())); + var controller = new BatchingTopicController(config, Map.of("key", "VALUE"), + new KubernetesHandler(config, metricsHolder, kubernetesClient), + new KafkaHandler(config, metricsHolder, kafkaAdmin), metricsHolder, + new CruiseControlHandler(config, metricsHolder, TopicOperatorUtil.createCruiseControlClient(config))); + + var batch = List.of(new ReconcilableTopic( + new Reconciliation("test", "KafkaTopic", NAMESPACE, TopicOperatorUtil.topicName(kafkaTopic)), kafkaTopic, TopicOperatorUtil.topicName(kafkaTopic))); assertThrows(InterruptedException.class, () -> controller.onUpdate(batch)); } @@ -153,46 +153,46 @@ private void assertOnUpdateThrowsInterruptedException(Admin kafkaAdminClient, Ka public void shouldHandleInterruptedExceptionFromDescribeTopics(KafkaCluster cluster) throws ExecutionException, InterruptedException { var topicName = "my-topic"; kafkaAdminClient[0] = Admin.create(Map.of(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.getBootstrapServers())); - var kafkaAdminClientSpy = Mockito.spy(kafkaAdminClient[0]); + var adminSpy = Mockito.spy(kafkaAdminClient[0]); var result = Mockito.mock(DescribeTopicsResult.class); Mockito.doReturn(interruptedFuture()).when(result).allTopicNames(); Mockito.doReturn(Map.of(topicName, interruptedFuture())).when(result).topicNameValues(); - Mockito.doReturn(result).when(kafkaAdminClientSpy).describeTopics(any(Collection.class)); + Mockito.doReturn(result).when(adminSpy).describeTopics(any(Collection.class)); - KafkaTopic kt = createKafkaTopic(topicName); - assertOnUpdateThrowsInterruptedException(kafkaAdminClientSpy, kt); + var kafkaTopic = createKafkaTopic(topicName); + assertOnUpdateThrowsInterruptedException(adminSpy, kafkaTopic); } @Test public void shouldHandleInterruptedExceptionFromDescribeConfigs(KafkaCluster cluster) throws ExecutionException, InterruptedException { var topicName = "my-topic"; kafkaAdminClient[0] = Admin.create(Map.of(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.getBootstrapServers())); - var kafkaAdminClientSpy = Mockito.spy(kafkaAdminClient[0]); + var adminSpy = Mockito.spy(kafkaAdminClient[0]); var result = Mockito.mock(DescribeConfigsResult.class); Mockito.doReturn(interruptedFuture()).when(result).all(); - Mockito.doReturn(Map.of(topicConfigResource(), interruptedFuture())).when(result).values(); - Mockito.doReturn(result).when(kafkaAdminClientSpy).describeConfigs(Mockito.argThat(a -> a.stream().anyMatch(x -> x.type() == ConfigResource.Type.TOPIC))); + Mockito.doReturn(Map.of(topicConfigResource(topicName), interruptedFuture())).when(result).values(); + Mockito.doReturn(result).when(adminSpy).describeConfigs(Mockito.argThat(a -> a.stream().anyMatch(x -> x.type() == ConfigResource.Type.TOPIC))); - KafkaTopic kafkaTopic = createKafkaTopic(topicName); - assertOnUpdateThrowsInterruptedException(kafkaAdminClientSpy, kafkaTopic); + var kafkaTopic = createKafkaTopic(topicName); + assertOnUpdateThrowsInterruptedException(adminSpy, kafkaTopic); } - private static ConfigResource topicConfigResource() { - return new ConfigResource(ConfigResource.Type.TOPIC, "my-topic"); + private static ConfigResource topicConfigResource(String topicName) { + return new ConfigResource(ConfigResource.Type.TOPIC, topicName); } @Test public void shouldHandleInterruptedExceptionFromCreateTopics(KafkaCluster cluster) throws ExecutionException, InterruptedException { var topicName = "my-topic"; kafkaAdminClient[0] = Admin.create(Map.of(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.getBootstrapServers())); - var kafkaAdminClientSpy = Mockito.spy(kafkaAdminClient[0]); + var adminSpy = Mockito.spy(kafkaAdminClient[0]); var result = Mockito.mock(CreateTopicsResult.class); Mockito.doReturn(interruptedFuture()).when(result).all(); Mockito.doReturn(Map.of(topicName, interruptedFuture())).when(result).values(); - Mockito.doReturn(result).when(kafkaAdminClientSpy).createTopics(any()); + Mockito.doReturn(result).when(adminSpy).createTopics(any()); - KafkaTopic kafkaTopic = createKafkaTopic(topicName); - assertOnUpdateThrowsInterruptedException(kafkaAdminClientSpy, kafkaTopic); + var kafkaTopic = createKafkaTopic(topicName); + assertOnUpdateThrowsInterruptedException(adminSpy, kafkaTopic); } @Test @@ -200,30 +200,30 @@ public void shouldHandleInterruptedExceptionFromIncrementalAlterConfigs(KafkaClu var topicName = "my-topic"; kafkaAdminClient[0] = Admin.create(Map.of(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.getBootstrapServers())); kafkaAdminClient[0].createTopics(List.of(new NewTopic(topicName, 1, (short) 1).configs(Map.of(TopicConfig.COMPRESSION_TYPE_CONFIG, "snappy")))).all().get(); - var kafkaAdminClientSpy = Mockito.spy(kafkaAdminClient[0]); + var adminSpy = Mockito.spy(kafkaAdminClient[0]); var result = Mockito.mock(AlterConfigsResult.class); Mockito.doReturn(interruptedFuture()).when(result).all(); - Mockito.doReturn(Map.of(topicConfigResource(), interruptedFuture())).when(result).values(); - Mockito.doReturn(result).when(kafkaAdminClientSpy).incrementalAlterConfigs(any()); + Mockito.doReturn(Map.of(topicConfigResource(topicName), interruptedFuture())).when(result).values(); + Mockito.doReturn(result).when(adminSpy).incrementalAlterConfigs(any()); - KafkaTopic kafkaTopic = createKafkaTopic(topicName); - assertOnUpdateThrowsInterruptedException(kafkaAdminClientSpy, kafkaTopic); + var kafkaTopic = createKafkaTopic(topicName); + assertOnUpdateThrowsInterruptedException(adminSpy, kafkaTopic); } @Test public void shouldHandleInterruptedExceptionFromCreatePartitions(KafkaCluster cluster) throws ExecutionException, InterruptedException { var topicName = "my-topic"; kafkaAdminClient[0] = Admin.create(Map.of(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.getBootstrapServers())); - kafkaAdminClient[0].createTopics(List.of(new NewTopic("my-topic", 1, (short) 1))).all().get(); + kafkaAdminClient[0].createTopics(List.of(new NewTopic(topicName, 1, (short) 1))).all().get(); - var kafkaAdminClientSpy = Mockito.spy(kafkaAdminClient[0]); + var adminSpy = Mockito.spy(kafkaAdminClient[0]); var result = Mockito.mock(CreatePartitionsResult.class); Mockito.doReturn(interruptedFuture()).when(result).all(); Mockito.doReturn(Map.of(topicName, interruptedFuture())).when(result).values(); - Mockito.doReturn(result).when(kafkaAdminClientSpy).createPartitions(any()); + Mockito.doReturn(result).when(adminSpy).createPartitions(any()); - KafkaTopic kafkaTopic = createKafkaTopic(topicName); - assertOnUpdateThrowsInterruptedException(kafkaAdminClientSpy, kafkaTopic); + var kafkaTopic = createKafkaTopic(topicName); + assertOnUpdateThrowsInterruptedException(adminSpy, kafkaTopic); } @Test @@ -234,13 +234,13 @@ public void shouldHandleInterruptedExceptionFromListReassignments( kafkaAdminClient[0] = Admin.create(Map.of(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.getBootstrapServers())); kafkaAdminClient[0].createTopics(List.of(new NewTopic(topicName, 2, (short) 2))).all().get(); - var kafkaAdminClientSpy = Mockito.spy(kafkaAdminClient[0]); + var adminSpy = Mockito.spy(kafkaAdminClient[0]); var result = Mockito.mock(ListPartitionReassignmentsResult.class); Mockito.doReturn(interruptedFuture()).when(result).reassignments(); - Mockito.doReturn(result).when(kafkaAdminClientSpy).listPartitionReassignments(any(Set.class)); + Mockito.doReturn(result).when(adminSpy).listPartitionReassignments(any(Set.class)); - KafkaTopic kafkaTopic = createKafkaTopic(topicName); - assertOnUpdateThrowsInterruptedException(kafkaAdminClientSpy, kafkaTopic); + var kafkaTopic = createKafkaTopic(topicName); + assertOnUpdateThrowsInterruptedException(adminSpy, kafkaTopic); } @Test @@ -248,20 +248,18 @@ public void shouldHandleInterruptedExceptionFromDeleteTopics(KafkaCluster cluste var topicName = "my-topic"; kafkaAdminClient[0] = Admin.create(Map.of(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.getBootstrapServers())); kafkaAdminClient[0].createTopics(List.of(new NewTopic(topicName, 1, (short) 1))).all().get(); - createKafkaTopic(topicName); - Crds.topicOperation(kubernetesClient).inNamespace(NAMESPACE).withName("my-topic").edit(theKt -> - new KafkaTopicBuilder(theKt).editOrNewMetadata().addToFinalizers(BatchingTopicController.FINALIZER).endMetadata().build()); - Crds.topicOperation(kubernetesClient).inNamespace(NAMESPACE).withName("my-topic").delete(); - var withDeletionTimestamp = Crds.topicOperation(kubernetesClient).inNamespace(NAMESPACE).withName("my-topic").get(); + Crds.topicOperation(kubernetesClient).inNamespace(NAMESPACE).withName(topicName).edit(theKt -> + new KafkaTopicBuilder(theKt).editOrNewMetadata().addToFinalizers(KubernetesHandler.FINALIZER_STRIMZI_IO_TO).endMetadata().build()); + Crds.topicOperation(kubernetesClient).inNamespace(NAMESPACE).withName(topicName).delete(); + var withDeletionTimestamp = Crds.topicOperation(kubernetesClient).inNamespace(NAMESPACE).withName(topicName).get(); - var kafkaAdminClientSpy = Mockito.spy(kafkaAdminClient[0]); + var adminSpy = Mockito.spy(kafkaAdminClient[0]); var result = Mockito.mock(DeleteTopicsResult.class); Mockito.doReturn(interruptedFuture()).when(result).all(); Mockito.doReturn(Map.of(topicName, interruptedFuture())).when(result).topicNameValues(); - Mockito.doReturn(result).when(kafkaAdminClientSpy).deleteTopics(any(TopicCollection.TopicNameCollection.class)); - - assertOnUpdateThrowsInterruptedException(kafkaAdminClientSpy, withDeletionTimestamp); + Mockito.doReturn(result).when(adminSpy).deleteTopics(any(TopicCollection.TopicNameCollection.class)); + assertOnUpdateThrowsInterruptedException(adminSpy, withDeletionTimestamp); } // TODO kube client interrupted exceptions @@ -272,7 +270,7 @@ public void replicasChangeShouldBeReconciled(boolean cruiseControlEnabled) { var topicName = "my-topic"; var replicationFactor = 1; - // setup + // setup: .spec.replicas != replicationFactor var config = Mockito.mock(TopicOperatorConfig.class); Mockito.doReturn(NAMESPACE).when(config).namespace(); Mockito.doReturn(true).when(config).useFinalizer(); @@ -285,11 +283,12 @@ public void replicasChangeShouldBeReconciled(boolean cruiseControlEnabled) { var partitionReassignmentResult = Mockito.mock(ListPartitionReassignmentsResult.class); var topicPartition = Mockito.mock(TopicPartition.class); var partitionReassignment = Mockito.mock(PartitionReassignment.class); - Mockito.doReturn(KafkaFuture.completedFuture(Map.of(topicPartition, partitionReassignment))).when(partitionReassignmentResult).reassignments(); + Mockito.doReturn(KafkaFuture.completedFuture(Map.of(topicPartition, partitionReassignment))) + .when(partitionReassignmentResult).reassignments(); - var kafkaAdminClient = Mockito.mock(Admin.class); - Mockito.doReturn(describeClusterResult).when(kafkaAdminClient).describeCluster(); - Mockito.doReturn(partitionReassignmentResult).when(kafkaAdminClient).listPartitionReassignments(any(Set.class)); + var kafkaAdmin = Mockito.mock(Admin.class); + Mockito.doReturn(describeClusterResult).when(kafkaAdmin).describeCluster(); + Mockito.doReturn(partitionReassignmentResult).when(kafkaAdmin).listPartitionReassignments(any(Set.class)); var topicDescription = Mockito.mock(TopicDescription.class); var topicPartitionInfo = Mockito.mock(TopicPartitionInfo.class); @@ -299,7 +298,7 @@ public void replicasChangeShouldBeReconciled(boolean cruiseControlEnabled) { Mockito.doReturn(replicationFactor).when(currentState).uniqueReplicationFactor(); Mockito.doReturn(topicDescription).when(currentState).description(); - var inputKt = new KafkaTopicBuilder() + var kafkaTopic = new KafkaTopicBuilder() .withNewMetadata() .withName(topicName) .withNamespace(NAMESPACE) @@ -310,45 +309,50 @@ public void replicasChangeShouldBeReconciled(boolean cruiseControlEnabled) { .withReplicas(++replicationFactor) .endSpec() .build(); - var inputRt = TopicOperatorTestUtil.reconcilableTopic(inputKt, NAMESPACE); - - var reconcilableTopics = List.of(inputRt); - var currentStatesOrError = new PartitionedByError<>(List.of(new Pair<>(inputRt, Either.ofRight(currentState))), List.of()); + var reconcilableTopic = new ReconcilableTopic( + new Reconciliation("test", KafkaTopic.RESOURCE_KIND, NAMESPACE, topicName), kafkaTopic, topicName); - var outputKt = new KafkaTopicBuilder(inputKt) - .withStatus(new KafkaTopicStatusBuilder() - .withReplicasChange(new ReplicasChangeStatusBuilder() - .withState(PENDING) - .withTargetReplicas(replicationFactor) - .build()) - .build()) - .build(); - var outputRt = TopicOperatorTestUtil.reconcilableTopic(outputKt, NAMESPACE); - - var replicasChangeHandler = Mockito.mock(ReplicasChangeHandler.class); - Mockito.doReturn(List.of(outputRt)).when(replicasChangeHandler).requestPendingChanges(anyList()); - Mockito.doReturn(List.of()).when(replicasChangeHandler).requestOngoingChanges(anyList()); - - // test - var metricsHolder = new TopicOperatorMetricsHolder(RESOURCE_KIND, null, new TopicOperatorMetricsProvider(new SimpleMeterRegistry())); - var controller = new BatchingTopicController(config, Map.of("key", "VALUE"), kafkaAdminClient, kubernetesClient, metricsHolder, replicasChangeHandler); - var results = controller.checkReplicasChanges(reconcilableTopics, currentStatesOrError); + var currentStatesOrError = new PartitionedByError<>( + List.of(new Pair<>(reconcilableTopic, Either.ofRight(currentState))), List.of()); + + var replicaChangeStatus = + new ReplicasChangeStatusBuilder() + .withState(ReplicasChangeState.PENDING) + .withTargetReplicas(replicationFactor) + .build(); + + var pendingResults = new Results(); + pendingResults.addRightResults(List.of(reconcilableTopic)); + pendingResults.addReplicasChange(reconcilableTopic, replicaChangeStatus); + + var cruiseControlHandler = Mockito.mock(CruiseControlHandler.class); + Mockito.doReturn(pendingResults).when(cruiseControlHandler).requestPendingChanges(anyList()); + Mockito.doReturn(new Results()).when(cruiseControlHandler).requestOngoingChanges(anyList()); + Mockito.doReturn(new Results()).when(cruiseControlHandler).completeZombieChanges(anyList()); + // run test + var metricsHolder = new TopicOperatorMetricsHolder(KafkaTopic.RESOURCE_KIND, null, new TopicOperatorMetricsProvider(new SimpleMeterRegistry())); + var controller = new BatchingTopicController(config, Map.of("key", "VALUE"), + new KubernetesHandler(config, metricsHolder, kubernetesClient), + new KafkaHandler(config, metricsHolder, kafkaAdmin), metricsHolder, + cruiseControlHandler); + var results = controller.checkReplicasChanges(currentStatesOrError.ok(), List.of(reconcilableTopic)); + if (cruiseControlEnabled) { - assertThat(results.ok().count(), is(1L)); - assertThat(results.ok().findFirst().get().getKey().kt(), is(outputKt)); + assertThat(results.size(), is(1)); + assertThat(results.getReplicasChanges().get(reconcilableTopic), is(replicaChangeStatus)); } else { - assertThat(results.errors().count(), is(1L)); - assertThat(results.errors().findFirst().get().getValue(), instanceOf(TopicOperatorException.NotSupported.class)); + assertThat(results.size(), is(1)); + results.forEachLeftResult((rt, e) -> assertThat(e, instanceOf(TopicOperatorException.NotSupported.class))); } } @Test public void replicasChangeShouldCompleteWhenSpecIsReverted() { var topicName = "my-topic"; - var replicationFactor = 3; + int replicationFactor = 3; - // setup: pending with error and .spec.replicas == uniqueReplicationFactor + // setup: pending with error and .spec.replicas == replicationFactor var config = Mockito.mock(TopicOperatorConfig.class); Mockito.doReturn(NAMESPACE).when(config).namespace(); Mockito.doReturn(true).when(config).useFinalizer(); @@ -361,11 +365,12 @@ public void replicasChangeShouldCompleteWhenSpecIsReverted() { var partitionReassignmentResult = Mockito.mock(ListPartitionReassignmentsResult.class); var topicPartition = Mockito.mock(TopicPartition.class); var partitionReassignment = Mockito.mock(PartitionReassignment.class); - Mockito.doReturn(KafkaFuture.completedFuture(Map.of(topicPartition, partitionReassignment))).when(partitionReassignmentResult).reassignments(); + Mockito.doReturn(KafkaFuture.completedFuture(Map.of(topicPartition, partitionReassignment))) + .when(partitionReassignmentResult).reassignments(); - var kafkaAdminClient = Mockito.mock(Admin.class); - Mockito.doReturn(describeClusterResult).when(kafkaAdminClient).describeCluster(); - Mockito.doReturn(partitionReassignmentResult).when(kafkaAdminClient).listPartitionReassignments(any(Set.class)); + var kafkaAdmin = Mockito.mock(Admin.class); + Mockito.doReturn(describeClusterResult).when(kafkaAdmin).describeCluster(); + Mockito.doReturn(partitionReassignmentResult).when(kafkaAdmin).listPartitionReassignments(any(Set.class)); var topicDescription = Mockito.mock(TopicDescription.class); var topicPartitionInfo = Mockito.mock(TopicPartitionInfo.class); @@ -388,26 +393,37 @@ public void replicasChangeShouldCompleteWhenSpecIsReverted() { .withStatus(new KafkaTopicStatusBuilder() .withReplicasChange(new ReplicasChangeStatusBuilder() .withMessage("Error message") - .withState(PENDING) + .withState(ReplicasChangeState.PENDING) .withTargetReplicas(replicationFactor) .build()) .build()) .build(); - var replicasChangeHandler = Mockito.mock(ReplicasChangeHandler.class); - Mockito.doReturn(List.of()).when(replicasChangeHandler).requestPendingChanges(anyList()); - Mockito.doReturn(List.of()).when(replicasChangeHandler).requestOngoingChanges(anyList()); + var reconcilableTopic = new ReconcilableTopic( + new Reconciliation("test", KafkaTopic.RESOURCE_KIND, NAMESPACE, topicName), kafkaTopic, topicName); - var reconcilableTopics = List.of(TopicOperatorTestUtil.reconcilableTopic(kafkaTopic, NAMESPACE)); + var completedResults = new Results(); + completedResults.addRightResults(List.of(reconcilableTopic)); + completedResults.addReplicasChange(reconcilableTopic, null); + + var cruiseControlHandler = Mockito.mock(CruiseControlHandler.class); + Mockito.doReturn(new Results()).when(cruiseControlHandler).requestPendingChanges(anyList()); + Mockito.doReturn(new Results()).when(cruiseControlHandler).requestOngoingChanges(anyList()); + Mockito.doReturn(completedResults).when(cruiseControlHandler).completeZombieChanges(anyList()); + + var reconcilableTopics = List.of(reconcilableTopic); PartitionedByError currentStatesOrError = new PartitionedByError<>(List.of(), List.of()); // run test - var metricsHolder = new TopicOperatorMetricsHolder(RESOURCE_KIND, null, new TopicOperatorMetricsProvider(new SimpleMeterRegistry())); - var controller = new BatchingTopicController(config, Map.of("key", "VALUE"), kafkaAdminClient, kubernetesClient, metricsHolder, new ReplicasChangeHandler(config, metricsHolder)); - var results = controller.checkReplicasChanges(reconcilableTopics, currentStatesOrError); - - assertThat(results.ok().count(), is(1L)); - assertThat(results.ok().findFirst().get().getKey().kt().getStatus().getReplicasChange(), is(nullValue())); + var metricsHolder = new TopicOperatorMetricsHolder(KafkaTopic.RESOURCE_KIND, null, new TopicOperatorMetricsProvider(new SimpleMeterRegistry())); + var controller = new BatchingTopicController(config, Map.of("key", "VALUE"), + new KubernetesHandler(config, metricsHolder, kubernetesClient), + new KafkaHandler(config, metricsHolder, kafkaAdmin), metricsHolder, + cruiseControlHandler); + var results = controller.checkReplicasChanges(currentStatesOrError.ok(), reconcilableTopics); + + assertThat(results.size(), is(1)); + assertThat(results.getReplicasChanges().get(reconcilableTopic), is(nullValue())); } @Test @@ -415,7 +431,7 @@ public void replicasChangeShouldCompleteWhenCruiseControlRestarts() { var topicName = "my-topic"; var replicationFactor = 1; - // setup: pending with .spec.replicas == uniqueReplicationFactor + // setup: pending with .spec.replicas == replicationFactor var config = Mockito.mock(TopicOperatorConfig.class); Mockito.doReturn(NAMESPACE).when(config).namespace(); Mockito.doReturn(true).when(config).useFinalizer(); @@ -428,11 +444,12 @@ public void replicasChangeShouldCompleteWhenCruiseControlRestarts() { var partitionReassignmentResult = Mockito.mock(ListPartitionReassignmentsResult.class); var topicPartition = Mockito.mock(TopicPartition.class); var partitionReassignment = Mockito.mock(PartitionReassignment.class); - Mockito.doReturn(KafkaFuture.completedFuture(Map.of(topicPartition, partitionReassignment))).when(partitionReassignmentResult).reassignments(); + Mockito.doReturn(KafkaFuture.completedFuture(Map.of(topicPartition, partitionReassignment))) + .when(partitionReassignmentResult).reassignments(); - var kafkaAdminClient = Mockito.mock(Admin.class); - Mockito.doReturn(describeClusterResult).when(kafkaAdminClient).describeCluster(); - Mockito.doReturn(partitionReassignmentResult).when(kafkaAdminClient).listPartitionReassignments(any(Set.class)); + var kafkaAdmin = Mockito.mock(Admin.class); + Mockito.doReturn(describeClusterResult).when(kafkaAdmin).describeCluster(); + Mockito.doReturn(partitionReassignmentResult).when(kafkaAdmin).listPartitionReassignments(any(Set.class)); var topicDescription = Mockito.mock(TopicDescription.class); var topicPartitionInfo = Mockito.mock(TopicPartitionInfo.class); @@ -454,193 +471,466 @@ public void replicasChangeShouldCompleteWhenCruiseControlRestarts() { .endSpec() .withStatus(new KafkaTopicStatusBuilder() .withReplicasChange(new ReplicasChangeStatusBuilder() - .withState(PENDING) + .withState(ReplicasChangeState.PENDING) .withTargetReplicas(replicationFactor) .build()) .build()) .build(); - var replicasChangeHandler = Mockito.mock(ReplicasChangeHandler.class); - Mockito.doReturn(List.of()).when(replicasChangeHandler).requestPendingChanges(anyList()); - Mockito.doReturn(List.of()).when(replicasChangeHandler).requestOngoingChanges(anyList()); + var reconcilableTopic = new ReconcilableTopic( + new Reconciliation("test", KafkaTopic.RESOURCE_KIND, NAMESPACE, topicName), kafkaTopic, topicName); + + var completedResults = new Results(); + completedResults.addRightResults(List.of(reconcilableTopic)); + completedResults.addReplicasChange(reconcilableTopic, null); + + var cruiseControlHandler = Mockito.mock(CruiseControlHandler.class); + Mockito.doReturn(new Results()).when(cruiseControlHandler).requestPendingChanges(anyList()); + Mockito.doReturn(new Results()).when(cruiseControlHandler).requestOngoingChanges(anyList()); + Mockito.doReturn(completedResults).when(cruiseControlHandler).completeZombieChanges(anyList()); - var reconcilableTopics = List.of(TopicOperatorTestUtil.reconcilableTopic(kafkaTopic, NAMESPACE)); + var reconcilableTopics = List.of(reconcilableTopic); PartitionedByError currentStatesOrError = new PartitionedByError<>(List.of(), List.of()); // run test - var metricsHolder = new TopicOperatorMetricsHolder(RESOURCE_KIND, null, new TopicOperatorMetricsProvider(new SimpleMeterRegistry())); - var controller = new BatchingTopicController(config, Map.of("key", "VALUE"), kafkaAdminClient, kubernetesClient, metricsHolder, new ReplicasChangeHandler(config, metricsHolder)); - var results = controller.checkReplicasChanges(reconcilableTopics, currentStatesOrError); + var metricsHolder = new TopicOperatorMetricsHolder(KafkaTopic.RESOURCE_KIND, null, new TopicOperatorMetricsProvider(new SimpleMeterRegistry())); + var controller = new BatchingTopicController(config, Map.of("key", "VALUE"), + new KubernetesHandler(config, metricsHolder, kubernetesClient), + new KafkaHandler(config, metricsHolder, kafkaAdmin), metricsHolder, + cruiseControlHandler); + var results = controller.checkReplicasChanges(currentStatesOrError.ok(), reconcilableTopics); - assertThat(results.ok().count(), is(1L)); - assertThat(results.ok().findFirst().get().getKey().kt().getStatus().getReplicasChange(), is(nullValue())); + assertThat(results.size(), is(1)); + assertThat(results.getReplicasChanges().get(reconcilableTopic), is(nullValue())); } @Test public void shouldNotCallGetClusterConfigWhenDisabled() { - var kafkaAdminClient = Mockito.mock(Admin.class); + var kafkaAdmin = Mockito.mock(Admin.class); var config = TopicOperatorConfig.buildFromMap(Map.of( TopicOperatorConfig.BOOTSTRAP_SERVERS.key(), "localhost:1234", - TopicOperatorConfig.NAMESPACE.key(), NAMESPACE, + TopicOperatorConfig.NAMESPACE.key(), "some-namespace", TopicOperatorConfig.SASL_ENABLED.key(), "true", TopicOperatorConfig.SKIP_CLUSTER_CONFIG_REVIEW.key(), "true" )); - var metricsHolder = new TopicOperatorMetricsHolder(RESOURCE_KIND, null, new TopicOperatorMetricsProvider(new SimpleMeterRegistry())); - new BatchingTopicController(config, Map.of("key", "VALUE"), kafkaAdminClient, kubernetesClient, metricsHolder, new ReplicasChangeHandler(config, metricsHolder)); - - verifyNoInteractions(kafkaAdminClient); + var metricsHolder = new TopicOperatorMetricsHolder(KafkaTopic.RESOURCE_KIND, null, new TopicOperatorMetricsProvider(new SimpleMeterRegistry())); + new BatchingTopicController(config, Map.of("key", "VALUE"), + new KubernetesHandler(config, metricsHolder, kubernetesClient), + new KafkaHandler(config, metricsHolder, kafkaAdmin), metricsHolder, + new CruiseControlHandler(config, metricsHolder, TopicOperatorUtil.createCruiseControlClient(config))); + + verifyNoInteractions(kafkaAdmin); } - @ParameterizedTest - @NullSource - @ValueSource(strings = { "ALL", "" }) - public void shouldUpdateProperties(String alterableTopicConfig, KafkaCluster cluster) throws InterruptedException, ExecutionException { - var topicName = "my-topic"; + @Test + public void shouldIgnoreWithCruiseControlThrottleConfigInKafka(KafkaCluster cluster) throws InterruptedException, ExecutionException { kafkaAdminClient[0] = Admin.create(Map.of(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.getBootstrapServers())); - kafkaAdminClient[0].createTopics(List.of(new NewTopic(topicName, 1, (short) 1).configs(Map.of(TopicConfig.COMPRESSION_TYPE_CONFIG, "snappy")))).all().get(); - var kafkaAdminClient = Mockito.spy(this.kafkaAdminClient[0]); - - // Setup the KafkaTopic with 1 property change that is not in the alterableTopicConfig list. - var kafkaTopic = Crds.topicOperation(kubernetesClient).resource( - new KafkaTopicBuilder() - .withNewMetadata() - .withName(topicName) - .withNamespace(NAMESPACE) - .addToLabels("key", "VALUE") - .endMetadata() - .withNewSpec() - .withConfig(Map.of( - TopicConfig.COMPRESSION_TYPE_CONFIG, "snappy", - TopicConfig.CLEANUP_POLICY_CONFIG, "compact")) - .withPartitions(2) - .withReplicas(1) - .endSpec() - .build()).create(); - + var kafkaAdminClientSpy = Mockito.spy(kafkaAdminClient[0]); var config = Mockito.mock(TopicOperatorConfig.class); Mockito.doReturn(NAMESPACE).when(config).namespace(); - Mockito.doReturn(true).when(config).skipClusterConfigReview(); - Mockito.doReturn(alterableTopicConfig).when(config).alterableTopicConfig(); + Mockito.doReturn(true).when(config).cruiseControlEnabled(); - var metricsHolder = new TopicOperatorMetricsHolder(RESOURCE_KIND, null, new TopicOperatorMetricsProvider(new SimpleMeterRegistry())); - var controller = new BatchingTopicController(config, Map.of("key", "VALUE"), kafkaAdminClient, kubernetesClient, metricsHolder, new ReplicasChangeHandler(config, metricsHolder)); - - var reconcilableTopics = List.of(TopicOperatorTestUtil.reconcilableTopic(kafkaTopic, NAMESPACE)); - controller.onUpdate(reconcilableTopics); + // setup topic in Kafka + kafkaAdminClient[0].createTopics(List.of(new NewTopic("my-topic", 2, (short) 1).configs(Map.of( + "leader.replication.throttled.replicas", "13:0,13:1,45:0,45:1", + "follower.replication.throttled.replicas", "13:0,13:1,45:0,45:1" + )))).all().get(); + + // setup topic in Kube + var testTopic = Crds.topicOperation(kubernetesClient).resource( + new KafkaTopicBuilder() + .withNewMetadata() + .withName("my-topic") + .withNamespace(NAMESPACE) + .addToLabels("key", "VALUE") + .endMetadata() + .withNewSpec() + .withPartitions(2) + .withReplicas(1) + .endSpec() + .build()).create(); + + // run test + var metricsHolder = new TopicOperatorMetricsHolder(KafkaTopic.RESOURCE_KIND, null, new TopicOperatorMetricsProvider(new SimpleMeterRegistry())); + var controller = new BatchingTopicController(config, Map.of("key", "VALUE"), + new KubernetesHandler(config, metricsHolder, kubernetesClient), + new KafkaHandler(config, metricsHolder, kafkaAdminClientSpy), metricsHolder, + new CruiseControlHandler(config, metricsHolder, TopicOperatorUtil.createCruiseControlClient(config))); + controller.onUpdate(List.of(new ReconcilableTopic( + new Reconciliation("test", KafkaTopic.RESOURCE_KIND, NAMESPACE, "my-topic"), testTopic, "my-topic"))); + + Mockito.verify(kafkaAdminClientSpy, Mockito.never()).incrementalAlterConfigs(any()); + + testTopic = Crds.topicOperation(kubernetesClient).inNamespace(NAMESPACE).withName("my-topic").get(); + assertEquals(1, testTopic.getStatus().getConditions().size()); + assertEquals("True", testTopic.getStatus().getConditions().get(0).getStatus()); + } - Mockito.verify(kafkaAdminClient).incrementalAlterConfigs(any()); + @Test + public void shouldReconcileAndWarnWithThrottleConfigInKube(KafkaCluster cluster) throws InterruptedException, ExecutionException { + kafkaAdminClient[0] = Admin.create(Map.of(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.getBootstrapServers())); + var kafkaAdminClientSpy = Mockito.spy(kafkaAdminClient[0]); + var config = Mockito.mock(TopicOperatorConfig.class); + Mockito.doReturn(NAMESPACE).when(config).namespace(); + Mockito.doReturn(true).when(config).cruiseControlEnabled(); - var updateTopic = Crds.topicOperation(kubernetesClient).inNamespace(NAMESPACE).withName(topicName).get(); + // setup topic in Kafka + kafkaAdminClient[0].createTopics(List.of(new NewTopic("my-topic", 2, (short) 1).configs(Map.of( + "leader.replication.throttled.replicas", "13:0,13:1,45:0,45:1", + "follower.replication.throttled.replicas", "13:0,13:1,45:0,45:1" + )))).all().get(); - var conditionList = updateTopic.getStatus().getConditions(); - assertEquals(1, conditionList.size()); + // setup topic in Kube + var testTopic = Crds.topicOperation(kubernetesClient).resource( + new KafkaTopicBuilder() + .withNewMetadata() + .withName("my-topic") + .withNamespace(NAMESPACE) + .addToLabels("key", "VALUE") + .endMetadata() + .withNewSpec() + .withConfig(Map.of( + "leader.replication.throttled.replicas", "10:1", + "follower.replication.throttled.replicas", "10:1")) + .withPartitions(2) + .withReplicas(1) + .endSpec() + .build()).create(); - var readyCondition = conditionList.stream().filter(condition -> condition.getType().equals("Ready")).findFirst().get(); - assertEquals("True", readyCondition.getStatus()); - } + // run test + var metricsHolder = new TopicOperatorMetricsHolder(KafkaTopic.RESOURCE_KIND, null, new TopicOperatorMetricsProvider(new SimpleMeterRegistry())); + var controller = new BatchingTopicController(config, Map.of("key", "VALUE"), + new KubernetesHandler(config, metricsHolder, kubernetesClient), + new KafkaHandler(config, metricsHolder, kafkaAdminClientSpy), metricsHolder, + new CruiseControlHandler(config, metricsHolder, TopicOperatorUtil.createCruiseControlClient(config))); + controller.onUpdate(List.of(new ReconcilableTopic(new Reconciliation("test", KafkaTopic.RESOURCE_KIND, NAMESPACE, "my-topic"), testTopic, "my-topic"))); + + Mockito.verify(kafkaAdminClientSpy, Mockito.times(1)).incrementalAlterConfigs(any()); + + testTopic = Crds.topicOperation(kubernetesClient).inNamespace(NAMESPACE).withName("my-topic").get(); + assertEquals(3, testTopic.getStatus().getConditions().size()); + assertEquals("True", testTopic.getStatus().getConditions().get(0).getStatus()); + + var warning1 = testTopic.getStatus().getConditions().get(1); + assertEquals("Property follower.replication.throttled.replicas may conflict with throttled rebalances", warning1.getMessage()); + assertEquals(BatchingTopicController.INVALID_CONFIG, warning1.getReason()); + assertEquals("True", warning1.getStatus()); + + var warning2 = testTopic.getStatus().getConditions().get(2); + assertEquals("Property leader.replication.throttled.replicas may conflict with throttled rebalances", warning2.getMessage()); + assertEquals(BatchingTopicController.INVALID_CONFIG, warning2.getReason()); + assertEquals("True", warning2.getStatus()); + + // remove warning condition + testTopic = Crds.topicOperation(kubernetesClient).resource( + new KafkaTopicBuilder() + .withNewMetadata() + .withName("my-topic") + .withNamespace(NAMESPACE) + .addToLabels("key", "VALUE") + .endMetadata() + .withNewSpec() + .withConfig(Map.of( + TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "1")) + .withPartitions(2) + .withReplicas(1) + .endSpec() + .build()).update(); + controller.onUpdate(List.of(new ReconcilableTopic( + new Reconciliation("test", KafkaTopic.RESOURCE_KIND, NAMESPACE, "my-topic"), testTopic, "my-topic"))); + testTopic = Crds.topicOperation(kubernetesClient).inNamespace(NAMESPACE).withName("my-topic").get(); + assertEquals(1, testTopic.getStatus().getConditions().size()); + assertEquals("True", testTopic.getStatus().getConditions().get(0).getStatus()); + } + @ParameterizedTest - @ValueSource(strings = { "NONE", "sdasdas", "retention.bytes; retention.ms" }) - public void shouldNotUpdateAnyPropertiesWarnOnAllProperties(String alterableTopicConfig, KafkaCluster cluster) throws InterruptedException, ExecutionException { - var topicName = "my-topic"; + @ValueSource(strings = { "min.insync.replicas, compression.type" }) + public void shouldIgnoreAndWarnWithAlterableConfigOnCreation(String alterableConfig, KafkaCluster cluster) throws InterruptedException { kafkaAdminClient[0] = Admin.create(Map.of(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.getBootstrapServers())); - kafkaAdminClient[0].createTopics(List.of(new NewTopic(topicName, 1, (short) 1).configs(Map.of(TopicConfig.COMPRESSION_TYPE_CONFIG, "snappy")))).all().get(); - var kafkaAdminClient = Mockito.spy(this.kafkaAdminClient[0]); - - // Setup the KafkaTopic with 2 property changes and an empty alterableTopicConfig list. - var kafkaTopic = Crds.topicOperation(kubernetesClient).resource( - new KafkaTopicBuilder() - .withNewMetadata() - .withName(topicName) - .withNamespace(NAMESPACE) - .addToLabels("key", "VALUE") - .endMetadata() - .withNewSpec() - .withConfig(Map.of( - TopicConfig.COMPRESSION_TYPE_CONFIG, "snappy", - TopicConfig.CLEANUP_POLICY_CONFIG, "compact")) - .withPartitions(2) - .withReplicas(1) - .endSpec() - .build()).create(); + var kafkaAdminClientSpy = Mockito.spy(kafkaAdminClient[0]); + var config = Mockito.mock(TopicOperatorConfig.class); + Mockito.doReturn(NAMESPACE).when(config).namespace(); + Mockito.doReturn(alterableConfig).when(config).alterableTopicConfig(); + // setup topic in Kube + var testTopic = Crds.topicOperation(kubernetesClient).resource( + new KafkaTopicBuilder() + .withNewMetadata() + .withName("my-topic") + .withNamespace(NAMESPACE) + .addToLabels("key", "VALUE") + .endMetadata() + .withNewSpec() + .withConfig(Map.of( + TopicConfig.COMPRESSION_TYPE_CONFIG, "snappy", + TopicConfig.CLEANUP_POLICY_CONFIG, "compact", + TopicConfig.SEGMENT_BYTES_CONFIG, "1073741824")) + .withPartitions(2) + .withReplicas(1) + .endSpec() + .build()).create(); + + // run test + var metricsHolder = new TopicOperatorMetricsHolder(KafkaTopic.RESOURCE_KIND, null, new TopicOperatorMetricsProvider(new SimpleMeterRegistry())); + var controller = new BatchingTopicController(config, Map.of("key", "VALUE"), + new KubernetesHandler(config, metricsHolder, kubernetesClient), + new KafkaHandler(config, metricsHolder, kafkaAdminClientSpy), metricsHolder, + new CruiseControlHandler(config, metricsHolder, TopicOperatorUtil.createCruiseControlClient(config))); + controller.onUpdate(List.of(new ReconcilableTopic(new Reconciliation("test", KafkaTopic.RESOURCE_KIND, NAMESPACE, "my-topic"), testTopic, "my-topic"))); + + Mockito.verify(kafkaAdminClientSpy, Mockito.never()).incrementalAlterConfigs(any()); + + testTopic = Crds.topicOperation(kubernetesClient).inNamespace(NAMESPACE).withName("my-topic").get(); + assertEquals(3, testTopic.getStatus().getConditions().size()); + assertEquals("True", testTopic.getStatus().getConditions().get(0).getStatus()); + + var warning1 = testTopic.getStatus().getConditions().get(1); + assertEquals("Property cleanup.policy is ignored according to alterable config", warning1.getMessage()); + assertEquals(BatchingTopicController.INVALID_CONFIG, warning1.getReason()); + assertEquals("True", warning1.getStatus()); + + var warning2 = testTopic.getStatus().getConditions().get(2); + assertEquals("Property segment.bytes is ignored according to alterable config", warning2.getMessage()); + assertEquals(BatchingTopicController.INVALID_CONFIG, warning2.getReason()); + assertEquals("True", warning2.getStatus()); + } + + @ParameterizedTest + @ValueSource(strings = { "compression.type, max.message.bytes, message.timestamp.difference.max.ms, message.timestamp.type, retention.bytes, retention.ms" }) + public void shouldReconcileWithAlterableConfigOnUpdate(String alterableConfig, KafkaCluster cluster) throws InterruptedException, ExecutionException { + kafkaAdminClient[0] = Admin.create(Map.of(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.getBootstrapServers())); + var kafkaAdminClientSpy = Mockito.spy(kafkaAdminClient[0]); var config = Mockito.mock(TopicOperatorConfig.class); Mockito.doReturn(NAMESPACE).when(config).namespace(); - Mockito.doReturn(true).when(config).skipClusterConfigReview(); - Mockito.doReturn(alterableTopicConfig).when(config).alterableTopicConfig(); + Mockito.doReturn(alterableConfig).when(config).alterableTopicConfig(); - var metricsHolder = new TopicOperatorMetricsHolder(RESOURCE_KIND, null, new TopicOperatorMetricsProvider(new SimpleMeterRegistry())); - var controller = new BatchingTopicController(config, Map.of("key", "VALUE"), kafkaAdminClient, kubernetesClient, metricsHolder, new ReplicasChangeHandler(config, metricsHolder)); + // setup topic in Kafka + kafkaAdminClient[0].createTopics(List.of(new NewTopic("my-topic", 2, (short) 1).configs(Map.of( + TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "2" + )))).all().get(); - var reconcilableTopics = List.of(TopicOperatorTestUtil.reconcilableTopic(kafkaTopic, NAMESPACE)); - controller.onUpdate(reconcilableTopics); + // setup topic in Kube + var testTopic = Crds.topicOperation(kubernetesClient).resource( + new KafkaTopicBuilder() + .withNewMetadata() + .withName("my-topic") + .withNamespace(NAMESPACE) + .addToLabels("key", "VALUE") + .endMetadata() + .withNewSpec() + .withConfig(Map.of( + TopicConfig.COMPRESSION_TYPE_CONFIG, "snappy", + TopicConfig.CLEANUP_POLICY_CONFIG, "compact", + TopicConfig.SEGMENT_BYTES_CONFIG, "1073741824")) + .withPartitions(2) + .withReplicas(1) + .endSpec() + .build()).create(); - Mockito.verify(kafkaAdminClient, Mockito.never()).incrementalAlterConfigs(any()); + // run test + var metricsHolder = new TopicOperatorMetricsHolder(KafkaTopic.RESOURCE_KIND, null, new TopicOperatorMetricsProvider(new SimpleMeterRegistry())); + var controller = new BatchingTopicController(config, Map.of("key", "VALUE"), + new KubernetesHandler(config, metricsHolder, kubernetesClient), + new KafkaHandler(config, metricsHolder, kafkaAdminClientSpy), metricsHolder, + new CruiseControlHandler(config, metricsHolder, TopicOperatorUtil.createCruiseControlClient(config))); + controller.onUpdate(List.of(new ReconcilableTopic(new Reconciliation("test", KafkaTopic.RESOURCE_KIND, NAMESPACE, "my-topic"), testTopic, "my-topic"))); + + Mockito.verify(kafkaAdminClientSpy, Mockito.times(1)).incrementalAlterConfigs(any()); + + testTopic = Crds.topicOperation(kubernetesClient).inNamespace(NAMESPACE).withName("my-topic").get(); + assertEquals(3, testTopic.getStatus().getConditions().size()); + assertEquals("True", testTopic.getStatus().getConditions().get(0).getStatus()); + + var warning1 = testTopic.getStatus().getConditions().get(1); + assertEquals("Property cleanup.policy is ignored according to alterable config", warning1.getMessage()); + assertEquals(BatchingTopicController.INVALID_CONFIG, warning1.getReason()); + assertEquals("True", warning1.getStatus()); + + var warning2 = testTopic.getStatus().getConditions().get(2); + assertEquals("Property segment.bytes is ignored according to alterable config", warning2.getMessage()); + assertEquals(BatchingTopicController.INVALID_CONFIG, warning2.getReason()); + assertEquals("True", warning2.getStatus()); + + // remove warning condition + testTopic = Crds.topicOperation(kubernetesClient).resource( + new KafkaTopicBuilder() + .withNewMetadata() + .withName("my-topic") + .withNamespace(NAMESPACE) + .addToLabels("key", "VALUE") + .endMetadata() + .withNewSpec() + .withConfig(Map.of( + TopicConfig.COMPRESSION_TYPE_CONFIG, "snappy")) + .withPartitions(2) + .withReplicas(1) + .endSpec() + .build()).update(); + controller.onUpdate(List.of(new ReconcilableTopic( + new Reconciliation("test", KafkaTopic.RESOURCE_KIND, NAMESPACE, "my-topic"), testTopic, "my-topic"))); - var updateTopic = Crds.topicOperation(kubernetesClient).inNamespace(NAMESPACE).withName(topicName).get(); + testTopic = Crds.topicOperation(kubernetesClient).inNamespace(NAMESPACE).withName("my-topic").get(); + assertEquals(1, testTopic.getStatus().getConditions().size()); + assertEquals("True", testTopic.getStatus().getConditions().get(0).getStatus()); + } + + @ParameterizedTest + @ValueSource(strings = { "ALL", "" }) + public void shouldReconcileWithAllOrEmptyAlterableConfig(String alterableConfig, KafkaCluster cluster) throws InterruptedException, ExecutionException { + kafkaAdminClient[0] = Admin.create(Map.of(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.getBootstrapServers())); + var kafkaAdminClientSpy = Mockito.spy(kafkaAdminClient[0]); + var config = Mockito.mock(TopicOperatorConfig.class); + Mockito.doReturn(NAMESPACE).when(config).namespace(); + Mockito.doReturn(alterableConfig).when(config).alterableTopicConfig(); - var conditionList = updateTopic.getStatus().getConditions(); - assertEquals(2, conditionList.size()); + // setup topic in Kafka + kafkaAdminClient[0].createTopics(List.of(new NewTopic("my-topic", 2, (short) 1).configs(Map.of( + TopicConfig.COMPRESSION_TYPE_CONFIG, "snappy" + )))).all().get(); - var readyCondition = conditionList.stream().filter(condition -> condition.getType().equals("Ready")).findFirst().get(); - assertEquals("True", readyCondition.getStatus()); + // setup topic in Kube + var testTopic = Crds.topicOperation(kubernetesClient).resource( + new KafkaTopicBuilder() + .withNewMetadata() + .withName("my-topic") + .withNamespace(NAMESPACE) + .addToLabels("key", "VALUE") + .endMetadata() + .withNewSpec() + .withConfig(Map.of( + TopicConfig.COMPRESSION_TYPE_CONFIG, "snappy", + TopicConfig.CLEANUP_POLICY_CONFIG, "compact")) + .withPartitions(2) + .withReplicas(1) + .endSpec() + .build()).create(); - var notConfiguredCondition = conditionList.stream().filter(condition -> condition.getType().equals("Warning")).findFirst().get(); - assertEquals("These .spec.config properties are not configurable: [cleanup.policy, compression.type]", notConfiguredCondition.getMessage()); - assertEquals("NotConfigurable", notConfiguredCondition.getReason()); - assertEquals("True", notConfiguredCondition.getStatus()); + // run test + var metricsHolder = new TopicOperatorMetricsHolder(KafkaTopic.RESOURCE_KIND, null, new TopicOperatorMetricsProvider(new SimpleMeterRegistry())); + var controller = new BatchingTopicController(config, Map.of("key", "VALUE"), + new KubernetesHandler(config, metricsHolder, kubernetesClient), + new KafkaHandler(config, metricsHolder, kafkaAdminClientSpy), metricsHolder, + new CruiseControlHandler(config, metricsHolder, TopicOperatorUtil.createCruiseControlClient(config))); + controller.onUpdate(List.of(new ReconcilableTopic(new Reconciliation("test", KafkaTopic.RESOURCE_KIND, NAMESPACE, "my-topic"), testTopic, "my-topic"))); + + Mockito.verify(kafkaAdminClientSpy, Mockito.times(1)).incrementalAlterConfigs(any()); + + testTopic = Crds.topicOperation(kubernetesClient).inNamespace(NAMESPACE).withName("my-topic").get(); + assertEquals(1, testTopic.getStatus().getConditions().size()); + assertEquals("True", testTopic.getStatus().getConditions().get(0).getStatus()); } - - @Test - public void shouldNotUpdatePropertiesNotInTheAlterableProperties(KafkaCluster cluster) throws InterruptedException, ExecutionException { - var topicName = "my-topic"; - var alterableTopicConfig = "compression.type, max.message.bytes, message.timestamp.difference.max.ms, message.timestamp.type, retention.bytes, retention.ms"; + + @ParameterizedTest + @ValueSource(strings = { "NONE" }) + public void shouldIgnoreAndWarnWithNoneAlterableConfig(String alterableConfig, KafkaCluster cluster) throws InterruptedException, ExecutionException { kafkaAdminClient[0] = Admin.create(Map.of(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.getBootstrapServers())); - kafkaAdminClient[0].createTopics(List.of(new NewTopic(topicName, 1, (short) 1).configs(Map.of(TopicConfig.COMPRESSION_TYPE_CONFIG, "snappy")))).all().get(); - var kafkaAdminClient = Mockito.spy(this.kafkaAdminClient[0]); - - // Setup the KafkaTopic with 1 property change that is not in the alterableTopicConfig list. - var kafkaTopic = Crds.topicOperation(kubernetesClient).resource( - new KafkaTopicBuilder() - .withNewMetadata() - .withName(topicName) - .withNamespace(NAMESPACE) - .addToLabels("key", "VALUE") - .endMetadata() - .withNewSpec() - .withConfig(Map.of( - TopicConfig.COMPRESSION_TYPE_CONFIG, "snappy", - TopicConfig.CLEANUP_POLICY_CONFIG, "compact")) - .withPartitions(2) - .withReplicas(1) - .endSpec() - .build()).create(); - + var kafkaAdminClientSpy = Mockito.spy(kafkaAdminClient[0]); var config = Mockito.mock(TopicOperatorConfig.class); Mockito.doReturn(NAMESPACE).when(config).namespace(); - Mockito.doReturn(true).when(config).skipClusterConfigReview(); - Mockito.doReturn(alterableTopicConfig).when(config).alterableTopicConfig(); + Mockito.doReturn(alterableConfig).when(config).alterableTopicConfig(); - var metricsHolder = new TopicOperatorMetricsHolder(RESOURCE_KIND, null, new TopicOperatorMetricsProvider(new SimpleMeterRegistry())); - var controller = new BatchingTopicController(config, Map.of("key", "VALUE"), kafkaAdminClient, kubernetesClient, metricsHolder, new ReplicasChangeHandler(config, metricsHolder)); + // setup topic in Kafka + kafkaAdminClient[0].createTopics(List.of(new NewTopic("my-topic", 2, (short) 1).configs(Map.of( + TopicConfig.COMPRESSION_TYPE_CONFIG, "snappy" + )))).all().get(); - var reconcilableTopics = List.of(TopicOperatorTestUtil.reconcilableTopic(kafkaTopic, NAMESPACE)); - controller.onUpdate(reconcilableTopics); - - Mockito.verify(kafkaAdminClient, Mockito.never()).incrementalAlterConfigs(any()); + // setup topic in Kube + var testTopic = Crds.topicOperation(kubernetesClient).resource( + new KafkaTopicBuilder() + .withNewMetadata() + .withName("my-topic") + .withNamespace(NAMESPACE) + .addToLabels("key", "VALUE") + .endMetadata() + .withNewSpec() + .withConfig(Map.of( + TopicConfig.COMPRESSION_TYPE_CONFIG, "snappy", + TopicConfig.CLEANUP_POLICY_CONFIG, "compact")) + .withPartitions(2) + .withReplicas(1) + .endSpec() + .build()).create(); - var updateTopic = Crds.topicOperation(kubernetesClient).inNamespace(NAMESPACE).withName(topicName).get(); + // run test + var metricsHolder = new TopicOperatorMetricsHolder(KafkaTopic.RESOURCE_KIND, null, new TopicOperatorMetricsProvider(new SimpleMeterRegistry())); + var controller = new BatchingTopicController(config, Map.of("key", "VALUE"), + new KubernetesHandler(config, metricsHolder, kubernetesClient), + new KafkaHandler(config, metricsHolder, kafkaAdminClientSpy), metricsHolder, + new CruiseControlHandler(config, metricsHolder, TopicOperatorUtil.createCruiseControlClient(config))); + controller.onUpdate(List.of(new ReconcilableTopic(new Reconciliation("test", KafkaTopic.RESOURCE_KIND, NAMESPACE, "my-topic"), testTopic, "my-topic"))); + + Mockito.verify(kafkaAdminClientSpy, Mockito.never()).incrementalAlterConfigs(any()); + + testTopic = Crds.topicOperation(kubernetesClient).inNamespace(NAMESPACE).withName("my-topic").get(); + assertEquals(3, testTopic.getStatus().getConditions().size()); + assertEquals("True", testTopic.getStatus().getConditions().get(0).getStatus()); + + var warning1 = testTopic.getStatus().getConditions().get(1); + assertEquals("Property cleanup.policy is ignored according to alterable config", warning1.getMessage()); + assertEquals(BatchingTopicController.INVALID_CONFIG, warning1.getReason()); + assertEquals("True", warning1.getStatus()); + + var warning2 = testTopic.getStatus().getConditions().get(2); + assertEquals("Property compression.type is ignored according to alterable config", warning2.getMessage()); + assertEquals(BatchingTopicController.INVALID_CONFIG, warning2.getReason()); + assertEquals("True", warning2.getStatus()); + } + + @ParameterizedTest + @ValueSource(strings = { "invalid", "compression.type; cleanup.policy" }) + public void shouldIgnoreAndWarnWithInvalidAlterableConfig(String alterableConfig, KafkaCluster cluster) throws InterruptedException, ExecutionException { + kafkaAdminClient[0] = Admin.create(Map.of(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.getBootstrapServers())); + var kafkaAdminClientSpy = Mockito.spy(kafkaAdminClient[0]); + var config = Mockito.mock(TopicOperatorConfig.class); + Mockito.doReturn(NAMESPACE).when(config).namespace(); + Mockito.doReturn(alterableConfig).when(config).alterableTopicConfig(); - var conditionList = updateTopic.getStatus().getConditions(); - assertEquals(2, conditionList.size()); + // setup topic in Kafka + kafkaAdminClient[0].createTopics(List.of(new NewTopic("my-topic", 2, (short) 1).configs(Map.of( + TopicConfig.COMPRESSION_TYPE_CONFIG, "snappy" + )))).all().get(); - var readyCondition = conditionList.stream().filter(condition -> condition.getType().equals("Ready")).findFirst().get(); - assertEquals("True", readyCondition.getStatus()); + // setup topic in Kube + var testTopic = Crds.topicOperation(kubernetesClient).resource( + new KafkaTopicBuilder() + .withNewMetadata() + .withName("my-topic") + .withNamespace(NAMESPACE) + .addToLabels("key", "VALUE") + .endMetadata() + .withNewSpec() + .withConfig(Map.of( + TopicConfig.COMPRESSION_TYPE_CONFIG, "snappy", + TopicConfig.CLEANUP_POLICY_CONFIG, "compact")) + .withPartitions(2) + .withReplicas(1) + .endSpec() + .build()).create(); - var notConfiguredCondition = conditionList.stream().filter(condition -> condition.getType().equals("Warning")).findFirst().get(); - assertEquals("These .spec.config properties are not configurable: [cleanup.policy]", notConfiguredCondition.getMessage()); - assertEquals("NotConfigurable", notConfiguredCondition.getReason()); - assertEquals("True", notConfiguredCondition.getStatus()); + // run test + var metricsHolder = new TopicOperatorMetricsHolder(KafkaTopic.RESOURCE_KIND, null, new TopicOperatorMetricsProvider(new SimpleMeterRegistry())); + var controller = new BatchingTopicController(config, Map.of("key", "VALUE"), + new KubernetesHandler(config, metricsHolder, kubernetesClient), + new KafkaHandler(config, metricsHolder, kafkaAdminClientSpy), metricsHolder, + new CruiseControlHandler(config, metricsHolder, TopicOperatorUtil.createCruiseControlClient(config))); + controller.onUpdate(List.of(new ReconcilableTopic(new Reconciliation("test", KafkaTopic.RESOURCE_KIND, NAMESPACE, "my-topic"), testTopic, "my-topic"))); + + Mockito.verify(kafkaAdminClientSpy, Mockito.never()).incrementalAlterConfigs(any()); + + testTopic = Crds.topicOperation(kubernetesClient).inNamespace(NAMESPACE).withName("my-topic").get(); + assertEquals(3, testTopic.getStatus().getConditions().size()); + assertEquals("True", testTopic.getStatus().getConditions().get(0).getStatus()); + + var warning1 = testTopic.getStatus().getConditions().get(1); + assertEquals("Property cleanup.policy is ignored according to alterable config", warning1.getMessage()); + assertEquals(BatchingTopicController.INVALID_CONFIG, warning1.getReason()); + assertEquals("True", warning1.getStatus()); + + var warning2 = testTopic.getStatus().getConditions().get(2); + assertEquals("Property compression.type is ignored according to alterable config", warning2.getMessage()); + assertEquals(BatchingTopicController.INVALID_CONFIG, warning2.getReason()); + assertEquals("True", warning2.getStatus()); } } diff --git a/topic-operator/src/test/java/io/strimzi/operator/topic/KafkaHandlerTest.java b/topic-operator/src/test/java/io/strimzi/operator/topic/KafkaHandlerTest.java new file mode 100644 index 00000000000..d4bd344c500 --- /dev/null +++ b/topic-operator/src/test/java/io/strimzi/operator/topic/KafkaHandlerTest.java @@ -0,0 +1,274 @@ +/* + * 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.topic; + +import io.kroxylicious.testing.kafka.api.KafkaCluster; +import io.kroxylicious.testing.kafka.common.BrokerCluster; +import io.kroxylicious.testing.kafka.common.BrokerConfig; +import io.kroxylicious.testing.kafka.junit5ext.KafkaClusterExtension; +import io.kroxylicious.testing.kafka.junit5ext.Topic; +import io.kroxylicious.testing.kafka.junit5ext.TopicPartitions; +import io.micrometer.core.instrument.simple.SimpleMeterRegistry; +import io.strimzi.api.kafka.model.topic.KafkaTopic; +import io.strimzi.api.kafka.model.topic.KafkaTopicBuilder; +import io.strimzi.operator.topic.metrics.TopicOperatorMetricsHolder; +import io.strimzi.operator.topic.metrics.TopicOperatorMetricsProvider; +import io.strimzi.operator.topic.model.Pair; +import io.strimzi.operator.topic.model.ReconcilableTopic; +import io.strimzi.operator.topic.model.TopicState; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.AdminClientConfig; +import org.apache.kafka.clients.admin.AlterConfigOp; +import org.apache.kafka.clients.admin.ConfigEntry; +import org.apache.kafka.clients.admin.NewPartitions; +import org.apache.kafka.clients.admin.TopicDescription; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.TopicCollection; +import org.apache.kafka.common.TopicPartitionInfo; +import org.apache.kafka.common.config.TopicConfig; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyCollection; +import static org.mockito.ArgumentMatchers.anySet; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +@ExtendWith(KafkaClusterExtension.class) +public class KafkaHandlerTest { + private static final String NAMESPACE = TopicOperatorTestUtil.namespaceName(KafkaHandlerTest.class); + + @Test + public void shouldGetClusterConfig( + @BrokerConfig(name = "auto.create.topics.enable", value = "false") + KafkaCluster cluster) { + var config = TopicOperatorConfig.buildFromMap(Map.of( + TopicOperatorConfig.BOOTSTRAP_SERVERS.key(), cluster.getBootstrapServers(), + TopicOperatorConfig.NAMESPACE.key(), NAMESPACE) + ); + var kafkaAdminClientSpy = spy(Admin.create(Map.of(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.getBootstrapServers()))); + + var kafkaHandler = new KafkaHandler(config, + new TopicOperatorMetricsHolder(KafkaTopic.RESOURCE_KIND, null, new TopicOperatorMetricsProvider(new SimpleMeterRegistry())), + kafkaAdminClientSpy); + var autoCreateValue = kafkaHandler.clusterConfig(KafkaHandler.AUTO_CREATE_TOPICS_ENABLE); + + verify(kafkaAdminClientSpy, times(1)).describeCluster(any()); + verify(kafkaAdminClientSpy, times(1)).describeConfigs(any()); + + assertThat(autoCreateValue.isPresent(), is(true)); + assertThat(autoCreateValue.get(), is("false")); + } + + @Test + public void shouldCreateTopics(KafkaCluster cluster) { + var config = TopicOperatorConfig.buildFromMap(Map.of( + TopicOperatorConfig.BOOTSTRAP_SERVERS.key(), cluster.getBootstrapServers(), + TopicOperatorConfig.NAMESPACE.key(), NAMESPACE) + ); + var kafkaAdminClientSpy = spy(Admin.create(Map.of(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.getBootstrapServers()))); + + var kafkaHandler = new KafkaHandler(config, + new TopicOperatorMetricsHolder(KafkaTopic.RESOURCE_KIND, null, new TopicOperatorMetricsProvider(new SimpleMeterRegistry())), + kafkaAdminClientSpy); + var reconcilableTopics = List.of( + TopicOperatorTestUtil.reconcilableTopic(buildTopic("t1", 1, 1), NAMESPACE), + TopicOperatorTestUtil.reconcilableTopic(buildTopic("t2", 1, 1), NAMESPACE) + ); + var result = kafkaHandler.createTopics(reconcilableTopics); + + verify(kafkaAdminClientSpy, times(1)).createTopics(any()); + + var resultTopicNames = result.ok() + .map(pair -> pair.getKey().kt().getMetadata().getName()) + .collect(Collectors.toSet()); + assertThat(resultTopicNames, is(Set.of("t1", "t2"))); + } + + @Test + public void shouldFilterByReassignmentTargetReplicas(@BrokerCluster(numBrokers = 2) KafkaCluster cluster) { + var config = TopicOperatorConfig.buildFromMap(Map.of( + TopicOperatorConfig.BOOTSTRAP_SERVERS.key(), cluster.getBootstrapServers(), + TopicOperatorConfig.NAMESPACE.key(), NAMESPACE) + ); + var kafkaAdminClientSpy = spy(Admin.create(Map.of(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.getBootstrapServers()))); + + var kafkaHandler = new KafkaHandler(config, + new TopicOperatorMetricsHolder(KafkaTopic.RESOURCE_KIND, null, new TopicOperatorMetricsProvider(new SimpleMeterRegistry())), + kafkaAdminClientSpy); + + // current RF = 2 + var topicName = "my-topic"; + var topicPartition0 = mock(TopicPartitionInfo.class); + doReturn(0).when(topicPartition0).partition(); + doReturn(parseNodes(cluster.getBootstrapServers())).when(topicPartition0).replicas(); + + // desired RF = 1 + List> pairs = List.of( + new Pair(TopicOperatorTestUtil.reconcilableTopic(buildTopic(topicName, 1, 1), NAMESPACE), + new TopicState(new TopicDescription(topicName, false, List.of(topicPartition0)), null)) + ); + kafkaHandler.filterByReassignmentTargetReplicas(pairs); + + verify(kafkaAdminClientSpy, times(1)).listPartitionReassignments(anySet()); + } + + @Test + public void shouldAlterConfigs(KafkaCluster cluster, + @io.kroxylicious.testing.kafka.junit5ext.TopicConfig(name = TopicConfig.RETENTION_MS_CONFIG, value = "604800000") Topic t1, + @io.kroxylicious.testing.kafka.junit5ext.TopicConfig(name = TopicConfig.CLEANUP_POLICY_CONFIG, value = "delete") Topic t2) { + var config = TopicOperatorConfig.buildFromMap(Map.of( + TopicOperatorConfig.BOOTSTRAP_SERVERS.key(), cluster.getBootstrapServers(), + TopicOperatorConfig.NAMESPACE.key(), NAMESPACE) + ); + var kafkaAdminClientSpy = spy(Admin.create(Map.of(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.getBootstrapServers()))); + + var kafkaHandler = new KafkaHandler(config, + new TopicOperatorMetricsHolder(KafkaTopic.RESOURCE_KIND, null, new TopicOperatorMetricsProvider(new SimpleMeterRegistry())), + kafkaAdminClientSpy); + + List>> pairs = List.of( + new Pair(TopicOperatorTestUtil.reconcilableTopic(buildTopic(t1.name(), 1, 1), NAMESPACE), + List.of(new AlterConfigOp(new ConfigEntry(TopicConfig.RETENTION_MS_CONFIG, "86400000"), AlterConfigOp.OpType.SET))), + new Pair(TopicOperatorTestUtil.reconcilableTopic(buildTopic(t2.name(), 1, 1), NAMESPACE), + List.of(new AlterConfigOp(new ConfigEntry(TopicConfig.CLEANUP_POLICY_CONFIG, "compact"), AlterConfigOp.OpType.SET))) + ); + var result = kafkaHandler.alterConfigs(pairs); + + verify(kafkaAdminClientSpy, times(1)).incrementalAlterConfigs(any()); + + var resultTopicNames = result.ok() + .map(pair -> pair.getKey().kt().getSpec().getTopicName()) + .collect(Collectors.toSet()); + assertThat(resultTopicNames, is(Set.of(t1.name(), t2.name()))); + } + + @Test + public void shouldCreatePartitions(KafkaCluster cluster, @TopicPartitions(1) Topic t1, @TopicPartitions(1) Topic t2) { + var config = TopicOperatorConfig.buildFromMap(Map.of( + TopicOperatorConfig.BOOTSTRAP_SERVERS.key(), cluster.getBootstrapServers(), + TopicOperatorConfig.NAMESPACE.key(), NAMESPACE) + ); + var kafkaAdminClientSpy = spy(Admin.create(Map.of(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.getBootstrapServers()))); + + var kafkaHandler = new KafkaHandler(config, + new TopicOperatorMetricsHolder(KafkaTopic.RESOURCE_KIND, null, new TopicOperatorMetricsProvider(new SimpleMeterRegistry())), + kafkaAdminClientSpy); + List> pairs = List.of( + new Pair(TopicOperatorTestUtil.reconcilableTopic(buildTopic(t1.name(), 1, 1), NAMESPACE), NewPartitions.increaseTo(2)), + new Pair(TopicOperatorTestUtil.reconcilableTopic(buildTopic(t2.name(), 1, 1), NAMESPACE), NewPartitions.increaseTo(2)) + ); + var result = kafkaHandler.createPartitions(pairs); + + verify(kafkaAdminClientSpy, times(1)).createPartitions(any()); + + var resultTopicNames = result.ok() + .map(pair -> pair.getKey().kt().getSpec().getTopicName()) + .collect(Collectors.toSet()); + assertThat(resultTopicNames, is(Set.of(t1.name(), t2.name()))); + } + + @Test + public void shouldDescribeTopics(KafkaCluster cluster, Topic t1, Topic t2) { + var config = TopicOperatorConfig.buildFromMap(Map.of( + TopicOperatorConfig.BOOTSTRAP_SERVERS.key(), cluster.getBootstrapServers(), + TopicOperatorConfig.NAMESPACE.key(), NAMESPACE) + ); + var kafkaAdminClientSpy = spy(Admin.create(Map.of(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.getBootstrapServers()))); + + var kafkaHandler = new KafkaHandler(config, + new TopicOperatorMetricsHolder(KafkaTopic.RESOURCE_KIND, null, new TopicOperatorMetricsProvider(new SimpleMeterRegistry())), + kafkaAdminClientSpy); + var reconcilableTopics = List.of( + TopicOperatorTestUtil.reconcilableTopic(buildTopic(t1.name(), 1, 1), NAMESPACE), + TopicOperatorTestUtil.reconcilableTopic(buildTopic(t2.name(), 1, 1), NAMESPACE) + ); + var result = kafkaHandler.describeTopics(reconcilableTopics); + + verify(kafkaAdminClientSpy, times(1)).describeTopics(anyCollection()); + verify(kafkaAdminClientSpy, times(1)).describeConfigs(any()); + + var t1State = result.ok() + .filter(pair -> Objects.equals(pair.getKey().kt().getSpec().getTopicName(), t1.name())) + .map(pair -> pair.getValue()).findFirst(); + assertThat(t1State.get().description().name(), is(t1.name())); + var t2State = result.ok() + .filter(pair -> Objects.equals(pair.getKey().kt().getSpec().getTopicName(), t2.name())) + .map(pair -> pair.getValue()).findFirst(); + assertThat(t2State.get().description().name(), is(t2.name())); + } + + @Test + public void shouldDeleteTopics(KafkaCluster cluster, Topic t1, Topic t2, Topic t3) { + var config = TopicOperatorConfig.buildFromMap(Map.of( + TopicOperatorConfig.BOOTSTRAP_SERVERS.key(), cluster.getBootstrapServers(), + TopicOperatorConfig.NAMESPACE.key(), NAMESPACE) + ); + var kafkaAdminClientSpy = spy(Admin.create(Map.of(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.getBootstrapServers()))); + + var kafkaHandler = new KafkaHandler(config, + new TopicOperatorMetricsHolder(KafkaTopic.RESOURCE_KIND, null, new TopicOperatorMetricsProvider(new SimpleMeterRegistry())), + kafkaAdminClientSpy); + var reconcilableTopics = List.of( + TopicOperatorTestUtil.reconcilableTopic(buildTopic(t1.name(), 1, 1), NAMESPACE), + TopicOperatorTestUtil.reconcilableTopic(buildTopic(t2.name(), 1, 1), NAMESPACE), + TopicOperatorTestUtil.reconcilableTopic(buildTopic(t3.name(), 1, 1), NAMESPACE) + ); + var topicNamesToDelete = reconcilableTopics.stream().map(ReconcilableTopic::topicName).collect(Collectors.toSet()); + topicNamesToDelete.removeIf(name -> Objects.equals(name, t3.name())); + var result = kafkaHandler.deleteTopics(reconcilableTopics, topicNamesToDelete); + + verify(kafkaAdminClientSpy, times(1)).deleteTopics(any(TopicCollection.TopicNameCollection.class)); + + var resultTopicNames = result.ok() + .map(pair -> pair.getKey().kt().getSpec().getTopicName()) + .collect(Collectors.toSet()); + assertThat(resultTopicNames, is(topicNamesToDelete)); + } + + private KafkaTopic buildTopic(String name, int partitions, int replicas) { + return new KafkaTopicBuilder() + .withNewMetadata() + .withName(name.replaceAll("_", "-")) + .withNamespace(NAMESPACE) + .addToLabels("key", "VALUE") + .endMetadata() + .withNewSpec() + .withTopicName(name) + .withPartitions(partitions) + .withReplicas(replicas) + .endSpec() + .build(); + } + + public static List parseNodes(String input) { + List nodes = new ArrayList<>(); + var nodeStrings = input.split(","); + var id = 0; + for (String nodeString : nodeStrings) { + String[] parts = nodeString.split(":"); + if (parts.length == 2) { + nodes.add(new Node(id++, parts[0], Integer.parseInt(parts[1]))); + } else { + throw new IllegalArgumentException("Invalid node string: " + nodeString); + } + } + return nodes; + } +} diff --git a/topic-operator/src/test/java/io/strimzi/operator/topic/KubernetesHandlerTest.java b/topic-operator/src/test/java/io/strimzi/operator/topic/KubernetesHandlerTest.java new file mode 100644 index 00000000000..967f8a7077d --- /dev/null +++ b/topic-operator/src/test/java/io/strimzi/operator/topic/KubernetesHandlerTest.java @@ -0,0 +1,208 @@ +/* + * 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.topic; + +import io.fabric8.kubernetes.client.KubernetesClient; +import io.micrometer.core.instrument.simple.SimpleMeterRegistry; +import io.strimzi.api.kafka.Crds; +import io.strimzi.api.kafka.model.common.ConditionBuilder; +import io.strimzi.api.kafka.model.topic.KafkaTopic; +import io.strimzi.api.kafka.model.topic.KafkaTopicBuilder; +import io.strimzi.api.kafka.model.topic.KafkaTopicStatusBuilder; +import io.strimzi.operator.common.model.StatusUtils; +import io.strimzi.operator.topic.metrics.TopicOperatorMetricsHolder; +import io.strimzi.operator.topic.metrics.TopicOperatorMetricsProvider; +import io.strimzi.test.mockkube3.MockKube3; +import org.apache.kafka.common.config.TopicConfig; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.List; +import java.util.Map; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; +import static org.junit.jupiter.api.Assertions.assertTrue; + +class KubernetesHandlerTest { + private static final String NAMESPACE = TopicOperatorTestUtil.namespaceName(KubernetesHandlerTest.class); + + private static MockKube3 mockKube; + private static KubernetesClient kubernetesClient; + private KubernetesHandler kubernetesHandler; + + @BeforeAll + public static void beforeAll() { + mockKube = new MockKube3.MockKube3Builder() + .withKafkaTopicCrd() + .withDeletionController() + .withNamespaces(NAMESPACE) + .build(); + mockKube.start(); + kubernetesClient = mockKube.client(); + } + + @AfterAll + public static void afterAll() { + mockKube.stop(); + } + + @BeforeEach + public void beforeEach() { + var config = TopicOperatorConfig.buildFromMap(Map.of( + TopicOperatorConfig.BOOTSTRAP_SERVERS.key(), "localhost:9092", + TopicOperatorConfig.NAMESPACE.key(), NAMESPACE + )); + var metricsHolder = new TopicOperatorMetricsHolder(KafkaTopic.RESOURCE_KIND, null, + new TopicOperatorMetricsProvider(new SimpleMeterRegistry())); + kubernetesHandler = new KubernetesHandler(config, metricsHolder, kubernetesClient); + } + + @AfterEach + public void afterEach() { + TopicOperatorTestUtil.cleanupNamespace(kubernetesClient, NAMESPACE); + } + + @Test + public void addFinalizerShouldWork() { + var kafkaTopic = createTopic("my-topic", false); + assertTrue(kafkaTopic.getMetadata().getFinalizers().isEmpty()); + + var update = kubernetesHandler.addFinalizer(TopicOperatorTestUtil.reconcilableTopic(kafkaTopic, NAMESPACE)); + assertThat(update.getMetadata().getFinalizers().get(0), is(KubernetesHandler.FINALIZER_STRIMZI_IO_TO)); + } + + @Test + public void addFinalizerShouldBeIdempotent() { + var kafkaTopic = createTopic("my-topic", false); + assertTrue(kafkaTopic.getMetadata().getFinalizers().isEmpty()); + + var update1 = kubernetesHandler.addFinalizer(TopicOperatorTestUtil.reconcilableTopic(kafkaTopic, NAMESPACE)); + var update2 = kubernetesHandler.addFinalizer(TopicOperatorTestUtil.reconcilableTopic(update1, NAMESPACE)); + assertThat(update2.getMetadata().getFinalizers().size(), is(1)); + } + + @Test + public void removeFinalizerShouldWork() { + var kafkaTopic = createTopic("my-topic", true); + assertThat(kafkaTopic.getMetadata().getFinalizers().get(0), is(KubernetesHandler.FINALIZER_STRIMZI_IO_TO)); + + var update = kubernetesHandler.removeFinalizer(TopicOperatorTestUtil.reconcilableTopic(kafkaTopic, NAMESPACE)); + assertTrue(update.getMetadata().getFinalizers().isEmpty()); + } + + @Test + public void shouldNotUpdateStatusWithNoChanges() { + var kafkaTopic = createTopicWithReadyState("my-topic", Map.of(TopicConfig.RETENTION_MS_CONFIG, "604800000")); + + var update1 = kubernetesHandler.updateStatus(TopicOperatorTestUtil.reconcilableTopic(kafkaTopic, NAMESPACE)); + assertThat(update1.getStatus().getObservedGeneration(), is(1L)); + + var change = TopicOperatorTestUtil.changeTopic(kubernetesClient, update1, kt -> kt); + + var update = kubernetesHandler.updateStatus(TopicOperatorTestUtil.reconcilableTopic(change, NAMESPACE)); + assertThat(update.getStatus().getObservedGeneration(), is(1L)); + } + + @Test + public void shouldUpdateStatusWithConfigChange() { + var kafkaTopic = createTopicWithReadyState("my-topic", Map.of(TopicConfig.RETENTION_MS_CONFIG, "604800000")); + + var update1 = kubernetesHandler.updateStatus(TopicOperatorTestUtil.reconcilableTopic(kafkaTopic, NAMESPACE)); + assertThat(update1.getStatus().getObservedGeneration(), is(1L)); + + var change = TopicOperatorTestUtil.changeTopic(kubernetesClient, update1, kt -> { + kt.getSpec().setConfig(Map.of(TopicConfig.RETENTION_MS_CONFIG, "86400000")); + return kt; + }); + + var update2 = kubernetesHandler.updateStatus(TopicOperatorTestUtil.reconcilableTopic(change, NAMESPACE)); + assertThat(update2.getStatus().getObservedGeneration(), is(2L)); + } + + @Test + public void shouldUpdateStatusWithPartitionChange() { + var kafkaTopic = createTopicWithReadyState("my-topic", Map.of(TopicConfig.RETENTION_MS_CONFIG, "604800000")); + + var update1 = kubernetesHandler.updateStatus(TopicOperatorTestUtil.reconcilableTopic(kafkaTopic, NAMESPACE)); + assertThat(update1.getStatus().getObservedGeneration(), is(1L)); + + var change = TopicOperatorTestUtil.changeTopic(kubernetesClient, update1, kt -> { + kt.getSpec().setPartitions(3); + return kt; + }); + + var update2 = kubernetesHandler.updateStatus(TopicOperatorTestUtil.reconcilableTopic(change, NAMESPACE)); + assertThat(update2.getStatus().getObservedGeneration(), is(2L)); + } + + @Test + public void shouldUpdateStatusWithReplicasChange() { + var kafkaTopic = createTopicWithReadyState("my-topic", Map.of(TopicConfig.RETENTION_MS_CONFIG, "604800000")); + + var update1 = kubernetesHandler.updateStatus(TopicOperatorTestUtil.reconcilableTopic(kafkaTopic, NAMESPACE)); + assertThat(update1.getStatus().getObservedGeneration(), is(1L)); + + var change = TopicOperatorTestUtil.changeTopic(kubernetesClient, update1, kt -> { + kt.getSpec().setReplicas(2); + return kt; + }); + + var update2 = kubernetesHandler.updateStatus(TopicOperatorTestUtil.reconcilableTopic(change, NAMESPACE)); + assertThat(update2.getStatus().getObservedGeneration(), is(2L)); + } + + private KafkaTopic createTopic(String name, boolean withFinalizer) { + var kafkaTopic = new KafkaTopicBuilder() + .withNewMetadata() + .withName(name) + .withNamespace(NAMESPACE) + .addToLabels("key", "VALUE") + .endMetadata() + .withNewSpec() + .withConfig(Map.of( + TopicConfig.COMPRESSION_TYPE_CONFIG, "snappy", + TopicConfig.CLEANUP_POLICY_CONFIG, "compact")) + .withPartitions(2) + .withReplicas(1) + .endSpec() + .build(); + if (withFinalizer) { + kafkaTopic.getMetadata().getFinalizers().add(KubernetesHandler.FINALIZER_STRIMZI_IO_TO); + } + return Crds.topicOperation(kubernetesClient).resource(kafkaTopic).create(); + } + + private static KafkaTopic createTopicWithReadyState(String name, Map config) { + var kafkaTopic = new KafkaTopicBuilder() + .withNewMetadata() + .withGeneration(1L) + .withName(name) + .withNamespace(NAMESPACE) + .addToLabels("key", "VALUE") + .endMetadata() + .withNewSpec() + .withConfig(config) + .withPartitions(2) + .withReplicas(1) + .endSpec() + .withStatus(new KafkaTopicStatusBuilder() + .withObservedGeneration(1L) + .withTopicName(name) + .withTopicId("WyhsoDQRSXqa8k2myxQrrA") + .withConditions(List.of(new ConditionBuilder() + .withType("Ready") + .withStatus("True") + .withLastTransitionTime(StatusUtils.iso8601Now()) + .build())) + .build()) + .build(); + Crds.topicOperation(kubernetesClient).resource(kafkaTopic).create(); + return Crds.topicOperation(kubernetesClient).resource(kafkaTopic).updateStatus(); + } +} diff --git a/topic-operator/src/test/java/io/strimzi/operator/topic/TopicControllerIT.java b/topic-operator/src/test/java/io/strimzi/operator/topic/TopicControllerIT.java index 1635e86414f..f836fdab16b 100644 --- a/topic-operator/src/test/java/io/strimzi/operator/topic/TopicControllerIT.java +++ b/topic-operator/src/test/java/io/strimzi/operator/topic/TopicControllerIT.java @@ -86,7 +86,6 @@ import java.util.function.UnaryOperator; import java.util.stream.Collectors; -import static java.lang.String.format; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertInstanceOf; @@ -289,7 +288,7 @@ private void maybeStartOperator(TopicOperatorConfig config) throws ExecutionExce } if (operator == null) { this.operatorConfig = config; - operator = TopicOperatorMain.operator(config, kubernetesClient, kafkaAdminClientOp[0]); + operator = TopicOperatorMain.operator(config, kafkaAdminClientOp[0]); assertFalse(operator.queue.isAlive()); assertFalse(operator.queue.isReady()); operator.start(); @@ -485,7 +484,7 @@ private void assertCreateSuccess(KafkaTopic kt, KafkaTopic reconciled, // Check updates to the KafkaTopic assertNotNull(reconciled.getMetadata().getFinalizers()); - assertEquals(operatorConfig.useFinalizer(), reconciled.getMetadata().getFinalizers().contains(BatchingTopicController.FINALIZER)); + assertEquals(operatorConfig.useFinalizer(), reconciled.getMetadata().getFinalizers().contains(KubernetesHandler.FINALIZER_STRIMZI_IO_TO)); assertEquals(expectedTopicName, reconciled.getStatus().getTopicName()); assertNotNull(reconciled.getStatus().getTopicId()); @@ -763,8 +762,8 @@ public void shouldNotUpdateTopicInKafkaWhenKafkaTopicBecomesUnselected( 5L, TimeUnit.SECONDS)) { createTopicAndAssertSuccess(kafkaCluster, kt); - assertTrue(operator.controller.topics.containsKey(expectedTopicName) - || operator.controller.topics.containsKey(expectedTopicName.toUpperCase(Locale.ROOT)), + assertTrue(operator.controller.topicRefs.containsKey(expectedTopicName) + || operator.controller.topicRefs.containsKey(expectedTopicName.toUpperCase(Locale.ROOT)), "Expect selected resource to be present in topics map"); // when @@ -779,7 +778,7 @@ public void shouldNotUpdateTopicInKafkaWhenKafkaTopicBecomesUnselected( LOGGER.debug("##Checking"); } assertNotNull(unmanaged.getMetadata().getFinalizers()); - assertTrue(unmanaged.getMetadata().getFinalizers().contains(BatchingTopicController.FINALIZER)); + assertTrue(unmanaged.getMetadata().getFinalizers().contains(KubernetesHandler.FINALIZER_STRIMZI_IO_TO)); assertNotNull(unmanaged.getStatus().getTopicName(), "Expect status.topicName to be unchanged from post-creation state"); var topicDescription = awaitTopicDescription(expectedTopicName); @@ -787,7 +786,7 @@ public void shouldNotUpdateTopicInKafkaWhenKafkaTopicBecomesUnselected( assertEquals(Set.of(kt.getSpec().getReplicas()), replicationFactors(topicDescription)); assertEquals(Map.of(), topicConfigMap(expectedTopicName)); - Map> topics = new HashMap<>(operator.controller.topics); + Map> topics = new HashMap<>(operator.controller.topicRefs); assertFalse(topics.containsKey(expectedTopicName) || topics.containsKey(expectedTopicName.toUpperCase(Locale.ROOT)), "Transition to a non-selected resource should result in removal from topics map: " + topics); @@ -820,8 +819,8 @@ public void shouldUpdateTopicInKafkaWhenKafkaTopicBecomesSelected( assertUnknownTopic(expectedTopicName); assertNull(created.getStatus(), "Expect status not to be set"); assertTrue(created.getMetadata().getFinalizers().isEmpty()); - assertFalse(operator.controller.topics.containsKey(expectedTopicName) - || operator.controller.topics.containsKey(expectedTopicName.toUpperCase(Locale.ROOT)), + assertFalse(operator.controller.topicRefs.containsKey(expectedTopicName) + || operator.controller.topicRefs.containsKey(expectedTopicName.toUpperCase(Locale.ROOT)), "Expect unselected resource to be absent from topics map"); // when @@ -834,18 +833,18 @@ public void shouldUpdateTopicInKafkaWhenKafkaTopicBecomesSelected( readyIsTrue()); // then - assertTrue(operator.controller.topics.containsKey(expectedTopicName) - || operator.controller.topics.containsKey(expectedTopicName.toUpperCase(Locale.ROOT)), + assertTrue(operator.controller.topicRefs.containsKey(expectedTopicName) + || operator.controller.topicRefs.containsKey(expectedTopicName.toUpperCase(Locale.ROOT)), "Expect selected resource to be present in topics map"); assertNotNull(managed.getMetadata().getFinalizers()); - assertTrue(managed.getMetadata().getFinalizers().contains(BatchingTopicController.FINALIZER)); + assertTrue(managed.getMetadata().getFinalizers().contains(KubernetesHandler.FINALIZER_STRIMZI_IO_TO)); assertNotNull(managed.getStatus().getTopicName(), "Expect status.topicName to be unchanged from post-creation state"); var topicDescription = awaitTopicDescription(expectedTopicName); assertEquals(3, numPartitions(topicDescription)); - assertTrue(operator.controller.topics.containsKey(expectedTopicName) - || operator.controller.topics.containsKey(expectedTopicName.toUpperCase(Locale.ROOT)), + assertTrue(operator.controller.topicRefs.containsKey(expectedTopicName) + || operator.controller.topicRefs.containsKey(expectedTopicName.toUpperCase(Locale.ROOT)), "Expect selected resource to be present in topics map"); } @@ -1189,7 +1188,7 @@ public void shouldRestoreFinalizerIfRemoved( // when: The finalizer is removed LOGGER.debug("Removing finalizer"); var postUpdate = TopicOperatorTestUtil.changeTopic(kubernetesClient, created, theKt1 -> { - theKt1.getMetadata().getFinalizers().remove(BatchingTopicController.FINALIZER); + theKt1.getMetadata().getFinalizers().remove(KubernetesHandler.FINALIZER_STRIMZI_IO_TO); return theKt1; }); var postUpdateGeneration = postUpdate.getMetadata().getGeneration(); @@ -1198,7 +1197,7 @@ public void shouldRestoreFinalizerIfRemoved( // then: We expect the operator to revert the finalizer waitUntil(postUpdate, theKt -> theKt.getStatus().getObservedGeneration() >= postUpdateGeneration - && theKt.getMetadata().getFinalizers().contains(BatchingTopicController.FINALIZER)); + && theKt.getMetadata().getFinalizers().contains(KubernetesHandler.FINALIZER_STRIMZI_IO_TO)); } @ParameterizedTest @@ -1428,7 +1427,7 @@ public void shouldDetectMultipleResourcesManagingSameTopicInKafka( // then assertNull(st1.getConditions().get(0).getReason()); assertEquals(TopicOperatorException.Reason.RESOURCE_CONFLICT.value, st2.getConditions().get(0).getReason()); - assertEquals(format("Managed by Ref{namespace='%s', name='%s'}", NAMESPACE, "kt1"), + assertEquals(String.format("Managed by Ref{namespace='%s', name='%s'}", NAMESPACE, "kt1"), st2.getConditions().get(0).getMessage()); } @@ -1515,6 +1514,7 @@ public void shouldFailChangeToSpecTopicName( operated -> { assertEquals("Changing spec.topicName is not supported", assertExactlyOneCondition(operated).getMessage()); assertEquals(TopicOperatorException.Reason.NOT_SUPPORTED.value, assertExactlyOneCondition(operated).getReason()); + assertEquals(expectedTopicName, operated.getStatus().getTopicName()); }, theKt -> { theKt.getSpec().setTopicName(expectedTopicName); @@ -1847,16 +1847,16 @@ private void invalidConfigFailsReconciliation( ) throws ExecutionException, InterruptedException { Map configs = new HashMap<>(); configs.put("cleanup.policy", policy); - KafkaTopic kafkaTopic = new KafkaTopicBuilder() + var kafkaTopic = new KafkaTopicBuilder() .withNewMetadata() - .withNamespace(NAMESPACE) - .withName("my-topic") - .withLabels(SELECTOR) + .withNamespace(NAMESPACE) + .withName("my-topic") + .withLabels(SELECTOR) .endMetadata() .withNewSpec() - .withConfig(configs) - .withPartitions(1) - .withReplicas(1) + .withConfig(configs) + .withPartitions(1) + .withReplicas(1) .endSpec() .build(); var created = createTopic(kafkaCluster, kafkaTopic); @@ -1957,7 +1957,7 @@ public void shouldFailIfNumPartitionsDivergedWithConfigChange(@BrokerConfig(name assertTrue(readyIsFalse().test(secondTopic)); var condition = assertExactlyOneCondition(secondTopic); assertEquals(TopicOperatorException.Reason.RESOURCE_CONFLICT.value, condition.getReason()); - assertEquals(format("Managed by Ref{namespace='%s', name='%s'}", NAMESPACE, firstTopicName), condition.getMessage()); + assertEquals(String.format("Managed by Ref{namespace='%s', name='%s'}", NAMESPACE, firstTopicName), condition.getMessage()); // increase partitions of topic LOGGER.info("Increase partitions of {}", firstTopicName); @@ -2013,7 +2013,7 @@ private void assertKafkaTopicConflict(KafkaTopic failed, KafkaTopic ready) { // the error message should refer to the ready resource name var condition = assertExactlyOneCondition(failed); assertEquals(TopicOperatorException.Reason.RESOURCE_CONFLICT.value, condition.getReason()); - assertEquals(format("Managed by Ref{namespace='%s', name='%s'}", + assertEquals(String.format("Managed by Ref{namespace='%s', name='%s'}", ready.getMetadata().getNamespace(), ready.getMetadata().getName()), condition.getMessage()); // the failed resource should become ready after we unmanage and delete the other @@ -2038,12 +2038,12 @@ private static KafkaFuture failedFuture(Throwable error) { @Test public void shouldLogWarningIfAutoCreateTopicsIsEnabled( - @BrokerConfig(name = BatchingTopicController.AUTO_CREATE_TOPICS_ENABLE, value = "true") + @BrokerConfig(name = KafkaHandler.AUTO_CREATE_TOPICS_ENABLE, value = "true") KafkaCluster kafkaCluster) throws Exception { try (var logCaptor = LogCaptor.logMessageMatches(BatchingTopicController.LOGGER, Level.WARN, - "It is recommended that " + BatchingTopicController.AUTO_CREATE_TOPICS_ENABLE + " is set to 'false' " + + "It is recommended that " + KafkaHandler.AUTO_CREATE_TOPICS_ENABLE + " is set to 'false' " + "to avoid races between the operator and Kafka applications auto-creating topics", 5L, TimeUnit.SECONDS)) { @@ -2112,7 +2112,7 @@ public void shouldTerminateIfQueueFull( @Test public void shouldNotReconcilePausedKafkaTopicOnAdd( - @BrokerConfig(name = BatchingTopicController.AUTO_CREATE_TOPICS_ENABLE, value = "false") + @BrokerConfig(name = KafkaHandler.AUTO_CREATE_TOPICS_ENABLE, value = "false") KafkaCluster kafkaCluster ) throws ExecutionException, InterruptedException { var topicName = "my-topic"; @@ -2280,7 +2280,7 @@ public void shouldReconcileOnTopicExistsException( var config = topicOperatorConfig(NAMESPACE, kafkaCluster); var creteTopicResult = mock(CreateTopicsResult.class); - var existsException = new TopicExistsException(format("Topic '%s' already exists.", topicName)); + var existsException = new TopicExistsException(String.format("Topic '%s' already exists.", topicName)); Mockito.doReturn(failedFuture(existsException)).when(creteTopicResult).all(); Mockito.doReturn(Map.of(topicName, failedFuture(existsException))).when(creteTopicResult).values(); kafkaAdminClientOp = new Admin[]{Mockito.spy(Admin.create(config.adminClientConfig()))}; diff --git a/topic-operator/src/test/java/io/strimzi/operator/topic/TopicOperatorMetricsTest.java b/topic-operator/src/test/java/io/strimzi/operator/topic/TopicOperatorMetricsTest.java index bf2d13c9a47..a973f187d23 100644 --- a/topic-operator/src/test/java/io/strimzi/operator/topic/TopicOperatorMetricsTest.java +++ b/topic-operator/src/test/java/io/strimzi/operator/topic/TopicOperatorMetricsTest.java @@ -10,11 +10,13 @@ import io.kroxylicious.testing.kafka.junit5ext.KafkaClusterExtension; import io.micrometer.core.instrument.search.MeterNotFoundException; import io.micrometer.core.instrument.simple.SimpleMeterRegistry; +import io.strimzi.api.ResourceAnnotations; import io.strimzi.api.kafka.Crds; import io.strimzi.api.kafka.model.topic.KafkaTopic; import io.strimzi.api.kafka.model.topic.KafkaTopicBuilder; import io.strimzi.operator.common.metrics.MetricsHolder; import io.strimzi.operator.topic.cruisecontrol.CruiseControlClient; +import io.strimzi.operator.topic.cruisecontrol.CruiseControlHandler; import io.strimzi.operator.topic.metrics.TopicOperatorMetricsHolder; import io.strimzi.operator.topic.metrics.TopicOperatorMetricsProvider; import io.strimzi.operator.topic.model.TopicEvent.TopicUpsert; @@ -38,30 +40,6 @@ import java.util.concurrent.TimeUnit; import java.util.function.UnaryOperator; -import static io.strimzi.api.ResourceAnnotations.ANNO_STRIMZI_IO_PAUSE_RECONCILIATION; -import static io.strimzi.api.kafka.model.topic.KafkaTopic.RESOURCE_KIND; -import static io.strimzi.operator.common.metrics.MetricsHolder.METRICS_RECONCILIATIONS; -import static io.strimzi.operator.common.metrics.MetricsHolder.METRICS_RECONCILIATIONS_DURATION; -import static io.strimzi.operator.common.metrics.MetricsHolder.METRICS_RECONCILIATIONS_FAILED; -import static io.strimzi.operator.common.metrics.MetricsHolder.METRICS_RECONCILIATIONS_LOCKED; -import static io.strimzi.operator.common.metrics.MetricsHolder.METRICS_RECONCILIATIONS_SUCCESSFUL; -import static io.strimzi.operator.common.metrics.MetricsHolder.METRICS_RESOURCES; -import static io.strimzi.operator.common.metrics.MetricsHolder.METRICS_RESOURCES_PAUSED; -import static io.strimzi.operator.topic.metrics.TopicOperatorMetricsHolder.METRICS_ADD_FINALIZER_DURATION; -import static io.strimzi.operator.topic.metrics.TopicOperatorMetricsHolder.METRICS_ALTER_CONFIGS_DURATION; -import static io.strimzi.operator.topic.metrics.TopicOperatorMetricsHolder.METRICS_CC_TOPIC_CONFIG_DURATION; -import static io.strimzi.operator.topic.metrics.TopicOperatorMetricsHolder.METRICS_CC_USER_TASKS_DURATION; -import static io.strimzi.operator.topic.metrics.TopicOperatorMetricsHolder.METRICS_CREATE_PARTITIONS_DURATION; -import static io.strimzi.operator.topic.metrics.TopicOperatorMetricsHolder.METRICS_CREATE_TOPICS_DURATION; -import static io.strimzi.operator.topic.metrics.TopicOperatorMetricsHolder.METRICS_DELETE_TOPICS_DURATION; -import static io.strimzi.operator.topic.metrics.TopicOperatorMetricsHolder.METRICS_DESCRIBE_CONFIGS_DURATION; -import static io.strimzi.operator.topic.metrics.TopicOperatorMetricsHolder.METRICS_DESCRIBE_TOPICS_DURATION; -import static io.strimzi.operator.topic.metrics.TopicOperatorMetricsHolder.METRICS_LIST_REASSIGNMENTS_DURATION; -import static io.strimzi.operator.topic.metrics.TopicOperatorMetricsHolder.METRICS_RECONCILIATIONS_MAX_BATCH_SIZE; -import static io.strimzi.operator.topic.metrics.TopicOperatorMetricsHolder.METRICS_RECONCILIATIONS_MAX_QUEUE_SIZE; -import static io.strimzi.operator.topic.metrics.TopicOperatorMetricsHolder.METRICS_REMOVE_FINALIZER_DURATION; -import static io.strimzi.operator.topic.metrics.TopicOperatorMetricsHolder.METRICS_UPDATE_TOPICS_DURATION; -import static java.lang.String.format; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.greaterThan; @@ -108,53 +86,45 @@ public void eventHandlerMetrics() throws InterruptedException { TopicOperatorConfig.BOOTSTRAP_SERVERS.key(), "localhost:9092", TopicOperatorConfig.NAMESPACE.key(), NAMESPACE) ); - var metricsHolder = new TopicOperatorMetricsHolder(RESOURCE_KIND, null, new TopicOperatorMetricsProvider(new SimpleMeterRegistry())); - var eventHandler = new TopicOperatorEventHandler(config, mock(BatchingLoop.class), metricsHolder); + var metricsHolder = new TopicOperatorMetricsHolder(KafkaTopic.RESOURCE_KIND, null, new TopicOperatorMetricsProvider(new SimpleMeterRegistry())); + var eventHandler = new TopicEventHandler(config, mock(BatchingLoop.class), metricsHolder); var numOfTestResources = 100; for (int i = 0; i < numOfTestResources; i++) { KafkaTopic kafkaTopic = buildTopicWithVersion("my-topic" + i); eventHandler.onAdd(kafkaTopic); } - assertMetricMatches(metricsHolder, METRICS_RESOURCES, "gauge", is(Double.valueOf(numOfTestResources))); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RESOURCES, "gauge", is(Double.valueOf(numOfTestResources))); for (int i = 0; i < numOfTestResources; i++) { KafkaTopic kafkaTopic = buildTopicWithVersion("my-topic" + i); eventHandler.onDelete(kafkaTopic, false); } - assertMetricMatches(metricsHolder, METRICS_RESOURCES, "gauge", is(0.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RESOURCES, "gauge", is(0.0)); var t1 = buildTopicWithVersion("my-topic-1"); var t2 = buildTopicWithVersion("my-topic-2"); - t2.getMetadata().setAnnotations(Map.of(ANNO_STRIMZI_IO_PAUSE_RECONCILIATION, "true")); + t2.getMetadata().setAnnotations(Map.of(ResourceAnnotations.ANNO_STRIMZI_IO_PAUSE_RECONCILIATION, "true")); eventHandler.onUpdate(t1, t2); - assertMetricMatches(metricsHolder, METRICS_RESOURCES_PAUSED, "gauge", is(1.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RESOURCES_PAUSED, "gauge", is(1.0)); var t3 = buildTopicWithVersion("t3"); - t3.getMetadata().setAnnotations(Map.of(ANNO_STRIMZI_IO_PAUSE_RECONCILIATION, "false")); + t3.getMetadata().setAnnotations(Map.of(ResourceAnnotations.ANNO_STRIMZI_IO_PAUSE_RECONCILIATION, "false")); eventHandler.onUpdate(t2, t3); - assertMetricMatches(metricsHolder, METRICS_RESOURCES_PAUSED, "gauge", is(0.0)); - } - - private KafkaTopic buildTopicWithVersion(String name) { - return new KafkaTopicBuilder() - .editOrNewMetadata() - .withNamespace(NAMESPACE) - .withName(name) - .withResourceVersion("100100") - .endMetadata() - .build(); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RESOURCES_PAUSED, "gauge", is(0.0)); } @Test public void batchingLoopMetrics() throws InterruptedException { - var controller = mock(BatchingTopicController.class); - var itemStore = mock(ItemStore.class); - var stop = mock(Runnable.class); - var metricsHolder = new TopicOperatorMetricsHolder(RESOURCE_KIND, null, - new TopicOperatorMetricsProvider(new SimpleMeterRegistry())); - var batchingLoop = new BatchingLoop(MAX_QUEUE_SIZE, controller, 1, - MAX_BATCH_SIZE, MAX_BATCH_LINGER_MS, itemStore, stop, metricsHolder, NAMESPACE); + var config = TopicOperatorConfig.buildFromMap(Map.of( + TopicOperatorConfig.BOOTSTRAP_SERVERS.key(), "localhost:9092", + TopicOperatorConfig.NAMESPACE.key(), NAMESPACE, + TopicOperatorConfig.MAX_QUEUE_SIZE.key(), String.valueOf(MAX_QUEUE_SIZE), + TopicOperatorConfig.MAX_BATCH_SIZE.key(), String.valueOf(MAX_BATCH_SIZE), + TopicOperatorConfig.MAX_BATCH_LINGER_MS.key(), String.valueOf(MAX_BATCH_LINGER_MS) + )); + var metricsHolder = new TopicOperatorMetricsHolder(KafkaTopic.RESOURCE_KIND, null, new TopicOperatorMetricsProvider(new SimpleMeterRegistry())); + var batchingLoop = new BatchingLoop(config, mock(BatchingTopicController.class), 1, mock(ItemStore.class), mock(Runnable.class), metricsHolder); batchingLoop.start(); int numOfTestResources = 100; @@ -166,11 +136,11 @@ public void batchingLoopMetrics() throws InterruptedException { } } - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS_MAX_QUEUE_SIZE, "gauge", greaterThan(0.0)); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS_MAX_QUEUE_SIZE, "gauge", lessThanOrEqualTo(Double.valueOf(MAX_QUEUE_SIZE))); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS_MAX_BATCH_SIZE, "gauge", greaterThan(0.0)); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS_MAX_BATCH_SIZE, "gauge", lessThanOrEqualTo(Double.valueOf(MAX_BATCH_SIZE))); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS_LOCKED, "counter", greaterThan(0.0)); + assertMetricMatches(metricsHolder, TopicOperatorMetricsHolder.METRICS_RECONCILIATIONS_MAX_QUEUE_SIZE, "gauge", greaterThan(0.0)); + assertMetricMatches(metricsHolder, TopicOperatorMetricsHolder.METRICS_RECONCILIATIONS_MAX_QUEUE_SIZE, "gauge", lessThanOrEqualTo(Double.valueOf(MAX_QUEUE_SIZE))); + assertMetricMatches(metricsHolder, TopicOperatorMetricsHolder.METRICS_RECONCILIATIONS_MAX_BATCH_SIZE, "gauge", greaterThan(0.0)); + assertMetricMatches(metricsHolder, TopicOperatorMetricsHolder.METRICS_RECONCILIATIONS_MAX_BATCH_SIZE, "gauge", lessThanOrEqualTo(Double.valueOf(MAX_BATCH_SIZE))); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS_LOCKED, "counter", greaterThan(0.0)); batchingLoop.stop(); } @@ -193,27 +163,31 @@ public void batchingTopicControllerMetrics(KafkaCluster cluster) throws Interrup new CruiseControlClient.UserTask("Active", null, null, userTaskId, System.currentTimeMillis())), 1); Mockito.doReturn(userTaskResponse).when(cruiseControlClient).userTasks(Set.of(userTaskId)); - var metricsHolder = new TopicOperatorMetricsHolder(RESOURCE_KIND, null, new TopicOperatorMetricsProvider(new SimpleMeterRegistry())); - var replicasChangeHandler = new ReplicasChangeHandler(config, metricsHolder, cruiseControlClient); - var controller = new BatchingTopicController(config, Map.of("key", "VALUE"), kafkaAdminClient, kubernetesClient, metricsHolder, replicasChangeHandler); + var metricsHolder = new TopicOperatorMetricsHolder(KafkaTopic.RESOURCE_KIND, null, + new TopicOperatorMetricsProvider(new SimpleMeterRegistry())); + var controller = new BatchingTopicController(config, Map.of("key", "VALUE"), + new KubernetesHandler(config, metricsHolder, kubernetesClient), + new KafkaHandler(config, metricsHolder, kafkaAdminClient), + metricsHolder, + new CruiseControlHandler(config, metricsHolder, cruiseControlClient)); // create topics - var t1 = createTopic("my-topic-a"); - var t2 = createTopic("my-topic-b"); - var t3 = createTopic("my-topic-c"); + var t1 = createTopic("t1"); + var t2 = createTopic("t2"); + var t3 = createTopic("t3"); controller.onUpdate(List.of( TopicOperatorTestUtil.reconcilableTopic(t1, NAMESPACE), TopicOperatorTestUtil.reconcilableTopic(t2, NAMESPACE), TopicOperatorTestUtil.reconcilableTopic(t3, NAMESPACE) )); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS, "counter", is(3.0)); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS_SUCCESSFUL, "counter", is(3.0)); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS_DURATION, "timer", greaterThan(0.0)); - assertMetricMatches(metricsHolder, METRICS_DESCRIBE_TOPICS_DURATION, "timer", greaterThan(0.0)); - assertMetricMatches(metricsHolder, METRICS_DESCRIBE_CONFIGS_DURATION, "timer", greaterThan(0.0)); - assertMetricMatches(metricsHolder, METRICS_CREATE_TOPICS_DURATION, "timer", greaterThan(0.0)); - assertMetricMatches(metricsHolder, METRICS_ADD_FINALIZER_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS, "counter", is(3.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS_SUCCESSFUL, "counter", is(3.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, TopicOperatorMetricsHolder.METRICS_DESCRIBE_TOPICS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, TopicOperatorMetricsHolder.METRICS_DESCRIBE_CONFIGS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, TopicOperatorMetricsHolder.METRICS_CREATE_TOPICS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, TopicOperatorMetricsHolder.METRICS_ADD_FINALIZER_DURATION, "timer", greaterThan(0.0)); // config change var t1ConfigChanged = updateTopic(TopicOperatorUtil.topicName(t1), kt -> { @@ -222,13 +196,13 @@ public void batchingTopicControllerMetrics(KafkaCluster cluster) throws Interrup }); controller.onUpdate(List.of(TopicOperatorTestUtil.reconcilableTopic(t1ConfigChanged, NAMESPACE))); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS, "counter", is(4.0)); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS_SUCCESSFUL, "counter", is(4.0)); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS_DURATION, "timer", greaterThan(0.0)); - assertMetricMatches(metricsHolder, METRICS_DESCRIBE_TOPICS_DURATION, "timer", greaterThan(0.0)); - assertMetricMatches(metricsHolder, METRICS_DESCRIBE_CONFIGS_DURATION, "timer", greaterThan(0.0)); - assertMetricMatches(metricsHolder, METRICS_ALTER_CONFIGS_DURATION, "timer", greaterThan(0.0)); - assertMetricMatches(metricsHolder, METRICS_UPDATE_TOPICS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS, "counter", is(4.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS_SUCCESSFUL, "counter", is(4.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, TopicOperatorMetricsHolder.METRICS_DESCRIBE_TOPICS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, TopicOperatorMetricsHolder.METRICS_DESCRIBE_CONFIGS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, TopicOperatorMetricsHolder.METRICS_ALTER_CONFIGS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, TopicOperatorMetricsHolder.METRICS_UPDATE_TOPICS_DURATION, "timer", greaterThan(0.0)); // increase partitions var t2PartIncreased = updateTopic(TopicOperatorUtil.topicName(t2), kt -> { @@ -237,12 +211,12 @@ public void batchingTopicControllerMetrics(KafkaCluster cluster) throws Interrup }); controller.onUpdate(List.of(TopicOperatorTestUtil.reconcilableTopic(t2PartIncreased, NAMESPACE))); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS, "counter", is(5.0)); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS_SUCCESSFUL, "counter", is(5.0)); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS_DURATION, "timer", greaterThan(0.0)); - assertMetricMatches(metricsHolder, METRICS_DESCRIBE_TOPICS_DURATION, "timer", greaterThan(0.0)); - assertMetricMatches(metricsHolder, METRICS_DESCRIBE_CONFIGS_DURATION, "timer", greaterThan(0.0)); - assertMetricMatches(metricsHolder, METRICS_CREATE_PARTITIONS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS, "counter", is(5.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS_SUCCESSFUL, "counter", is(5.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, TopicOperatorMetricsHolder.METRICS_DESCRIBE_TOPICS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, TopicOperatorMetricsHolder.METRICS_DESCRIBE_CONFIGS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, TopicOperatorMetricsHolder.METRICS_CREATE_PARTITIONS_DURATION, "timer", greaterThan(0.0)); // decrease partitions (failure) var t2PartDecreased = updateTopic(TopicOperatorUtil.topicName(t2), kt -> { @@ -251,29 +225,31 @@ public void batchingTopicControllerMetrics(KafkaCluster cluster) throws Interrup }); controller.onUpdate(List.of(TopicOperatorTestUtil.reconcilableTopic(t2PartDecreased, NAMESPACE))); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS, "counter", is(6.0)); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS_SUCCESSFUL, "counter", is(5.0)); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS_FAILED, "counter", is(1.0)); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS_DURATION, "timer", greaterThan(0.0)); - assertMetricMatches(metricsHolder, METRICS_DESCRIBE_TOPICS_DURATION, "timer", greaterThan(0.0)); - assertMetricMatches(metricsHolder, METRICS_DESCRIBE_CONFIGS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS, "counter", is(6.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS_SUCCESSFUL, "counter", is(5.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS_FAILED, "counter", is(1.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, TopicOperatorMetricsHolder.METRICS_DESCRIBE_TOPICS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, TopicOperatorMetricsHolder.METRICS_DESCRIBE_CONFIGS_DURATION, "timer", greaterThan(0.0)); // increase replicas + // we reconcile two times to trigger requestOngoingChanges operation var t3ReplIncreased = updateTopic(TopicOperatorUtil.topicName(t3), kt -> { kt.getSpec().setReplicas(2); return kt; }); controller.onUpdate(List.of(TopicOperatorTestUtil.reconcilableTopic(t3ReplIncreased, NAMESPACE))); + controller.onUpdate(List.of(TopicOperatorTestUtil.reconcilableTopic(t3ReplIncreased, NAMESPACE))); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS, "counter", is(7.0)); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS_SUCCESSFUL, "counter", is(6.0)); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS_FAILED, "counter", is(1.0)); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS_DURATION, "timer", greaterThan(0.0)); - assertMetricMatches(metricsHolder, METRICS_DESCRIBE_TOPICS_DURATION, "timer", greaterThan(0.0)); - assertMetricMatches(metricsHolder, METRICS_DESCRIBE_CONFIGS_DURATION, "timer", greaterThan(0.0)); - assertMetricMatches(metricsHolder, METRICS_LIST_REASSIGNMENTS_DURATION, "timer", greaterThan(0.0)); - assertMetricMatches(metricsHolder, METRICS_CC_TOPIC_CONFIG_DURATION, "timer", greaterThan(0.0)); - assertMetricMatches(metricsHolder, METRICS_CC_USER_TASKS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS, "counter", is(8.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS_SUCCESSFUL, "counter", is(7.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS_FAILED, "counter", is(1.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, TopicOperatorMetricsHolder.METRICS_DESCRIBE_TOPICS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, TopicOperatorMetricsHolder.METRICS_DESCRIBE_CONFIGS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, TopicOperatorMetricsHolder.METRICS_LIST_REASSIGNMENTS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, TopicOperatorMetricsHolder.METRICS_CC_TOPIC_CONFIG_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, TopicOperatorMetricsHolder.METRICS_CC_USER_TASKS_DURATION, "timer", greaterThan(0.0)); // unmanage topic var t1Unmanaged = updateTopic(TopicOperatorUtil.topicName(t1), kt -> { @@ -282,62 +258,72 @@ public void batchingTopicControllerMetrics(KafkaCluster cluster) throws Interrup }); controller.onUpdate(List.of(TopicOperatorTestUtil.reconcilableTopic(t1Unmanaged, NAMESPACE))); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS, "counter", is(8.0)); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS_SUCCESSFUL, "counter", is(7.0)); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS_FAILED, "counter", is(1.0)); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS_DURATION, "timer", greaterThan(0.0)); - assertMetricMatches(metricsHolder, METRICS_DESCRIBE_TOPICS_DURATION, "timer", greaterThan(0.0)); - assertMetricMatches(metricsHolder, METRICS_DESCRIBE_CONFIGS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS, "counter", is(9.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS_SUCCESSFUL, "counter", is(8.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS_FAILED, "counter", is(1.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, TopicOperatorMetricsHolder.METRICS_DESCRIBE_TOPICS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, TopicOperatorMetricsHolder.METRICS_DESCRIBE_CONFIGS_DURATION, "timer", greaterThan(0.0)); // delete managed topics controller.onDelete(List.of(TopicOperatorTestUtil.reconcilableTopic( Crds.topicOperation(kubernetesClient).resource(t2).get(), NAMESPACE))); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS, "counter", is(9.0)); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS_SUCCESSFUL, "counter", is(8.0)); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS_FAILED, "counter", is(1.0)); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS_DURATION, "timer", greaterThan(0.0)); - assertMetricMatches(metricsHolder, METRICS_DESCRIBE_TOPICS_DURATION, "timer", greaterThan(0.0)); - assertMetricMatches(metricsHolder, METRICS_DESCRIBE_CONFIGS_DURATION, "timer", greaterThan(0.0)); - assertMetricMatches(metricsHolder, METRICS_DELETE_TOPICS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS, "counter", is(10.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS_SUCCESSFUL, "counter", is(9.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS_FAILED, "counter", is(1.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, TopicOperatorMetricsHolder.METRICS_DESCRIBE_TOPICS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, TopicOperatorMetricsHolder.METRICS_DESCRIBE_CONFIGS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, TopicOperatorMetricsHolder.METRICS_DELETE_TOPICS_DURATION, "timer", greaterThan(0.0)); // delete unmanaged topic controller.onDelete(List.of(TopicOperatorTestUtil.reconcilableTopic( Crds.topicOperation(kubernetesClient).resource(t1).get(), NAMESPACE))); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS, "counter", is(10.0)); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS_SUCCESSFUL, "counter", is(9.0)); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS_FAILED, "counter", is(1.0)); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS_DURATION, "timer", greaterThan(0.0)); - assertMetricMatches(metricsHolder, METRICS_DESCRIBE_TOPICS_DURATION, "timer", greaterThan(0.0)); - assertMetricMatches(metricsHolder, METRICS_DESCRIBE_CONFIGS_DURATION, "timer", greaterThan(0.0)); - assertMetricMatches(metricsHolder, METRICS_REMOVE_FINALIZER_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS, "counter", is(11.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS_SUCCESSFUL, "counter", is(10.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS_FAILED, "counter", is(1.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, TopicOperatorMetricsHolder.METRICS_DESCRIBE_TOPICS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, TopicOperatorMetricsHolder.METRICS_DESCRIBE_CONFIGS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, TopicOperatorMetricsHolder.METRICS_REMOVE_FINALIZER_DURATION, "timer", greaterThan(0.0)); // pause topic var t3Paused = updateTopic(TopicOperatorUtil.topicName(t3), kt -> { - kt.getMetadata().setAnnotations(Map.of(ANNO_STRIMZI_IO_PAUSE_RECONCILIATION, "true")); + kt.getMetadata().setAnnotations(Map.of(ResourceAnnotations.ANNO_STRIMZI_IO_PAUSE_RECONCILIATION, "true")); return kt; }); controller.onUpdate(List.of(TopicOperatorTestUtil.reconcilableTopic(t3Paused, NAMESPACE))); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS, "counter", is(11.0)); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS_SUCCESSFUL, "counter", is(10.0)); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS_FAILED, "counter", is(1.0)); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS_DURATION, "timer", greaterThan(0.0)); - assertMetricMatches(metricsHolder, METRICS_DESCRIBE_TOPICS_DURATION, "timer", greaterThan(0.0)); - assertMetricMatches(metricsHolder, METRICS_DESCRIBE_CONFIGS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS, "counter", is(12.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS_SUCCESSFUL, "counter", is(11.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS_FAILED, "counter", is(1.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, TopicOperatorMetricsHolder.METRICS_DESCRIBE_TOPICS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, TopicOperatorMetricsHolder.METRICS_DESCRIBE_CONFIGS_DURATION, "timer", greaterThan(0.0)); // delete paused topic controller.onDelete(List.of(TopicOperatorTestUtil.reconcilableTopic( Crds.topicOperation(kubernetesClient).resource(t3).get(), NAMESPACE))); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS, "counter", is(12.0)); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS_SUCCESSFUL, "counter", is(11.0)); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS_FAILED, "counter", is(1.0)); - assertMetricMatches(metricsHolder, METRICS_RECONCILIATIONS_DURATION, "timer", greaterThan(0.0)); - assertMetricMatches(metricsHolder, METRICS_DESCRIBE_TOPICS_DURATION, "timer", greaterThan(0.0)); - assertMetricMatches(metricsHolder, METRICS_DESCRIBE_CONFIGS_DURATION, "timer", greaterThan(0.0)); - assertMetricMatches(metricsHolder, METRICS_REMOVE_FINALIZER_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS, "counter", is(13.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS_SUCCESSFUL, "counter", is(12.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS_FAILED, "counter", is(1.0)); + assertMetricMatches(metricsHolder, MetricsHolder.METRICS_RECONCILIATIONS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, TopicOperatorMetricsHolder.METRICS_DESCRIBE_TOPICS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, TopicOperatorMetricsHolder.METRICS_DESCRIBE_CONFIGS_DURATION, "timer", greaterThan(0.0)); + assertMetricMatches(metricsHolder, TopicOperatorMetricsHolder.METRICS_REMOVE_FINALIZER_DURATION, "timer", greaterThan(0.0)); + } + + private KafkaTopic buildTopicWithVersion(String name) { + return new KafkaTopicBuilder() + .editOrNewMetadata() + .withNamespace(NAMESPACE) + .withName(name) + .withResourceVersion("100100") + .endMetadata() + .build(); } private KafkaTopic createTopic(String name) { @@ -367,7 +353,7 @@ private void assertMetricMatches(MetricsHolder metricsHolder, String name, Strin try { LOGGER.info("Searching for metric {}", name); var requiredSearch = metricsHolder.metricsProvider().meterRegistry().get(name) - .tags("kind", RESOURCE_KIND, "namespace", NAMESPACE); + .tags("kind", KafkaTopic.RESOURCE_KIND, "namespace", NAMESPACE); switch (type) { case "counter": assertThat(requiredSearch.counter().count(), matcher); @@ -379,7 +365,7 @@ private void assertMetricMatches(MetricsHolder metricsHolder, String name, Strin assertThat(requiredSearch.timer().totalTime(TimeUnit.MILLISECONDS), matcher); break; default: - throw new RuntimeException(format("Unknown metric type %s", type)); + throw new RuntimeException(String.format("Unknown metric type %s", type)); } found = true; } catch (MeterNotFoundException mnfe) { @@ -388,7 +374,7 @@ private void assertMetricMatches(MetricsHolder metricsHolder, String name, Strin } } if (!found) { - throw new RuntimeException(format("Unable to find metric %s", name)); + throw new RuntimeException(String.format("Unable to find metric %s", name)); } } } diff --git a/topic-operator/src/test/java/io/strimzi/operator/topic/TopicOperatorTestUtil.java b/topic-operator/src/test/java/io/strimzi/operator/topic/TopicOperatorTestUtil.java index 013554e074c..8f81728b80c 100644 --- a/topic-operator/src/test/java/io/strimzi/operator/topic/TopicOperatorTestUtil.java +++ b/topic-operator/src/test/java/io/strimzi/operator/topic/TopicOperatorTestUtil.java @@ -16,6 +16,7 @@ import io.strimzi.api.kafka.model.topic.KafkaTopic; import io.strimzi.operator.common.Reconciliation; import io.strimzi.operator.topic.model.ReconcilableTopic; +import io.strimzi.test.TestUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -24,6 +25,7 @@ import java.io.IOException; import java.net.HttpURLConnection; import java.net.URL; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Paths; import java.util.List; @@ -33,11 +35,6 @@ import java.util.function.Predicate; import java.util.function.UnaryOperator; -import static io.strimzi.api.kafka.model.topic.KafkaTopic.RESOURCE_KIND; -import static io.strimzi.test.TestUtils.CRD_TOPIC; -import static io.strimzi.test.TestUtils.USER_PATH; -import static java.lang.String.format; -import static java.nio.charset.StandardCharsets.UTF_8; import static org.junit.jupiter.api.Assertions.fail; public class TopicOperatorTestUtil { @@ -52,10 +49,10 @@ public static String namespaceName(Class clazz) { public static void setupKubeCluster(KubernetesClient kubernetesClient, String namespace) { deleteNamespace(kubernetesClient, namespace); createNamespace(kubernetesClient, namespace); - createOrReplace(kubernetesClient, "file://" + USER_PATH + "/../packaging/install/topic-operator/01-ServiceAccount-strimzi-topic-operator.yaml", namespace); - createOrReplace(kubernetesClient, "file://" + USER_PATH + "/../packaging/install/topic-operator/02-Role-strimzi-topic-operator.yaml", namespace); - createOrReplace(kubernetesClient, "file://" + USER_PATH + "/../packaging/install/topic-operator/03-RoleBinding-strimzi-topic-operator.yaml", namespace); - createOrReplace(kubernetesClient, "file://" + CRD_TOPIC); + createOrReplace(kubernetesClient, "file://" + TestUtils.USER_PATH + "/../packaging/install/topic-operator/01-ServiceAccount-strimzi-topic-operator.yaml", namespace); + createOrReplace(kubernetesClient, "file://" + TestUtils.USER_PATH + "/../packaging/install/topic-operator/02-Role-strimzi-topic-operator.yaml", namespace); + createOrReplace(kubernetesClient, "file://" + TestUtils.USER_PATH + "/../packaging/install/topic-operator/03-RoleBinding-strimzi-topic-operator.yaml", namespace); + createOrReplace(kubernetesClient, "file://" + TestUtils.CRD_TOPIC); } private static void createOrReplace(KubernetesClient kubernetesClient, String resourcesPath) { @@ -137,7 +134,7 @@ public static void cleanupNamespace(KubernetesClient kubernetesClient, String na } public static ReconcilableTopic reconcilableTopic(KafkaTopic kafkaTopic, String namespace) { - return new ReconcilableTopic(new Reconciliation("test", RESOURCE_KIND, namespace, + return new ReconcilableTopic(new Reconciliation("test", KafkaTopic.RESOURCE_KIND, namespace, TopicOperatorUtil.topicName(kafkaTopic)), kafkaTopic, TopicOperatorUtil.topicName(kafkaTopic)); } @@ -182,9 +179,9 @@ public static KafkaTopic findKafkaTopicByName(List topics, String na public static String contentFromTextFile(File filePath) { try { var resourceURI = Objects.requireNonNull(filePath).toURI(); - try (var lines = Files.lines(Paths.get(resourceURI), UTF_8)) { + try (var lines = Files.lines(Paths.get(resourceURI), StandardCharsets.UTF_8)) { var content = lines.reduce((x, y) -> x + y); - return content.orElseThrow(() -> new IOException(format("File %s is empty", filePath.getAbsolutePath()))); + return content.orElseThrow(() -> new IOException(String.format("File %s is empty", filePath.getAbsolutePath()))); } } catch (Throwable t) { throw new RuntimeException(t); diff --git a/topic-operator/src/test/java/io/strimzi/operator/topic/ReplicasChangeHandlerTest.java b/topic-operator/src/test/java/io/strimzi/operator/topic/cruisecontrol/CruiseControlHandlerTest.java similarity index 78% rename from topic-operator/src/test/java/io/strimzi/operator/topic/ReplicasChangeHandlerTest.java rename to topic-operator/src/test/java/io/strimzi/operator/topic/cruisecontrol/CruiseControlHandlerTest.java index 3ce30e21146..6d5b9260d42 100644 --- a/topic-operator/src/test/java/io/strimzi/operator/topic/ReplicasChangeHandlerTest.java +++ b/topic-operator/src/test/java/io/strimzi/operator/topic/cruisecontrol/CruiseControlHandlerTest.java @@ -2,21 +2,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.topic; +package io.strimzi.operator.topic.cruisecontrol; import io.micrometer.core.instrument.simple.SimpleMeterRegistry; import io.strimzi.api.kafka.model.common.ConditionBuilder; +import io.strimzi.api.kafka.model.topic.KafkaTopic; import io.strimzi.api.kafka.model.topic.KafkaTopicBuilder; import io.strimzi.api.kafka.model.topic.KafkaTopicStatusBuilder; +import io.strimzi.api.kafka.model.topic.ReplicasChangeState; import io.strimzi.api.kafka.model.topic.ReplicasChangeStatusBuilder; import io.strimzi.certs.Subject; import io.strimzi.operator.common.Reconciliation; import io.strimzi.operator.common.model.StatusUtils; import io.strimzi.operator.common.operator.MockCertManager; -import io.strimzi.operator.topic.cruisecontrol.MockCruiseControl; +import io.strimzi.operator.topic.TopicOperatorConfig; +import io.strimzi.operator.topic.TopicOperatorTestUtil; +import io.strimzi.operator.topic.TopicOperatorUtil; import io.strimzi.operator.topic.metrics.TopicOperatorMetricsHolder; import io.strimzi.operator.topic.metrics.TopicOperatorMetricsProvider; import io.strimzi.operator.topic.model.ReconcilableTopic; +import io.strimzi.operator.topic.model.Results; import io.strimzi.test.TestUtils; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; @@ -32,17 +37,16 @@ import java.util.List; import java.util.Map; -import static io.strimzi.api.kafka.model.topic.KafkaTopic.RESOURCE_KIND; -import static io.strimzi.api.kafka.model.topic.ReplicasChangeState.ONGOING; import static java.util.Map.entry; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.CoreMatchers.notNullValue; import static org.hamcrest.CoreMatchers.nullValue; import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.number.OrderingComparison.greaterThan; import static org.junit.jupiter.api.Assertions.assertThrows; -public class ReplicasChangeHandlerTest { - private static final String NAMESPACE = TopicOperatorTestUtil.namespaceName(ReplicasChangeHandlerTest.class); +public class CruiseControlHandlerTest { + private static final String NAMESPACE = TopicOperatorTestUtil.namespaceName(CruiseControlHandlerTest.class); private static TopicOperatorMetricsHolder metricsHolder; private static int serverPort; @@ -53,19 +57,19 @@ public class ReplicasChangeHandlerTest { @BeforeAll public static void beforeAll() throws IOException { - metricsHolder = new TopicOperatorMetricsHolder(RESOURCE_KIND, null, + metricsHolder = new TopicOperatorMetricsHolder(KafkaTopic.RESOURCE_KIND, null, new TopicOperatorMetricsProvider(new SimpleMeterRegistry())); serverPort = TestUtils.getFreePort(); - File tlsKeyFile = TestUtils.tempFile(ReplicasChangeHandlerTest.class.getSimpleName(), ".key"); - tlsCrtFile = TestUtils.tempFile(ReplicasChangeHandlerTest.class.getSimpleName(), ".crt"); + File tlsKeyFile = TestUtils.tempFile(CruiseControlHandlerTest.class.getSimpleName(), ".key"); + tlsCrtFile = TestUtils.tempFile(CruiseControlHandlerTest.class.getSimpleName(), ".crt"); new MockCertManager().generateSelfSignedCert(tlsKeyFile, tlsCrtFile, new Subject.Builder().withCommonName("Trusted Test CA").build(), 365); - apiUserFile = TestUtils.tempFile(ReplicasChangeHandlerTest.class.getSimpleName(), ".username"); + apiUserFile = TestUtils.tempFile(CruiseControlHandlerTest.class.getSimpleName(), ".username"); try (PrintWriter out = new PrintWriter(apiUserFile.getAbsolutePath())) { out.print("topic-operator-admin"); } - apiPassFile = TestUtils.tempFile(ReplicasChangeHandlerTest.class.getSimpleName(), ".password"); + apiPassFile = TestUtils.tempFile(CruiseControlHandlerTest.class.getSimpleName(), ".password"); try (PrintWriter out = new PrintWriter(apiPassFile.getAbsolutePath())) { out.print("changeit"); } @@ -89,7 +93,7 @@ public void beforeEach() { @ParameterizedTest @MethodSource("validConfigs") public void shouldSucceedWithValidConfig(TopicOperatorConfig config) { - var handler = new ReplicasChangeHandler(config, metricsHolder); + var handler = new CruiseControlHandler(config, metricsHolder, TopicOperatorUtil.createCruiseControlClient(config)); server.expectTopicConfigSuccessResponse(apiUserFile, apiPassFile); var pending = buildPendingReconcilableTopics(); @@ -99,7 +103,7 @@ public void shouldSucceedWithValidConfig(TopicOperatorConfig config) { server.expectUserTasksSuccessResponse(apiUserFile, apiPassFile); var ongoing = buildOngoingReconcilableTopics(); var completedAndFailed = handler.requestOngoingChanges(ongoing); - assertCompleted(completedAndFailed); + assertCompleted(ongoing, completedAndFailed); } @Test @@ -113,7 +117,8 @@ public void shouldFailWithSslEnabledAndMissingCrtFile() { entry(TopicOperatorConfig.CRUISE_CONTROL_CRT_FILE_PATH.key(), "/invalid/ca.crt") )); - RuntimeException thrown = assertThrows(RuntimeException.class, () -> new ReplicasChangeHandler(config, metricsHolder)); + RuntimeException thrown = assertThrows(RuntimeException.class, () -> + new CruiseControlHandler(config, metricsHolder, TopicOperatorUtil.createCruiseControlClient(config))); assertThat(thrown.getMessage(), is("File not found: /invalid/ca.crt")); } @@ -129,7 +134,8 @@ public void shouldFailWithAuthEnabledAndUsernameFileNotFound() { entry(TopicOperatorConfig.CRUISE_CONTROL_API_PASS_PATH.key(), apiPassFile.getAbsolutePath()) )); - RuntimeException thrown = assertThrows(RuntimeException.class, () -> new ReplicasChangeHandler(config, metricsHolder)); + RuntimeException thrown = assertThrows(RuntimeException.class, () -> + new CruiseControlHandler(config, metricsHolder, TopicOperatorUtil.createCruiseControlClient(config))); assertThat(thrown.getMessage(), is("File not found: /invalid/username")); } @@ -145,12 +151,13 @@ public void shouldFailWithAuthEnabledAndPasswordFileNotFound() { entry(TopicOperatorConfig.CRUISE_CONTROL_API_PASS_PATH.key(), "/invalid/password") )); - RuntimeException thrown = assertThrows(RuntimeException.class, () -> new ReplicasChangeHandler(config, metricsHolder)); + RuntimeException thrown = assertThrows(RuntimeException.class, () -> + new CruiseControlHandler(config, metricsHolder, TopicOperatorUtil.createCruiseControlClient(config))); assertThat(thrown.getMessage(), is("File not found: /invalid/password")); } @Test - public void shouldFailWhenCruiseControlEndpointNotReachable() { + public void replicasChangeShouldFailWhenCruiseControlEndpointNotReachable() { var config = TopicOperatorConfig.buildFromMap(Map.ofEntries( entry(TopicOperatorConfig.BOOTSTRAP_SERVERS.key(), "localhost:9092"), entry(TopicOperatorConfig.NAMESPACE.key(), NAMESPACE), @@ -158,19 +165,19 @@ public void shouldFailWhenCruiseControlEndpointNotReachable() { entry(TopicOperatorConfig.CRUISE_CONTROL_PORT.key(), String.valueOf(serverPort)) )); - var handler = new ReplicasChangeHandler(config, metricsHolder); + var handler = new CruiseControlHandler(config, metricsHolder, TopicOperatorUtil.createCruiseControlClient(config)); var pending = buildPendingReconcilableTopics(); var pendingAndOngoing = handler.requestPendingChanges(pending); - assertFailedWithMessage(pendingAndOngoing, "Replicas change failed, Connection failed"); + assertFailedWithMessage(pending, pendingAndOngoing, "Replicas change failed, Connection failed"); var ongoing = buildOngoingReconcilableTopics(); var completedAndFailed = handler.requestOngoingChanges(ongoing); - assertFailedWithMessage(completedAndFailed, "Replicas change failed, Connection failed"); + assertFailedWithMessage(ongoing, completedAndFailed, "Replicas change failed, Connection failed"); } @Test - public void shouldFailWhenCruiseControlReturnsErrorResponse() { + public void replicasChangeShouldFailWhenCruiseControlReturnsErrorResponse() { var config = TopicOperatorConfig.buildFromMap(Map.ofEntries( entry(TopicOperatorConfig.BOOTSTRAP_SERVERS.key(), "localhost:9092"), entry(TopicOperatorConfig.NAMESPACE.key(), NAMESPACE), @@ -183,22 +190,22 @@ public void shouldFailWhenCruiseControlReturnsErrorResponse() { entry(TopicOperatorConfig.CRUISE_CONTROL_API_PASS_PATH.key(), apiPassFile.getAbsolutePath()) )); - var handler = new ReplicasChangeHandler(config, metricsHolder); + var handler = new CruiseControlHandler(config, metricsHolder, TopicOperatorUtil.createCruiseControlClient(config)); server.expectTopicConfigErrorResponse(apiUserFile, apiPassFile); var pending = buildPendingReconcilableTopics(); var pendingAndOngoing = handler.requestPendingChanges(pending); - assertFailedWithMessage(pendingAndOngoing, "Replicas change failed, Request failed (500), Cluster model not ready"); + assertFailedWithMessage(pending, pendingAndOngoing, "Replicas change failed, Request failed (500), Cluster model not ready"); server.expectUserTasksErrorResponse(apiUserFile, apiPassFile); var ongoing = buildOngoingReconcilableTopics(); var completedAndFailed = handler.requestOngoingChanges(ongoing); - assertFailedWithMessage(completedAndFailed, "Replicas change failed, Request failed (500), " + + assertFailedWithMessage(ongoing, completedAndFailed, "Replicas change failed, Request failed (500), " + "Error processing GET request '/user_tasks' due to: 'Error happened in fetching response for task 9730e4fb-ea41-4e2d-b053-9be2310589b5'."); } @Test - public void shouldFailWhenTheRequestTimesOut() { + public void replicasChangeShouldFailWhenTheRequestTimesOut() { var config = TopicOperatorConfig.buildFromMap(Map.ofEntries( entry(TopicOperatorConfig.BOOTSTRAP_SERVERS.key(), "localhost:9092"), entry(TopicOperatorConfig.NAMESPACE.key(), NAMESPACE), @@ -211,21 +218,21 @@ public void shouldFailWhenTheRequestTimesOut() { entry(TopicOperatorConfig.CRUISE_CONTROL_API_PASS_PATH.key(), apiPassFile.getAbsolutePath()) )); - var handler = new ReplicasChangeHandler(config, metricsHolder); + var handler = new CruiseControlHandler(config, metricsHolder, TopicOperatorUtil.createCruiseControlClient(config)); server.expectTopicConfigRequestTimeout(apiUserFile, apiPassFile); var pending = buildPendingReconcilableTopics(); var pendingAndOngoing = handler.requestPendingChanges(pending); - assertFailedWithMessage(pendingAndOngoing, "Replicas change failed, Request failed (408)"); + assertFailedWithMessage(pending, pendingAndOngoing, "Replicas change failed, Request failed (408)"); server.expectUserTasksRequestTimeout(apiUserFile, apiPassFile); var ongoing = buildOngoingReconcilableTopics(); var completedAndFailed = handler.requestOngoingChanges(ongoing); - assertFailedWithMessage(completedAndFailed, "Replicas change failed, Request failed (408)"); + assertFailedWithMessage(ongoing, completedAndFailed, "Replicas change failed, Request failed (408)"); } @Test - public void shouldFailWhenTheRequestIsUnauthorized() { + public void replicasChangeShouldFailWhenTheRequestIsUnauthorized() { var config = TopicOperatorConfig.buildFromMap(Map.ofEntries( entry(TopicOperatorConfig.BOOTSTRAP_SERVERS.key(), "localhost:9092"), entry(TopicOperatorConfig.NAMESPACE.key(), NAMESPACE), @@ -238,46 +245,48 @@ public void shouldFailWhenTheRequestIsUnauthorized() { entry(TopicOperatorConfig.CRUISE_CONTROL_API_PASS_PATH.key(), apiPassFile.getAbsolutePath()) )); - var handler = new ReplicasChangeHandler(config, metricsHolder); + var handler = new CruiseControlHandler(config, metricsHolder, TopicOperatorUtil.createCruiseControlClient(config)); server.expectTopicConfigRequestUnauthorized(apiUserFile, apiPassFile); var pending = buildPendingReconcilableTopics(); var pendingAndOngoing = handler.requestPendingChanges(pending); - assertFailedWithMessage(pendingAndOngoing, "Replicas change failed, Request failed (401), Authorization error"); + assertFailedWithMessage(pending, pendingAndOngoing, "Replicas change failed, Request failed (401), Authorization error"); server.expectUserTasksRequestUnauthorized(apiUserFile, apiPassFile); var ongoing = buildOngoingReconcilableTopics(); var completedAndFailed = handler.requestOngoingChanges(ongoing); - assertFailedWithMessage(completedAndFailed, "Replicas change failed, Request failed (401), Authorization error"); + assertFailedWithMessage(ongoing, completedAndFailed, "Replicas change failed, Request failed (401), Authorization error"); } - private static void assertOngoing(List input, List output) { - assertThat(output.isEmpty(), is(false)); - var outputKt = output.stream().findFirst().get().kt(); - assertThat(outputKt.getStatus().getReplicasChange(), is(notNullValue())); - assertThat(outputKt.getStatus().getReplicasChange().getMessage(), is(nullValue())); - assertThat(outputKt.getStatus().getReplicasChange().getSessionId(), is(notNullValue())); - assertThat(outputKt.getStatus().getReplicasChange().getState(), is(ONGOING)); - var inputKt = input.stream().findFirst().get().kt(); - assertThat(outputKt.getStatus().getReplicasChange().getTargetReplicas(), is(inputKt.getSpec().getReplicas())); + private static void assertOngoing(List input, Results output) { + assertThat(output.size(), greaterThan(0)); + var inputRt = input.get(0); + var outputRcs = output.getReplicasChange(inputRt); + assertThat(outputRcs, is(notNullValue())); + assertThat(outputRcs.getMessage(), is(nullValue())); + assertThat(outputRcs.getSessionId(), is(notNullValue())); + assertThat(outputRcs.getState(), is(ReplicasChangeState.ONGOING)); + assertThat(outputRcs.getTargetReplicas(), is(inputRt.kt().getSpec().getReplicas())); } - private static void assertCompleted(List output) { - assertThat(output.isEmpty(), is(false)); - var kt = output.stream().findFirst().get().kt(); - assertThat(kt.getStatus().getReplicasChange(), is(nullValue())); + private static void assertCompleted(List input, Results output) { + assertThat(output.size(), greaterThan(0)); + var inputRt = input.get(0); + var outputRcs = output.getReplicasChange(inputRt); + assertThat(outputRcs, is(nullValue())); } - private static void assertFailedWithMessage(List output, String message) { - assertThat(output.isEmpty(), is(false)); - var outputKt = output.stream().findFirst().get().kt(); - assertThat(outputKt.getStatus().getReplicasChange(), is(notNullValue())); - assertThat(outputKt.getStatus().getReplicasChange().getMessage(), is(message)); + private static void assertFailedWithMessage(List input, Results output, String message) { + assertThat(output.size(), greaterThan(0)); + var inputRt = input.get(0); + var outputRcs = output.getReplicasChange(inputRt); + assertThat(outputRcs, is(notNullValue())); + assertThat(outputRcs.getMessage(), is(message)); } private List buildPendingReconcilableTopics() { var topicName = "my-topic"; - int replicationFactor = 2; + var replicationFactor = 2; var status = new KafkaTopicStatusBuilder() .withConditions(List.of(new ConditionBuilder() .withType("Ready") @@ -298,13 +307,13 @@ private List buildPendingReconcilableTopics() { .withStatus(status) .build(); return List.of(new ReconcilableTopic( - new Reconciliation("test", RESOURCE_KIND, NAMESPACE, topicName), - kafkaTopic, TopicOperatorUtil.topicName(kafkaTopic))); + new Reconciliation("test", KafkaTopic.RESOURCE_KIND, NAMESPACE, topicName), + kafkaTopic, topicName)); } private List buildOngoingReconcilableTopics() { var topicName = "my-topic"; - int replicationFactor = 3; + var replicationFactor = 3; var status = new KafkaTopicStatusBuilder() .withConditions(List.of(new ConditionBuilder() .withType("Ready") @@ -313,7 +322,7 @@ private List buildOngoingReconcilableTopics() { .build())) .withReplicasChange(new ReplicasChangeStatusBuilder() .withSessionId("8911ca89-351f-888-8d0f-9aade00e098h") - .withState(ONGOING) + .withState(ReplicasChangeState.ONGOING) .withTargetReplicas(replicationFactor) .build()) .build(); @@ -330,8 +339,8 @@ private List buildOngoingReconcilableTopics() { .withStatus(status) .build(); return List.of(new ReconcilableTopic( - new Reconciliation("test", RESOURCE_KIND, NAMESPACE, topicName), - kafkaTopic, TopicOperatorUtil.topicName(kafkaTopic))); + new Reconciliation("test", KafkaTopic.RESOURCE_KIND, NAMESPACE, topicName), + kafkaTopic, topicName)); } private static List validConfigs() { diff --git a/topic-operator/src/test/java/io/strimzi/operator/topic/cruisecontrol/MockCruiseControl.java b/topic-operator/src/test/java/io/strimzi/operator/topic/cruisecontrol/MockCruiseControl.java index 695c77402bc..9b2b88024ef 100644 --- a/topic-operator/src/test/java/io/strimzi/operator/topic/cruisecontrol/MockCruiseControl.java +++ b/topic-operator/src/test/java/io/strimzi/operator/topic/cruisecontrol/MockCruiseControl.java @@ -7,6 +7,7 @@ import io.strimzi.operator.common.CruiseControlUtil; import io.strimzi.operator.common.model.cruisecontrol.CruiseControlEndpoints; import io.strimzi.operator.common.model.cruisecontrol.CruiseControlParameters; +import io.strimzi.operator.topic.TopicOperatorTestUtil; import io.strimzi.test.TestUtils; import org.mockserver.configuration.ConfigurationProperties; import org.mockserver.integration.ClientAndServer; @@ -20,12 +21,10 @@ import java.io.ByteArrayInputStream; import java.io.File; +import java.nio.charset.StandardCharsets; import java.util.concurrent.TimeUnit; import java.util.logging.LogManager; -import static io.strimzi.operator.topic.TopicOperatorTestUtil.contentFromTextFile; -import static java.nio.charset.StandardCharsets.UTF_8; - /** * Cruise Control mock. */ @@ -53,7 +52,8 @@ public MockCruiseControl(int serverPort, File tlsKeyFile, File tlsCrtFile) { "java.util.logging.SimpleFormatter.format=%1$tF %1$tT %3$s %4$s %5$s %6$s%n\n" + ".level=" + ConfigurationProperties.javaLoggerLogLevel() + "\n" + "io.netty.handler.ssl.SslHandler.level=WARNING"; - LogManager.getLogManager().readConfiguration(new ByteArrayInputStream(loggingConfiguration.getBytes(UTF_8))); + LogManager.getLogManager().readConfiguration( + new ByteArrayInputStream(loggingConfiguration.getBytes(StandardCharsets.UTF_8))); this.server = new ClientAndServer(serverPort); } catch (Exception e) { @@ -101,8 +101,9 @@ public void expectTopicConfigSuccessResponse(File apiUserFile, File apiPassFile) .withQueryStringParameter(Parameter.param(CruiseControlParameters.JSON.toString(), "true")) .withPath(CruiseControlEndpoints.TOPIC_CONFIGURATION.toString()) .withContentType(MediaType.APPLICATION_JSON) - .withHeader(new Header("Authorization", - CruiseControlUtil.buildBasicAuthValue(contentFromTextFile(apiUserFile), contentFromTextFile(apiPassFile)))) + .withHeader(new Header("Authorization", CruiseControlUtil.buildBasicAuthValue( + TopicOperatorTestUtil.contentFromTextFile(apiUserFile), TopicOperatorTestUtil.contentFromTextFile(apiPassFile)) + )) .withSecure(true)) .respond( HttpResponse.response() @@ -139,8 +140,9 @@ public void expectTopicConfigSuccessResponse(File apiUserFile, File apiPassFile) .withQueryStringParameter(Parameter.param(CruiseControlParameters.JSON.toString(), "true")) .withPath(CruiseControlEndpoints.TOPIC_CONFIGURATION.toString()) .withContentType(MediaType.APPLICATION_JSON) - .withHeader(new Header("Authorization", - CruiseControlUtil.buildBasicAuthValue(contentFromTextFile(apiUserFile), contentFromTextFile(apiPassFile))))) + .withHeader(new Header("Authorization", CruiseControlUtil.buildBasicAuthValue( + TopicOperatorTestUtil.contentFromTextFile(apiUserFile), TopicOperatorTestUtil.contentFromTextFile(apiPassFile))) + )) .respond( HttpResponse.response() .withStatusCode(HttpStatusCode.OK_200.code()) @@ -159,8 +161,9 @@ public void expectTopicConfigErrorResponse(File apiUserFile, File apiPassFile) { .withQueryStringParameter(Parameter.param(CruiseControlParameters.JSON.toString(), "true")) .withPath(CruiseControlEndpoints.TOPIC_CONFIGURATION.toString()) .withContentType(MediaType.APPLICATION_JSON) - .withHeader(new Header("Authorization", - CruiseControlUtil.buildBasicAuthValue(contentFromTextFile(apiUserFile), contentFromTextFile(apiPassFile)))) + .withHeader(new Header("Authorization", CruiseControlUtil.buildBasicAuthValue( + TopicOperatorTestUtil.contentFromTextFile(apiUserFile), TopicOperatorTestUtil.contentFromTextFile(apiPassFile)) + )) .withSecure(true)) .respond( HttpResponse.response() @@ -180,8 +183,9 @@ public void expectTopicConfigRequestTimeout(File apiUserFile, File apiPassFile) .withQueryStringParameter(Parameter.param(CruiseControlParameters.JSON.toString(), "true")) .withPath(CruiseControlEndpoints.TOPIC_CONFIGURATION.toString()) .withContentType(MediaType.APPLICATION_JSON) - .withHeader(new Header("Authorization", - CruiseControlUtil.buildBasicAuthValue(contentFromTextFile(apiUserFile), contentFromTextFile(apiPassFile)))) + .withHeader(new Header("Authorization", CruiseControlUtil.buildBasicAuthValue( + TopicOperatorTestUtil.contentFromTextFile(apiUserFile), TopicOperatorTestUtil.contentFromTextFile(apiPassFile)) + )) .withSecure(true)) .respond( HttpResponse.response() @@ -199,8 +203,9 @@ public void expectTopicConfigRequestUnauthorized(File apiUserFile, File apiPassF .withQueryStringParameter(Parameter.param(CruiseControlParameters.JSON.toString(), "true")) .withPath(CruiseControlEndpoints.TOPIC_CONFIGURATION.toString()) .withContentType(MediaType.APPLICATION_JSON) - .withHeader(new Header("Authorization", - CruiseControlUtil.buildBasicAuthValue(contentFromTextFile(apiUserFile), contentFromTextFile(apiPassFile)))) + .withHeader(new Header("Authorization", CruiseControlUtil.buildBasicAuthValue( + TopicOperatorTestUtil.contentFromTextFile(apiUserFile), TopicOperatorTestUtil.contentFromTextFile(apiPassFile)) + )) .withSecure(true)) .respond( HttpResponse.response() @@ -231,8 +236,9 @@ public void expectUserTasksSuccessResponse(File apiUserFile, File apiPassFile) { .withQueryStringParameter(Parameter.param(CruiseControlParameters.USER_TASK_IDS.toString(), "8911ca89-351f-888-8d0f-9aade00e098h")) .withQueryStringParameter(Parameter.param(CruiseControlParameters.JSON.toString(), "true")) .withPath(CruiseControlEndpoints.USER_TASKS.toString()) - .withHeader(new Header("Authorization", - CruiseControlUtil.buildBasicAuthValue(contentFromTextFile(apiUserFile), contentFromTextFile(apiPassFile)))) + .withHeader(new Header("Authorization", CruiseControlUtil.buildBasicAuthValue( + TopicOperatorTestUtil.contentFromTextFile(apiUserFile), TopicOperatorTestUtil.contentFromTextFile(apiPassFile)) + )) .withSecure(true)) .respond( HttpResponse.response() @@ -263,8 +269,9 @@ public void expectUserTasksSuccessResponse(File apiUserFile, File apiPassFile) { .withQueryStringParameter(Parameter.param(CruiseControlParameters.USER_TASK_IDS.toString(), "8911ca89-351f-888-8d0f-9aade00e098h")) .withQueryStringParameter(Parameter.param(CruiseControlParameters.JSON.toString(), "true")) .withPath(CruiseControlEndpoints.USER_TASKS.toString()) - .withHeader(new Header("Authorization", - CruiseControlUtil.buildBasicAuthValue(contentFromTextFile(apiUserFile), contentFromTextFile(apiPassFile))))) + .withHeader(new Header("Authorization", CruiseControlUtil.buildBasicAuthValue( + TopicOperatorTestUtil.contentFromTextFile(apiUserFile), TopicOperatorTestUtil.contentFromTextFile(apiPassFile))) + )) .respond( HttpResponse.response() .withStatusCode(HttpStatusCode.OK_200.code()) @@ -280,8 +287,9 @@ public void expectUserTasksErrorResponse(File apiUserFile, File apiPassFile) { .withQueryStringParameter(Parameter.param(CruiseControlParameters.USER_TASK_IDS.toString(), "8911ca89-351f-888-8d0f-9aade00e098h")) .withQueryStringParameter(Parameter.param(CruiseControlParameters.JSON.toString(), "true")) .withPath(CruiseControlEndpoints.USER_TASKS.toString()) - .withHeader(new Header("Authorization", - CruiseControlUtil.buildBasicAuthValue(contentFromTextFile(apiUserFile), contentFromTextFile(apiPassFile)))) + .withHeader(new Header("Authorization", CruiseControlUtil.buildBasicAuthValue( + TopicOperatorTestUtil.contentFromTextFile(apiUserFile), TopicOperatorTestUtil.contentFromTextFile(apiPassFile)) + )) .withSecure(true)) .respond( HttpResponse.response() @@ -298,8 +306,9 @@ public void expectUserTasksRequestTimeout(File apiUserFile, File apiPassFile) { .withQueryStringParameter(Parameter.param(CruiseControlParameters.USER_TASK_IDS.toString(), "8911ca89-351f-888-8d0f-9aade00e098h")) .withQueryStringParameter(Parameter.param(CruiseControlParameters.JSON.toString(), "true")) .withPath(CruiseControlEndpoints.USER_TASKS.toString()) - .withHeader(new Header("Authorization", - CruiseControlUtil.buildBasicAuthValue(contentFromTextFile(apiUserFile), contentFromTextFile(apiPassFile)))) + .withHeader(new Header("Authorization", CruiseControlUtil.buildBasicAuthValue( + TopicOperatorTestUtil.contentFromTextFile(apiUserFile), TopicOperatorTestUtil.contentFromTextFile(apiPassFile)) + )) .withSecure(true)) .respond( HttpResponse.response() @@ -315,8 +324,9 @@ public void expectUserTasksRequestUnauthorized(File apiUserFile, File apiPassFil .withQueryStringParameter(Parameter.param(CruiseControlParameters.USER_TASK_IDS.toString(), "8911ca89-351f-888-8d0f-9aade00e098h")) .withQueryStringParameter(Parameter.param(CruiseControlParameters.JSON.toString(), "true")) .withPath(CruiseControlEndpoints.USER_TASKS.toString()) - .withHeader(new Header("Authorization", - CruiseControlUtil.buildBasicAuthValue(contentFromTextFile(apiUserFile), contentFromTextFile(apiPassFile)))) + .withHeader(new Header("Authorization", CruiseControlUtil.buildBasicAuthValue( + TopicOperatorTestUtil.contentFromTextFile(apiUserFile), TopicOperatorTestUtil.contentFromTextFile(apiPassFile)) + )) .withSecure(true)) .respond( HttpResponse.response() diff --git a/topic-operator/src/test/java/io/strimzi/operator/topic/TopicUpsertTest.java b/topic-operator/src/test/java/io/strimzi/operator/topic/model/TopicUpsertTest.java similarity index 95% rename from topic-operator/src/test/java/io/strimzi/operator/topic/TopicUpsertTest.java rename to topic-operator/src/test/java/io/strimzi/operator/topic/model/TopicUpsertTest.java index 4ff46af2ff0..d853260d004 100644 --- a/topic-operator/src/test/java/io/strimzi/operator/topic/TopicUpsertTest.java +++ b/topic-operator/src/test/java/io/strimzi/operator/topic/model/TopicUpsertTest.java @@ -2,7 +2,7 @@ * 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.topic; +package io.strimzi.operator.topic.model; import io.strimzi.operator.topic.model.TopicEvent.TopicUpsert; import org.junit.jupiter.api.Test;