Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Improve min.insync.replicas parsing in warnTooLargeMinIsr #10844

Merged
merged 4 commits into from
Nov 28, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -684,11 +684,11 @@ private static Either<TopicOperatorException, NewPartitions> partitionChanges(
Reconciliation reconciliation, KafkaTopic kafkaTopic, int currentNumPartitions
) {
var requested = kafkaTopic.getSpec() == null || kafkaTopic.getSpec().getPartitions() == null
? KafkaHandler.BROKER_DEFAULT : kafkaTopic.getSpec().getPartitions();
? KafkaHandler.DEFAULT_PARTITIONS : kafkaTopic.getSpec().getPartitions();
if (requested > currentNumPartitions) {
LOGGER.debugCr(reconciliation, "Partition increase from {} to {}", currentNumPartitions, requested);
return Either.ofRight(NewPartitions.increaseTo(requested));
} else if (requested != KafkaHandler.BROKER_DEFAULT && requested < currentNumPartitions) {
} else if (requested != KafkaHandler.DEFAULT_PARTITIONS && requested < currentNumPartitions) {
LOGGER.debugCr(reconciliation, "Partition decrease from {} to {}", currentNumPartitions, requested);
return Either.ofLeft(new TopicOperatorException.NotSupported("Decreasing partitions not supported"));
} else {
Expand Down Expand Up @@ -820,13 +820,15 @@ private void warnTooLargeMinIsr(List<ReconcilableTopic> reconcilableTopics) {
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 topicMinIsr = topicConfig.get(KafkaHandler.MIN_INSYNC_REPLICAS);
var configuredMinIsr = topicMinIsr != null
? Integer.parseInt(TopicOperatorUtil.configValueAsString(KafkaHandler.MIN_INSYNC_REPLICAS, topicMinIsr))
: clusterMinIsr.map(Integer::parseInt).orElse(KafkaHandler.DEFAULT_MIN_ISR);
fvaleri marked this conversation as resolved.
Show resolved Hide resolved
var targetRf = reconcilableTopic.kt().getSpec().getReplicas();
if (targetRf < minIsr) {
if (targetRf < configuredMinIsr) {
LOGGER.warnCr(reconcilableTopic.reconciliation(),
"The target replication factor ({}) is below the configured {} ({})",
targetRf, KafkaHandler.MIN_INSYNC_REPLICAS, minIsr);
targetRf, KafkaHandler.MIN_INSYNC_REPLICAS, configuredMinIsr);
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,13 +50,16 @@
*/
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";

/** Default value for partitions. */
public static final int DEFAULT_PARTITIONS = -1;
/** Default value for min.insync.replicas. */
public static final int DEFAULT_MIN_ISR = 1;

private final TopicOperatorConfig config;
private final TopicOperatorMetricsHolder metricsHolder;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -267,8 +267,6 @@ public void shouldHandleInterruptedExceptionFromDeleteTopics() throws ExecutionE
assertOnUpdateThrowsInterruptedException(adminSpy, withDeletionTimestamp);
}

// TODO kube client interrupted exceptions

@ParameterizedTest
@ValueSource(booleans = { true, false })
public void replicasChangeShouldBeReconciled(boolean cruiseControlEnabled) {
Expand Down Expand Up @@ -310,6 +308,8 @@ public void replicasChangeShouldBeReconciled(boolean cruiseControlEnabled) {
.addToLabels("key", "VALUE")
.endMetadata()
.withNewSpec()
// we also support string values
.withConfig(Map.of("min.insync.replicas", "1"))
.withPartitions(25)
.withReplicas(++replicationFactor)
.endSpec()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -994,7 +994,7 @@ public void shouldUpdateTopicInKafkaWhenConfigRemovedInKube() throws ExecutionEx
return theKt;
},
expectedCreateConfigs -> {
Map<String, String> expectedUpdatedConfigs = new HashMap<>(expectedCreateConfigs);
var expectedUpdatedConfigs = new HashMap<>(expectedCreateConfigs);
expectedUpdatedConfigs.remove(TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG);
return expectedUpdatedConfigs;
});
Expand Down Expand Up @@ -1550,7 +1550,7 @@ private KafkaTopic modifyTopicAndAwait(KafkaTopic kt, UnaryOperator<KafkaTopic>
@Override
public boolean test(KafkaTopic theKt) {
return theKt.getStatus() != null
&& (theKt.getStatus().getObservedGeneration() >= postUpdateGeneration
&& (theKt.getStatus().getObservedGeneration() == postUpdateGeneration
|| !TopicOperatorUtil.isManaged(theKt) || TopicOperatorUtil.isPaused(theKt))
&& predicate.test(theKt);
}
Expand Down
Loading