Skip to content

Commit

Permalink
Fix flaky Cruise Control test and fix error when RF is not configured (
Browse files Browse the repository at this point in the history
…#9674)

Signed-off-by: Jakub Scholz <www@scholzj.com>
  • Loading branch information
scholzj committed Feb 13, 2024
1 parent dbcb208 commit 51486c7
Show file tree
Hide file tree
Showing 5 changed files with 14 additions and 15 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -212,8 +212,7 @@ public static CruiseControl fromCrd(
result.image = image;

KafkaConfiguration kafkaConfiguration = new KafkaConfiguration(reconciliation, kafkaClusterSpec.getConfig().entrySet());
CruiseControlConfiguration cruiseControlConfiguration = new CruiseControlConfiguration(reconciliation, ccSpec.getConfig().entrySet());
result.updateConfigurationWithDefaults(cruiseControlConfiguration, kafkaConfiguration);
result.updateConfigurationWithDefaults(ccSpec, kafkaConfiguration);

CruiseControlConfiguration ccConfiguration = result.configuration;
result.sslEnabled = ccConfiguration.isApiSslEnabled();
Expand Down Expand Up @@ -246,16 +245,15 @@ public static CruiseControl fromCrd(
}
}

private void updateConfigurationWithDefaults(CruiseControlConfiguration cruiseControlConfiguration, KafkaConfiguration kafkaConfiguration) {
private void updateConfigurationWithDefaults(CruiseControlSpec ccSpec, KafkaConfiguration kafkaConfiguration) {
Map<String, String> defaultCruiseControlProperties = new HashMap<>(CruiseControlConfiguration.getCruiseControlDefaultPropertiesMap());
defaultCruiseControlProperties.put(CruiseControlConfigurationParameters.SAMPLE_STORE_TOPIC_REPLICATION_FACTOR.getValue(), kafkaConfiguration.getConfigOption(KafkaConfiguration.DEFAULT_REPLICATION_FACTOR));
for (Map.Entry<String, String> defaultEntry : defaultCruiseControlProperties.entrySet()) {
if (cruiseControlConfiguration.getConfigOption(defaultEntry.getKey()) == null) {
cruiseControlConfiguration.setConfigOption(defaultEntry.getKey(), defaultEntry.getValue());
}
if (kafkaConfiguration.getConfigOption(KafkaConfiguration.DEFAULT_REPLICATION_FACTOR) != null) {
defaultCruiseControlProperties.put(CruiseControlConfigurationParameters.SAMPLE_STORE_TOPIC_REPLICATION_FACTOR.getValue(), kafkaConfiguration.getConfigOption(KafkaConfiguration.DEFAULT_REPLICATION_FACTOR));
}
// Ensure that the configured anomaly.detection.goals are a sub-set of the default goals

CruiseControlConfiguration cruiseControlConfiguration = new CruiseControlConfiguration(reconciliation, ccSpec.getConfig().entrySet(), defaultCruiseControlProperties);
checkGoals(cruiseControlConfiguration);

this.configuration = cruiseControlConfiguration;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -90,8 +90,7 @@ public class CruiseControlConfiguration extends AbstractConfiguration {
Map.entry(CruiseControlConfigurationParameters.WEBSERVER_SSL_ENABLE.getValue(), Boolean.toString(CruiseControlConfigurationParameters.DEFAULT_WEBSERVER_SSL_ENABLED)),
Map.entry(CruiseControlConfigurationParameters.PARTITION_METRIC_TOPIC_NAME.getValue(), CruiseControlConfigurationParameters.DEFAULT_PARTITION_METRIC_TOPIC_NAME),
Map.entry(CruiseControlConfigurationParameters.BROKER_METRIC_TOPIC_NAME.getValue(), CruiseControlConfigurationParameters.DEFAULT_BROKER_METRIC_TOPIC_NAME),
Map.entry(CruiseControlConfigurationParameters.METRIC_REPORTER_TOPIC_NAME.getValue(), CruiseControlConfigurationParameters.DEFAULT_METRIC_REPORTER_TOPIC_NAME),
Map.entry(CruiseControlConfigurationParameters.CAPACITY_CONFIG_FILE.getValue(), CruiseControl.CONFIG_VOLUME_MOUNT + CruiseControl.CAPACITY_CONFIG_FILENAME)
Map.entry(CruiseControlConfigurationParameters.METRIC_REPORTER_TOPIC_NAME.getValue(), CruiseControlConfigurationParameters.DEFAULT_METRIC_REPORTER_TOPIC_NAME)
)));

private static final List<String> FORBIDDEN_PREFIXES = AbstractConfiguration.splitPrefixesToList(CruiseControlSpec.FORBIDDEN_PREFIXES);
Expand All @@ -103,9 +102,10 @@ public class CruiseControlConfiguration extends AbstractConfiguration {
*
* @param reconciliation The reconciliation
* @param jsonOptions Json object with configuration options as key ad value pairs.
* @param defaults Default configuration values
*/
public CruiseControlConfiguration(Reconciliation reconciliation, Iterable<Map.Entry<String, Object>> jsonOptions) {
super(reconciliation, jsonOptions, FORBIDDEN_PREFIXES, FORBIDDEN_PREFIX_EXCEPTIONS);
public CruiseControlConfiguration(Reconciliation reconciliation, Iterable<Map.Entry<String, Object>> jsonOptions, Map<String, String> defaults) {
super(reconciliation, jsonOptions, FORBIDDEN_PREFIXES, FORBIDDEN_PREFIX_EXCEPTIONS, defaults);
}

protected static Map<String, String> getCruiseControlDefaultPropertiesMap() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1235,7 +1235,7 @@ private Future<MapAndStatus<ConfigMap, KafkaRebalanceStatus>> onReady(Reconcilia
return Future.failedFuture(Util.missingSecretException(clusterNamespace, ccApiSecretName));
}

CruiseControlConfiguration ccConfig = new CruiseControlConfiguration(reconciliation, kafka.getSpec().getCruiseControl().getConfig().entrySet());
CruiseControlConfiguration ccConfig = new CruiseControlConfiguration(reconciliation, kafka.getSpec().getCruiseControl().getConfig().entrySet(), Map.of());
boolean apiAuthEnabled = ccConfig.isApiAuthEnabled();
boolean apiSslEnabled = ccConfig.isApiSslEnabled();
CruiseControlApi apiClient = cruiseControlClientProvider(ccSecret, ccApiSecret, apiAuthEnabled, apiSslEnabled);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -154,7 +154,7 @@ public void reconcileEnabledCruiseControl(VertxTestContext context) {
// Verify deployment
assertThat(deployCaptor.getAllValues().size(), is(1));
assertThat(deployCaptor.getValue(), is(notNullValue()));
assertThat(deployCaptor.getAllValues().get(0).getSpec().getTemplate().getMetadata().getAnnotations().get(CruiseControl.ANNO_STRIMZI_SERVER_CONFIGURATION_HASH), is("fc0ad847"));
assertThat(deployCaptor.getAllValues().get(0).getSpec().getTemplate().getMetadata().getAnnotations().get(CruiseControl.ANNO_STRIMZI_SERVER_CONFIGURATION_HASH), is("096591fb"));
assertThat(deployCaptor.getAllValues().get(0).getSpec().getTemplate().getMetadata().getAnnotations().get(CruiseControl.ANNO_STRIMZI_CAPACITY_CONFIGURATION_HASH), is("1eb49220"));
assertThat(deployCaptor.getAllValues().get(0).getSpec().getTemplate().getMetadata().getAnnotations().get(Annotations.ANNO_STRIMZI_AUTH_HASH), is("27ada64b"));

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,5 +26,6 @@ ssl.truststore.type=PKCS12
ssl.truststore.location=/tmp/cruise-control/replication.truststore.p12
ssl.truststore.password=$CERTS_STORE_PASSWORD
kafka.broker.failure.detection.enable=true
capacity.config.file=/opt/cruise-control/custom-config/capacity.json
${CRUISE_CONTROL_CONFIGURATION}
EOF

0 comments on commit 51486c7

Please sign in to comment.