From 5ae4d8374abed13e9c12705f59e35ba02965dc3c Mon Sep 17 00:00:00 2001 From: Morgan Galpin Date: Mon, 20 Mar 2023 16:07:51 -0700 Subject: [PATCH 1/5] Persist demoted and removed brokers * Simplified KafkaZkClient creation for debugging. * Added persistence for demoted and removed brokers. * Added tests and conformed to lint tests. --- config/cruisecontrol.properties | 46 +++ .../cruisecontrol/KafkaCruiseControl.java | 13 +- .../KafkaCruiseControlUtils.java | 175 +++++---- .../config/KafkaCruiseControlConfig.java | 66 +++- .../config/constants/PersistedDataConfig.java | 157 ++++++++ .../detector/MaintenanceEventTopicReader.java | 10 +- .../cruisecontrol/executor/Executor.java | 141 ++++--- .../persisteddata/BackingMethod.java | 38 ++ .../persisteddata/KeyMappedMap.java | 139 +++++++ .../persisteddata/MapDecorator.java | 93 +++++ .../persisteddata/PersistedMap.java | 35 ++ .../persisteddata/PersistedMapFactory.java | 72 ++++ .../persisteddata/ValueMappedMap.java | 136 +++++++ .../kafka/KafkaPersistedMap.java | 348 ++++++++++++++++++ .../kafka/KafkaPersistedMapException.java | 11 + .../namespace/ExecutorPersistedData.java | 169 +++++++++ .../persisteddata/namespace/Namespace.java | 53 +++ .../servlet/handler/sync/AdminRequest.java | 57 ++- .../cruisecontrol/KafkaCruiseControlTest.java | 9 +- .../config/KafkaCruiseControlConfigTest.java | 81 ++++ .../cruisecontrol/executor/ExecutorTest.java | 49 ++- .../persisteddata/BackingMethodTest.java | 40 ++ .../persisteddata/KeyMappedMapTest.java | 198 ++++++++++ .../persisteddata/MapDecoratorTest.java | 182 +++++++++ .../PersistedMapFactoryTest.java | 56 +++ .../persisteddata/PersistedMapTest.java | 69 ++++ .../persisteddata/ValueMappedMapTest.java | 197 ++++++++++ .../kafka/KafkaPersistedMapTest.java | 294 +++++++++++++++ .../namespace/ExecutorPersistedDataTest.java | 197 ++++++++++ .../namespace/NamespaceTest.java | 51 +++ 30 files changed, 2985 insertions(+), 197 deletions(-) create mode 100644 cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/config/constants/PersistedDataConfig.java create mode 100644 cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/BackingMethod.java create mode 100644 cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/KeyMappedMap.java create mode 100644 cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/MapDecorator.java create mode 100644 cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistedMap.java create mode 100644 cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistedMapFactory.java create mode 100644 cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/ValueMappedMap.java create mode 100644 cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/kafka/KafkaPersistedMap.java create mode 100644 cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/kafka/KafkaPersistedMapException.java create mode 100644 cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/namespace/ExecutorPersistedData.java create mode 100644 cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/namespace/Namespace.java create mode 100644 cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/config/KafkaCruiseControlConfigTest.java create mode 100644 cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/BackingMethodTest.java create mode 100644 cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/KeyMappedMapTest.java create mode 100644 cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/MapDecoratorTest.java create mode 100644 cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistedMapFactoryTest.java create mode 100644 cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistedMapTest.java create mode 100644 cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/ValueMappedMapTest.java create mode 100644 cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/kafka/KafkaPersistedMapTest.java create mode 100644 cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/namespace/ExecutorPersistedDataTest.java create mode 100644 cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/namespace/NamespaceTest.java diff --git a/config/cruisecontrol.properties b/config/cruisecontrol.properties index a0dd8cffa3..062ce32915 100644 --- a/config/cruisecontrol.properties +++ b/config/cruisecontrol.properties @@ -198,6 +198,52 @@ num.concurrent.leader.movements=1000 execution.progress.check.interval.ms=10000 +# Configurations for persistent data +# ======================================= + +# The method to use to store persisted data. This is the first "persisted.data" config to set which +# will determine which other configs of the series should be configured. The available options are: +# "kafka", +# "memory" +# The default is "memory", which doesn't durably persist any runtime data. +#persisted.data.backing.method=kafka + +# The name of the kafka topic to use to persist data when "persisted.data.backing.method" is set to +# "kafka". If the topic is not present, then it will be created. +#persisted.data.kafka.topic.name=__CruiseControlPersistentData + +# The number of partitions to ensure are present for the kafka topic. Only applies when +# "persisted.data.backing.method" is set to "kafka". If the topic has fewer than this number of +# partitions, then partitions will be added. +#persisted.data.kafka.topic.partition.count=2 + +# The replication factor to use for the kafka topic. Only applies when +# "persisted.data.backing.method" is set to "kafka". Multiple partition replicas are desirable to +# ensure the topic is reasonably available. +#persisted.data.kafka.topic.replication.factor=2 + +# The configs to apply to the kafka topic used to persist Cruise Control data. Only applies if +# "persisted.data.backing.method" is set to "kafka". This "list" should be a semicolon separated +# string of 'key=value' pairs. The keys and values need to be valid Kafka Topic configs. +# See: https://kafka.apache.org/documentation/#topicconfigs +# e.g. persisted.data.kafka.topic.additional.configs.map=max.compaction.lag.ms=600000;min.insync.replicas=2 +#persisted.data.kafka.topic.additional.configs.map= + +# The additional configs to use when creating the kafka producer to persist Cruise Control data. +# Only applies if "persisted.data.backing.method" is set to "kafka". This "list" should be a +# semicolon separated string of 'key=value' pairs. The keys and values need to be valid Kafka +# Producer configs. See: https://kafka.apache.org/documentation/#producerconfigs +# e.g. persisted.data.kafka.producer.additional.configs.map=buffer.memory=100000000;batch.size=900000 +#persisted.data.kafka.producer.additional.configs.map= + +# The additional configs to use when creating the kafka consumer to read persisted Cruise Control +# data. Only applies if "persisted.data.backing.method" is set to "kafka". This "list" should be a +# semicolon separated string of 'key=value' pairs. The keys and values need to be valid Kafka +# Consumer configs. See: https://kafka.apache.org/documentation/#consumerconfigs +# e.g. persisted.data.kafka.consumer.additional.configs.map=group.id=my-group-id;fetch.max.bytes=2000000 +#persisted.data.kafka.consumer.additional.configs.map= + + # Configurations for anomaly detector # ======================================= diff --git a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/KafkaCruiseControl.java b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/KafkaCruiseControl.java index 921e84e042..904d131255 100644 --- a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/KafkaCruiseControl.java +++ b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/KafkaCruiseControl.java @@ -41,6 +41,8 @@ import com.linkedin.kafka.cruisecontrol.monitor.MonitorUtils; import com.linkedin.kafka.cruisecontrol.monitor.metricdefinition.KafkaMetricDef; import com.linkedin.kafka.cruisecontrol.monitor.task.LoadMonitorTaskRunner; +import com.linkedin.kafka.cruisecontrol.persisteddata.PersistedMapFactory; +import com.linkedin.kafka.cruisecontrol.persisteddata.namespace.ExecutorPersistedData; import com.linkedin.kafka.cruisecontrol.servlet.UserTaskManager; import com.linkedin.kafka.cruisecontrol.servlet.response.stats.BrokerStats; import java.io.InputStream; @@ -82,6 +84,7 @@ public class KafkaCruiseControl { private final GoalOptimizer _goalOptimizer; private final ExecutorService _goalOptimizerExecutor; private final Executor _executor; + private final PersistedMapFactory _persistedMapFactory; private final AnomalyDetectorManager _anomalyDetectorManager; private final Time _time; private final AdminClient _adminClient; @@ -122,7 +125,9 @@ public KafkaCruiseControl(KafkaCruiseControlConfig config, MetricRegistry dropwi Provisioner.class, Collections.singletonMap(KAFKA_CRUISE_CONTROL_OBJECT_CONFIG, this)); _anomalyDetectorManager = new AnomalyDetectorManager(this, _time, dropwizardMetricRegistry); - _executor = new Executor(config, _time, dropwizardMetricRegistry, _anomalyDetectorManager); + _persistedMapFactory = new PersistedMapFactory(config, _adminClient); + _executor = new Executor(config, _time, dropwizardMetricRegistry, _anomalyDetectorManager, + new ExecutorPersistedData(_persistedMapFactory.instance())); _loadMonitor = new LoadMonitor(config, _time, dropwizardMetricRegistry, KafkaMetricDef.commonMetricDef()); _goalOptimizerExecutor = Executors.newSingleThreadExecutor(new KafkaCruiseControlThreadFactory("GoalOptimizerExecutor")); _goalOptimizer = new GoalOptimizer(config, _loadMonitor, _time, dropwizardMetricRegistry, _executor, _adminClient); @@ -138,11 +143,13 @@ public KafkaCruiseControl(KafkaCruiseControlConfig config, MetricRegistry dropwi LoadMonitor loadMonitor, ExecutorService goalOptimizerExecutor, GoalOptimizer goalOptimizer, + PersistedMapFactory persistedMapFactory, Provisioner provisioner) { _config = config; _time = time; _adminClient = createAdminClient(KafkaCruiseControlUtils.parseAdminClientConfigs(config)); _anomalyDetectorManager = anomalyDetectorManager; + _persistedMapFactory = persistedMapFactory; _executor = executor; _loadMonitor = loadMonitor; _goalOptimizerExecutor = goalOptimizerExecutor; @@ -468,9 +475,9 @@ public boolean dropRecentBrokers(Set brokersToDrop, boolean isRemoved) */ public void addRecentBrokersPermanently(Set brokersToAdd, boolean isRemoved) { if (isRemoved) { - _executor.addRecentlyRemovedBrokers(brokersToAdd); + _executor.addRecentlyRemovedBrokersPermanently(brokersToAdd); } else { - _executor.addRecentlyDemotedBrokers(brokersToAdd); + _executor.addRecentlyDemotedBrokersPermanently(brokersToAdd); } } diff --git a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/KafkaCruiseControlUtils.java b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/KafkaCruiseControlUtils.java index 50d6f53972..ce6fd17757 100644 --- a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/KafkaCruiseControlUtils.java +++ b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/KafkaCruiseControlUtils.java @@ -11,6 +11,7 @@ import com.linkedin.kafka.cruisecontrol.config.constants.AnalyzerConfig; import com.linkedin.kafka.cruisecontrol.config.constants.ExecutorConfig; import com.linkedin.kafka.cruisecontrol.config.constants.WebServerConfig; +import com.linkedin.kafka.cruisecontrol.detector.TopicAnomalyDetector; import com.linkedin.kafka.cruisecontrol.exception.PartitionNotExistsException; import com.linkedin.kafka.cruisecontrol.exception.SamplingException; import com.linkedin.kafka.cruisecontrol.metricsreporter.CruiseControlMetricsUtils; @@ -18,6 +19,7 @@ import com.linkedin.kafka.cruisecontrol.monitor.ModelCompletenessRequirements; import com.linkedin.kafka.cruisecontrol.monitor.task.LoadMonitorTaskRunner; import kafka.zk.KafkaZkClient; +import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.AdminClientConfig; import org.apache.kafka.clients.admin.AlterConfigOp; @@ -45,6 +47,7 @@ import org.apache.kafka.common.config.SaslConfigs; import org.apache.kafka.common.config.SslConfigs; import org.apache.kafka.common.config.TopicConfig; +import org.apache.kafka.common.config.types.Password; import org.apache.kafka.common.errors.ReassignmentInProgressException; import org.apache.kafka.common.errors.TopicExistsException; import org.apache.kafka.common.message.MetadataResponseData; @@ -63,7 +66,6 @@ import java.io.InputStream; import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; import java.time.Duration; import java.util.Collections; import java.util.HashMap; @@ -71,15 +73,17 @@ import java.util.List; import java.util.Map; import java.util.NoSuchElementException; +import java.util.Objects; import java.util.Properties; import java.util.Random; import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.function.Function; import java.util.stream.Collectors; -import scala.Option; +import static com.linkedin.kafka.cruisecontrol.config.constants.MonitorConfig.DEFAULT_RECONNECT_BACKOFF_MS; import static com.linkedin.kafka.cruisecontrol.config.constants.MonitorConfig.RECONNECT_BACKOFF_MS_CONFIG; import static com.linkedin.kafka.cruisecontrol.servlet.parameters.ParameterUtils.SKIP_HARD_GOAL_CHECK_PARAM; @@ -204,12 +208,39 @@ public static boolean createTopic(AdminClient adminClient, NewTopic topicToBeCre return true; } + /** + * If the topic does not exist, create it with the requested name, partition count, replication + * factor, and topic configs. Otherwise, if the existing topic has + *
    + *
  • fewer than the requested number of partitions, increase the count to the requested value
  • + *
  • different topic configs, then update the topic configs to match the desired configs
  • + *
+ * + * Note that the replication factor is not adjusted if an existing maintenance event topic has + * a different replication factor. Automated handling of replication factor inconsistencies is + * the responsibility of {@link TopicAnomalyDetector}. + * + * @param adminClient Admin client to use in topic creation, config checks, and required updates + * (if any). + * @param topic Desired topic and configuration. + */ + public static void maybeCreateOrUpdateTopic(AdminClient adminClient, NewTopic topic) { + if (!createTopic(adminClient, topic)) { + // Update topic config and partition count to ensure desired properties. + maybeUpdateTopicConfig(adminClient, topic); + maybeIncreasePartitionCount(adminClient, topic); + } + } + /** * Describe cluster configs. * * @param adminClient The adminClient to send describeConfigs request. * @param timeout Timeout to describe cluster configs. * @return Cluster configs, or {@code null} if there is a timeout. + * @throws InterruptedException If the thread is interrupted while waiting for the + * AdminClient.describeConfigs response. + * @throws ExecutionException If the AdminClient.describeConfigs response is failed. */ public static Config describeClusterConfigs(AdminClient adminClient, Duration timeout) throws InterruptedException, ExecutionException { @@ -362,6 +393,7 @@ public static CompletionType maybeIncreasePartitionCount(AdminClient adminClient * * @param endOffsets End offsets retrieved by consumer. * @param offsetsForTimes Offsets for times retrieved by consumer. + * @throws SamplingException if there are any partitions that failed to fetch offsets. */ public static void sanityCheckOffsetFetch(Map endOffsets, Map offsetsForTimes) @@ -542,34 +574,17 @@ public static KafkaZkClient createKafkaZkClient(String connectString, boolean zkSecurityEnabled, ZKClientConfig zkClientConfig) { KafkaZkClient kafkaZkClient = null; + String zkClientName = String.format("%s-%s", metricGroup, metricType); try { - String zkClientName = String.format("%s-%s", metricGroup, metricType); - Method kafka31PlusMet = KafkaZkClient.class.getMethod("apply", String.class, boolean.class, int.class, int.class, int.class, - org.apache.kafka.common.utils.Time.class, String.class, ZKClientConfig.class, - String.class, String.class, boolean.class); - kafkaZkClient = (KafkaZkClient) kafka31PlusMet.invoke(null, connectString, zkSecurityEnabled, ZK_SESSION_TIMEOUT, ZK_CONNECTION_TIMEOUT, - Integer.MAX_VALUE, new SystemTime(), zkClientName, zkClientConfig, metricGroup, - metricType, false); - } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) { - LOG.debug("Unable to find apply method in KafkaZkClient for Kafka 3.1+.", e); - } - if (kafkaZkClient == null) { - try { - Option zkClientName = Option.apply(String.format("%s-%s", metricGroup, metricType)); - Option zkConfig = Option.apply(zkClientConfig); - Method kafka31MinusMet = KafkaZkClient.class.getMethod("apply", String.class, boolean.class, int.class, int.class, int.class, - org.apache.kafka.common.utils.Time.class, String.class, String.class, Option.class, - Option.class); - kafkaZkClient = (KafkaZkClient) kafka31MinusMet.invoke(null, connectString, zkSecurityEnabled, ZK_SESSION_TIMEOUT, ZK_CONNECTION_TIMEOUT, - Integer.MAX_VALUE, new SystemTime(), metricGroup, metricType, zkClientName, zkConfig); - } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) { - LOG.debug("Unable to find apply method in KafkaZkClient for Kafka 3.1-.", e); - } - } - if (kafkaZkClient != null) { + LOG.info("Connecting to zookeeper, {}, with zkSecurityEnabled={}, zkClientConfig={}", + connectString, zkSecurityEnabled, zkClientConfig.toString()); + kafkaZkClient = KafkaZkClient.apply(connectString, zkSecurityEnabled, ZK_SESSION_TIMEOUT, + ZK_CONNECTION_TIMEOUT, Integer.MAX_VALUE, new SystemTime(), zkClientName, + zkClientConfig, metricGroup, metricType, false); return kafkaZkClient; - } else { - throw new NoSuchElementException("Unable to find viable apply function version for the KafkaZkClient class "); + } catch (Exception e) { + LOG.error("Unable to create KafkaZkClient.", e); + throw new NoSuchElementException("Unable to create KafkaZkClient"); } } @@ -626,37 +641,46 @@ public static Map parseAdminClientConfigs(KafkaCruiseControlConf adminClientConfigs.put(AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG, configs.getInt(ExecutorConfig.ADMIN_CLIENT_REQUEST_TIMEOUT_MS_CONFIG)); adminClientConfigs.put(AdminClientConfig.RECONNECT_BACKOFF_MS_CONFIG, configs.getLong(RECONNECT_BACKOFF_MS_CONFIG)); - // Add security protocol (if specified). - try { - String securityProtocol = configs.getString(AdminClientConfig.SECURITY_PROTOCOL_CONFIG); - adminClientConfigs.put(AdminClientConfig.SECURITY_PROTOCOL_CONFIG, securityProtocol); - setStringConfigIfExists(configs, adminClientConfigs, SaslConfigs.SASL_MECHANISM); - setClassConfigIfExists(configs, adminClientConfigs, SaslConfigs.SASL_LOGIN_CALLBACK_HANDLER_CLASS); - setClassConfigIfExists(configs, adminClientConfigs, SaslConfigs.SASL_CLIENT_CALLBACK_HANDLER_CLASS); - setPasswordConfigIfExists(configs, adminClientConfigs, SaslConfigs.SASL_JAAS_CONFIG); - setStringConfigIfExists(configs, adminClientConfigs, SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL); - - // Configure SSL configs (if security protocol is SSL or SASL_SSL) - if (securityProtocol.equals(SecurityProtocol.SSL.name) || securityProtocol.equals(SecurityProtocol.SASL_SSL.name)) { - setStringConfigIfExists(configs, adminClientConfigs, SslConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG); - setStringConfigIfExists(configs, adminClientConfigs, SslConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG); - setStringConfigIfExists(configs, adminClientConfigs, SslConfigs.SSL_KEYSTORE_TYPE_CONFIG); - setStringConfigIfExists(configs, adminClientConfigs, SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG); - setStringConfigIfExists(configs, adminClientConfigs, SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG); - setStringConfigIfExists(configs, adminClientConfigs, SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG); - setStringConfigIfExists(configs, adminClientConfigs, SslConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_CONFIG); - setStringConfigIfExists(configs, adminClientConfigs, SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG); - setPasswordConfigIfExists(configs, adminClientConfigs, SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG); - setPasswordConfigIfExists(configs, adminClientConfigs, SslConfigs.SSL_KEY_PASSWORD_CONFIG); - setPasswordConfigIfExists(configs, adminClientConfigs, SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG); - } - } catch (ConfigException ce) { - // let it go. - } + maybeAddSecurityConfig(configs, adminClientConfigs); return adminClientConfigs; } + /** + * Add security config, if specified, to the target {@code config} map. Security config + * parameters in {@code cruiseControlConfig} are copied into {@code config}. + * + * @param cruiseControlConfig The application config to copy the configured security settings + * from. + * @param config The target configuration map to copy the security settings from. + * @return The modified {@code config} map with the security config added. + */ + public static Map maybeAddSecurityConfig(AbstractConfig cruiseControlConfig, Map config) { + String securityProtocol = setStringConfigIfExists(cruiseControlConfig, config, CommonClientConfigs.SECURITY_PROTOCOL_CONFIG); + setStringConfigIfExists(cruiseControlConfig, config, SaslConfigs.SASL_MECHANISM); + setClassConfigIfExists(cruiseControlConfig, config, SaslConfigs.SASL_LOGIN_CALLBACK_HANDLER_CLASS); + setClassConfigIfExists(cruiseControlConfig, config, SaslConfigs.SASL_CLIENT_CALLBACK_HANDLER_CLASS); + setPasswordConfigIfExists(cruiseControlConfig, config, SaslConfigs.SASL_JAAS_CONFIG); + setStringConfigIfExists(cruiseControlConfig, config, SaslConfigs.SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL); + + // Configure SSL config (if security protocol is SSL or SASL_SSL) + if (SecurityProtocol.SSL.name.equals(securityProtocol) + || SecurityProtocol.SASL_SSL.name.equals(securityProtocol)) { + setStringConfigIfExists(cruiseControlConfig, config, SslConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG); + setStringConfigIfExists(cruiseControlConfig, config, SslConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG); + setStringConfigIfExists(cruiseControlConfig, config, SslConfigs.SSL_KEYSTORE_TYPE_CONFIG); + setStringConfigIfExists(cruiseControlConfig, config, SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG); + setStringConfigIfExists(cruiseControlConfig, config, SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG); + setStringConfigIfExists(cruiseControlConfig, config, SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG); + setStringConfigIfExists(cruiseControlConfig, config, SslConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_CONFIG); + setStringConfigIfExists(cruiseControlConfig, config, SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG); + setPasswordConfigIfExists(cruiseControlConfig, config, SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG); + setPasswordConfigIfExists(cruiseControlConfig, config, SslConfigs.SSL_KEY_PASSWORD_CONFIG); + setPasswordConfigIfExists(cruiseControlConfig, config, SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG); + } + return config; + } + /** * Generate a {@link MetadataResponseData} with the given information -- e.g. for creating bootstrap and test response. * @@ -752,28 +776,28 @@ private static MetadataResponseData.MetadataResponseTopic prepareMetadataRespons return metadataResponseTopic; } - private static void setPasswordConfigIfExists(KafkaCruiseControlConfig configs, Map props, String name) { + private static T setTypedConfigIfExists(Function typedGetter, Map props, String name) { + final T result; try { - props.put(name, configs.getPassword(name)); + result = typedGetter.apply(name); } catch (ConfigException ce) { - // let it go. + // Config key doesn't exist. + return null; } + props.put(name, result); + return result; } - private static void setStringConfigIfExists(KafkaCruiseControlConfig configs, Map props, String name) { - try { - props.put(name, configs.getString(name)); - } catch (ConfigException ce) { - // let it go. - } + public static Password setPasswordConfigIfExists(AbstractConfig configs, Map props, String name) { + return setTypedConfigIfExists(configs::getPassword, props, name); } - private static void setClassConfigIfExists(KafkaCruiseControlConfig configs, Map props, String name) { - try { - props.put(name, configs.getClass(name)); - } catch (ConfigException ce) { - // let it go. - } + public static String setStringConfigIfExists(AbstractConfig configs, Map props, String name) { + return setTypedConfigIfExists(configs::getString, props, name); + } + + public static Class setClassConfigIfExists(AbstractConfig configs, Map props, String name) { + return setTypedConfigIfExists(configs::getClass, props, name); } /** @@ -781,9 +805,10 @@ private static void setClassConfigIfExists(KafkaCruiseControlConfig configs, Map * @param cluster The current cluster state. * @param tp The topic partition to check. * @return {@code true} if the partition is currently under replicated. + * @throws PartitionNotExistsException if the partition does not exist. */ - public static boolean isPartitionUnderReplicated(Cluster cluster, TopicPartition tp) throws - PartitionNotExistsException { + public static boolean isPartitionUnderReplicated(Cluster cluster, TopicPartition tp) + throws PartitionNotExistsException { PartitionInfo partitionInfo = cluster.partition(tp); if (partitionInfo == null) { throw new PartitionNotExistsException("Partition " + tp + " does not exist."); @@ -905,19 +930,19 @@ public static , V, VT extends Deserializer> Con consumerProps.setProperty(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, Integer.toString(Integer.MAX_VALUE)); consumerProps.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializer.getName()); consumerProps.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializer.getName()); - consumerProps.setProperty(ConsumerConfig.RECONNECT_BACKOFF_MS_CONFIG, configs.get(RECONNECT_BACKOFF_MS_CONFIG).toString()); + consumerProps.setProperty(ConsumerConfig.RECONNECT_BACKOFF_MS_CONFIG, + Objects.requireNonNullElse(configs.get(RECONNECT_BACKOFF_MS_CONFIG), DEFAULT_RECONNECT_BACKOFF_MS).toString()); return new KafkaConsumer<>(consumerProps); } /** - * * Returns the right KafkaCruiseControl app, depending on the vertx.enabled property. * * @param config The configurations for Cruise Control. * @param port The port for the REST API. * @param hostname The hostname for the REST API. * @return KafkaCruiseControlApp class depending on the vertx.enabled property. - * @throws ServletException + * @throws ServletException if there is an error during configuration. */ public static KafkaCruiseControlApp getCruiseControlApp(KafkaCruiseControlConfig config, Integer port, String hostname) throws ServletException { if (config.getBoolean(WebServerConfig.VERTX_ENABLED_CONFIG)) { diff --git a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/config/KafkaCruiseControlConfig.java b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/config/KafkaCruiseControlConfig.java index b422e7a01e..df58fc9be1 100644 --- a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/config/KafkaCruiseControlConfig.java +++ b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/config/KafkaCruiseControlConfig.java @@ -7,6 +7,7 @@ import com.linkedin.cruisecontrol.common.CruiseControlConfigurable; import com.linkedin.kafka.cruisecontrol.config.constants.AnalyzerConfig; import com.linkedin.kafka.cruisecontrol.config.constants.AnomalyDetectorConfig; +import com.linkedin.kafka.cruisecontrol.config.constants.PersistedDataConfig; import com.linkedin.kafka.cruisecontrol.config.constants.CruiseControlParametersConfig; import com.linkedin.kafka.cruisecontrol.config.constants.CruiseControlRequestConfig; import com.linkedin.kafka.cruisecontrol.config.constants.ExecutorConfig; @@ -16,6 +17,7 @@ import com.linkedin.kafka.cruisecontrol.metricsreporter.CruiseControlMetricsReporterConfig; import java.nio.file.Files; import java.nio.file.Paths; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Set; @@ -23,6 +25,7 @@ import com.linkedin.kafka.cruisecontrol.servlet.security.BasicSecurityProvider; import com.linkedin.kafka.cruisecontrol.servlet.security.SecurityProvider; import com.linkedin.kafka.cruisecontrol.servlet.security.jwt.JwtSecurityProvider; +import org.apache.commons.lang3.StringUtils; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; import java.util.Map; @@ -38,9 +41,18 @@ public class KafkaCruiseControlConfig extends AbstractConfig { private static final ConfigDef CONFIG; static { - CONFIG = CruiseControlRequestConfig.define(CruiseControlParametersConfig.define(AnomalyDetectorConfig.define( - AnalyzerConfig.define(ExecutorConfig.define(MonitorConfig.define(WebServerConfig.define( - UserTaskManagerConfig.define(new ConfigDef())))))))).withClientSslSupport().withClientSaslSupport(); + ConfigDef configDef = new ConfigDef(); + configDef = AnalyzerConfig.define(configDef); + configDef = AnomalyDetectorConfig.define(configDef); + configDef = CruiseControlParametersConfig.define(configDef); + configDef = CruiseControlRequestConfig.define(configDef); + configDef = ExecutorConfig.define(configDef); + configDef = MonitorConfig.define(configDef); + configDef = PersistedDataConfig.define(configDef); + configDef = UserTaskManagerConfig.define(configDef); + configDef = WebServerConfig.define(configDef); + configDef.withClientSslSupport().withClientSaslSupport(); + CONFIG = configDef; } public KafkaCruiseControlConfig(Map originals) { @@ -59,6 +71,18 @@ public KafkaCruiseControlConfig(Map originals, boolean doLog) { sanityCheckWebServerUrlPrefix(); } + /** + * Package private for testing. Since configDef controls which configurations can be set, being + * able to set a simple or mocked one is helpful for unit testing. + * + * @param configDef Overriding the parse() method allows control over which configs end up in + * {@link KafkaCruiseControlConfig}'s map of configurations. + * @param originals The key/value pairs of configs to set. + */ + KafkaCruiseControlConfig(ConfigDef configDef, Map originals) { + super(configDef, originals, false); + } + /** * @return Merged config values. */ @@ -122,6 +146,42 @@ public List getConfiguredInstances(String key, Class t, Map getMap(String key) { + Map result = new HashMap<>(); + String value = getString(key); + if (StringUtils.isBlank(value)) { + return result; + } + + for (String pair : value.split(";")) { + if (StringUtils.isNotBlank(pair)) { + final String[] parts = pair.split("="); + + // An empty value is allowed, but not an empty key. + if (parts.length == 2 && StringUtils.isNotBlank(parts[0])) { + result.put(parts[0].trim(), parts[1].trim()); + } else if (parts.length == 1 && pair.stripTrailing().equals(parts[0] + "=")) { + result.put(parts[0].trim(), ""); + } else { + throw new ConfigException(String.format( + "Invalid configuration map entry: \"%s\", found for key: \"%s\". " + + "Each entry must be a key=value pair.", + pair, key)); + } + } + } + return result; + } + /** * Sanity check to ensure that {@link ExecutorConfig#ZOOKEEPER_CONNECT_CONFIG} is set if *
    diff --git a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/config/constants/PersistedDataConfig.java b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/config/constants/PersistedDataConfig.java new file mode 100644 index 0000000000..80b9ed438d --- /dev/null +++ b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/config/constants/PersistedDataConfig.java @@ -0,0 +1,157 @@ +/* + * Copyright 2024 LinkedIn Corp. Licensed under the BSD 2-Clause License (the "License"). See License in the project root for license information. + */ +package com.linkedin.kafka.cruisecontrol.config.constants; + +import com.linkedin.kafka.cruisecontrol.persisteddata.BackingMethod; +import org.apache.kafka.common.config.ConfigDef; + +import static org.apache.kafka.common.config.ConfigDef.Range.atLeast; + + +/** + * A class to keep Cruise Control persisted data configs and defaults. + * DO NOT CHANGE EXISTING CONFIG NAMES AS CHANGES WOULD BREAK USER CODE. + */ +public final class PersistedDataConfig { + + public static final String CONFIG_PREFIX = "persisted.data."; + + /** + * persisted.data.backing.method + */ + public static final String BACKING_METHOD_CONFIG = CONFIG_PREFIX + "backing.method"; + public static final String DEFAULT_BACKING_METHOD = "memory"; + public static final String BACKING_METHOD_DOC = + "The method to use to store persisted data. This is the first " + + "\"persisted.data\" config to set which will determine which other configs " + + "of the series should be configured. The available options are: " + + BackingMethod.stringValues().toString() + + ". The default is \"" + DEFAULT_BACKING_METHOD + "\", which doesn't durably " + + "persist any runtime data."; + + /** + * persisted.data.kafka.topic.name + */ + public static final String KAFKA_TOPIC_NAME_CONFIG = CONFIG_PREFIX + "kafka.topic.name"; + public static final String DEFAULT_KAFKA_TOPIC_NAME = "__CruiseControlPersistentData"; + public static final String KAFKA_TOPIC_NAME_DOC = + "The name of the kafka topic to use to persist data when " + "\"" + + BACKING_METHOD_CONFIG + "\" is set to \"kafka\". If the topic is not " + + "present, then it will be created."; + + /** + * persisted.data.kafka.topic.partition.count + */ + public static final String KAFKA_TOPIC_PARTITION_COUNT_CONFIG = + CONFIG_PREFIX + "kafka.topic.partition.count"; + public static final int DEFAULT_KAFKA_TOPIC_PARTITION_COUNT = 2; + public static final String KAFKA_TOPIC_PARTITION_COUNT_DOC = + "The number of partitions to ensure are present " + + "for the kafka topic. Only applies when \"" + BACKING_METHOD_CONFIG + + "\" is set to \"kafka\". If the topic has fewer than this number of " + + "partitions, then partitions will be added."; + + /** + * persisted.data.kafka.topic.replication.factor + */ + public static final String KAFKA_TOPIC_REPLICATION_FACTOR_CONFIG = + CONFIG_PREFIX + "kafka.topic.replication.factor"; + public static final short DEFAULT_KAFKA_TOPIC_REPLICATION_FACTOR = 2; + public static final String KAFKA_TOPIC_REPLICATION_FACTOR_DOC = + "The replication factor to use for the kafka " + "topic. Only applies when \"" + + BACKING_METHOD_CONFIG + "\" is set to \"kafka\". Multiple partition " + + "replicas are desirable to ensure the topic is reasonably available."; + + /** + * persisted.data.kafka.topic.additional.configs.map + */ + public static final String KAFKA_TOPIC_ADDITIONAL_CONFIGS_MAP_CONFIG = + CONFIG_PREFIX + "kafka.topic.additional.configs.map"; + public static final String DEFAULT_KAFKA_TOPIC_ADDITIONAL_CONFIGS_MAP = ""; + public static final String KAFKA_TOPIC_ADDITIONAL_CONFIGS_MAP_DOC = + "The configs to apply to the kafka topic used to " + + "persist Cruise Control data. Only applies if \"" + BACKING_METHOD_CONFIG + + "\" is set to \"kafka\". This \"list\" should be a semicolon separated " + + "string of 'key=value' pairs. The keys and values need to be valid Kafka " + + "Topic configs. See: " + + "https://kafka.apache.org/documentation/#topicconfigs"; + + /** + * persisted.data.kafka.producer.additional.configs.map + */ + public static final String KAFKA_PRODUCER_ADDITIONAL_CONFIGS_MAP_CONFIG = + CONFIG_PREFIX + "kafka.producer.additional.configs.map"; + public static final String DEFAULT_KAFKA_PRODUCER_ADDITIONAL_CONFIGS_MAP = ""; + public static final String KAFKA_PRODUCER_ADDITIONAL_CONFIGS_MAP_DOC = + "The additional configs to use when creating the kafka " + + "producer to persist Cruise Control data. Only applies if \"" + + BACKING_METHOD_CONFIG + "\" is set to \"kafka\". This \"list\" should be a " + + "semicolon separated string of 'key=value' pairs. The keys and values need " + + "to be valid Kafka Producer configs. See: " + + "https://kafka.apache.org/documentation/#producerconfigs"; + + /** + * persisted.data.kafka.consumer.additional.configs.map + */ + public static final String KAFKA_CONSUMER_ADDITIONAL_CONFIGS_MAP_CONFIG = + CONFIG_PREFIX + "kafka.consumer.additional.configs.map"; + public static final String DEFAULT_KAFKA_CONSUMER_ADDITIONAL_CONFIGS_MAP = ""; + public static final String KAFKA_CONSUMER_ADDITIONAL_CONFIGS_MAP_DOC = + "The additional configs to use when creating the kafka " + + "consumer to read persisted Cruise Control data. Only applies if \"" + + BACKING_METHOD_CONFIG + "\" is set to \"kafka\". This \"list\" should be a " + + "semicolon separated string of 'key=value' pairs. The keys and values need " + + "to be valid Kafka Consumer configs. See: " + + "https://kafka.apache.org/documentation/#consumerconfigs"; + + private PersistedDataConfig() { + } + + /** + * Define persisted data configs. + * + * @param configDef Config definition. + * @return The given ConfigDef after defining the common configs. + */ + public static ConfigDef define(ConfigDef configDef) { + return configDef + .define(BACKING_METHOD_CONFIG, + ConfigDef.Type.STRING, + DEFAULT_BACKING_METHOD, + ConfigDef.Importance.MEDIUM, + BACKING_METHOD_DOC) + .define(KAFKA_TOPIC_NAME_CONFIG, + ConfigDef.Type.STRING, + DEFAULT_KAFKA_TOPIC_NAME, + ConfigDef.Importance.LOW, + KAFKA_TOPIC_NAME_DOC) + .define(KAFKA_TOPIC_PARTITION_COUNT_CONFIG, + ConfigDef.Type.INT, + DEFAULT_KAFKA_TOPIC_PARTITION_COUNT, + atLeast(1), + ConfigDef.Importance.LOW, + KAFKA_TOPIC_PARTITION_COUNT_DOC) + .define(KAFKA_TOPIC_REPLICATION_FACTOR_CONFIG, + ConfigDef.Type.SHORT, + DEFAULT_KAFKA_TOPIC_REPLICATION_FACTOR, + atLeast(1), + ConfigDef.Importance.MEDIUM, + KAFKA_TOPIC_REPLICATION_FACTOR_DOC) + .define(KAFKA_TOPIC_ADDITIONAL_CONFIGS_MAP_CONFIG, + ConfigDef.Type.STRING, + DEFAULT_KAFKA_TOPIC_ADDITIONAL_CONFIGS_MAP, + ConfigDef.Importance.LOW, + KAFKA_TOPIC_ADDITIONAL_CONFIGS_MAP_DOC) + .define(KAFKA_PRODUCER_ADDITIONAL_CONFIGS_MAP_CONFIG, + ConfigDef.Type.STRING, + DEFAULT_KAFKA_PRODUCER_ADDITIONAL_CONFIGS_MAP, + ConfigDef.Importance.LOW, + KAFKA_PRODUCER_ADDITIONAL_CONFIGS_MAP_DOC) + .define(KAFKA_CONSUMER_ADDITIONAL_CONFIGS_MAP_CONFIG, + ConfigDef.Type.STRING, + DEFAULT_KAFKA_CONSUMER_ADDITIONAL_CONFIGS_MAP, + ConfigDef.Importance.LOW, + KAFKA_CONSUMER_ADDITIONAL_CONFIGS_MAP_DOC); + } +} diff --git a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/detector/MaintenanceEventTopicReader.java b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/detector/MaintenanceEventTopicReader.java index c9bc2cd2bb..5d18b6802f 100644 --- a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/detector/MaintenanceEventTopicReader.java +++ b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/detector/MaintenanceEventTopicReader.java @@ -5,6 +5,7 @@ package com.linkedin.kafka.cruisecontrol.detector; import com.linkedin.kafka.cruisecontrol.KafkaCruiseControl; +import com.linkedin.kafka.cruisecontrol.KafkaCruiseControlUtils; import com.linkedin.kafka.cruisecontrol.config.constants.AnomalyDetectorConfig; import com.linkedin.kafka.cruisecontrol.exception.SamplingException; import java.time.Duration; @@ -30,9 +31,6 @@ import static com.linkedin.kafka.cruisecontrol.KafkaCruiseControlUtils.CLIENT_REQUEST_TIMEOUT_MS; import static com.linkedin.kafka.cruisecontrol.KafkaCruiseControlUtils.consumptionDone; -import static com.linkedin.kafka.cruisecontrol.KafkaCruiseControlUtils.createTopic; -import static com.linkedin.kafka.cruisecontrol.KafkaCruiseControlUtils.maybeIncreasePartitionCount; -import static com.linkedin.kafka.cruisecontrol.KafkaCruiseControlUtils.maybeUpdateTopicConfig; import static com.linkedin.kafka.cruisecontrol.KafkaCruiseControlUtils.sanityCheckOffsetFetch; import static com.linkedin.kafka.cruisecontrol.KafkaCruiseControlUtils.wrapTopic; import static com.linkedin.kafka.cruisecontrol.detector.AnomalyDetectorUtils.ANOMALY_DETECTION_TIME_MS_OBJECT_CONFIG; @@ -333,11 +331,7 @@ protected void ensureTopicCreated(Map config) { * @param maintenanceEventTopic Maintenance event topic. */ protected void maybeCreateOrUpdateTopic(AdminClient adminClient, NewTopic maintenanceEventTopic) { - if (!createTopic(adminClient, maintenanceEventTopic)) { - // Update topic config and partition count to ensure desired properties. - maybeUpdateTopicConfig(adminClient, maintenanceEventTopic); - maybeIncreasePartitionCount(adminClient, maintenanceEventTopic); - } + KafkaCruiseControlUtils.maybeCreateOrUpdateTopic(adminClient, maintenanceEventTopic); } @Override diff --git a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/executor/Executor.java b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/executor/Executor.java index bf948f4a0c..d2c519f2ba 100644 --- a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/executor/Executor.java +++ b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/executor/Executor.java @@ -22,6 +22,8 @@ import com.linkedin.kafka.cruisecontrol.executor.strategy.StrategyOptions; import com.linkedin.kafka.cruisecontrol.model.ReplicaPlacementInfo; import com.linkedin.kafka.cruisecontrol.monitor.LoadMonitor; +import com.linkedin.kafka.cruisecontrol.persisteddata.namespace.ExecutorPersistedData; +import com.linkedin.kafka.cruisecontrol.persisteddata.namespace.ExecutorPersistedData.BrokerMapKey; import com.linkedin.kafka.cruisecontrol.servlet.UserTaskManager; import java.time.Duration; import java.util.ArrayList; @@ -116,11 +118,9 @@ public class Executor { private final AtomicInteger _numExecutionStartedInNonKafkaAssignerMode; private volatile boolean _isKafkaAssignerMode; private volatile boolean _skipInterBrokerReplicaConcurrencyAdjustment; - // TODO: Execution history is currently kept in memory, but ideally we should move it to a persistent store. private final long _demotionHistoryRetentionTimeMs; private final long _removalHistoryRetentionTimeMs; - private final ConcurrentMap _latestDemoteStartTimeMsByBrokerId; - private final ConcurrentMap _latestRemoveStartTimeMsByBrokerId; + private final ExecutorPersistedData _persistedData; private final ScheduledExecutorService _executionHistoryScannerExecutor; private UserTaskManager _userTaskManager; private final AnomalyDetectorManager _anomalyDetectorManager; @@ -154,8 +154,10 @@ public class Executor { public Executor(KafkaCruiseControlConfig config, Time time, MetricRegistry dropwizardMetricRegistry, - AnomalyDetectorManager anomalyDetectorManager) { - this(config, time, dropwizardMetricRegistry, null, null, anomalyDetectorManager); + AnomalyDetectorManager anomalyDetectorManager, + ExecutorPersistedData persistedData) { + this(config, time, dropwizardMetricRegistry, null, null, anomalyDetectorManager, + persistedData); } /** @@ -169,7 +171,8 @@ public Executor(KafkaCruiseControlConfig config, MetricRegistry dropwizardMetricRegistry, MetadataClient metadataClient, ExecutorNotifier executorNotifier, - AnomalyDetectorManager anomalyDetectorManager) { + AnomalyDetectorManager anomalyDetectorManager, + ExecutorPersistedData persistedData) { _numExecutionStopped = new AtomicInteger(0); _numExecutionStoppedByUser = new AtomicInteger(0); _executionStoppedByUser = new AtomicBoolean(false); @@ -202,8 +205,7 @@ public Executor(KafkaCruiseControlConfig config, _requestedExecutionProgressCheckIntervalMs = null; _proposalExecutor = Executors.newSingleThreadExecutor(new KafkaCruiseControlThreadFactory("ProposalExecutor", false, LOG)); - _latestDemoteStartTimeMsByBrokerId = new ConcurrentHashMap<>(); - _latestRemoveStartTimeMsByBrokerId = new ConcurrentHashMap<>(); + _persistedData = persistedData; _executorState = ExecutorState.noTaskInProgress(recentlyDemotedBrokers(), recentlyRemovedBrokers()); _stopSignal = new AtomicInteger(NO_STOP_EXECUTION); _hasOngoingExecution = false; @@ -421,22 +423,6 @@ private void registerGaugeSensors(MetricRegistry dropwizardMetricRegistry) { .getAvgExecutionConcurrency(ConcurrencyType.LEADERSHIP_BROKER)); } - private void removeExpiredDemotionHistory() { - LOG.debug("Remove expired demotion history"); - _latestDemoteStartTimeMsByBrokerId.entrySet().removeIf(entry -> { - long startTime = entry.getValue(); - return startTime != ExecutionUtils.PERMANENT_TIMESTAMP && startTime + _demotionHistoryRetentionTimeMs < _time.milliseconds(); - }); - } - - private void removeExpiredRemovalHistory() { - LOG.debug("Remove expired broker removal history"); - _latestRemoveStartTimeMsByBrokerId.entrySet().removeIf(entry -> { - long startTime = entry.getValue(); - return startTime != ExecutionUtils.PERMANENT_TIMESTAMP && startTime + _removalHistoryRetentionTimeMs < _time.milliseconds(); - }); - } - /** * A runnable class to remove expired execution history. */ @@ -444,8 +430,8 @@ private class ExecutionHistoryScanner implements Runnable { @Override public void run() { try { - removeExpiredDemotionHistory(); - removeExpiredRemovalHistory(); + removeExpiredBrokerHistory(BrokerMapKey.DEMOTE, _demotionHistoryRetentionTimeMs); + removeExpiredBrokerHistory(BrokerMapKey.REMOVE, _removalHistoryRetentionTimeMs); } catch (Throwable t) { LOG.warn("Received exception when trying to expire execution history.", t); } @@ -677,66 +663,113 @@ public void run() { * Recently demoted brokers are the ones *
      *
    • for which a broker demotion was started, regardless of how the corresponding process was completed, or
    • - *
    • that are explicitly marked so -- e.g. via a pluggable component using {@link #addRecentlyDemotedBrokers(Set)}.
    • + *
    • that are explicitly marked so -- e.g. via a pluggable component using {@link #addRecentlyDemotedBrokersPermanently(Set)}.
    • *
    * * @return IDs of recently demoted brokers -- i.e. demoted within the last {@link #_demotionHistoryRetentionTimeMs}. */ public Set recentlyDemotedBrokers() { - return Collections.unmodifiableSet(_latestDemoteStartTimeMsByBrokerId.keySet()); + return _persistedData.getDemotedOrRemovedBrokers(BrokerMapKey.DEMOTE).keySet(); } /** * Recently removed brokers are the ones *
      *
    • for which a broker removal was started, regardless of how the corresponding process was completed, or
    • - *
    • that are explicitly marked so -- e.g. via a pluggable component using {@link #addRecentlyRemovedBrokers(Set)}.
    • + *
    • that are explicitly marked so -- e.g. via a pluggable component using {@link #addRecentlyRemovedBrokersPermanently(Set)}.
    • *
    * * @return IDs of recently removed brokers -- i.e. removed within the last {@link #_removalHistoryRetentionTimeMs}. */ public Set recentlyRemovedBrokers() { - return Collections.unmodifiableSet(_latestRemoveStartTimeMsByBrokerId.keySet()); + return _persistedData.getDemotedOrRemovedBrokers(BrokerMapKey.REMOVE).keySet(); } /** - * Drop the given brokers from the recently removed brokers. + * Removes expired brokers from the broker history map. Permanent brokers are not removed. + * Package protected for testing. * - * @param brokersToDrop Brokers to drop from the {@link #_latestRemoveStartTimeMsByBrokerId}. - * @return {@code true} if any elements were removed from {@link #_latestRemoveStartTimeMsByBrokerId}. + * @param persistenceKey The history map key. + * @param retentionTimeMs The configured retention time in ms to evict brokers after this much + * time has elapsed. */ - public boolean dropRecentlyRemovedBrokers(Set brokersToDrop) { - return _latestRemoveStartTimeMsByBrokerId.entrySet().removeIf(entry -> (brokersToDrop.contains(entry.getKey()))); + void removeExpiredBrokerHistory(BrokerMapKey persistenceKey, long retentionTimeMs) { + LOG.debug("Remove expired {} history", persistenceKey); + _persistedData.modifyDemotedOrRemovedBrokers(persistenceKey, + data -> data.values().removeIf(startTime -> + startTime != ExecutionUtils.PERMANENT_TIMESTAMP + && startTime + retentionTimeMs < _time.milliseconds())); + } + + /** + * Drop the given brokers from the recent brokers. + * + * @param persistenceKey The history map key. + * @param brokers The brokers to remove from the map of recent brokers. + * @return {@code true} if any elements were removed from the collection. + */ + private boolean dropRecentBrokers(BrokerMapKey persistenceKey, Collection brokers) { + return _persistedData.modifyDemotedOrRemovedBrokers(persistenceKey, + data -> data.keySet().removeAll(brokers)); } /** * Drop the given brokers from the recently demoted brokers. * - * @param brokersToDrop Brokers to drop from the {@link #_latestDemoteStartTimeMsByBrokerId}. - * @return {@code true} if any elements were removed from {@link #_latestDemoteStartTimeMsByBrokerId}. + * @param brokersToDrop Brokers to drop from the collection of demoted brokers. + * @return {@code true} if any elements were removed from the collection. */ public boolean dropRecentlyDemotedBrokers(Set brokersToDrop) { - return _latestDemoteStartTimeMsByBrokerId.entrySet().removeIf(entry -> (brokersToDrop.contains(entry.getKey()))); + return dropRecentBrokers(BrokerMapKey.DEMOTE, brokersToDrop); } /** - * Add the given brokers to the recently removed brokers permanently -- i.e. until they are explicitly dropped by user. - * If given set has brokers that were already removed recently, make them a permanent part of recently removed brokers. + * Drop the given brokers from the recently removed brokers. * - * @param brokersToAdd Brokers to add to the {@link #_latestRemoveStartTimeMsByBrokerId}. + * @param brokersToDrop Brokers to drop from the collection of removed brokers. + * @return {@code true} if any elements were removed from the collection. */ - public void addRecentlyRemovedBrokers(Set brokersToAdd) { - brokersToAdd.forEach(brokerId -> _latestRemoveStartTimeMsByBrokerId.put(brokerId, ExecutionUtils.PERMANENT_TIMESTAMP)); + public boolean dropRecentlyRemovedBrokers(Set brokersToDrop) { + return dropRecentBrokers(BrokerMapKey.REMOVE, brokersToDrop); } /** * Add the given brokers from the recently demoted brokers permanently -- i.e. until they are explicitly dropped by user. * If given set has brokers that were already demoted recently, make them a permanent part of recently demoted brokers. * - * @param brokersToAdd Brokers to add to the {@link #_latestDemoteStartTimeMsByBrokerId}. + * @param brokersToAdd Brokers to add to the collection of demoted brokers. */ - public void addRecentlyDemotedBrokers(Set brokersToAdd) { - brokersToAdd.forEach(brokerId -> _latestDemoteStartTimeMsByBrokerId.put(brokerId, ExecutionUtils.PERMANENT_TIMESTAMP)); + public void addRecentlyDemotedBrokersPermanently(Set brokersToAdd) { + addRecentBrokers(BrokerMapKey.DEMOTE, brokersToAdd, ExecutionUtils.PERMANENT_TIMESTAMP); + } + + /** + * Add the given brokers to the recently removed brokers permanently -- i.e. until they are explicitly dropped by user. + * If given set has brokers that were already removed recently, make them a permanent part of recently removed brokers. + * + * @param brokersToAdd Brokers to add to the collection of removed brokers. + */ + public void addRecentlyRemovedBrokersPermanently(Set brokersToAdd) { + addRecentBrokers(BrokerMapKey.REMOVE, brokersToAdd, ExecutionUtils.PERMANENT_TIMESTAMP); + } + + /** + * Add/overwrite the latest time of recent brokers (if any). Package protected for testing. + * + * @param persistenceKey The history map key. + * @param brokers The brokers to add the map of recent brokers. + * @param timestamp The time in unix milliseconds when the broker can be removed from the set. + */ + void addRecentBrokers(BrokerMapKey persistenceKey, Collection brokers, + Long timestamp) { + _persistedData.modifyDemotedOrRemovedBrokers(persistenceKey, + data -> { + brokers.forEach(id -> + data.compute(id, (brokerId, startTime) -> + startTime == null || startTime != ExecutionUtils.PERMANENT_TIMESTAMP + ? timestamp : startTime)); + return null; + }); } /** @@ -1319,22 +1352,10 @@ private class ProposalExecutionRunnable implements Runnable { throw new IllegalStateException("User task manager cannot be null."); } if (_demotedBrokers != null) { - // Add/overwrite the latest demotion time of (non-permanent) demoted brokers (if any). - _demotedBrokers.forEach(id -> { - Long demoteStartTime = _latestDemoteStartTimeMsByBrokerId.get(id); - if (demoteStartTime == null || demoteStartTime != ExecutionUtils.PERMANENT_TIMESTAMP) { - _latestDemoteStartTimeMsByBrokerId.put(id, _time.milliseconds()); - } - }); + addRecentBrokers(BrokerMapKey.DEMOTE, _demotedBrokers, _time.milliseconds()); } if (_removedBrokers != null) { - // Add/overwrite the latest removal time of (non-permanent) removed brokers (if any). - _removedBrokers.forEach(id -> { - Long removeStartTime = _latestRemoveStartTimeMsByBrokerId.get(id); - if (removeStartTime == null || removeStartTime != ExecutionUtils.PERMANENT_TIMESTAMP) { - _latestRemoveStartTimeMsByBrokerId.put(id, _time.milliseconds()); - } - }); + addRecentBrokers(BrokerMapKey.REMOVE, _removedBrokers, _time.milliseconds()); } _recentlyDemotedBrokers = recentlyDemotedBrokers(); _recentlyRemovedBrokers = recentlyRemovedBrokers(); diff --git a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/BackingMethod.java b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/BackingMethod.java new file mode 100644 index 0000000000..5148ddd3ad --- /dev/null +++ b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/BackingMethod.java @@ -0,0 +1,38 @@ +/* + * Copyright 2024 LinkedIn Corp. Licensed under the BSD 2-Clause License (the "License"). See License in the project root for license information. + */ +package com.linkedin.kafka.cruisecontrol.persisteddata; + +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +/** + * Defines the backing method types of persistent data. + */ +public enum BackingMethod { + MEMORY, + KAFKA; + + /** + * Parse the string version of a backing method into the enum. + * + * @param value The value to parse. + * @return The correctly parsed enum. + */ + public static BackingMethod fromString(String value) { + return valueOf(value.toUpperCase()); + } + + /** + * Get the lower case string values of all backing methods. + * + * @return The list of all the backing methods as lower case strings. + */ + public static List stringValues() { + return Arrays.stream(BackingMethod.values()) + .map(backingMethod -> backingMethod.toString().toLowerCase()) + .sorted() + .collect(Collectors.toList()); + } +} diff --git a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/KeyMappedMap.java b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/KeyMappedMap.java new file mode 100644 index 0000000000..638b102fb0 --- /dev/null +++ b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/KeyMappedMap.java @@ -0,0 +1,139 @@ +/* + * Copyright 2024 LinkedIn Corp. Licensed under the BSD 2-Clause License (the "License"). See License in the project root for license information. + */ +package com.linkedin.kafka.cruisecontrol.persisteddata; + +import java.util.Collection; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; +import javax.annotation.Nonnull; +import org.apache.commons.lang3.tuple.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A {@link Map} where the keys stored internally are a different type or format than those that are + * externally used. For example, a number can be stored, but internally it is stored as a string. + * The functions that do the mapping between the two representations just need to be 1:1 inverses of + * each other. + */ +public class KeyMappedMap implements Map { + + private static final Logger LOG = LoggerFactory.getLogger(KeyMappedMap.class); + + // Stores the actual internal data. + private final Map _child; + + // Converts an external key to an internal key. + private final Function _keyMapper; + + // Converts an internal key back to an external key. + private final Function _keyBackMapper; + + /** + * Wraps all key reference methods of the child map by first applying the + * keyMapper to the key. + * + * @param child The map that stores all the actual data. + * @param keyMapper Used to remap incoming keys to the internally stored keys. + * @param keyBackMapper Used to remap outgoing keys from the internally stored keys. To ensure + * keys are correctly mapped back, they should validate that the key is coming from the correct + * name/number space, and return null if the validation fails. For example, for a String mapper + * that adds a prefix string to each key, the back mapper should return null if that same prefix + * is not present on the mapped key. + */ + public KeyMappedMap(@Nonnull Map child, + Function keyMapper, + Function keyBackMapper) { + this._child = child; + this._keyMapper = keyMapper; + this._keyBackMapper = keyBackMapper; + } + + @Override + public V get(Object externalKey) { + final INTERNALKEY internalKey = this._keyMapper.apply(externalKey); + final V value = this._child.get(internalKey); + LOG.debug("Getting externalKey={}, internalKey={} value={}", + externalKey, internalKey, value); + return value; + } + + @Override + public V put(EXTERNALKEY externalKey, V value) { + final INTERNALKEY internalKey = this._keyMapper.apply(externalKey); + LOG.debug("Putting externalKey={}, internalKey={} value={}", + externalKey, internalKey, value); + return this._child.put(internalKey, value); + } + + @Override + public boolean containsKey(Object key) { + return this._child.containsKey(this._keyMapper.apply(key)); + } + + @Override + public V remove(Object key) { + return this._child.remove(this._keyMapper.apply(key)); + } + + @Override + public int size() { + return this.keySet().size(); + } + + @Override + public boolean isEmpty() { + return this.keySet().isEmpty(); + } + + @Override + public boolean containsValue(Object value) { + return this._child.entrySet().stream() + .filter(e -> Objects.nonNull(this._keyBackMapper.apply(e.getKey()))) + .map(Entry::getValue) + .anyMatch(v -> Objects.equals(v, value)); + } + + @Override + public void putAll(@Nonnull Map map) { + this._child.putAll( + map.entrySet().stream().collect( + Collectors.toMap(e -> this._keyMapper.apply(e.getKey()), Entry::getValue) + )); + } + + @Override + public void clear() { + this.keySet().stream() + .map(this._keyMapper) + .forEach(this._child::remove); + } + + @Override + public Set keySet() { + return this._child.keySet().stream() + .map(this._keyBackMapper) + .filter(Objects::nonNull) + .collect(Collectors.toSet()); + } + + @Override + public Collection values() { + return this._child.entrySet().stream() + .filter(e -> Objects.nonNull(this._keyBackMapper.apply(e.getKey()))) + .map(Entry::getValue) + .collect(Collectors.toList()); + } + + @Override + public Set> entrySet() { + return this._child.entrySet().stream() + .map(e -> Pair.of(this._keyBackMapper.apply(e.getKey()), e.getValue())) + .filter(e -> Objects.nonNull(e.getKey())) + .collect(Collectors.toSet()); + } +} diff --git a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/MapDecorator.java b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/MapDecorator.java new file mode 100644 index 0000000000..3ba31faa53 --- /dev/null +++ b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/MapDecorator.java @@ -0,0 +1,93 @@ +/* + * Copyright 2024 LinkedIn Corp. Licensed under the BSD 2-Clause License (the "License"). See License in the project root for license information. + */ +package com.linkedin.kafka.cruisecontrol.persisteddata; + +import java.util.Collection; +import java.util.Map; +import java.util.Set; +import javax.annotation.Nonnull; + +/** + * Decorates a standard {@link Map} allowing a subset of methods to be overridden instead of all of + * them. + * + * @param The map key type. + * @param The map value type. + */ +public abstract class MapDecorator implements Map { + + protected Map _child; + + /** + * Wraps the child map so accesses to it can be decorated. + * + * @param child The other map instance to decorate. + */ + public MapDecorator(Map child) { + this._child = child; + } + + @Override + public V get(Object key) { + return this._child.get(key); + } + + @Override + public V put(K key, V value) { + return this._child.put(key, value); + } + + @Override + public int size() { + return this._child.size(); + } + + @Override + public boolean isEmpty() { + return this._child.isEmpty(); + } + + @Override + public boolean containsKey(Object key) { + return this._child.containsKey(key); + } + + @Override + public boolean containsValue(Object value) { + return this._child.containsValue(value); + } + + @Override + public V remove(Object key) { + return this._child.remove(key); + } + + @Override + public void putAll(@Nonnull Map map) { + this._child.putAll(map); + } + + @Override + public void clear() { + this._child.clear(); + } + + @Nonnull + @Override + public Set keySet() { + return this._child.keySet(); + } + + @Nonnull + @Override + public Collection values() { + return this._child.values(); + } + + @Nonnull + @Override + public Set> entrySet() { + return this._child.entrySet(); + } +} diff --git a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistedMap.java b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistedMap.java new file mode 100644 index 0000000000..9681877dad --- /dev/null +++ b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistedMap.java @@ -0,0 +1,35 @@ +/* + * Copyright 2024 LinkedIn Corp. Licensed under the BSD 2-Clause License (the "License"). See License in the project root for license information. + */ +package com.linkedin.kafka.cruisecontrol.persisteddata; + +import java.util.Map; +import javax.annotation.Nonnull; + +public class PersistedMap extends MapDecorator { + + /** + * Wraps the child map so accesses to it can be decorated. + * + * @param child The other map instance to decorate. + */ + public PersistedMap(Map child) { + super(child); + } + + @Override + public String remove(@Nonnull Object key) { + return this.put(key.toString(), null); + } + + @Override + public void putAll(@Nonnull Map map) { + map.forEach(this::put); + } + + @Override + public void clear() { + // This only really works if other write operations have stopped. + this.keySet().forEach(this::remove); + } +} diff --git a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistedMapFactory.java b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistedMapFactory.java new file mode 100644 index 0000000000..6600ffd2a6 --- /dev/null +++ b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistedMapFactory.java @@ -0,0 +1,72 @@ +/* + * Copyright 2024 LinkedIn Corp. Licensed under the BSD 2-Clause License (the "License"). See License in the project root for license information. + */ +package com.linkedin.kafka.cruisecontrol.persisteddata; + +import com.linkedin.kafka.cruisecontrol.config.KafkaCruiseControlConfig; +import com.linkedin.kafka.cruisecontrol.config.constants.PersistedDataConfig; +import com.linkedin.kafka.cruisecontrol.persisteddata.kafka.KafkaPersistedMap; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Supplier; +import org.apache.kafka.clients.admin.AdminClient; + +import static com.linkedin.kafka.cruisecontrol.persisteddata.BackingMethod.KAFKA; +import static com.linkedin.kafka.cruisecontrol.persisteddata.BackingMethod.MEMORY; + +/** + * Constructs the correct {@link PersistedMap} implementation based on the provided + * {@link KafkaCruiseControlConfig}. In particular, it uses the value of + * {@link PersistedDataConfig#BACKING_METHOD_CONFIG} to determine which implementation to + * construct. + */ +public class PersistedMapFactory { + + // The overall configuration is used when creating the {@link PersistedMap} implementation. + private final KafkaCruiseControlConfig _config; + + // Keeps the suppliers for implementation-specific instances. + private final Map> _suppliers; + + /** + * Creates an instance that is able to construct the correct {@link PersistedMap} + * implementation. + * + * @param config The complete program configuration to evaluate. Specifically, the + * {@link PersistedDataConfig#BACKING_METHOD_CONFIG} config is needed. + * @param adminClient The admin client to pass to {@link KafkaPersistedMap}, if needed. + */ + public PersistedMapFactory(KafkaCruiseControlConfig config, AdminClient adminClient) { + this(config, + () -> new KafkaPersistedMap(config, adminClient), + () -> new PersistedMap(new ConcurrentHashMap<>())); + } + + /** + * Creates an instance that is able to construct the correct {@link PersistedMap} + * implementation. + * + * @param kafkaSupplier The supplier for {@link KafkaPersistedMap}. + * @param memoryAndDefaultSupplier The supplier for {@link PersistedMap} and the default + * implementation. + */ + PersistedMapFactory(KafkaCruiseControlConfig config, Supplier kafkaSupplier, + Supplier memoryAndDefaultSupplier) { + this._config = config; + this._suppliers = Map.of( + KAFKA, kafkaSupplier, + MEMORY, memoryAndDefaultSupplier); + } + + /** + * Constructs the correct {@link PersistedMap} implementation based on the configured + * {@link PersistedDataConfig#BACKING_METHOD_CONFIG}. + * + * @return The {@link PersistedMap} implementation. + */ + public PersistedMap instance() { + BackingMethod backingMethod = BackingMethod.fromString( + _config.getString(PersistedDataConfig.BACKING_METHOD_CONFIG)); + return this._suppliers.getOrDefault(backingMethod, this._suppliers.get(MEMORY)).get(); + } +} diff --git a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/ValueMappedMap.java b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/ValueMappedMap.java new file mode 100644 index 0000000000..a6604ce184 --- /dev/null +++ b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/ValueMappedMap.java @@ -0,0 +1,136 @@ +/* + * Copyright 2024 LinkedIn Corp. Licensed under the BSD 2-Clause License (the "License"). See License in the project root for license information. + */ +package com.linkedin.kafka.cruisecontrol.persisteddata; + +import java.util.Collection; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; +import javax.annotation.Nonnull; +import org.apache.commons.lang3.tuple.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A {@link Map} where the values stored internally are a different type or format than those that + * are externally used. For example, a number can be stored, but internally it is stored as a + * string. The functions that do the mapping between the two representations just need to be 1:1 + * inverses of each other. + */ +public class ValueMappedMap implements Map { + + private static final Logger LOG = LoggerFactory.getLogger(ValueMappedMap.class); + + // Stores the actual internal data. + private final Map _child; + + // Converts an external value to an internal value. + private final Function _valueMapper; + + // Converts an internal value back to an external value. + private final Function _valueBackMapper; + + /** + * Wraps all value reference methods of the child map by first applying the + * {@code valueMapper} to the value. + * + * @param child The map that stores all the actual data. + * @param valueMapper Used to remap incoming values to the internally stored values. + * @param valueBackMapper Used to remap outgoing values from the internally stored values. + */ + public ValueMappedMap(@Nonnull Map child, + Function valueMapper, + Function valueBackMapper) { + this._child = child; + this._valueMapper = valueMapper; + this._valueBackMapper = valueBackMapper; + } + + @Override + public EXTERNALVALUE get(Object key) { + final INTERNALVALUE internalValue = this._child.get(key); + if (internalValue == null) { + return null; + } + final EXTERNALVALUE externalValue = this._valueBackMapper.apply(internalValue); + LOG.debug("Getting key={}, internalValue={}, externalValue={}", + key, internalValue, externalValue); + return externalValue; + } + + @Override + public EXTERNALVALUE put(K key, EXTERNALVALUE externalValue) { + final INTERNALVALUE internalValue = this._valueMapper.apply(externalValue); + final INTERNALVALUE previousInternalValue = this._child.put(key, internalValue); + LOG.debug("Putting key={}, externalValue={}, internalValue={}, previousInternalValue={}", + key, externalValue, internalValue, previousInternalValue); + if (previousInternalValue == null) { + return null; + } + return this._valueBackMapper.apply(previousInternalValue); + } + + @Override + public boolean containsKey(Object key) { + return this._child.containsKey(key); + } + + @Override + public EXTERNALVALUE remove(Object key) { + final INTERNALVALUE internalvalue = this._child.remove(key); + if (internalvalue == null) { + return null; + } + return this._valueBackMapper.apply(internalvalue); + } + + @Override + public int size() { + return this._child.size(); + } + + @Override + public boolean isEmpty() { + return this._child.isEmpty(); + } + + @Override + public boolean containsValue(Object value) { + return this._child.containsValue(this._valueMapper.apply(value)); + } + + @Override + public void putAll(@Nonnull Map map) { + final Map remapped = + map.entrySet().stream().collect( + Collectors.toMap(Entry::getKey, + e -> this._valueMapper.apply(e.getValue())) + ); + this._child.putAll(remapped); + } + + @Override + public void clear() { + this._child.clear(); + } + + @Override + public Set keySet() { + return this._child.keySet(); + } + + @Override + public Collection values() { + return this._child.values().stream().map(this._valueBackMapper) + .collect(Collectors.toList()); + } + + @Override + public Set> entrySet() { + return this._child.entrySet().stream() + .map(e -> Pair.of(e.getKey(), this._valueBackMapper.apply(e.getValue()))) + .collect(Collectors.toSet()); + } +} diff --git a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/kafka/KafkaPersistedMap.java b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/kafka/KafkaPersistedMap.java new file mode 100644 index 0000000000..64e8f9d863 --- /dev/null +++ b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/kafka/KafkaPersistedMap.java @@ -0,0 +1,348 @@ +/* + * Copyright 2024 LinkedIn Corp. Licensed under the BSD 2-Clause License (the "License"). See License in the project root for license information. + */ +package com.linkedin.kafka.cruisecontrol.persisteddata.kafka; + + +import com.google.common.base.Suppliers; +import com.google.common.collect.ImmutableMap; +import com.linkedin.kafka.cruisecontrol.KafkaCruiseControlUtils; +import com.linkedin.kafka.cruisecontrol.config.KafkaCruiseControlConfig; +import com.linkedin.kafka.cruisecontrol.config.constants.MonitorConfig; +import com.linkedin.kafka.cruisecontrol.config.constants.PersistedDataConfig; +import com.linkedin.kafka.cruisecontrol.persisteddata.PersistedMap; +import java.io.Closeable; +import java.io.IOException; +import java.time.Duration; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.function.Function; +import java.util.function.Supplier; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.config.TopicConfig; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.StringSerializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Stores persistent map data in a compacted Kafka topic. The topic is treated like an event store + * so any other writes to the topic are dynamically picked up and made available for reading. + */ +public class KafkaPersistedMap extends PersistedMap implements Closeable { + + private static final Logger LOG = LoggerFactory.getLogger(KafkaPersistedMap.class); + + // Keys and values are stored as strings within the kafka topic. + private static final Class SERIALIZER_CLASS = StringSerializer.class; + private static final Class DESERIALIZER_CLASS = StringDeserializer.class; + + // The hard-coded producer config. This is overridable. + private static final Map DEFAULT_PRODUCER_CONFIG = ImmutableMap.builder() + .put(ProducerConfig.ACKS_CONFIG, "all") + // 2MB + .put(ProducerConfig.BUFFER_MEMORY_CONFIG, "2000000") + .put(ProducerConfig.CLIENT_ID_CONFIG, + "kafka-cruise-control.kafka-persisted-map.producer") + .put(ProducerConfig.COMPRESSION_TYPE_CONFIG, "gzip") + // 1 hour + .put(ProducerConfig.DELIVERY_TIMEOUT_MS_CONFIG, "3600000") + .put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, SERIALIZER_CLASS.getName()) + .put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, "2") + // 1MB + .put(ProducerConfig.MAX_REQUEST_SIZE_CONFIG, "1000000") + .put(ProducerConfig.RECONNECT_BACKOFF_MS_CONFIG, "1000") + .put(ProducerConfig.RETRY_BACKOFF_MS_CONFIG, "1000") + .put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, SERIALIZER_CLASS.getName()) + .build(); + + // The hard-coded consumer config. This is overridable. + private static final Map DEFAULT_CONSUMER_CONFIG = ImmutableMap.builder() + .put(ConsumerConfig.ALLOW_AUTO_CREATE_TOPICS_CONFIG, "false") + .put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") + .put(ConsumerConfig.CLIENT_ID_CONFIG, + "kafka-cruise-control.kafka-persisted-map.consumer") + .put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") + .put(ConsumerConfig.EXCLUDE_INTERNAL_TOPICS_CONFIG, "false") + .put(ConsumerConfig.GROUP_ID_CONFIG, + "kafka-cruise-control.kafka-persisted-map.consumer-group") + .put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed") + .put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, DESERIALIZER_CLASS.getName()) + .put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, Integer.toString(Integer.MAX_VALUE)) + .put(ConsumerConfig.RECONNECT_BACKOFF_MS_CONFIG, "1000") + .put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, DESERIALIZER_CLASS.getName()) + .build(); + + // The configuration which must be set on the data storage kafka topic. + private static final Map REQUIRED_TOPIC_CONFIG = Map.of( + TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT); + + // The configured data storage kafka topic name. + private final String _topic; + + // The number of partitions that the data storage kafka topic should have. This can be increased + // if additional performance is necessary but can't be reduced without data loss. + private final int _topicPartitions; + + // The configured number of brokers should host each partition of the data storage kafka topic. + private final short _topicReplicationFactor; + + // Additional topic configuration configured by an administrator. + private final Map _topicAdditionalConfigs; + + // The producer instance to send data updates to kafka. + private final Supplier> _producer; + + // The consumer instance to read updates from the kafka topic. + private final Supplier> _consumerFactory; + + // True while the background consumer thread is active. When it is false, the consumer thread + // will terminate. + private volatile boolean _active = true; + + // Holds a reference to the background thread. + private final Thread _cacheUpdater; + + /** + * Reads and writes data in a kafka topic and keeps an eventually consistent view of the + * persisted data. + */ + public KafkaPersistedMap(KafkaCruiseControlConfig config, AdminClient adminClient) { + this(config.getString(PersistedDataConfig.KAFKA_TOPIC_NAME_CONFIG), + config.getInt(PersistedDataConfig.KAFKA_TOPIC_PARTITION_COUNT_CONFIG), + config.getShort(PersistedDataConfig.KAFKA_TOPIC_REPLICATION_FACTOR_CONFIG), + config.getMap(PersistedDataConfig.KAFKA_TOPIC_ADDITIONAL_CONFIGS_MAP_CONFIG), + config.getMap(PersistedDataConfig.KAFKA_PRODUCER_ADDITIONAL_CONFIGS_MAP_CONFIG), + config.getMap(PersistedDataConfig.KAFKA_CONSUMER_ADDITIONAL_CONFIGS_MAP_CONFIG), + KafkaCruiseControlUtils.maybeAddSecurityConfig(config, new HashMap<>()), + String.join(",", config.getList(MonitorConfig.BOOTSTRAP_SERVERS_CONFIG)), + adminClient); + } + + /** + * Reads and writes data in the kafka topic and keeps an eventually consistent view of the + * persisted data. + * + * @param topic The topic name to use and to ensure exists. + * @param topicPartitions The number of partitions to ensure the topic has. + * @param topicReplicationFactor Number of partition replicas to use for the topic. + * @param topicAdditionalConfigs The additional topic configuration to apply to the topic. + * @param producerAdditionalConfigs The additional producer configuration to use. + * @param consumerAdditionalConfigs The additional consumer configuration to use. + * @param commonSecurityConfigs The security configuration to use for the producer and consumer. + * e.g. client SSL options. + * @param bootstrapServers bootstrap.servers configuration to use for the producer and + * consumer. + * @param adminClient Kafka AdminClient used to create the backing Kafka topic. + */ + public KafkaPersistedMap(String topic, + int topicPartitions, + short topicReplicationFactor, + Map topicAdditionalConfigs, + Map producerAdditionalConfigs, + Map consumerAdditionalConfigs, + Map commonSecurityConfigs, + String bootstrapServers, + AdminClient adminClient) { + this(new ConcurrentHashMap<>(), topic, topicPartitions, topicReplicationFactor, + topicAdditionalConfigs, + newTopic -> KafkaCruiseControlUtils.maybeCreateOrUpdateTopic(adminClient, newTopic), + () -> createKafkaProducer(bootstrapServers, producerAdditionalConfigs, + commonSecurityConfigs), + () -> createKafkaConsumer(bootstrapServers, consumerAdditionalConfigs, + commonSecurityConfigs)); + } + + /** + * Package private for testing. + */ + KafkaPersistedMap(Map child, + String topic, + int topicPartitions, + short topicReplicationFactor, + Map topicAdditionalConfigs, + java.util.function.Consumer topicCreator, + Supplier> producer, + Supplier> consumerFactory) { + super(child); + this._topic = topic; + this._topicPartitions = topicPartitions; + this._topicReplicationFactor = topicReplicationFactor; + this._topicAdditionalConfigs = topicAdditionalConfigs; + ensureTopicIsPresentAndConfigured(topicCreator); + + // This odd setup is to lazy-load the producer and also adapt the java Supplier to a + // guava Supplier. + this._producer = Suppliers.memoize(producer::get); + this._consumerFactory = consumerFactory; + + this._cacheUpdater = Executors.defaultThreadFactory() + .newThread(this::consumeAndUpdateCache); + this._cacheUpdater.start(); + } + + /** + * Returns the background thread. Package private for testing. + * + * @return the background thread that updates the cache. + */ + Thread getCacheUpdater() { + return _cacheUpdater; + } + + /** + * Configures and creates the requested Kafka client instance. Package private for testing. + * + * @param bootstrapServers The configured {@code bootstrap.servers} config to use. + * @param defaultConfig The default config values to configure the client with. + * @param securityConfig The security options the client should use when connecting to the Kafka + * cluster. + * @param additionalConfig Any additional config to override the default and security config + * with. + * @param clientFactory Function that takes a config map and returns the Kafka client instance. + * @param The type of Kafka client to return. + * @return A fully configured Kafka client instance. + */ + static C createKafkaClient(String bootstrapServers, Map defaultConfig, + Map securityConfig, Map additionalConfig, + Function, C> clientFactory) { + // Configure the client by combining the default, security and additional config. + Map config = mergeConfig(defaultConfig, securityConfig); + config.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); + config.putAll(additionalConfig); + LOG.debug("KafkaPersistedMap.createKafkaClient(bootstrapServers={}, defaultConfig={}, " + + "securityConfig={}, additionalConfig={}, clientFactory={})", bootstrapServers, + defaultConfig, securityConfig, additionalConfig, clientFactory); + return clientFactory.apply(config); + } + + /** + * Configure and create the KafkaConsumer instance. + * + * @param bootstrapServers The configured {@code bootstrap.servers} config to use. + * @param consumerAdditionalConfigs Any additional {@link KafkaConsumer} configs to use to + * override the default consumer config. + * @param commonSecurityConfigs The kafka client security configuration options. + * @return The {@link Consumer} instance. + */ + private static Consumer createKafkaConsumer(String bootstrapServers, + Map consumerAdditionalConfigs, + Map commonSecurityConfigs) { + return createKafkaClient(bootstrapServers, DEFAULT_CONSUMER_CONFIG, commonSecurityConfigs, + consumerAdditionalConfigs, KafkaConsumer::new); + } + + /** + * Configure and create the KafkaProducer instance. + * + * @param bootstrapServers The configured {@code bootstrap.servers} config to use. + * @param producerAdditionalConfigs Any additional {@link KafkaProducer} configs to use to + * override the default producer config. + * @param commonSecurityConfigs The kafka client security configuration options. + * @return The {@link Producer} instance. + */ + private static Producer createKafkaProducer(String bootstrapServers, + Map producerAdditionalConfigs, + Map commonSecurityConfigs) { + return createKafkaClient(bootstrapServers, DEFAULT_PRODUCER_CONFIG, commonSecurityConfigs, + producerAdditionalConfigs, KafkaProducer::new); + } + + /** + * Merge the two maps into a new one overwriting any default values with those from overrides. + * + * @param defaults The base key/values to overwrite if necessary. + * @param overrides Any key/values to merge over the values in defaults. + * @param The defaults map value type. + * @param The overrides map value type. + * @return The combined map of key/values. + */ + private static Map mergeConfig( + Map defaults, + Map overrides) { + final Map result = new HashMap<>(defaults); + result.putAll(overrides); + return result; + } + + /** + * Create the topic if it isn't present, or update the topic config if it is present. + * + * @param topicCreator Used to create the topic. This uses a functional interface to make + * testing easier to mock. + */ + private void ensureTopicIsPresentAndConfigured( + java.util.function.Consumer topicCreator) { + Map config = new HashMap<>(this._topicAdditionalConfigs); + config.putAll(REQUIRED_TOPIC_CONFIG); + NewTopic topic = new NewTopic(this._topic, this._topicPartitions, + this._topicReplicationFactor); + topic.configs(config); + topicCreator.accept(topic); + } + + /** + * Keep updating the cached aggregate data in a background thread. + */ + private void consumeAndUpdateCache() { + try (Consumer consumer = this._consumerFactory.get()) { + if (this._active) { + // Start the consumer's subscription to the topic. + consumer.subscribe(Collections.singleton(this._topic)); + + // Keep polling for updates. Ignore committing offsets for compacted topic. + while (this._active) { + try { + ConsumerRecords records = consumer.poll( + Duration.ofSeconds(10)); + records.forEach(record -> { + this._child.put(record.key(), record.value()); + LOG.debug("Retrieved record: key={}, value={}", + record.key(), record.value()); + }); + } catch (Exception e) { + LOG.warn(String.format("Error while consuming records from Kafka topic=%s", + this._topic), e); + } + } + } + } + } + + @Override + public void close() throws IOException { + // Signal that the consumer poll loop should stop. + this._active = false; + } + + @Override + public String put(String key, String value) { + String oldValue = this.get(key); + ProducerRecord record = new ProducerRecord<>(this._topic, key, value); + final Future future = this._producer.get().send(record); + try { + future.get(); + LOG.debug("Saved record: key={}, value={}", key, value); + } catch (InterruptedException | ExecutionException e) { + throw new KafkaPersistedMapException( + String.format("Failed to save data to Kafka: key=%s, value=%s", key, value), e); + } + return oldValue; + } +} diff --git a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/kafka/KafkaPersistedMapException.java b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/kafka/KafkaPersistedMapException.java new file mode 100644 index 0000000000..d07d5dde14 --- /dev/null +++ b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/kafka/KafkaPersistedMapException.java @@ -0,0 +1,11 @@ +/* + * Copyright 2024 LinkedIn Corp. Licensed under the BSD 2-Clause License (the "License"). See License in the project root for license information. + */ +package com.linkedin.kafka.cruisecontrol.persisteddata.kafka; + +public class KafkaPersistedMapException extends RuntimeException { + + public KafkaPersistedMapException(String message, Exception cause) { + super(message, cause); + } +} diff --git a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/namespace/ExecutorPersistedData.java b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/namespace/ExecutorPersistedData.java new file mode 100644 index 0000000000..dcfa116dfa --- /dev/null +++ b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/namespace/ExecutorPersistedData.java @@ -0,0 +1,169 @@ +/* + * Copyright 2024 LinkedIn Corp. Licensed under the BSD 2-Clause License (the "License"). See License in the project root for license information. + */ +package com.linkedin.kafka.cruisecontrol.persisteddata.namespace; + +import com.linkedin.kafka.cruisecontrol.executor.Executor; +import com.linkedin.kafka.cruisecontrol.persisteddata.ValueMappedMap; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Objects; +import java.util.function.Function; +import java.util.stream.Collectors; +import javax.annotation.Nonnull; +import org.apache.commons.lang3.tuple.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Persisted data used by an {@link Executor} instance. + */ +public class ExecutorPersistedData { + + private static final Logger LOG = LoggerFactory.getLogger(ExecutorPersistedData.class); + + /** + * Keys to reference the particular broker map. For use with the "get" and "set" methods. + */ + public enum BrokerMapKey { + /** + * Key to reference the "demote broker" map. + */ + DEMOTE, + + /** + * Key to reference the "remove broker" map. + */ + REMOVE; + } + + // Holds the maps for the demoted brokers and the removed brokers. + private final Map> _demotedOrRemovedBrokers; + + // Used to make thread safe changes to _demotedOrRemovedBrokers. + private final Map _locks = Map.of( + BrokerMapKey.DEMOTE, new Object(), + BrokerMapKey.REMOVE, new Object() + ); + + /** + * Executor data that needs persisting is stored in the given map. It assumes that the map's + * data is persisted independently. This class is concerned with providing a clean view of that + * data and not with how it is stored. + * + * @param persistedMap The map to store {@link Executor} data in. + */ + public ExecutorPersistedData(Map persistedMap) { + this._demotedOrRemovedBrokers = Namespace.EXECUTOR.embed( + new ValueMappedMap<>( + persistedMap, + ExecutorPersistedData::serializeNumberMap, + ExecutorPersistedData::deserializeNumberMap)); + } + + /** + * Package private for testing. Used to serialize a full number map to a single string. + * + * @param map Map data to serialize. + * @return The string view of the map data. + */ + static String serializeNumberMap(Object map) { + if (!(map instanceof Map)) { + return null; + } + + return ((Map) map).entrySet().stream() + .map(e -> e.getKey() + ":" + e.getValue()) + .collect(Collectors.joining(",")); + } + + /** + * Package private for testing. Used to convert a map serialized by {@code serializeNumberMap} + * back to an equivalent map. + * + * @param str The serialized map data to convert back into a map. + * @return The deserialized string as a map or null if the data could not be deserialized. + */ + static Map deserializeNumberMap(String str) { + if (str == null) { + return null; + } + if (str.isBlank()) { + return new HashMap<>(); + } + return Arrays.stream(str.split(",")) + .map(e -> { + final String[] parts = e.split(":"); + if (parts.length != 2) { + return null; + } + try { + return Pair.of(Integer.parseInt(parts[0]), Long.parseLong(parts[1])); + } catch (NumberFormatException nfe) { + return null; + } + }) + .filter(Objects::nonNull) + .collect(Collectors.toMap(Entry::getKey, Entry::getValue)); + } + + /** + * Gets the Demoted Brokers map or the Removed Brokers map depending on the key. If the map + * hasn't been set yet, an empty map is returned. The returned map is a copy of the internal + * data so any changes to it are not stored. Any changes to the map must be stored using + * {@code setDemotedOrRemovedBrokers()}. + * + * @param key The key for which broker map to return. + * @return Non-null broker map associated with the key. + */ + public Map getDemotedOrRemovedBrokers(@Nonnull BrokerMapKey key) { + final Map value = Objects.requireNonNullElseGet( + this._demotedOrRemovedBrokers.get(key.toString()), HashMap::new); + LOG.debug("Getting key={}, value={}", key, value); + return value; + } + + /** + * Overwrite the stored broker map with the given map. + * + * @param key Key for the broker map to update. + * @param newValue The new map of broker keys and values to store. + */ + public void setDemotedOrRemovedBrokers(@Nonnull BrokerMapKey key, Map newValue) { + LOG.debug("Setting key={}, value={}", key.toString(), newValue); + this._demotedOrRemovedBrokers.put(key.toString(), newValue); + } + + /** + * Get the lock for the associated key so the broker maps can be updated with thread safety. + * + * @param key Key for the broker map to lock. + * @return The object used to lock the associated broker map. + */ + public Object getDemotedOrRemovedBrokersLock(BrokerMapKey key) { + return this._locks.get(key); + } + + /** + * Modifies the requested broker map using the provided function. This ensures the map is + * modified in a thread safe manner. + * + * @param key The key of the specific map to modify. + * @param mapModifier Used to modify the current values of the requested map. + * @param The return value resulting from applying {@code mapModifier} to the current map. + * @return The value returned by the {@code mapModifier} function. + */ + public R modifyDemotedOrRemovedBrokers(BrokerMapKey key, + @Nonnull Function, R> mapModifier) { + R result; + synchronized (this.getDemotedOrRemovedBrokersLock(key)) { + final Map currentValue = this.getDemotedOrRemovedBrokers(key); + result = mapModifier.apply(currentValue); + this.setDemotedOrRemovedBrokers(key, currentValue); + } + return result; + } + +} diff --git a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/namespace/Namespace.java b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/namespace/Namespace.java new file mode 100644 index 0000000000..0a3c095d66 --- /dev/null +++ b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/namespace/Namespace.java @@ -0,0 +1,53 @@ +/* + * Copyright 2024 LinkedIn Corp. Licensed under the BSD 2-Clause License (the "License"). See License in the project root for license information. + */ +package com.linkedin.kafka.cruisecontrol.persisteddata.namespace; + +import com.linkedin.kafka.cruisecontrol.persisteddata.KeyMappedMap; +import java.util.Locale; +import java.util.Map; +import javax.annotation.Nonnull; + +/** + * Allows multiple use cases by keeping them cleanly and consistently separated. New use cases can + * be added by adding their own enum constant. + */ +public enum Namespace { + // The defined namespaces. There needs to be a unique one for each use case. + EXECUTOR; + + // This is the cached string version of the name of the enum used as the map key prefix. + private final String _prefix; + + Namespace() { + this._prefix = this.name().toLowerCase(Locale.ROOT) + "."; + } + + /** + * Embed the namespace into the provided map. The returned map is a wrapper around the provided + * map. + * + * @param backingStore The map to embed the namespace into. + * @param The type of values stored in the map. + * @return A wrapped view of the provided map that keeps keys and values in their own namespace. + */ + public Map embed(Map backingStore) { + return new KeyMappedMap<>( + backingStore, + this::keyToNamespaceMapper, + this::namespaceToKeyMapper); + } + + // Package private for testing. + @Nonnull + String keyToNamespaceMapper(Object key) { + return this._prefix + key; + } + + // Package private for testing. + String namespaceToKeyMapper(@Nonnull String namespace) { + return namespace.startsWith(this._prefix) + ? namespace.substring(this._prefix.length()) + : null; + } +} diff --git a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/servlet/handler/sync/AdminRequest.java b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/servlet/handler/sync/AdminRequest.java index 377dce7e08..ce77b06309 100644 --- a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/servlet/handler/sync/AdminRequest.java +++ b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/servlet/handler/sync/AdminRequest.java @@ -191,42 +191,35 @@ protected String processDropRecentBrokersRequest() { return null; } - StringBuilder sb = new StringBuilder(); - Set brokersToDropFromRecentlyRemoved = dropRecentBrokersParameters.dropRecentlyRemovedBrokers(); - if (!brokersToDropFromRecentlyRemoved.isEmpty()) { - if (!_kafkaCruiseControl.dropRecentBrokers(brokersToDropFromRecentlyRemoved, true)) { - Set recentlyRemovedBrokers = _kafkaCruiseControl.recentBrokers(true); - sb.append(String.format("None of the brokers to drop (%s) are in the recently removed broker set" - + " (%s).%n", brokersToDropFromRecentlyRemoved, recentlyRemovedBrokers)); - LOG.warn("None of the user-requested brokers to drop ({}) are in the recently removed broker set ({}).", - brokersToDropFromRecentlyRemoved, recentlyRemovedBrokers); - } else { - Set recentlyRemovedBrokers = _kafkaCruiseControl.recentBrokers(true); - sb.append(String.format("Dropped recently removed brokers (requested: %s after-dropping: %s).%n", - brokersToDropFromRecentlyRemoved, recentlyRemovedBrokers)); - LOG.info("Recently removed brokers are dropped by user (requested: {} after-dropping: {}).", - brokersToDropFromRecentlyRemoved, recentlyRemovedBrokers); - } - } + StringBuilder resultDescription = new StringBuilder(); + dropBrokersFromRecentSet(dropRecentBrokersParameters.dropRecentlyRemovedBrokers(), "removed", + true, resultDescription); + dropBrokersFromRecentSet(dropRecentBrokersParameters.dropRecentlyDemotedBrokers(), "demoted", + false, resultDescription); - Set brokersToDropFromRecentlyDemoted = dropRecentBrokersParameters.dropRecentlyDemotedBrokers(); - if (!brokersToDropFromRecentlyDemoted.isEmpty()) { - if (!_kafkaCruiseControl.dropRecentBrokers(brokersToDropFromRecentlyDemoted, false)) { - Set recentlyDemotedBrokers = _kafkaCruiseControl.recentBrokers(false); - sb.append(String.format("None of the brokers to drop (%s) are in the recently demoted broker set" - + " (%s).%n", brokersToDropFromRecentlyDemoted, recentlyDemotedBrokers)); - LOG.warn("None of the user-requested brokers to drop ({}) are in the recently demoted broker set ({}).", - brokersToDropFromRecentlyDemoted, recentlyDemotedBrokers); + return resultDescription.toString(); + } + + void dropBrokersFromRecentSet(Set brokersToDrop, String brokerSetName, boolean isRemoved, + StringBuilder resultDescription) { + if (!brokersToDrop.isEmpty()) { + Set recentBrokers = _kafkaCruiseControl.recentBrokers(isRemoved); + if (!_kafkaCruiseControl.dropRecentBrokers(brokersToDrop, isRemoved)) { + resultDescription.append(String.format("None of the brokers to drop (%s) are in the " + + "recently %s broker set (%s).%n", brokerSetName, brokersToDrop, recentBrokers)); + LOG.warn("None of the user-requested brokers to drop ({}) are in the " + + "recently {} broker set ({}).", brokersToDrop, brokerSetName, recentBrokers); } else { - Set recentlyDemotedBrokers = _kafkaCruiseControl.recentBrokers(false); - sb.append(String.format("Dropped recently demoted brokers (requested: %s after-dropping: %s).%n", - brokersToDropFromRecentlyDemoted, recentlyDemotedBrokers)); - LOG.info("Recently demoted brokers are dropped by user (requested: {} after-dropping: {}).", - brokersToDropFromRecentlyDemoted, recentlyDemotedBrokers); + // The sets of recent brokers are updated eventually so this in-memory set math is necessary + // for the request to complete quickly. + recentBrokers.removeAll(brokersToDrop); + resultDescription.append(String.format("Dropped recently %s brokers " + + "(requested: %s after-dropping: %s).%n", + brokerSetName, brokersToDrop, recentBrokers)); + LOG.info("Recently {} brokers are dropped by user (requested: {} after-dropping: {}).", + brokerSetName, brokersToDrop, recentBrokers); } } - - return sb.toString(); } @Override diff --git a/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/KafkaCruiseControlTest.java b/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/KafkaCruiseControlTest.java index 60cc801c13..75a91f993e 100644 --- a/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/KafkaCruiseControlTest.java +++ b/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/KafkaCruiseControlTest.java @@ -13,6 +13,7 @@ import com.linkedin.kafka.cruisecontrol.executor.Executor; import com.linkedin.kafka.cruisecontrol.monitor.LoadMonitor; import com.linkedin.kafka.cruisecontrol.monitor.sampling.KafkaSampleStore; +import com.linkedin.kafka.cruisecontrol.persisteddata.PersistedMapFactory; import java.util.Collections; import java.util.Properties; import java.util.Set; @@ -54,6 +55,7 @@ public void testSanityCheckDryRun() throws InterruptedException, ExecutionExcept Time time = EasyMock.mock(Time.class); AnomalyDetectorManager anomalyDetectorManager = EasyMock.mock(AnomalyDetectorManager.class); Executor executor = EasyMock.strictMock(Executor.class); + PersistedMapFactory persistedMapFactory = EasyMock.mock(PersistedMapFactory.class); LoadMonitor loadMonitor = EasyMock.mock(LoadMonitor.class); ExecutorService goalOptimizerExecutor = EasyMock.mock(ExecutorService.class); GoalOptimizer goalOptimizer = EasyMock.mock(GoalOptimizer.class); @@ -73,7 +75,8 @@ public void testSanityCheckDryRun() throws InterruptedException, ExecutionExcept EasyMock.replay(time, anomalyDetectorManager, executor, loadMonitor, goalOptimizerExecutor, goalOptimizer); KafkaCruiseControl kafkaCruiseControl = new KafkaCruiseControl(_config, time, anomalyDetectorManager, executor, - loadMonitor, goalOptimizerExecutor, goalOptimizer, new NoopProvisioner()); + loadMonitor, goalOptimizerExecutor, goalOptimizer, + persistedMapFactory, new NoopProvisioner()); // Expect no failure (dryrun = true) regardless of ongoing executions. kafkaCruiseControl.sanityCheckDryRun(true, false); @@ -103,6 +106,7 @@ public void testDisableAutoStopExternalAgent() throws InterruptedException, Exec Time time = EasyMock.mock(Time.class); AnomalyDetectorManager anomalyDetectorManager = EasyMock.mock(AnomalyDetectorManager.class); Executor executor = EasyMock.strictMock(Executor.class); + PersistedMapFactory persistedMapFactory = EasyMock.mock(PersistedMapFactory.class); LoadMonitor loadMonitor = EasyMock.mock(LoadMonitor.class); ExecutorService goalOptimizerExecutor = EasyMock.mock(ExecutorService.class); GoalOptimizer goalOptimizer = EasyMock.mock(GoalOptimizer.class); @@ -120,7 +124,8 @@ public void testDisableAutoStopExternalAgent() throws InterruptedException, Exec EasyMock.replay(time, anomalyDetectorManager, executor, loadMonitor, goalOptimizerExecutor, goalOptimizer); KafkaCruiseControl kafkaCruiseControl = new KafkaCruiseControl(_config, time, anomalyDetectorManager, executor, - loadMonitor, goalOptimizerExecutor, goalOptimizer, new NoopProvisioner()); + loadMonitor, goalOptimizerExecutor, goalOptimizer, + persistedMapFactory, new NoopProvisioner()); // Expect failure (dryrun = false), if there is no execution started by CC, but ongoing replica reassignment, request to stop is irrelevant. assertThrows(IllegalStateException.class, () -> kafkaCruiseControl.sanityCheckDryRun(false, false)); diff --git a/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/config/KafkaCruiseControlConfigTest.java b/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/config/KafkaCruiseControlConfigTest.java new file mode 100644 index 0000000000..a4022c8ae4 --- /dev/null +++ b/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/config/KafkaCruiseControlConfigTest.java @@ -0,0 +1,81 @@ +/* + * Copyright 2024 LinkedIn Corp. Licensed under the BSD 2-Clause License (the "License"). See License in the project root for license information. + */ +package com.linkedin.kafka.cruisecontrol.config; + +import java.util.Collections; +import java.util.Map; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigException; +import org.junit.Test; + +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertThrows; + +public class KafkaCruiseControlConfigTest { + + private static final String KEY = "key"; + + /** + * getMap() should parse valid values. + */ + @Test + public void getMapParsesValidValues() { + assertGetMapResultMatchesExpected("", Collections.emptyMap()); + assertGetMapResultMatchesExpected(" ", Collections.emptyMap()); + assertGetMapResultMatchesExpected(" ;", Collections.emptyMap()); + assertGetMapResultMatchesExpected("k=", Map.of("k", "")); + assertGetMapResultMatchesExpected(" k = ", Map.of("k", "")); + assertGetMapResultMatchesExpected("k=v", Map.of("k", "v")); + assertGetMapResultMatchesExpected(" k = v ", Map.of("k", "v")); + assertGetMapResultMatchesExpected("k1=v1;k2=v2a,v2b", Map.of("k1", "v1", "k2", "v2a,v2b")); + assertGetMapResultMatchesExpected(" k1 = v1 ; k2 = v2 ", Map.of("k1", "v1", "k2", "v2")); + } + + /** + * getMap() should reject invalid values. + */ + @Test + public void getMapThrowsExceptionForInvalidValues() { + assertGetMapThrowsConfigException("k"); + assertGetMapThrowsConfigException(" k "); + assertGetMapThrowsConfigException("="); + assertGetMapThrowsConfigException(" = "); + assertGetMapThrowsConfigException("=v"); + assertGetMapThrowsConfigException(" = v "); + } + + private void assertGetMapResultMatchesExpected(String rawValue, + Map expectedValue) { + final KafkaCruiseControlConfig config = new KafkaCruiseControlConfig( + PermissiveConfigDef.of(KEY), Map.of(KEY, rawValue)); + assertThat(config.getMap(KEY), is(expectedValue)); + } + + private void assertGetMapThrowsConfigException(String rawValue) { + final KafkaCruiseControlConfig config = new KafkaCruiseControlConfig( + PermissiveConfigDef.of(KEY), Map.of(KEY, rawValue)); + assertThrows(ConfigException.class, () -> config.getMap(KEY)); + } + + private static class PermissiveConfigDef extends ConfigDef { + + /** + * Create an instance with a single key of type STRING. + * @param key The config key name. + * @return The new instance, ready for use with a KafkaCruiseControlConfig instance. + */ + public static PermissiveConfigDef of(String key) { + final PermissiveConfigDef result = new PermissiveConfigDef(); + result.define(key, Type.STRING, Importance.LOW, ""); + return result; + } + + @SuppressWarnings("unchecked") + @Override + public Map parse(Map props) { + return (Map) props; + } + } +} diff --git a/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/executor/ExecutorTest.java b/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/executor/ExecutorTest.java index 5f5f3e3a40..52c3f1061f 100644 --- a/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/executor/ExecutorTest.java +++ b/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/executor/ExecutorTest.java @@ -22,6 +22,8 @@ import com.linkedin.kafka.cruisecontrol.monitor.LoadMonitor; import com.linkedin.kafka.cruisecontrol.monitor.sampling.NoopSampler; import com.linkedin.kafka.cruisecontrol.monitor.task.LoadMonitorTaskRunner; +import com.linkedin.kafka.cruisecontrol.persisteddata.namespace.ExecutorPersistedData; +import com.linkedin.kafka.cruisecontrol.persisteddata.namespace.ExecutorPersistedData.BrokerMapKey; import com.linkedin.kafka.cruisecontrol.servlet.UserTaskManager; import java.util.ArrayList; import java.util.Arrays; @@ -81,7 +83,8 @@ public class ExecutorTest extends CCKafkaClientsIntegrationTestHarness { private static final int BROKER_ID_0 = 0; private static final int BROKER_ID_1 = 1; private static final Random RANDOM = new Random(0xDEADBEEF); - private static final int MOCK_BROKER_ID_TO_DROP = 1; + private static final int MOCK_BROKER_ID1 = 1; + private static final int MOCK_BROKER_ID2 = 2; private static final long MOCK_CURRENT_TIME = 1596842708000L; private final ZKClientConfig _zkClientConfig = ZKConfigUtils.zkClientConfigFromKafkaConfig( new KafkaCruiseControlConfig(KafkaCruiseControlUnitTestUtils.getKafkaCruiseControlProperties())); @@ -371,7 +374,7 @@ private static boolean verifyFutureError(Future future, Class())); long minExecutionProgressCheckIntervalMs = config.getLong(ExecutorConfig.MIN_EXECUTION_PROGRESS_CHECK_INTERVAL_MS_CONFIG); // RequestedExecutionProgressCheckIntervalMs has to be larger than minExecutionProgressCheckIntervalMs @@ -385,7 +388,7 @@ public void testSetRequestedExecutionProgressCheckIntervalMs() { public void testSetExecutionProgressCheckIntervalMsWithRequestedValue() { KafkaCruiseControlConfig config = new KafkaCruiseControlConfig(getExecutorProperties()); Executor executor = new Executor(config, null, new MetricRegistry(), EasyMock.mock(MetadataClient.class), - null, EasyMock.mock(AnomalyDetectorManager.class)); + null, EasyMock.mock(AnomalyDetectorManager.class), new ExecutorPersistedData(new HashMap<>())); long defaultExecutionProgressCheckIntervalMs = config.getLong(ExecutorConfig.EXECUTION_PROGRESS_CHECK_INTERVAL_MS_CONFIG); long minExecutionProgressCheckIntervalMs = config.getLong(ExecutorConfig.MIN_EXECUTION_PROGRESS_CHECK_INTERVAL_MS_CONFIG); @@ -406,7 +409,7 @@ public void testSetExecutionProgressCheckIntervalMsWithRequestedValue() { public void testSetExecutionProgressCheckIntervalMsWithNoRequestedValue() { KafkaCruiseControlConfig config = new KafkaCruiseControlConfig(getExecutorProperties()); Executor executor = new Executor(config, null, new MetricRegistry(), EasyMock.mock(MetadataClient.class), - null, EasyMock.mock(AnomalyDetectorManager.class)); + null, EasyMock.mock(AnomalyDetectorManager.class), new ExecutorPersistedData(new HashMap<>())); long defaultExecutionProgressCheckIntervalMs = config.getLong(ExecutorConfig.EXECUTION_PROGRESS_CHECK_INTERVAL_MS_CONFIG); long minExecutionProgressCheckIntervalMs = config.getLong(ExecutorConfig.MIN_EXECUTION_PROGRESS_CHECK_INTERVAL_MS_CONFIG); @@ -427,7 +430,7 @@ public void testSetExecutionProgressCheckIntervalMsWithNoRequestedValue() { public void testResetExecutionProgressCheckIntervalMs() { KafkaCruiseControlConfig config = new KafkaCruiseControlConfig(getExecutorProperties()); Executor executor = new Executor(config, null, new MetricRegistry(), EasyMock.mock(MetadataClient.class), - null, EasyMock.mock(AnomalyDetectorManager.class)); + null, EasyMock.mock(AnomalyDetectorManager.class), new ExecutorPersistedData(new HashMap<>())); long defaultExecutionProgressCheckIntervalMs = config.getLong(ExecutorConfig.EXECUTION_PROGRESS_CHECK_INTERVAL_MS_CONFIG); executor.resetExecutionProgressCheckIntervalMs(); assertEquals(defaultExecutionProgressCheckIntervalMs, executor.executionProgressCheckIntervalMs()); @@ -442,17 +445,35 @@ public void testResetExecutionProgressCheckIntervalMs() { public void testExecutionKnobs() { KafkaCruiseControlConfig config = new KafkaCruiseControlConfig(getExecutorProperties()); assertThrows(IllegalStateException.class, - () -> new Executor(config, null, new MetricRegistry(), EasyMock.mock(MetadataClient.class), null, null)); - Executor executor = new Executor(config, null, new MetricRegistry(), EasyMock.mock(MetadataClient.class), - null, EasyMock.mock(AnomalyDetectorManager.class)); + () -> new Executor(config, Time.SYSTEM, new MetricRegistry(), EasyMock.mock(MetadataClient.class), null, null, + new ExecutorPersistedData(new HashMap<>()))); + Executor executor = new Executor(config, Time.SYSTEM, new MetricRegistry(), EasyMock.mock(MetadataClient.class), + null, EasyMock.mock(AnomalyDetectorManager.class), + new ExecutorPersistedData(new HashMap<>())); // Verify correctness of add/drop recently removed/demoted brokers. assertFalse(executor.dropRecentlyRemovedBrokers(Collections.emptySet())); assertFalse(executor.dropRecentlyDemotedBrokers(Collections.emptySet())); - executor.addRecentlyRemovedBrokers(Collections.singleton(MOCK_BROKER_ID_TO_DROP)); - executor.addRecentlyDemotedBrokers(Collections.singleton(MOCK_BROKER_ID_TO_DROP)); - assertTrue(executor.dropRecentlyRemovedBrokers(Collections.singleton(MOCK_BROKER_ID_TO_DROP))); - assertTrue(executor.dropRecentlyDemotedBrokers(Collections.singleton(MOCK_BROKER_ID_TO_DROP))); + executor.addRecentlyRemovedBrokersPermanently(Collections.singleton(MOCK_BROKER_ID1)); + executor.addRecentlyDemotedBrokersPermanently(Collections.singleton(MOCK_BROKER_ID1)); + assertTrue(executor.dropRecentlyRemovedBrokers(Collections.singleton(MOCK_BROKER_ID1))); + assertTrue(executor.dropRecentlyDemotedBrokers(Collections.singleton(MOCK_BROKER_ID1))); + } + + @Test + public void testExpiredBrokerRemoval() { + KafkaCruiseControlConfig config = new KafkaCruiseControlConfig(getExecutorProperties()); + final Time time = Time.SYSTEM; + final long startTime = time.milliseconds(); + Executor executor = new Executor(config, time, new MetricRegistry(), null, null, + EasyMock.mock(AnomalyDetectorManager.class), + new ExecutorPersistedData(new HashMap<>())); + + executor.addRecentBrokers(BrokerMapKey.DEMOTE, Collections.singleton(MOCK_BROKER_ID1), + startTime); + executor.addRecentlyDemotedBrokersPermanently(Collections.singleton(MOCK_BROKER_ID2)); + executor.removeExpiredBrokerHistory(BrokerMapKey.DEMOTE, -1); + assertEquals(executor.recentlyDemotedBrokers(), Collections.singleton(MOCK_BROKER_ID2)); } @Test @@ -489,7 +510,7 @@ public void testTimeoutAndExecutionStop() throws InterruptedException, OngoingEx Collection proposalsToExecute = Collections.singletonList(proposal); Executor executor = new Executor(configs, time, new MetricRegistry(), mockMetadataClient, null, - mockAnomalyDetectorManager); + mockAnomalyDetectorManager, new ExecutorPersistedData(new HashMap<>())); executor.setUserTaskManager(mockUserTaskManager); executor.setGeneratingProposalsForExecution(RANDOM_UUID, ExecutorTest.class::getSimpleName, true); @@ -781,7 +802,7 @@ private void executeAndVerifyProposals(KafkaZkClient kafkaZkClient, } MetricRegistry metricRegistry = new MetricRegistry(); Executor executor = new Executor(configs, new SystemTime(), metricRegistry, null, mockExecutorNotifier, - mockAnomalyDetectorManager); + mockAnomalyDetectorManager, new ExecutorPersistedData(new HashMap<>())); executor.setUserTaskManager(mockUserTaskManager); Map replicationFactors = new HashMap<>(); for (ExecutionProposal proposal : proposalsToCheck) { diff --git a/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/BackingMethodTest.java b/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/BackingMethodTest.java new file mode 100644 index 0000000000..987c9c3966 --- /dev/null +++ b/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/BackingMethodTest.java @@ -0,0 +1,40 @@ +/* + * Copyright 2024 LinkedIn Corp. Licensed under the BSD 2-Clause License (the "License"). See License in the project root for license information. + */ +package com.linkedin.kafka.cruisecontrol.persisteddata; + +import java.util.List; +import org.junit.Test; + +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; + +public class BackingMethodTest { + + /** + * Ensure fromString accepts valid values ignoring case. + */ + @Test + public void fromStringAcceptsValidValuesIgnoringCase() { + assertThat(BackingMethod.fromString("kafka"), is(BackingMethod.KAFKA)); + assertThat(BackingMethod.fromString("Kafka"), is(BackingMethod.KAFKA)); + } + + /** + * Ensure fromString rejects invalid values. + */ + @Test(expected = IllegalArgumentException.class) + public void fromStringRejectsInvalidValues() { + BackingMethod.fromString("invalid"); + } + + /** + * Ensure stringValues gets all possible backing method options. + */ + @Test + public void stringValuesGetsAllEnumsLowercaseAndSorted() { + List expected = List.of("kafka", "memory"); + List actual = BackingMethod.stringValues(); + assertThat(actual, is(expected)); + } +} diff --git a/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/KeyMappedMapTest.java b/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/KeyMappedMapTest.java new file mode 100644 index 0000000000..7c14aef496 --- /dev/null +++ b/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/KeyMappedMapTest.java @@ -0,0 +1,198 @@ +/* + * Copyright 2024 LinkedIn Corp. Licensed under the BSD 2-Clause License (the "License"). See License in the project root for license information. + */ +package com.linkedin.kafka.cruisecontrol.persisteddata; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import org.apache.commons.lang3.tuple.Pair; +import org.junit.Before; +import org.junit.Test; + +import static org.hamcrest.CoreMatchers.allOf; +import static org.hamcrest.CoreMatchers.hasItems; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.not; +import static org.hamcrest.CoreMatchers.nullValue; +import static org.hamcrest.MatcherAssert.assertThat; + +public class KeyMappedMapTest { + + private static final Integer EXTERNAL_KEY1 = 1; + private static final Integer EXTERNAL_KEY2 = 3; + private static final Integer INTERNAL_KEY1 = 2; + private static final Integer INTERNAL_KEY2 = 6; + private static final Integer VALUE1 = 11; + private static final Integer VALUE2 = 22; + private static final Map EXTERNAL_KEYS_MAP = + Map.of(EXTERNAL_KEY1, VALUE1, EXTERNAL_KEY2, VALUE2); + private static final Map INTERNAL_KEYS_MAP = + Map.of(INTERNAL_KEY1, VALUE1, INTERNAL_KEY2, VALUE2); + + private Map _backingMap; + private Map _mappedValues; + + /** + * Set a common starting point. + */ + @Before + public void beforeTest() { + this._backingMap = new HashMap<>(); + this._mappedValues = new KeyMappedMap<>(this._backingMap, + KeyMappedMapTest::doubleInteger, KeyMappedMapTest::halveInteger); + } + + // The key mapper function. + private static Integer doubleInteger(Object o) { + if (!(o instanceof Integer)) { + throw new IllegalArgumentException( + String.format("Key must be an Integer but was: %s", + o.getClass().getSimpleName())); + } + return (Integer) o * 2; + } + + // The key back mapper function. + private static Integer halveInteger(Integer i) { + return i % 2 == 0 ? i / 2 : null; + } + + /** + * Ensure get() maps external key values to internal ones and passes calls to the child map. + */ + @Test + public void getAppliesMapperToKey() { + assertThat(this._mappedValues.get(EXTERNAL_KEY1), is(nullValue())); + this._backingMap.put(INTERNAL_KEY1, VALUE1); + assertThat(this._mappedValues.get(EXTERNAL_KEY1), is(VALUE1)); + } + + /** + * Ensure put() maps external key values to internal ones and passes calls to the child map. + */ + @Test + public void putAppliesMapperToKey() { + assertThat(this._mappedValues.put(EXTERNAL_KEY1, VALUE1), is(nullValue())); + assertThat(this._backingMap.get(INTERNAL_KEY1), is(VALUE1)); + assertThat(this._mappedValues.put(EXTERNAL_KEY1, VALUE1), is(VALUE1)); + } + + /** + * Ensure containsKey() maps external key values to internal ones and passes calls to the child map. + */ + @Test + public void containsKeyAppliesMapperToKey() { + assertThat(this._mappedValues.containsKey(EXTERNAL_KEY1), is(false)); + this._backingMap.put(INTERNAL_KEY1, VALUE1); + assertThat(this._mappedValues.containsKey(EXTERNAL_KEY1), is(true)); + } + + /** + * Ensure remove() maps external key values to internal ones and passes calls to the child map. + */ + @Test + public void removeAppliesMapperToKey() { + assertThat(this._mappedValues.remove(EXTERNAL_KEY1), is(nullValue())); + this._backingMap.put(INTERNAL_KEY1, VALUE1); + assertThat(this._mappedValues.remove(EXTERNAL_KEY1), is(VALUE1)); + } + + /** + * Ensure size() passes calls to the child map. + */ + @Test + public void sizePassesCallToChildMap() { + assertThat(this._mappedValues.size(), is(0)); + this._backingMap.put(INTERNAL_KEY1, VALUE1); + assertThat(this._mappedValues.size(), is(1)); + } + + /** + * Ensure size() only counts entries where the key works with the mapper and ignores the others. + */ + @Test + public void sizeOnlyIncludesMappableKeys() { + this._backingMap.put(INTERNAL_KEY1, VALUE1); + this._backingMap.put(EXTERNAL_KEY1, VALUE1); + assertThat(this._mappedValues.size(), is(1)); + } + + /** + * Ensure isEmpty() passes calls to the child map. + */ + @Test + public void isEmptyPassesCallToChildMap() { + assertThat(this._mappedValues.isEmpty(), is(true)); + this._backingMap.put(INTERNAL_KEY1, VALUE1); + assertThat(this._mappedValues.isEmpty(), is(false)); + } + + /** + * Ensure containsValue() finds values belonging to the used keyMapper and ignores other mapped + * keys. + */ + @Test + public void containsValueFindsKeyMappedKeyValues() { + assertThat(this._mappedValues.containsValue(VALUE1), is(false)); + this._backingMap.put(INTERNAL_KEY1, VALUE1); + this._backingMap.put(EXTERNAL_KEY2, VALUE2); + assertThat(this._mappedValues.containsValue(VALUE1), is(true)); + assertThat(this._mappedValues.containsValue(VALUE2), is(false)); + } + + /** + * Ensure putAll() maps external key values to internal ones and passes them to the child map. + */ + @Test + public void putAllMapsAllKeys() { + this._mappedValues.putAll(EXTERNAL_KEYS_MAP); + assertThat(this._backingMap, is(INTERNAL_KEYS_MAP)); + } + + /** + * Ensure clear() removes all entries with keyMapped keys. + */ + @Test + public void clearRemovesKeyMappedKeys() { + this._backingMap.put(INTERNAL_KEY1, VALUE1); + this._backingMap.put(EXTERNAL_KEY2, VALUE2); + this._mappedValues.clear(); + assertThat(this._backingMap.containsKey(INTERNAL_KEY1), is(false)); + assertThat(this._backingMap.containsKey(EXTERNAL_KEY2), is(true)); + } + + /** + * Ensure keySet() maps external keys to internal ones and only including those that are + * keyMappable. + */ + @Test + public void keySetMapsAllKeys() { + assertThat(this._mappedValues.keySet().isEmpty(), is(true)); + this._backingMap.put(INTERNAL_KEY1, VALUE1); + this._backingMap.put(EXTERNAL_KEY2, VALUE2); + assertThat(this._mappedValues.keySet(), is(Set.of(EXTERNAL_KEY1))); + } + + /** + * Ensure values() returns only values of key mapped entries. + */ + @Test + public void valuesReturnsValuesOfKeyMappedEntries() { + assertThat(this._mappedValues.values().isEmpty(), is(true)); + this._backingMap.put(INTERNAL_KEY1, VALUE1); + this._backingMap.put(EXTERNAL_KEY2, VALUE2); + assertThat(this._mappedValues.values(), allOf(hasItems(VALUE1), not(hasItems(VALUE2)))); + } + + /** + * Ensure entrySet() includes all keyMapped entries. + */ + @Test + public void entrySetMapsAllKeys() { + assertThat(this._mappedValues.entrySet().isEmpty(), is(true)); + this._backingMap.put(INTERNAL_KEY1, VALUE1); + this._backingMap.put(EXTERNAL_KEY2, VALUE2); + assertThat(this._mappedValues.entrySet(), is(Set.of(Pair.of(EXTERNAL_KEY1, VALUE1)))); + } +} diff --git a/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/MapDecoratorTest.java b/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/MapDecoratorTest.java new file mode 100644 index 0000000000..a4c0c92f02 --- /dev/null +++ b/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/MapDecoratorTest.java @@ -0,0 +1,182 @@ +/* + * Copyright 2024 LinkedIn Corp. Licensed under the BSD 2-Clause License (the "License"). See License in the project root for license information. + */ +package com.linkedin.kafka.cruisecontrol.persisteddata; + +import java.util.Collections; +import java.util.Map; +import java.util.Set; +import org.apache.commons.lang3.tuple.Pair; +import org.easymock.EasyMockRunner; +import org.easymock.Mock; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; + +import static org.easymock.EasyMock.eq; +import static org.easymock.EasyMock.expect; +import static org.easymock.EasyMock.expectLastCall; +import static org.easymock.EasyMock.replay; +import static org.easymock.EasyMock.verify; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.nullValue; +import static org.hamcrest.MatcherAssert.assertThat; + +@RunWith(EasyMockRunner.class) +public class MapDecoratorTest { + + private static final Integer KEY = 1; + private static final String VALUE = "value"; + + @Mock + private Map _mockChild; + + private Map _decorator; + + /** + * Common setup for each test. + */ + @Before + public void beforeTest() { + this._decorator = new MapDecorator<>(this._mockChild) { + }; + } + + /** + * Passes the call to get() on the decorated child map. + */ + @Test + public void getDecoratesChildMap() { + expect(this._mockChild.get(eq(KEY))).andReturn(VALUE); + replay(this._mockChild); + assertThat(this._decorator.get(KEY), is(VALUE)); + verify(this._mockChild); + } + + /** + * Passes the call to put() on the decorated child map. + */ + @Test + public void putDecoratesChildMap() { + expect(this._mockChild.put(eq(KEY), eq(VALUE))).andReturn(null); + replay(this._mockChild); + assertThat(this._decorator.put(KEY, VALUE), is(nullValue())); + verify(this._mockChild); + } + + /** + * Passes the call to size() on the decorated child map. + */ + @Test + public void sizeDecoratesChildMap() { + expect(this._mockChild.size()).andReturn(1); + replay(this._mockChild); + assertThat(this._decorator.size(), is(1)); + verify(this._mockChild); + } + + /** + * Passes the call to isEmpty() on the decorated child map. + */ + @Test + public void isEmptyDecoratesChildMap() { + expect(this._mockChild.isEmpty()).andReturn(true); + replay(this._mockChild); + assertThat(this._decorator.isEmpty(), is(true)); + verify(this._mockChild); + } + + /** + * Passes the call to containsKey() on the decorated child map. + */ + @Test + public void containsKeyDecoratesChildMap() { + expect(this._mockChild.containsKey(eq(KEY))).andReturn(true); + replay(this._mockChild); + assertThat(this._decorator.containsKey(KEY), is(true)); + verify(this._mockChild); + } + + /** + * Passes the call to containsValue() on the decorated child map. + */ + @Test + public void containsValueDecoratesChildMap() { + expect(this._mockChild.containsValue(eq(VALUE))).andReturn(true); + replay(this._mockChild); + assertThat(this._decorator.containsValue(VALUE), is(true)); + verify(this._mockChild); + } + + /** + * Passes the call to remove() on the decorated child map. + */ + @Test + public void removeDecoratesChildMap() { + expect(this._mockChild.remove(eq(KEY))).andReturn(VALUE); + replay(this._mockChild); + assertThat(this._decorator.remove(KEY), is(VALUE)); + verify(this._mockChild); + } + + /** + * Passes the call to putAll() on the decorated child map. + */ + @Test + public void putAllDecoratesChildMap() { + final Map data = Map.of(KEY, VALUE); + this._mockChild.putAll(eq(data)); + expectLastCall(); + replay(this._mockChild); + this._decorator.putAll(data); + verify(this._mockChild); + } + + /** + * Passes the call to clear() on the decorated child map. + */ + @Test + public void clearDecoratesChildMap() { + this._mockChild.clear(); + expectLastCall(); + replay(this._mockChild); + this._decorator.clear(); + verify(this._mockChild); + } + + /** + * Passes the call to keySet() on the decorated child map. + */ + @Test + public void keySetDecoratesChildMap() { + Set keys = Collections.singleton(KEY); + expect(this._mockChild.keySet()).andReturn(keys); + replay(this._mockChild); + assertThat(this._decorator.keySet(), is(keys)); + verify(this._mockChild); + } + + /** + * Passes the call to values() on the decorated child map. + */ + @Test + public void valuesDecoratesChildMap() { + Set values = Collections.singleton(VALUE); + expect(this._mockChild.values()).andReturn(values); + replay(this._mockChild); + assertThat(this._decorator.values(), is(values)); + verify(this._mockChild); + } + + /** + * Passes the call to entrySet() on the decorated child map. + */ + @Test + public void entrySetDecoratesChildMap() { + Set> entries = Collections.singleton(Pair.of(KEY, VALUE)); + expect(this._mockChild.entrySet()).andReturn(entries); + replay(this._mockChild); + assertThat(this._decorator.entrySet(), is(entries)); + verify(this._mockChild); + } +} diff --git a/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistedMapFactoryTest.java b/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistedMapFactoryTest.java new file mode 100644 index 0000000000..6bf551ac2e --- /dev/null +++ b/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistedMapFactoryTest.java @@ -0,0 +1,56 @@ +/* + * Copyright 2024 LinkedIn Corp. Licensed under the BSD 2-Clause License (the "License"). See License in the project root for license information. + */ +package com.linkedin.kafka.cruisecontrol.persisteddata; + +import com.linkedin.kafka.cruisecontrol.config.KafkaCruiseControlConfig; +import com.linkedin.kafka.cruisecontrol.config.constants.ExecutorConfig; +import com.linkedin.kafka.cruisecontrol.config.constants.PersistedDataConfig; +import com.linkedin.kafka.cruisecontrol.persisteddata.kafka.KafkaPersistedMap; +import java.util.HashMap; +import java.util.Map; +import org.apache.kafka.clients.CommonClientConfigs; +import org.junit.Test; + +import static org.easymock.EasyMock.mock; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.core.Is.is; +import static org.hamcrest.core.Is.isA; + + +public class PersistedMapFactoryTest { + + /** + * Ensure setting the backing method config results in the factory producing an instance of the + * right type. + */ + @Test + public void instanceReturnsKafkaPersistedMapWhenConfiguredForKafka() { + final PersistedMapFactory factory = configureAndGetPersistedMapFactory( + "kafka"); + PersistedMap map = factory.instance(); + assertThat(map instanceof KafkaPersistedMap, is(true)); + } + + /** + * Ensure setting the backing method config results in the factory producing an instance of the + * right type. + */ + @Test + public void instanceReturnsPersistedMapWhenConfiguredForMemory() { + final PersistedMapFactory factory = configureAndGetPersistedMapFactory( + "memory"); + PersistedMap map = factory.instance(); + assertThat(map, isA(PersistedMap.class)); + } + + private static PersistedMapFactory configureAndGetPersistedMapFactory(String backingMethod) { + KafkaCruiseControlConfig config = new KafkaCruiseControlConfig( + Map.of(PersistedDataConfig.BACKING_METHOD_CONFIG, backingMethod, + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "fake", + ExecutorConfig.ZOOKEEPER_CONNECT_CONFIG, "connect:1234")); + return new PersistedMapFactory(config, + () -> mock(KafkaPersistedMap.class), + () -> new PersistedMap(new HashMap<>())); + } +} diff --git a/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistedMapTest.java b/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistedMapTest.java new file mode 100644 index 0000000000..13364f4ec3 --- /dev/null +++ b/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistedMapTest.java @@ -0,0 +1,69 @@ +/* + * Copyright 2024 LinkedIn Corp. Licensed under the BSD 2-Clause License (the "License"). See License in the project root for license information. + */ +package com.linkedin.kafka.cruisecontrol.persisteddata; + +import java.util.HashMap; +import java.util.Map; +import org.junit.Before; +import org.junit.Test; + +import static org.hamcrest.CoreMatchers.nullValue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.core.Is.is; + +public class PersistedMapTest { + + private static final String KEY_1 = "key1"; + private static final String VALUE_1 = "value1"; + private static final String KEY_2 = "key2"; + private static final String VALUE_2 = "value2"; + + private Map _backingMap; + private PersistedMap _persistedMap; + + /** + * Setup for the tests. + */ + @Before + public void setUp() { + _backingMap = new HashMap<>(); + _persistedMap = new PersistedMap(this._backingMap); + } + + /** + * The mapping is removed by putting a null value for the key. + */ + @Test + public void removeMeansTheValueIsNullForTheKey() { + _backingMap.put(KEY_1, VALUE_1); + _persistedMap.remove(KEY_1); + assertThat(_backingMap.get(KEY_1), is(nullValue())); + } + + /** + * All the keys and values are produced in order. + */ + @Test + public void putAllPutsEachKeyAndValue() { + final Map data = Map.of( + KEY_1, VALUE_1, + KEY_2, VALUE_2); + _persistedMap.putAll(data); + assertThat(_backingMap, is(data)); + } + + /** + * Clear should remove all the values. + */ + @Test + public void clearRemovesAllTheValues() { + final Map data = Map.of( + KEY_1, VALUE_1, + KEY_2, VALUE_2); + _backingMap.putAll(data); + _persistedMap.clear(); + assertThat(_backingMap.get(KEY_1), is(nullValue())); + assertThat(_backingMap.get(KEY_2), is(nullValue())); + } +} diff --git a/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/ValueMappedMapTest.java b/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/ValueMappedMapTest.java new file mode 100644 index 0000000000..dd11cd2cd7 --- /dev/null +++ b/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/ValueMappedMapTest.java @@ -0,0 +1,197 @@ +/* + * Copyright 2024 LinkedIn Corp. Licensed under the BSD 2-Clause License (the "License"). See License in the project root for license information. + */ +package com.linkedin.kafka.cruisecontrol.persisteddata; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.commons.lang3.tuple.Pair; +import org.junit.Before; +import org.junit.Test; + +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.nullValue; +import static org.hamcrest.MatcherAssert.assertThat; + +public class ValueMappedMapTest { + + private static final Integer KEY1 = 1; + private static final Integer KEY2 = 2; + private static final Integer EXTERNAL_VALUE1 = 3; + private static final Integer EXTERNAL_VALUE2 = 6; + private static final Map EXTERNAL_VALUES_MAP = + Map.of(KEY1, EXTERNAL_VALUE1, KEY2, EXTERNAL_VALUE2); + private static final Integer INTERNAL_VALUE1 = 4; + private static final Integer INTERNAL_VALUE2 = 7; + private static final Map INTERNAL_VALUES_MAP = + Map.of(KEY1, INTERNAL_VALUE1, KEY2, INTERNAL_VALUE2); + + private Map _backingMap; + private Map _mappedValues; + + /** + * Set a common starting point. + */ + @Before + public void beforeTest() { + this._backingMap = new HashMap<>(); + this._mappedValues = new ValueMappedMap<>(this._backingMap, + ValueMappedMapTest::incrementInteger, ValueMappedMapTest::decrementInteger); + } + + // The value mapper function. + private static Integer incrementInteger(Object o) { + if (!(o instanceof Integer)) { + throw new IllegalArgumentException( + String.format("Value must be an Integer but was: %s", + o.getClass().getSimpleName())); + } + return (Integer) o + 1; + } + + // The value back mapper function. + private static Integer decrementInteger(Integer i) { + return i - 1; + } + + /** + * Ensure get() maps external values to internal ones and passes calls onto the child map. + */ + @Test + public void getAppliesBackMapperToValue() { + this._backingMap.put(KEY1, INTERNAL_VALUE1); + assertThat(this._mappedValues.get(KEY1), is(EXTERNAL_VALUE1)); + } + + /** + * Ensure get() doesn't choke on a null value. + */ + @Test + public void getHandlesNullValue() { + assertThat(this._mappedValues.get(KEY1), is(nullValue())); + } + + /** + * Ensure put() maps external values to internal ones and passes calls onto the child map. + */ + @Test + public void putStoresInternalValue() { + assertThat(this._mappedValues.put(KEY1, EXTERNAL_VALUE1), is(nullValue())); + assertThat(this._backingMap.get(KEY1), is(INTERNAL_VALUE1)); + } + + /** + * Ensure put() maps external values to internal ones and passes calls onto the child map. + */ + @Test + public void putBackMapsPreviousValue() { + this._backingMap.put(KEY1, INTERNAL_VALUE1); + assertThat(this._mappedValues.put(KEY1, EXTERNAL_VALUE1), is(EXTERNAL_VALUE1)); + } + + /** + * Ensure containsKey() passes calls onto the child map. + */ + @Test + public void containsKeyPassesCallToChildMap() { + assertThat(this._mappedValues.containsKey(KEY1), is(false)); + this._backingMap.put(KEY1, INTERNAL_VALUE1); + assertThat(this._mappedValues.containsKey(KEY1), is(true)); + } + + /** + * Ensure remove() maps external values to internal ones and passes calls onto the child map. + */ + @Test + public void removeDeletesKeyFromChildMapAndBackMapsOldValue() { + assertThat(this._mappedValues.remove(KEY1), is(nullValue())); + this._backingMap.put(KEY1, INTERNAL_VALUE1); + assertThat(this._mappedValues.remove(KEY1), is(EXTERNAL_VALUE1)); + assertThat(this._backingMap.containsKey(KEY1), is(false)); + } + + /** + * Ensure size() passes calls onto the child map. + */ + @Test + public void sizePassesCallToChildMap() { + assertThat(this._mappedValues.size(), is(0)); + this._backingMap.put(KEY1, INTERNAL_VALUE1); + assertThat(this._mappedValues.size(), is(1)); + } + + /** + * Ensure isEmpty() passes calls onto the child map. + */ + @Test + public void isEmptyPassesCallToChildMap() { + assertThat(this._mappedValues.isEmpty(), is(true)); + this._backingMap.put(KEY1, INTERNAL_VALUE1); + assertThat(this._mappedValues.isEmpty(), is(false)); + } + + /** + * Ensure containsValue() maps external values to internal ones and passes calls onto the child map. + */ + @Test + public void containsValueMapsQueriedValue() { + assertThat(this._mappedValues.containsValue(EXTERNAL_VALUE1), is(false)); + this._backingMap.put(KEY1, INTERNAL_VALUE1); + assertThat(this._mappedValues.containsValue(EXTERNAL_VALUE1), is(true)); + } + + /** + * Ensure putAll() maps external values to internal ones and passes calls onto the child map. + */ + @Test + public void putAllMapsAllValues() { + this._mappedValues.putAll(EXTERNAL_VALUES_MAP); + assertThat(this._backingMap, is(INTERNAL_VALUES_MAP)); + } + + /** + * Ensure clear() passes calls onto the child map. + */ + @Test + public void clearPassesCallToChildMap() { + this._backingMap.put(KEY1, INTERNAL_VALUE1); + this._mappedValues.clear(); + assertThat(this._backingMap.isEmpty(), is(true)); + } + + /** + * Ensure keySet() passes calls onto the child map. + */ + @Test + public void keySetPassesCallToChildMap() { + assertThat(this._mappedValues.keySet(), is(Collections.emptySet())); + this._backingMap.put(KEY1, INTERNAL_VALUE1); + assertThat(this._mappedValues.keySet(), is(Collections.singleton(KEY1))); + } + + /** + * Ensure values() maps external values to internal ones and passes calls onto the child map. + */ + @Test + public void valuesBackMapsAllValues() { + assertThat(this._mappedValues.values().isEmpty(), is(true)); + this._backingMap.putAll( + INTERNAL_VALUES_MAP); + assertThat(this._mappedValues.values(), is( + List.of(EXTERNAL_VALUE1, EXTERNAL_VALUE2))); + } + + /** + * Ensure entrySet() maps external values to internal ones and passes calls onto the child map. + */ + @Test + public void entrySetBackMapsAllValues() { + assertThat(this._mappedValues.entrySet().isEmpty(), is(true)); + this._backingMap.putAll(INTERNAL_VALUES_MAP); + assertThat(this._mappedValues.entrySet(), is( + Set.of(Pair.of(KEY1, EXTERNAL_VALUE1), Pair.of(KEY2, EXTERNAL_VALUE2)))); + } +} diff --git a/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/kafka/KafkaPersistedMapTest.java b/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/kafka/KafkaPersistedMapTest.java new file mode 100644 index 0000000000..c6424f3bae --- /dev/null +++ b/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/kafka/KafkaPersistedMapTest.java @@ -0,0 +1,294 @@ +/* + * Copyright 2024 LinkedIn Corp. Licensed under the BSD 2-Clause License (the "License"). See License in the project root for license information. + */ +package com.linkedin.kafka.cruisecontrol.persisteddata.kafka; + +import com.linkedin.kafka.cruisecontrol.persisteddata.MapDecorator; +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.TopicPartition; +import org.easymock.EasyMock; +import org.easymock.EasyMockRunner; +import org.easymock.IExpectationSetters; +import org.easymock.Mock; +import org.junit.Test; +import org.junit.runner.RunWith; + +import static org.easymock.EasyMock.anyObject; +import static org.easymock.EasyMock.eq; +import static org.easymock.EasyMock.expect; +import static org.easymock.EasyMock.expectLastCall; +import static org.easymock.EasyMock.replay; +import static org.easymock.EasyMock.verify; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +@RunWith(EasyMockRunner.class) +public class KafkaPersistedMapTest { + + private static final String TOPIC = "topic"; + private static final int NUM_PARTITIONS = 1; + private static final TopicPartition PARTITION_0 = new TopicPartition(TOPIC, 0); + private static final CompletableFuture SUCCESSFUL_PRODUCER_SEND_RESPONSE = CompletableFuture.completedFuture( + new RecordMetadata(PARTITION_0, 1, 0, System.currentTimeMillis(), 0, 0)); + private static final Map ADDITIONAL_CONFIGS = Map.of("max.message.bytes", + "10000"); + private static final String KEY_1 = "key1"; + private static final String VALUE_1 = "value1"; + private static final String KEY_2 = "key2"; + private static final String VALUE_2 = "value2"; + private static final String KEY_3 = "key3"; + private static final String VALUE_3 = "value3"; + private static final long TEST_WAIT_TIMEOUT_MS = TimeUnit.SECONDS.toMillis(60); + + @Mock + private Producer _mockProducer; + @Mock + private Consumer _mockConsumer; + + private KeyQueueMapDecorator _backingMap; + private KafkaPersistedMap _persistedMap; + + private void initPersistedMap() { + // Set up calls to the consumer in the background thread. + setUpConsumerThread(); + + // Persisted data is written to the backingMap. + this._backingMap = new KeyQueueMapDecorator<>(new ConcurrentHashMap<>()); + + // Initialize the object under test. + this._persistedMap = new KafkaPersistedMap(this._backingMap, TOPIC, NUM_PARTITIONS, + (short) 1, ADDITIONAL_CONFIGS, t -> { + }, + () -> this._mockProducer, () -> this._mockConsumer); + } + + private void setUpConsumerThread() { + // Set up the consumer thread. + this._mockConsumer.subscribe(eq(Set.of(TOPIC))); + expectLastCall(); + this._mockConsumer.seekToBeginning(EasyMock.anyObject()); + expectLastCall(); + this._mockConsumer.close(); + expectLastCall(); + + // Consumer returns no data for successive calls. + expect(this._mockConsumer.poll(anyObject())) + .andReturn(new ConsumerRecords<>(Collections.emptyMap())).anyTimes(); + replay(this._mockConsumer); + } + + private void addConsumerData(Map data) { + expect(this._mockConsumer.poll(anyObject())) + .andReturn(getConsumerRecordsForPartition(data)).times(1); + } + + private static ConsumerRecords getConsumerRecordsForPartition( + Map data) { + final List> consumerRecords = data.entrySet().stream() + .map(e -> new ConsumerRecord<>(PARTITION_0.topic(), + PARTITION_0.partition(), + 1, e.getKey(), e.getValue())) + .collect(Collectors.toList()); + return new ConsumerRecords<>(Map.of(PARTITION_0, consumerRecords)); + } + + private void expectProducerSuccess(String key, String value) { + expectForProducer(key, value) + .andReturn(SUCCESSFUL_PRODUCER_SEND_RESPONSE); + replay(this._mockProducer); + } + + private IExpectationSetters> expectForProducer(String key, + String value) { + return expect(this._mockProducer.send(eq(new ProducerRecord<>(TOPIC, key, value)))); + } + + private static class KeyQueueMapDecorator extends MapDecorator { + + // Maintains a queue of keys that were added to the map. + private final BlockingQueue _keyQueue; + + /** + * Wraps the child map so accesses to it can be decorated. + * + * @param child The other map instance to decorate. + */ + KeyQueueMapDecorator(Map child) { + super(child); + this._keyQueue = new LinkedBlockingQueue<>(); + } + + @Override + public V put(K key, V value) { + // Store the value before notifying queue consumers. + final V result = super.put(key, value); + this._keyQueue.add(key); + return result; + } + + public K getKeyFromQueue() throws InterruptedException { + return _keyQueue.poll(TEST_WAIT_TIMEOUT_MS, TimeUnit.MILLISECONDS); + } + } + + /** + * Make sure all the config is applied and in the correct overriding order. Also make sure it is + * passed to the clientFactory. + */ + @Test + public void createKafkaClientAppliesAllTheConfig() { + final String bootstrapServers = "bootstrapServers"; + final Map defaultConfig = Map.of( + KEY_1, VALUE_1, + KEY_2, VALUE_1, + KEY_3, VALUE_1); + final Map securityConfig = Map.of( + KEY_2, VALUE_2, + KEY_3, VALUE_2); + final Map additionalConfig = Map.of( + KEY_3, VALUE_3); + final Function, Map> clientFactory = + config -> { + config.put("called", true); + return config; + }; + assertThat(KafkaPersistedMap.createKafkaClient(bootstrapServers, + defaultConfig, securityConfig, additionalConfig, clientFactory), equalTo(Map.of( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers, + KEY_1, VALUE_1, + KEY_2, VALUE_2, + KEY_3, VALUE_3, + "called", true))); + } + + /** + * Ensure that closing the persisted map object, stops the background thread and closes the + * kafka consumer. + */ + @Test + public void closeStopsTheConsumerThread() throws IOException, InterruptedException { + // Consumer returns no data. + initPersistedMap(); + + // The background thread starts alive, then dies after close() is called. + final Thread cacheUpdater = this._persistedMap.getCacheUpdater(); + assertTrue(cacheUpdater.isAlive()); + this._persistedMap.close(); + cacheUpdater.join(TEST_WAIT_TIMEOUT_MS); + assertFalse(cacheUpdater.isAlive()); + } + + /** + * All consumed data ends up in the backing map. + */ + @Test + public void backgroundThreadPutsConsumedDataIntoBackingMap() throws InterruptedException { + // Consumer returns two records, then nothing. + Map data1 = Map.of(KEY_1, VALUE_1); + Map data2 = Map.of(KEY_2, VALUE_2); + addConsumerData(data1); + addConsumerData(data2); + initPersistedMap(); + + // Wait for the backing map to contain the data. + assertThat(this._backingMap.getKeyFromQueue(), equalTo(KEY_1)); + assertThat(this._backingMap.getKeyFromQueue(), equalTo(KEY_2)); + + // Make sure both keys and values have been inserted. + assertThat(this._backingMap.get(KEY_1), equalTo(VALUE_1)); + assertThat(this._backingMap.get(KEY_2), equalTo(VALUE_2)); + } + + /** + * All consumed data ends up in the backing map. + */ + @Test + public void backgroundThreadKeepsConsumingEvenAfterAnException() throws InterruptedException { + // Consumer receives an exception then some valid data. + Map data1 = Map.of(KEY_1, VALUE_1); + expect(this._mockConsumer.poll(anyObject())) + .andThrow(new RuntimeException( + "For test: backgroundThreadKeepsConsumingEvenAfterAnException()")).times(1); + addConsumerData(data1); + initPersistedMap(); + + // Wait for the backing map to contain the data. + assertThat(this._backingMap.getKeyFromQueue(), equalTo(KEY_1)); + + // Make sure keys and values have been inserted. + assertThat(this._backingMap.get(KEY_1), equalTo(VALUE_1)); + } + + /** + * The put method should return the previous value for a key. + */ + @Test + public void putReturnsPreviousValue() { + initPersistedMap(); + expect(this._mockProducer.send(anyObject())) + .andReturn(SUCCESSFUL_PRODUCER_SEND_RESPONSE) + .times(2); + replay(this._mockProducer); + + // Backing map is initially empty so null value should be returned. + assertThat(this._persistedMap.put(KEY_1, VALUE_1), equalTo(null)); + + // Manually place a value into the backing store, then see if it is returned on the next put. + this._backingMap.put(KEY_1, VALUE_1); + assertThat(this._persistedMap.put(KEY_1, VALUE_2), equalTo(VALUE_1)); + } + + /** + * Put produces the data to kafka. + */ + @Test + public void putGivesDataToProducer() { + initPersistedMap(); + expectProducerSuccess(KEY_1, VALUE_1); + + // This test will fail if the above mock call is not used. + this._persistedMap.put(KEY_1, VALUE_1); + verify(this._mockProducer); + } + + /** + * Put throws an exception if persisting the data fails. + */ + @Test(expected = KafkaPersistedMapException.class) + public void putHandlesFailedProduce() { + initPersistedMap(); + expectForProducer(KEY_1, VALUE_1) + .andReturn(CompletableFuture.failedFuture( + new ExecutionException("test exception", null))); + replay(this._mockProducer); + + try { + this._persistedMap.put(KEY_1, VALUE_1); + } finally { + // This test will fail if the above mock call is not used. + verify(this._mockProducer); + } + } +} diff --git a/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/namespace/ExecutorPersistedDataTest.java b/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/namespace/ExecutorPersistedDataTest.java new file mode 100644 index 0000000000..7f6858f838 --- /dev/null +++ b/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/namespace/ExecutorPersistedDataTest.java @@ -0,0 +1,197 @@ +/* + * Copyright 2024 LinkedIn Corp. Licensed under the BSD 2-Clause License (the "License"). See License in the project root for license information. + */ +package com.linkedin.kafka.cruisecontrol.persisteddata.namespace; + +import com.linkedin.kafka.cruisecontrol.persisteddata.namespace.ExecutorPersistedData.BrokerMapKey; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; +import org.junit.Test; + +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.not; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.CoreMatchers.sameInstance; +import static org.hamcrest.MatcherAssert.assertThat; + +public class ExecutorPersistedDataTest { + + private static final int SINGLE_KEY = 1; + private static final long SINGLE_VALUE = 1L; + private static final Map SINGLE_ENTRY_MAP = Map.of(SINGLE_KEY, SINGLE_VALUE); + private ExecutorPersistedData _executorData; + + /** + * Set up the object under test, when the object is needed. + */ + public void setUp() { + _executorData = new ExecutorPersistedData(new HashMap<>()); + } + + /** + * Helper for ensuring serialized data is deserialized back to be the same as the original + * data. + * + * @param data data to serialize then compare against the deserialized version. + */ + private static void assertThatSerializedDataCanBeDeserialized(Map data) { + final String serialized = ExecutorPersistedData.serializeNumberMap(data); + assertThat(ExecutorPersistedData.deserializeNumberMap(serialized), is(data)); + } + + /** + * Ensure the executor's data is namespaced within the backing map. + */ + @Test + public void executorDataIsCorrectlyNamespacedWithinTheBackingStore() { + final HashMap backingMap = new HashMap<>(); + final ExecutorPersistedData executorData = new ExecutorPersistedData(backingMap); + assertThat(backingMap.size(), is(0)); + final Map demotedBrokers = executorData.getDemotedOrRemovedBrokers( + BrokerMapKey.DEMOTE); + demotedBrokers.put(1, 2L); + executorData.setDemotedOrRemovedBrokers(BrokerMapKey.DEMOTE, demotedBrokers); + assertThat(backingMap.size(), is(1)); + final String key = backingMap.keySet().stream().findFirst().orElseThrow(); + assertThat(key, containsString(Namespace.EXECUTOR.toString().toLowerCase())); + } + + /** + * A null should come back as a null. + */ + @Test + public void serializeAndDeserializeNumberMapWorksForNull() { + assertThatSerializedDataCanBeDeserialized(null); + } + + /** + * An empty map should come back as an empty one. + */ + @Test + public void serializeAndDeserializeNumberMapWorksForEmptyMap() { + assertThatSerializedDataCanBeDeserialized(Collections.emptyMap()); + } + + /** + * A map with a single entry should work. + */ + @Test + public void serializeAndDeserializeNumberMapWorksForSingleValueMap() { + assertThatSerializedDataCanBeDeserialized(SINGLE_ENTRY_MAP); + } + + /** + * Multiple entries and values along the entire number range should work. + */ + @Test + public void serializeAndDeserializeNumberMapWorksForMultiValueMap() { + assertThatSerializedDataCanBeDeserialized(Map.of( + Integer.MIN_VALUE, Long.MIN_VALUE, + 0, 0L, + Integer.MAX_VALUE, Long.MAX_VALUE)); + } + + /** + * Invalid serialized number map data should be ignored. + */ + @Test + public void deserializeNumberMapReturnsNullForInvalidSerializedData() { + assertThat(ExecutorPersistedData.deserializeNumberMap("1"), is(Collections.emptyMap())); + assertThat(ExecutorPersistedData.deserializeNumberMap("1:a"), is(Collections.emptyMap())); + assertThat(ExecutorPersistedData.deserializeNumberMap("a:1"), is(Collections.emptyMap())); + } + + /** + * The first request for a key should return an empty map rather than null. + */ + @Test + public void getDemotedOrRemovedBrokersReturnsNonNullMap() { + setUp(); + assertThat(_executorData.getDemotedOrRemovedBrokers(BrokerMapKey.DEMOTE), notNullValue()); + } + + /** + * Data written using setDemotedOrRemovedBrokers() should readable using + * getDemotedOrRemovedBrokers() for the same key without mixing up the keys. + */ + @Test + public void setDemotedOrRemovedBrokersDataCanBeGottenBack() { + setUp(); + _executorData.setDemotedOrRemovedBrokers(BrokerMapKey.DEMOTE, SINGLE_ENTRY_MAP); + assertThat(_executorData.getDemotedOrRemovedBrokers(BrokerMapKey.DEMOTE), + is(SINGLE_ENTRY_MAP)); + assertThat(_executorData.getDemotedOrRemovedBrokers(BrokerMapKey.REMOVE).isEmpty(), + is(true)); + } + + /** + * The lock objects need to be consistent for locking to work correctly. + */ + @Test + public void getDemotedOrRemovedBrokersLockReturnsSameObjectInstanceEachTime() { + setUp(); + assertThat(_executorData.getDemotedOrRemovedBrokersLock(BrokerMapKey.DEMOTE), + sameInstance(_executorData.getDemotedOrRemovedBrokersLock(BrokerMapKey.DEMOTE))); + } + + /** + * Each key needs its own lock object for locking to work correctly. + */ + @Test + public void getDemotedOrRemovedBrokersLockReturnsUniqueInstanceForEachKey() { + setUp(); + assertThat(_executorData.getDemotedOrRemovedBrokersLock(BrokerMapKey.DEMOTE), + not(sameInstance( + _executorData.getDemotedOrRemovedBrokersLock(BrokerMapKey.REMOVE)))); + } + + /** + * modifyDemotedOrRemovedBrokers() must apply the modify function to the key, and only the + * key's, map. + */ + @Test + public void modifyDemotedOrRemovedBrokersSetsTheModifiedValueForAKey() { + setUp(); + _executorData.modifyDemotedOrRemovedBrokers(BrokerMapKey.DEMOTE, + map -> map.put(SINGLE_KEY, SINGLE_VALUE)); + assertThat(_executorData.getDemotedOrRemovedBrokers(BrokerMapKey.DEMOTE), + is(SINGLE_ENTRY_MAP)); + assertThat(_executorData.getDemotedOrRemovedBrokers(BrokerMapKey.REMOVE).isEmpty(), + is(true)); + } + + /** + * Multiple threads reading/writing the same data should do so atomically. + */ + @Test + public void modifyDemotedOrRemovedBrokersSynchronizesModifications() + throws InterruptedException { + setUp(); + _executorData.setDemotedOrRemovedBrokers(BrokerMapKey.DEMOTE, SINGLE_ENTRY_MAP); + + // Have 2 workers continually increment the same key/value. The result should be the same + // as if they ran in series. + final int numWorkers = 2; + final int numIterations = 10; + Thread[] workers = new Thread[numWorkers]; + final ThreadFactory threadFactory = Executors.defaultThreadFactory(); + for (int i = 0; i < workers.length; i++) { + workers[i] = threadFactory.newThread(() -> { + for (int j = 0; j < numIterations; j++) { + _executorData.modifyDemotedOrRemovedBrokers(BrokerMapKey.DEMOTE, + map -> map.compute(SINGLE_KEY, (k, v) -> v == null ? 0 : v + 1)); + } + }); + workers[i].start(); + } + for (Thread worker : workers) { + worker.join(); + } + assertThat(_executorData.getDemotedOrRemovedBrokers(BrokerMapKey.DEMOTE).get(SINGLE_KEY), + is(SINGLE_VALUE + numWorkers * numIterations)); + } +} diff --git a/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/namespace/NamespaceTest.java b/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/namespace/NamespaceTest.java new file mode 100644 index 0000000000..ed196aa4e6 --- /dev/null +++ b/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/namespace/NamespaceTest.java @@ -0,0 +1,51 @@ +/* + * Copyright 2024 LinkedIn Corp. Licensed under the BSD 2-Clause License (the "License"). See License in the project root for license information. + */ +package com.linkedin.kafka.cruisecontrol.persisteddata.namespace; + +import java.util.HashMap; +import java.util.Map; +import org.junit.Test; + +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.nullValue; +import static org.hamcrest.MatcherAssert.assertThat; + +public class NamespaceTest { + + /** + * All, and only, keys mapped by the mapper should be retrievable by the back-mapper. + */ + @Test + public void namespaceKeyMapperFunctionIsOneToOneReversible() { + final String key = "key"; + for (Namespace namespace : Namespace.values()) { + String mappedKey = namespace.keyToNamespaceMapper(key); + + // The mapped key should map back to the key. + assertThat(namespace.namespaceToKeyMapper(mappedKey), is(key)); + + // An unmapped key should map to null to indicate key failed the back-mapper's + // validation. + assertThat(namespace.namespaceToKeyMapper(key), is(nullValue())); + } + } + + /** + * Make sure embed() returns a wrapped view of the provided map that keeps keys and values in + * their own key namespace. + */ + @Test + public void embedReturnsNamespacedViewOfMap() { + final String key = "key"; + final String value = "value"; + final String embeddedValue = "embeddedValue"; + final Namespace namespace = Namespace.EXECUTOR; + Map map = new HashMap<>(); + Map embeddedMap = namespace.embed(map); + map.put(key, value); + embeddedMap.put(key, embeddedValue); + assertThat(map.get(key), is(value)); + assertThat(embeddedMap.get(key), is(embeddedValue)); + } +} From 86058f30c81d37c80b3ef7179110f6e9d35de72e Mon Sep 17 00:00:00 2001 From: Morgan Galpin Date: Fri, 9 Feb 2024 14:43:35 -0800 Subject: [PATCH 2/5] Renamed 'backing.method' to 'persist.method'. --- config/cruisecontrol.properties | 14 ++++---- .../config/constants/PersistedDataConfig.java | 32 +++++++++---------- ...{BackingMethod.java => PersistMethod.java} | 16 +++++----- .../persisteddata/PersistedMapFactory.java | 16 +++++----- ...MethodTest.java => PersistMethodTest.java} | 10 +++--- .../PersistedMapFactoryTest.java | 2 +- 6 files changed, 45 insertions(+), 45 deletions(-) rename cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/{BackingMethod.java => PersistMethod.java} (59%) rename cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/{BackingMethodTest.java => PersistMethodTest.java} (78%) diff --git a/config/cruisecontrol.properties b/config/cruisecontrol.properties index 062ce32915..e52df22191 100644 --- a/config/cruisecontrol.properties +++ b/config/cruisecontrol.properties @@ -206,38 +206,38 @@ execution.progress.check.interval.ms=10000 # "kafka", # "memory" # The default is "memory", which doesn't durably persist any runtime data. -#persisted.data.backing.method=kafka +#persisted.data.persist.method=kafka -# The name of the kafka topic to use to persist data when "persisted.data.backing.method" is set to +# The name of the kafka topic to use to persist data when "persisted.data.persist.method" is set to # "kafka". If the topic is not present, then it will be created. #persisted.data.kafka.topic.name=__CruiseControlPersistentData # The number of partitions to ensure are present for the kafka topic. Only applies when -# "persisted.data.backing.method" is set to "kafka". If the topic has fewer than this number of +# "persisted.data.persist.method" is set to "kafka". If the topic has fewer than this number of # partitions, then partitions will be added. #persisted.data.kafka.topic.partition.count=2 # The replication factor to use for the kafka topic. Only applies when -# "persisted.data.backing.method" is set to "kafka". Multiple partition replicas are desirable to +# "persisted.data.persist.method" is set to "kafka". Multiple partition replicas are desirable to # ensure the topic is reasonably available. #persisted.data.kafka.topic.replication.factor=2 # The configs to apply to the kafka topic used to persist Cruise Control data. Only applies if -# "persisted.data.backing.method" is set to "kafka". This "list" should be a semicolon separated +# "persisted.data.persist.method" is set to "kafka". This "list" should be a semicolon separated # string of 'key=value' pairs. The keys and values need to be valid Kafka Topic configs. # See: https://kafka.apache.org/documentation/#topicconfigs # e.g. persisted.data.kafka.topic.additional.configs.map=max.compaction.lag.ms=600000;min.insync.replicas=2 #persisted.data.kafka.topic.additional.configs.map= # The additional configs to use when creating the kafka producer to persist Cruise Control data. -# Only applies if "persisted.data.backing.method" is set to "kafka". This "list" should be a +# Only applies if "persisted.data.persist.method" is set to "kafka". This "list" should be a # semicolon separated string of 'key=value' pairs. The keys and values need to be valid Kafka # Producer configs. See: https://kafka.apache.org/documentation/#producerconfigs # e.g. persisted.data.kafka.producer.additional.configs.map=buffer.memory=100000000;batch.size=900000 #persisted.data.kafka.producer.additional.configs.map= # The additional configs to use when creating the kafka consumer to read persisted Cruise Control -# data. Only applies if "persisted.data.backing.method" is set to "kafka". This "list" should be a +# data. Only applies if "persisted.data.persist.method" is set to "kafka". This "list" should be a # semicolon separated string of 'key=value' pairs. The keys and values need to be valid Kafka # Consumer configs. See: https://kafka.apache.org/documentation/#consumerconfigs # e.g. persisted.data.kafka.consumer.additional.configs.map=group.id=my-group-id;fetch.max.bytes=2000000 diff --git a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/config/constants/PersistedDataConfig.java b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/config/constants/PersistedDataConfig.java index 80b9ed438d..fc88d4dcc6 100644 --- a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/config/constants/PersistedDataConfig.java +++ b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/config/constants/PersistedDataConfig.java @@ -3,7 +3,7 @@ */ package com.linkedin.kafka.cruisecontrol.config.constants; -import com.linkedin.kafka.cruisecontrol.persisteddata.BackingMethod; +import com.linkedin.kafka.cruisecontrol.persisteddata.PersistMethod; import org.apache.kafka.common.config.ConfigDef; import static org.apache.kafka.common.config.ConfigDef.Range.atLeast; @@ -18,16 +18,16 @@ public final class PersistedDataConfig { public static final String CONFIG_PREFIX = "persisted.data."; /** - * persisted.data.backing.method + * persisted.data.persist.method */ - public static final String BACKING_METHOD_CONFIG = CONFIG_PREFIX + "backing.method"; - public static final String DEFAULT_BACKING_METHOD = "memory"; - public static final String BACKING_METHOD_DOC = + public static final String PERSIST_METHOD_CONFIG = CONFIG_PREFIX + "persist.method"; + public static final String DEFAULT_PERSIST_METHOD = "memory"; + public static final String PERSIST_METHOD_DOC = "The method to use to store persisted data. This is the first " + "\"persisted.data\" config to set which will determine which other configs " + "of the series should be configured. The available options are: " - + BackingMethod.stringValues().toString() - + ". The default is \"" + DEFAULT_BACKING_METHOD + "\", which doesn't durably " + + PersistMethod.stringValues().toString() + + ". The default is \"" + DEFAULT_PERSIST_METHOD + "\", which doesn't durably " + "persist any runtime data."; /** @@ -37,7 +37,7 @@ public final class PersistedDataConfig { public static final String DEFAULT_KAFKA_TOPIC_NAME = "__CruiseControlPersistentData"; public static final String KAFKA_TOPIC_NAME_DOC = "The name of the kafka topic to use to persist data when " + "\"" - + BACKING_METHOD_CONFIG + "\" is set to \"kafka\". If the topic is not " + + PERSIST_METHOD_CONFIG + "\" is set to \"kafka\". If the topic is not " + "present, then it will be created."; /** @@ -48,7 +48,7 @@ public final class PersistedDataConfig { public static final int DEFAULT_KAFKA_TOPIC_PARTITION_COUNT = 2; public static final String KAFKA_TOPIC_PARTITION_COUNT_DOC = "The number of partitions to ensure are present " - + "for the kafka topic. Only applies when \"" + BACKING_METHOD_CONFIG + + "for the kafka topic. Only applies when \"" + PERSIST_METHOD_CONFIG + "\" is set to \"kafka\". If the topic has fewer than this number of " + "partitions, then partitions will be added."; @@ -60,7 +60,7 @@ public final class PersistedDataConfig { public static final short DEFAULT_KAFKA_TOPIC_REPLICATION_FACTOR = 2; public static final String KAFKA_TOPIC_REPLICATION_FACTOR_DOC = "The replication factor to use for the kafka " + "topic. Only applies when \"" - + BACKING_METHOD_CONFIG + "\" is set to \"kafka\". Multiple partition " + + PERSIST_METHOD_CONFIG + "\" is set to \"kafka\". Multiple partition " + "replicas are desirable to ensure the topic is reasonably available."; /** @@ -71,7 +71,7 @@ public final class PersistedDataConfig { public static final String DEFAULT_KAFKA_TOPIC_ADDITIONAL_CONFIGS_MAP = ""; public static final String KAFKA_TOPIC_ADDITIONAL_CONFIGS_MAP_DOC = "The configs to apply to the kafka topic used to " - + "persist Cruise Control data. Only applies if \"" + BACKING_METHOD_CONFIG + + "persist Cruise Control data. Only applies if \"" + PERSIST_METHOD_CONFIG + "\" is set to \"kafka\". This \"list\" should be a semicolon separated " + "string of 'key=value' pairs. The keys and values need to be valid Kafka " + "Topic configs. See: " @@ -86,7 +86,7 @@ public final class PersistedDataConfig { public static final String KAFKA_PRODUCER_ADDITIONAL_CONFIGS_MAP_DOC = "The additional configs to use when creating the kafka " + "producer to persist Cruise Control data. Only applies if \"" - + BACKING_METHOD_CONFIG + "\" is set to \"kafka\". This \"list\" should be a " + + PERSIST_METHOD_CONFIG + "\" is set to \"kafka\". This \"list\" should be a " + "semicolon separated string of 'key=value' pairs. The keys and values need " + "to be valid Kafka Producer configs. See: " + "https://kafka.apache.org/documentation/#producerconfigs"; @@ -100,7 +100,7 @@ public final class PersistedDataConfig { public static final String KAFKA_CONSUMER_ADDITIONAL_CONFIGS_MAP_DOC = "The additional configs to use when creating the kafka " + "consumer to read persisted Cruise Control data. Only applies if \"" - + BACKING_METHOD_CONFIG + "\" is set to \"kafka\". This \"list\" should be a " + + PERSIST_METHOD_CONFIG + "\" is set to \"kafka\". This \"list\" should be a " + "semicolon separated string of 'key=value' pairs. The keys and values need " + "to be valid Kafka Consumer configs. See: " + "https://kafka.apache.org/documentation/#consumerconfigs"; @@ -116,11 +116,11 @@ private PersistedDataConfig() { */ public static ConfigDef define(ConfigDef configDef) { return configDef - .define(BACKING_METHOD_CONFIG, + .define(PERSIST_METHOD_CONFIG, ConfigDef.Type.STRING, - DEFAULT_BACKING_METHOD, + DEFAULT_PERSIST_METHOD, ConfigDef.Importance.MEDIUM, - BACKING_METHOD_DOC) + PERSIST_METHOD_DOC) .define(KAFKA_TOPIC_NAME_CONFIG, ConfigDef.Type.STRING, DEFAULT_KAFKA_TOPIC_NAME, diff --git a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/BackingMethod.java b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistMethod.java similarity index 59% rename from cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/BackingMethod.java rename to cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistMethod.java index 5148ddd3ad..d8af37c9d3 100644 --- a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/BackingMethod.java +++ b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistMethod.java @@ -8,30 +8,30 @@ import java.util.stream.Collectors; /** - * Defines the backing method types of persistent data. + * Defines the persist method types of persistent data. */ -public enum BackingMethod { +public enum PersistMethod { MEMORY, KAFKA; /** - * Parse the string version of a backing method into the enum. + * Parse the string version of a persist method into the enum. * * @param value The value to parse. * @return The correctly parsed enum. */ - public static BackingMethod fromString(String value) { + public static PersistMethod fromString(String value) { return valueOf(value.toUpperCase()); } /** - * Get the lower case string values of all backing methods. + * Get the lower case string values of all persist methods. * - * @return The list of all the backing methods as lower case strings. + * @return The list of all the persist methods as lower case strings. */ public static List stringValues() { - return Arrays.stream(BackingMethod.values()) - .map(backingMethod -> backingMethod.toString().toLowerCase()) + return Arrays.stream(PersistMethod.values()) + .map(persistMethod -> persistMethod.toString().toLowerCase()) .sorted() .collect(Collectors.toList()); } diff --git a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistedMapFactory.java b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistedMapFactory.java index 6600ffd2a6..c161654fd4 100644 --- a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistedMapFactory.java +++ b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistedMapFactory.java @@ -11,13 +11,13 @@ import java.util.function.Supplier; import org.apache.kafka.clients.admin.AdminClient; -import static com.linkedin.kafka.cruisecontrol.persisteddata.BackingMethod.KAFKA; -import static com.linkedin.kafka.cruisecontrol.persisteddata.BackingMethod.MEMORY; +import static com.linkedin.kafka.cruisecontrol.persisteddata.PersistMethod.KAFKA; +import static com.linkedin.kafka.cruisecontrol.persisteddata.PersistMethod.MEMORY; /** * Constructs the correct {@link PersistedMap} implementation based on the provided * {@link KafkaCruiseControlConfig}. In particular, it uses the value of - * {@link PersistedDataConfig#BACKING_METHOD_CONFIG} to determine which implementation to + * {@link PersistedDataConfig#PERSIST_METHOD_CONFIG} to determine which implementation to * construct. */ public class PersistedMapFactory { @@ -26,14 +26,14 @@ public class PersistedMapFactory { private final KafkaCruiseControlConfig _config; // Keeps the suppliers for implementation-specific instances. - private final Map> _suppliers; + private final Map> _suppliers; /** * Creates an instance that is able to construct the correct {@link PersistedMap} * implementation. * * @param config The complete program configuration to evaluate. Specifically, the - * {@link PersistedDataConfig#BACKING_METHOD_CONFIG} config is needed. + * {@link PersistedDataConfig#PERSIST_METHOD_CONFIG} config is needed. * @param adminClient The admin client to pass to {@link KafkaPersistedMap}, if needed. */ public PersistedMapFactory(KafkaCruiseControlConfig config, AdminClient adminClient) { @@ -60,13 +60,13 @@ public PersistedMapFactory(KafkaCruiseControlConfig config, AdminClient adminCli /** * Constructs the correct {@link PersistedMap} implementation based on the configured - * {@link PersistedDataConfig#BACKING_METHOD_CONFIG}. + * {@link PersistedDataConfig#PERSIST_METHOD_CONFIG}. * * @return The {@link PersistedMap} implementation. */ public PersistedMap instance() { - BackingMethod backingMethod = BackingMethod.fromString( - _config.getString(PersistedDataConfig.BACKING_METHOD_CONFIG)); + PersistMethod backingMethod = PersistMethod.fromString( + _config.getString(PersistedDataConfig.PERSIST_METHOD_CONFIG)); return this._suppliers.getOrDefault(backingMethod, this._suppliers.get(MEMORY)).get(); } } diff --git a/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/BackingMethodTest.java b/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistMethodTest.java similarity index 78% rename from cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/BackingMethodTest.java rename to cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistMethodTest.java index 987c9c3966..9d1f7076c5 100644 --- a/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/BackingMethodTest.java +++ b/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistMethodTest.java @@ -9,15 +9,15 @@ import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; -public class BackingMethodTest { +public class PersistMethodTest { /** * Ensure fromString accepts valid values ignoring case. */ @Test public void fromStringAcceptsValidValuesIgnoringCase() { - assertThat(BackingMethod.fromString("kafka"), is(BackingMethod.KAFKA)); - assertThat(BackingMethod.fromString("Kafka"), is(BackingMethod.KAFKA)); + assertThat(PersistMethod.fromString("kafka"), is(PersistMethod.KAFKA)); + assertThat(PersistMethod.fromString("Kafka"), is(PersistMethod.KAFKA)); } /** @@ -25,7 +25,7 @@ public void fromStringAcceptsValidValuesIgnoringCase() { */ @Test(expected = IllegalArgumentException.class) public void fromStringRejectsInvalidValues() { - BackingMethod.fromString("invalid"); + PersistMethod.fromString("invalid"); } /** @@ -34,7 +34,7 @@ public void fromStringRejectsInvalidValues() { @Test public void stringValuesGetsAllEnumsLowercaseAndSorted() { List expected = List.of("kafka", "memory"); - List actual = BackingMethod.stringValues(); + List actual = PersistMethod.stringValues(); assertThat(actual, is(expected)); } } diff --git a/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistedMapFactoryTest.java b/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistedMapFactoryTest.java index 6bf551ac2e..db243ed74e 100644 --- a/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistedMapFactoryTest.java +++ b/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistedMapFactoryTest.java @@ -46,7 +46,7 @@ public void instanceReturnsPersistedMapWhenConfiguredForMemory() { private static PersistedMapFactory configureAndGetPersistedMapFactory(String backingMethod) { KafkaCruiseControlConfig config = new KafkaCruiseControlConfig( - Map.of(PersistedDataConfig.BACKING_METHOD_CONFIG, backingMethod, + Map.of(PersistedDataConfig.PERSIST_METHOD_CONFIG, backingMethod, CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "fake", ExecutorConfig.ZOOKEEPER_CONNECT_CONFIG, "connect:1234")); return new PersistedMapFactory(config, From 336c579b9feb6a67669ef6386b222dbb3d4710f0 Mon Sep 17 00:00:00 2001 From: Morgan Galpin Date: Mon, 8 Jul 2024 14:40:58 -0700 Subject: [PATCH 3/5] Reverted formatting change to ConfigDef initialization. --- .../config/KafkaCruiseControlConfig.java | 16 ++++------------ 1 file changed, 4 insertions(+), 12 deletions(-) diff --git a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/config/KafkaCruiseControlConfig.java b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/config/KafkaCruiseControlConfig.java index df58fc9be1..af96f325a8 100644 --- a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/config/KafkaCruiseControlConfig.java +++ b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/config/KafkaCruiseControlConfig.java @@ -41,18 +41,10 @@ public class KafkaCruiseControlConfig extends AbstractConfig { private static final ConfigDef CONFIG; static { - ConfigDef configDef = new ConfigDef(); - configDef = AnalyzerConfig.define(configDef); - configDef = AnomalyDetectorConfig.define(configDef); - configDef = CruiseControlParametersConfig.define(configDef); - configDef = CruiseControlRequestConfig.define(configDef); - configDef = ExecutorConfig.define(configDef); - configDef = MonitorConfig.define(configDef); - configDef = PersistedDataConfig.define(configDef); - configDef = UserTaskManagerConfig.define(configDef); - configDef = WebServerConfig.define(configDef); - configDef.withClientSslSupport().withClientSaslSupport(); - CONFIG = configDef; + CONFIG = CruiseControlRequestConfig.define(CruiseControlParametersConfig.define(AnomalyDetectorConfig.define( + AnalyzerConfig.define(ExecutorConfig.define(MonitorConfig.define(WebServerConfig.define( + UserTaskManagerConfig.define(PersistedDataConfig.define(new ConfigDef()))))))))) + .withClientSslSupport().withClientSaslSupport(); } public KafkaCruiseControlConfig(Map originals) { From b671de05b15c4f3a4101cba7f5c8cb75e0877ada Mon Sep 17 00:00:00 2001 From: Morgan Galpin Date: Mon, 8 Jul 2024 16:47:20 -0700 Subject: [PATCH 4/5] Enhanced client config documentation. --- config/cruisecontrol.properties | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 deletions(-) diff --git a/config/cruisecontrol.properties b/config/cruisecontrol.properties index e52df22191..6c99bbbe62 100644 --- a/config/cruisecontrol.properties +++ b/config/cruisecontrol.properties @@ -226,21 +226,33 @@ execution.progress.check.interval.ms=10000 # "persisted.data.persist.method" is set to "kafka". This "list" should be a semicolon separated # string of 'key=value' pairs. The keys and values need to be valid Kafka Topic configs. # See: https://kafka.apache.org/documentation/#topicconfigs -# e.g. persisted.data.kafka.topic.additional.configs.map=max.compaction.lag.ms=600000;min.insync.replicas=2 +# To enhance readability, the backslash (\) character can be used at the end of lines to allow +# line-continuation. +# e.g. persisted.data.kafka.topic.additional.configs.map = \ +# max.compaction.lag.ms = 600000; \ +# min.insync.replicas = 2 #persisted.data.kafka.topic.additional.configs.map= # The additional configs to use when creating the kafka producer to persist Cruise Control data. # Only applies if "persisted.data.persist.method" is set to "kafka". This "list" should be a # semicolon separated string of 'key=value' pairs. The keys and values need to be valid Kafka # Producer configs. See: https://kafka.apache.org/documentation/#producerconfigs -# e.g. persisted.data.kafka.producer.additional.configs.map=buffer.memory=100000000;batch.size=900000 +# To enhance readability, the backslash (\) character can be used at the end of lines to allow +# line-continuation. +# e.g. persisted.data.kafka.producer.additional.configs.map = \ +# buffer.memory = 100000000; \ +# interceptor.classes = com.example.Interceptor1,com.example.Interceptor2 #persisted.data.kafka.producer.additional.configs.map= # The additional configs to use when creating the kafka consumer to read persisted Cruise Control # data. Only applies if "persisted.data.persist.method" is set to "kafka". This "list" should be a # semicolon separated string of 'key=value' pairs. The keys and values need to be valid Kafka # Consumer configs. See: https://kafka.apache.org/documentation/#consumerconfigs -# e.g. persisted.data.kafka.consumer.additional.configs.map=group.id=my-group-id;fetch.max.bytes=2000000 +# To enhance readability, the backslash (\) character can be used at the end of lines to allow +# line-continuation. +# e.g. persisted.data.kafka.consumer.additional.configs.map = \ +# group.id = my-group-id; \ +# fetch.max.bytes = 2000000 #persisted.data.kafka.consumer.additional.configs.map= From 9a9a9749bc3d0bc716d24846aefed3f578893d64 Mon Sep 17 00:00:00 2001 From: Morgan Galpin Date: Tue, 23 Jul 2024 14:46:05 -0700 Subject: [PATCH 5/5] Refactored how PersistedData is configured to use a prefixed config pattern. Removed the, now-unneeded, Map config value type. --- config/cruisecontrol.properties | 103 ++++---- .../config/KafkaCruiseControlConfig.java | 38 --- .../config/constants/PersistedDataConfig.java | 95 ++++--- .../config/constants/TopicConfigDef.java | 232 ++++++++++++++++++ .../persisteddata/PersistedMapFactory.java | 47 ++-- .../kafka/KafkaPersistedMap.java | 151 +++--------- .../config/KafkaCruiseControlConfigTest.java | 81 ------ .../PersistedMapFactoryTest.java | 22 +- .../kafka/KafkaPersistedMapTest.java | 17 +- 9 files changed, 425 insertions(+), 361 deletions(-) create mode 100644 cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/config/constants/TopicConfigDef.java delete mode 100644 cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/config/KafkaCruiseControlConfigTest.java diff --git a/config/cruisecontrol.properties b/config/cruisecontrol.properties index 6c99bbbe62..3dded1a669 100644 --- a/config/cruisecontrol.properties +++ b/config/cruisecontrol.properties @@ -198,7 +198,7 @@ num.concurrent.leader.movements=1000 execution.progress.check.interval.ms=10000 -# Configurations for persistent data +# Configuration for persistent data # ======================================= # The method to use to store persisted data. This is the first "persisted.data" config to set which @@ -206,54 +206,73 @@ execution.progress.check.interval.ms=10000 # "kafka", # "memory" # The default is "memory", which doesn't durably persist any runtime data. -#persisted.data.persist.method=kafka +persisted.data.persist.method=memory -# The name of the kafka topic to use to persist data when "persisted.data.persist.method" is set to -# "kafka". If the topic is not present, then it will be created. -#persisted.data.kafka.topic.name=__CruiseControlPersistentData +# The name of the kafka topic to use to persist data when "persisted.data.persist.method=kafka". +# If the topic is not present, then it will be created. +persisted.data.kafka.topic.name=__CruiseControlPersistentData # The number of partitions to ensure are present for the kafka topic. Only applies when -# "persisted.data.persist.method" is set to "kafka". If the topic has fewer than this number of -# partitions, then partitions will be added. -#persisted.data.kafka.topic.partition.count=2 +# "persisted.data.persist.method=kafka". If the topic has fewer than this number of partitions, +# then partitions will be added. +persisted.data.kafka.topic.partition.count=1 # The replication factor to use for the kafka topic. Only applies when -# "persisted.data.persist.method" is set to "kafka". Multiple partition replicas are desirable to -# ensure the topic is reasonably available. -#persisted.data.kafka.topic.replication.factor=2 +# "persisted.data.persist.method=kafka". Multiple partition replicas are desirable to ensure the +# topic is reasonably available. +persisted.data.kafka.topic.replication.factor=3 # The configs to apply to the kafka topic used to persist Cruise Control data. Only applies if -# "persisted.data.persist.method" is set to "kafka". This "list" should be a semicolon separated -# string of 'key=value' pairs. The keys and values need to be valid Kafka Topic configs. -# See: https://kafka.apache.org/documentation/#topicconfigs -# To enhance readability, the backslash (\) character can be used at the end of lines to allow -# line-continuation. -# e.g. persisted.data.kafka.topic.additional.configs.map = \ -# max.compaction.lag.ms = 600000; \ -# min.insync.replicas = 2 -#persisted.data.kafka.topic.additional.configs.map= - -# The additional configs to use when creating the kafka producer to persist Cruise Control data. -# Only applies if "persisted.data.persist.method" is set to "kafka". This "list" should be a -# semicolon separated string of 'key=value' pairs. The keys and values need to be valid Kafka -# Producer configs. See: https://kafka.apache.org/documentation/#producerconfigs -# To enhance readability, the backslash (\) character can be used at the end of lines to allow -# line-continuation. -# e.g. persisted.data.kafka.producer.additional.configs.map = \ -# buffer.memory = 100000000; \ -# interceptor.classes = com.example.Interceptor1,com.example.Interceptor2 -#persisted.data.kafka.producer.additional.configs.map= - -# The additional configs to use when creating the kafka consumer to read persisted Cruise Control -# data. Only applies if "persisted.data.persist.method" is set to "kafka". This "list" should be a -# semicolon separated string of 'key=value' pairs. The keys and values need to be valid Kafka -# Consumer configs. See: https://kafka.apache.org/documentation/#consumerconfigs -# To enhance readability, the backslash (\) character can be used at the end of lines to allow -# line-continuation. -# e.g. persisted.data.kafka.consumer.additional.configs.map = \ -# group.id = my-group-id; \ -# fetch.max.bytes = 2000000 -#persisted.data.kafka.consumer.additional.configs.map= +# "persisted.data.persist.method=kafka". All valid topic config keys and values can be used to +# apply to the "persisted.data.kafka.topic.name" topic. Use this prefix string with the topic +# config keys. +# e.g. persisted.data.kafka.topic.config.min.insync.replicas=2 +# See: Kafka topic config +# documentation. +# +# persisted.data.kafka.topic.config.K=V +persisted.data.kafka.topic.config.cleanup.policy=compact + +# The configs to use when creating the kafka producer to persist Cruise Control data. Only +# applies if "persisted.data.persist.method=kafka". The keys and values need to be valid Kafka +# Producer configs. All valid Kafka producer config keys +# can be prefixed, and they will be applied to the producer. +# e.g. persisted.data.kafka.producer.config.compression.type=gzip +# +# persisted.data.kafka.producer.config.K=V +persisted.data.kafka.producer.config.acks=all +persisted.data.kafka.producer.config.buffer.memory=2000000 +persisted.data.kafka.producer.config.client.id=kafka-cruise-control.kafka-persisted-map.producer +persisted.data.kafka.producer.config.compression.type=gzip +persisted.data.kafka.producer.config.delivery.timeout.ms=3600000 +persisted.data.kafka.producer.config.key.serializer=StringSerializer +persisted.data.kafka.producer.config.max.in.flight.requests.per.connection=2 +persisted.data.kafka.producer.config.max.request.size=1000000 +persisted.data.kafka.producer.config.reconnect.backoff.ms=1000 +persisted.data.kafka.producer.config.retries=2147483647 +persisted.data.kafka.producer.config.retry.backoff.ms=1000 +persisted.data.kafka.producer.config.value.serializer=StringSerializer + +# The configs to use when creating the kafka consumer to read persisted Cruise Control data. +# Only applies if "persisted.data.persist.method=kafka". The keys and values need to be valid +# Kafka Consumer configs. All valid Kafka consumer config keys +# can be prefixed, and they will be applied to the consumer. +# e.g. persisted.data.kafka.consumer.config.max.poll.records=100 +# +# persisted.data.kafka.consumer.K=V +persisted.data.kafka.consumer.allow.auto.create.topics=false +persisted.data.kafka.consumer.auto.offset.reset=earliest +persisted.data.kafka.consumer.client.id=kafka-cruise-control.kafka-persisted-map.consumer +persisted.data.kafka.consumer.enable.auto.commit=false +persisted.data.kafka.consumer.exclude.internal.topics=false +persisted.data.kafka.consumer.group.id=kafka-cruise-control.kafka-persisted-map.consumer-group +persisted.data.kafka.consumer.isolation.level=read_committed +persisted.data.kafka.consumer.key.deserializer=StringDeserializer +persisted.data.kafka.consumer.max.poll.records=2147483647 +persisted.data.kafka.consumer.reconnect.backoff.ms=1000 +persisted.data.kafka.consumer.value.deserializer=StringDeserializer # Configurations for anomaly detector diff --git a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/config/KafkaCruiseControlConfig.java b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/config/KafkaCruiseControlConfig.java index af96f325a8..aab92e4ff0 100644 --- a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/config/KafkaCruiseControlConfig.java +++ b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/config/KafkaCruiseControlConfig.java @@ -17,7 +17,6 @@ import com.linkedin.kafka.cruisecontrol.metricsreporter.CruiseControlMetricsReporterConfig; import java.nio.file.Files; import java.nio.file.Paths; -import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Set; @@ -25,7 +24,6 @@ import com.linkedin.kafka.cruisecontrol.servlet.security.BasicSecurityProvider; import com.linkedin.kafka.cruisecontrol.servlet.security.SecurityProvider; import com.linkedin.kafka.cruisecontrol.servlet.security.jwt.JwtSecurityProvider; -import org.apache.commons.lang3.StringUtils; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; import java.util.Map; @@ -138,42 +136,6 @@ public List getConfiguredInstances(String key, Class t, Map getMap(String key) { - Map result = new HashMap<>(); - String value = getString(key); - if (StringUtils.isBlank(value)) { - return result; - } - - for (String pair : value.split(";")) { - if (StringUtils.isNotBlank(pair)) { - final String[] parts = pair.split("="); - - // An empty value is allowed, but not an empty key. - if (parts.length == 2 && StringUtils.isNotBlank(parts[0])) { - result.put(parts[0].trim(), parts[1].trim()); - } else if (parts.length == 1 && pair.stripTrailing().equals(parts[0] + "=")) { - result.put(parts[0].trim(), ""); - } else { - throw new ConfigException(String.format( - "Invalid configuration map entry: \"%s\", found for key: \"%s\". " - + "Each entry must be a key=value pair.", - pair, key)); - } - } - } - return result; - } - /** * Sanity check to ensure that {@link ExecutorConfig#ZOOKEEPER_CONNECT_CONFIG} is set if *
      diff --git a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/config/constants/PersistedDataConfig.java b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/config/constants/PersistedDataConfig.java index fc88d4dcc6..17a580bd1c 100644 --- a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/config/constants/PersistedDataConfig.java +++ b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/config/constants/PersistedDataConfig.java @@ -4,6 +4,8 @@ package com.linkedin.kafka.cruisecontrol.config.constants; import com.linkedin.kafka.cruisecontrol.persisteddata.PersistMethod; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.config.ConfigDef; import static org.apache.kafka.common.config.ConfigDef.Range.atLeast; @@ -16,6 +18,8 @@ public final class PersistedDataConfig { public static final String CONFIG_PREFIX = "persisted.data."; + public static final String KAFKA_PREFIX = CONFIG_PREFIX + "kafka."; + public static final String KAFKA_TOPIC_PREFIX = KAFKA_PREFIX + "topic."; /** * persisted.data.persist.method @@ -33,7 +37,7 @@ public final class PersistedDataConfig { /** * persisted.data.kafka.topic.name */ - public static final String KAFKA_TOPIC_NAME_CONFIG = CONFIG_PREFIX + "kafka.topic.name"; + public static final String KAFKA_TOPIC_NAME_CONFIG = KAFKA_TOPIC_PREFIX + "name"; public static final String DEFAULT_KAFKA_TOPIC_NAME = "__CruiseControlPersistentData"; public static final String KAFKA_TOPIC_NAME_DOC = "The name of the kafka topic to use to persist data when " + "\"" @@ -44,8 +48,8 @@ public final class PersistedDataConfig { * persisted.data.kafka.topic.partition.count */ public static final String KAFKA_TOPIC_PARTITION_COUNT_CONFIG = - CONFIG_PREFIX + "kafka.topic.partition.count"; - public static final int DEFAULT_KAFKA_TOPIC_PARTITION_COUNT = 2; + KAFKA_TOPIC_PREFIX + "partition.count"; + public static final int DEFAULT_KAFKA_TOPIC_PARTITION_COUNT = 1; public static final String KAFKA_TOPIC_PARTITION_COUNT_DOC = "The number of partitions to ensure are present " + "for the kafka topic. Only applies when \"" + PERSIST_METHOD_CONFIG @@ -56,54 +60,48 @@ public final class PersistedDataConfig { * persisted.data.kafka.topic.replication.factor */ public static final String KAFKA_TOPIC_REPLICATION_FACTOR_CONFIG = - CONFIG_PREFIX + "kafka.topic.replication.factor"; - public static final short DEFAULT_KAFKA_TOPIC_REPLICATION_FACTOR = 2; + KAFKA_TOPIC_PREFIX + "replication.factor"; + public static final short DEFAULT_KAFKA_TOPIC_REPLICATION_FACTOR = 3; public static final String KAFKA_TOPIC_REPLICATION_FACTOR_DOC = "The replication factor to use for the kafka " + "topic. Only applies when \"" + PERSIST_METHOD_CONFIG + "\" is set to \"kafka\". Multiple partition " + "replicas are desirable to ensure the topic is reasonably available."; /** - * persisted.data.kafka.topic.additional.configs.map + * persisted.data.kafka.topic.config.* + *
      + * The configs to apply to the kafka topic used to persist Cruise Control data. Only applies if + * "persisted.data.persist.method=kafka". All valid topic config keys and values can be used to + * apply to the "persisted.data.kafka.topic.name" topic. Use this prefix string with the topic + * config keys. e.g. "persisted.data.kafka.topic.config.min.insync.replicas=2" See: Kafka topic config + * documentation. */ - public static final String KAFKA_TOPIC_ADDITIONAL_CONFIGS_MAP_CONFIG = - CONFIG_PREFIX + "kafka.topic.additional.configs.map"; - public static final String DEFAULT_KAFKA_TOPIC_ADDITIONAL_CONFIGS_MAP = ""; - public static final String KAFKA_TOPIC_ADDITIONAL_CONFIGS_MAP_DOC = - "The configs to apply to the kafka topic used to " - + "persist Cruise Control data. Only applies if \"" + PERSIST_METHOD_CONFIG - + "\" is set to \"kafka\". This \"list\" should be a semicolon separated " - + "string of 'key=value' pairs. The keys and values need to be valid Kafka " - + "Topic configs. See: " - + "https://kafka.apache.org/documentation/#topicconfigs"; + public static final String KAFKA_TOPIC_CONFIG_PREFIX = KAFKA_TOPIC_PREFIX + "config."; /** - * persisted.data.kafka.producer.additional.configs.map + * persisted.data.kafka.producer.config. + *
      + * The configs to use when creating the kafka producer to persist Cruise Control data. Only + * applies if "persisted.data.persist.method=kafka". The keys and values need to be valid Kafka + * Producer configs. All valid Kafka producer config keys + * can be prefixed, and they will be applied to the producer. e.g. + * "persisted.data.kafka.producer.config.compression.type=gzip" */ - public static final String KAFKA_PRODUCER_ADDITIONAL_CONFIGS_MAP_CONFIG = - CONFIG_PREFIX + "kafka.producer.additional.configs.map"; - public static final String DEFAULT_KAFKA_PRODUCER_ADDITIONAL_CONFIGS_MAP = ""; - public static final String KAFKA_PRODUCER_ADDITIONAL_CONFIGS_MAP_DOC = - "The additional configs to use when creating the kafka " - + "producer to persist Cruise Control data. Only applies if \"" - + PERSIST_METHOD_CONFIG + "\" is set to \"kafka\". This \"list\" should be a " - + "semicolon separated string of 'key=value' pairs. The keys and values need " - + "to be valid Kafka Producer configs. See: " - + "https://kafka.apache.org/documentation/#producerconfigs"; + public static final String KAFKA_PRODUCER_CONFIG_PREFIX = KAFKA_PREFIX + "producer.config."; /** - * persisted.data.kafka.consumer.additional.configs.map + * persisted.data.kafka.consumer.config.prefix. + *
      + * The configs to use when creating the kafka consumer to read persisted Cruise Control data. + * Only applies if "persisted.data.persist.method=kafka". The keys and values need to be valid + * Kafka Consumer configs. All valid Kafka consumer config keys + * can be prefixed, and they will be applied to the consumer. e.g. + * "persisted.data.kafka.consumer.config.max.poll.records=100" */ - public static final String KAFKA_CONSUMER_ADDITIONAL_CONFIGS_MAP_CONFIG = - CONFIG_PREFIX + "kafka.consumer.additional.configs.map"; - public static final String DEFAULT_KAFKA_CONSUMER_ADDITIONAL_CONFIGS_MAP = ""; - public static final String KAFKA_CONSUMER_ADDITIONAL_CONFIGS_MAP_DOC = - "The additional configs to use when creating the kafka " - + "consumer to read persisted Cruise Control data. Only applies if \"" - + PERSIST_METHOD_CONFIG + "\" is set to \"kafka\". This \"list\" should be a " - + "semicolon separated string of 'key=value' pairs. The keys and values need " - + "to be valid Kafka Consumer configs. See: " - + "https://kafka.apache.org/documentation/#consumerconfigs"; + public static final String KAFKA_CONSUMER_CONFIG_PREFIX = KAFKA_PREFIX + "consumer.config."; private PersistedDataConfig() { } @@ -115,6 +113,10 @@ private PersistedDataConfig() { * @return The given ConfigDef after defining the common configs. */ public static ConfigDef define(ConfigDef configDef) { + configDef.embed(KAFKA_TOPIC_CONFIG_PREFIX, "", -1, TopicConfigDef.configDef()); + configDef.embed(KAFKA_PRODUCER_CONFIG_PREFIX, "", -1, ProducerConfig.configDef()); + configDef.embed(KAFKA_CONSUMER_CONFIG_PREFIX, "", -1, ConsumerConfig.configDef()); + return configDef .define(PERSIST_METHOD_CONFIG, ConfigDef.Type.STRING, @@ -137,21 +139,6 @@ public static ConfigDef define(ConfigDef configDef) { DEFAULT_KAFKA_TOPIC_REPLICATION_FACTOR, atLeast(1), ConfigDef.Importance.MEDIUM, - KAFKA_TOPIC_REPLICATION_FACTOR_DOC) - .define(KAFKA_TOPIC_ADDITIONAL_CONFIGS_MAP_CONFIG, - ConfigDef.Type.STRING, - DEFAULT_KAFKA_TOPIC_ADDITIONAL_CONFIGS_MAP, - ConfigDef.Importance.LOW, - KAFKA_TOPIC_ADDITIONAL_CONFIGS_MAP_DOC) - .define(KAFKA_PRODUCER_ADDITIONAL_CONFIGS_MAP_CONFIG, - ConfigDef.Type.STRING, - DEFAULT_KAFKA_PRODUCER_ADDITIONAL_CONFIGS_MAP, - ConfigDef.Importance.LOW, - KAFKA_PRODUCER_ADDITIONAL_CONFIGS_MAP_DOC) - .define(KAFKA_CONSUMER_ADDITIONAL_CONFIGS_MAP_CONFIG, - ConfigDef.Type.STRING, - DEFAULT_KAFKA_CONSUMER_ADDITIONAL_CONFIGS_MAP, - ConfigDef.Importance.LOW, - KAFKA_CONSUMER_ADDITIONAL_CONFIGS_MAP_DOC); + KAFKA_TOPIC_REPLICATION_FACTOR_DOC); } } diff --git a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/config/constants/TopicConfigDef.java b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/config/constants/TopicConfigDef.java new file mode 100644 index 0000000000..4e2811d869 --- /dev/null +++ b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/config/constants/TopicConfigDef.java @@ -0,0 +1,232 @@ +/* + * Copyright 2024 LinkedIn Corp. Licensed under the BSD 2-Clause License (the "License"). See License in the project root for license information. + */ + +package com.linkedin.kafka.cruisecontrol.config.constants; + +import kafka.log.Defaults; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigDef.Importance; +import org.apache.kafka.common.config.ConfigDef.Type; +import org.apache.kafka.common.config.TopicConfig; + +/** + * Generates the ConfigDef instance for the TopicConfig class provided by Kafka. It may only be + * needed for Kafka versions < 3.5 since there is some refactoring wrt LogConfig after that. The + * following code was generated using the following prompt for ChatGPT: + * + *
      + *     The following class file exists, named ProducerConfig.java:
      + *     ```java
      + *      ... contents of org/apache/kafka/clients/producer/ProducerConfig.java ...
      + *     ```
      + *
      + *     It defines `_CONFIG` and `_DOC` String values, then initializes a `ConfigDef`
      + *     instance starting with the `CONFIG = new ConfigDef().define(...` code. I can
      + *     then get a copy of the `CONFIG` variable using the `configDef()` static method.
      + *
      + *     I also have the `TopicConfig.java` file which contains the following:
      + *     ```java
      + *      ... contents of org/apache/kafka/common/config/TopicConfig.java ...
      + *     ```
      + *
      + *     Default values can be referenced from the `LogConfig.scala` file which contains the
      + *     following:
      + *     ```scala
      + *      ... contents of org/apache/kafka/log/LogConfig.scala...
      + *     ```
      + *
      + *     I want to create a new class named `TopicConfigDef` which uses the `_CONFIG` and `_DOC`
      + *     variables defined in `TopicConfig` to initialize a `ConfigDef` instance. The `ConfigDef`
      + *     instance should be created and initialized in a private static method, so the deprecation
      + *     warning suppression annotation can be applied to it, rather than in a `static`
      + *     initialization block. Do not include a `static` initialization block, but instead assign to
      + *     the `CONFIG` member variable. It must also have a `configDef()` static method to get a copy
      + *     of the statically created `ConfigDef` instance. Each of the calls to the `define()` method
      + *     should include an argument to the `defaultValue` parameter. Reference the correct value from
      + *     the `kafka.log.Defaults` object as though it is a method call. e.g.
      + *     `kafka.log.Defaults.SegmentSize()`
      + *  
      + * + * See Kafka topic configuration + * documentation for more details. + */ +public final class TopicConfigDef { + + private static final ConfigDef CONFIG = initConfig(); + + private TopicConfigDef() { + } + + @SuppressWarnings("deprecation") + private static ConfigDef initConfig() { + + return new ConfigDef() + .define( + TopicConfig.CLEANUP_POLICY_CONFIG, + Type.STRING, + Defaults.CleanupPolicy(), + Importance.HIGH, + TopicConfig.CLEANUP_POLICY_DOC) + .define( + TopicConfig.COMPRESSION_TYPE_CONFIG, + Type.STRING, + Defaults.CompressionType(), + Importance.MEDIUM, + TopicConfig.COMPRESSION_TYPE_DOC) + .define( + TopicConfig.DELETE_RETENTION_MS_CONFIG, + Type.LONG, + Defaults.DeleteRetentionMs(), + Importance.LOW, + TopicConfig.DELETE_RETENTION_MS_DOC) + .define( + TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, + Type.LONG, + Defaults.FileDeleteDelayMs(), + Importance.LOW, + TopicConfig.FILE_DELETE_DELAY_MS_DOC) + .define( + TopicConfig.FLUSH_MESSAGES_INTERVAL_CONFIG, + Type.LONG, + Defaults.FlushInterval(), + Importance.LOW, + TopicConfig.FLUSH_MESSAGES_INTERVAL_DOC) + .define( + TopicConfig.FLUSH_MS_CONFIG, + Type.LONG, + Defaults.FlushMs(), + Importance.LOW, + TopicConfig.FLUSH_MS_DOC) + .define( + TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, + Type.INT, + Defaults.IndexInterval(), + Importance.MEDIUM, + TopicConfig.INDEX_INTERVAL_BYTES_DOCS) + .define( + TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, + Type.LONG, + Defaults.LocalRetentionBytes(), + Importance.MEDIUM, + TopicConfig.LOCAL_LOG_RETENTION_BYTES_DOC) + .define( + TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, + Type.LONG, + Defaults.LocalRetentionMs(), + Importance.MEDIUM, + TopicConfig.LOCAL_LOG_RETENTION_MS_DOC) + .define( + TopicConfig.MAX_COMPACTION_LAG_MS_CONFIG, + Type.LONG, + Defaults.MaxCompactionLagMs(), + Importance.LOW, + TopicConfig.MAX_COMPACTION_LAG_MS_DOC) + .define( + TopicConfig.MAX_MESSAGE_BYTES_CONFIG, + Type.INT, + Defaults.MaxMessageSize(), + Importance.MEDIUM, + TopicConfig.MAX_MESSAGE_BYTES_DOC) + .define( + TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_CONFIG, + Type.BOOLEAN, + Defaults.MessageDownConversionEnable(), + Importance.MEDIUM, + TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_DOC) + .define( + TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, + Type.STRING, + Defaults.MessageFormatVersion(), + Importance.MEDIUM, + TopicConfig.MESSAGE_FORMAT_VERSION_DOC) + .define( + TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, + Type.LONG, + Defaults.MessageTimestampDifferenceMaxMs(), + Importance.MEDIUM, + TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_DOC) + .define( + TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG, + Type.STRING, + Defaults.MessageTimestampType(), + Importance.MEDIUM, + TopicConfig.MESSAGE_TIMESTAMP_TYPE_DOC) + .define( + TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG, + Type.DOUBLE, + Defaults.MinCleanableDirtyRatio(), + Importance.LOW, + TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_DOC) + .define( + TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG, + Type.LONG, + Defaults.MinCompactionLagMs(), + Importance.LOW, + TopicConfig.MIN_COMPACTION_LAG_MS_DOC) + .define( + TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, + Type.INT, + Defaults.MinInSyncReplicas(), + Importance.HIGH, + TopicConfig.MIN_IN_SYNC_REPLICAS_DOC) + .define( + TopicConfig.PREALLOCATE_CONFIG, + Type.BOOLEAN, + Defaults.PreAllocateEnable(), + Importance.LOW, + TopicConfig.PREALLOCATE_DOC) + .define( + TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, + Type.BOOLEAN, + Defaults.RemoteLogStorageEnable(), + Importance.MEDIUM, + TopicConfig.REMOTE_LOG_STORAGE_ENABLE_DOC) + .define( + TopicConfig.RETENTION_BYTES_CONFIG, + Type.LONG, + Defaults.RetentionSize(), + Importance.HIGH, + TopicConfig.RETENTION_BYTES_DOC) + .define( + TopicConfig.RETENTION_MS_CONFIG, + Type.LONG, + Defaults.RetentionMs(), + Importance.HIGH, + TopicConfig.RETENTION_MS_DOC) + .define( + TopicConfig.SEGMENT_BYTES_CONFIG, + Type.INT, + Defaults.SegmentSize(), + Importance.HIGH, + TopicConfig.SEGMENT_BYTES_DOC) + .define( + TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, + Type.INT, + Defaults.MaxIndexSize(), + Importance.MEDIUM, + TopicConfig.SEGMENT_INDEX_BYTES_DOC) + .define( + TopicConfig.SEGMENT_JITTER_MS_CONFIG, + Type.LONG, + Defaults.SegmentJitterMs(), + Importance.MEDIUM, + TopicConfig.SEGMENT_JITTER_MS_DOC) + .define( + TopicConfig.SEGMENT_MS_CONFIG, + Type.LONG, + Defaults.SegmentMs(), + Importance.HIGH, + TopicConfig.SEGMENT_MS_DOC) + .define( + TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, + Type.BOOLEAN, + Defaults.UncleanLeaderElectionEnable(), + Importance.MEDIUM, + TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_DOC); + } + + public static ConfigDef configDef() { + return new ConfigDef(CONFIG); + } +} diff --git a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistedMapFactory.java b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistedMapFactory.java index c161654fd4..d18821cec5 100644 --- a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistedMapFactory.java +++ b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistedMapFactory.java @@ -15,10 +15,9 @@ import static com.linkedin.kafka.cruisecontrol.persisteddata.PersistMethod.MEMORY; /** - * Constructs the correct {@link PersistedMap} implementation based on the provided - * {@link KafkaCruiseControlConfig}. In particular, it uses the value of - * {@link PersistedDataConfig#PERSIST_METHOD_CONFIG} to determine which implementation to - * construct. + * Constructs the correct {@link PersistedMap} implementation based on the provided {@link + * KafkaCruiseControlConfig}. In particular, it uses the value of {@link + * PersistedDataConfig#PERSIST_METHOD_CONFIG} to determine which implementation to construct. */ public class PersistedMapFactory { @@ -32,12 +31,13 @@ public class PersistedMapFactory { * Creates an instance that is able to construct the correct {@link PersistedMap} * implementation. * - * @param config The complete program configuration to evaluate. Specifically, the - * {@link PersistedDataConfig#PERSIST_METHOD_CONFIG} config is needed. + * @param config The complete program configuration to evaluate. Specifically, the {@link + * PersistedDataConfig#PERSIST_METHOD_CONFIG} config is needed. * @param adminClient The admin client to pass to {@link KafkaPersistedMap}, if needed. */ public PersistedMapFactory(KafkaCruiseControlConfig config, AdminClient adminClient) { - this(config, + this( + config, () -> new KafkaPersistedMap(config, adminClient), () -> new PersistedMap(new ConcurrentHashMap<>())); } @@ -47,26 +47,35 @@ public PersistedMapFactory(KafkaCruiseControlConfig config, AdminClient adminCli * implementation. * * @param kafkaSupplier The supplier for {@link KafkaPersistedMap}. - * @param memoryAndDefaultSupplier The supplier for {@link PersistedMap} and the default - * implementation. + * @param memorySupplier The supplier for {@link PersistedMap}. */ - PersistedMapFactory(KafkaCruiseControlConfig config, Supplier kafkaSupplier, - Supplier memoryAndDefaultSupplier) { + PersistedMapFactory( + KafkaCruiseControlConfig config, + Supplier kafkaSupplier, + Supplier memorySupplier) { this._config = config; - this._suppliers = Map.of( - KAFKA, kafkaSupplier, - MEMORY, memoryAndDefaultSupplier); + this._suppliers = + Map.of( + KAFKA, kafkaSupplier, + MEMORY, memorySupplier); } /** - * Constructs the correct {@link PersistedMap} implementation based on the configured - * {@link PersistedDataConfig#PERSIST_METHOD_CONFIG}. + * Constructs the correct {@link PersistedMap} implementation based on the configured {@link + * PersistedDataConfig#PERSIST_METHOD_CONFIG}. The {@link + * PersistedDataConfig#DEFAULT_PERSIST_METHOD} is used if PERSIST_METHOD_CONFIG is + * not specified. * * @return The {@link PersistedMap} implementation. */ public PersistedMap instance() { - PersistMethod backingMethod = PersistMethod.fromString( - _config.getString(PersistedDataConfig.PERSIST_METHOD_CONFIG)); - return this._suppliers.getOrDefault(backingMethod, this._suppliers.get(MEMORY)).get(); + PersistMethod backingMethod = + PersistMethod.fromString( + _config.getString(PersistedDataConfig.PERSIST_METHOD_CONFIG)); + PersistMethod defaultBackingMethod = + PersistMethod.fromString(PersistedDataConfig.DEFAULT_PERSIST_METHOD); + return this._suppliers + .getOrDefault(backingMethod, this._suppliers.get(defaultBackingMethod)) + .get(); } } diff --git a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/kafka/KafkaPersistedMap.java b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/kafka/KafkaPersistedMap.java index 64e8f9d863..56afd98f2b 100644 --- a/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/kafka/KafkaPersistedMap.java +++ b/cruise-control/src/main/java/com/linkedin/kafka/cruisecontrol/persisteddata/kafka/KafkaPersistedMap.java @@ -5,7 +5,6 @@ import com.google.common.base.Suppliers; -import com.google.common.collect.ImmutableMap; import com.linkedin.kafka.cruisecontrol.KafkaCruiseControlUtils; import com.linkedin.kafka.cruisecontrol.config.KafkaCruiseControlConfig; import com.linkedin.kafka.cruisecontrol.config.constants.MonitorConfig; @@ -27,17 +26,12 @@ import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.clients.consumer.Consumer; -import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.Producer; -import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.RecordMetadata; -import org.apache.kafka.common.config.TopicConfig; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -49,50 +43,6 @@ public class KafkaPersistedMap extends PersistedMap implements Closeable { private static final Logger LOG = LoggerFactory.getLogger(KafkaPersistedMap.class); - // Keys and values are stored as strings within the kafka topic. - private static final Class SERIALIZER_CLASS = StringSerializer.class; - private static final Class DESERIALIZER_CLASS = StringDeserializer.class; - - // The hard-coded producer config. This is overridable. - private static final Map DEFAULT_PRODUCER_CONFIG = ImmutableMap.builder() - .put(ProducerConfig.ACKS_CONFIG, "all") - // 2MB - .put(ProducerConfig.BUFFER_MEMORY_CONFIG, "2000000") - .put(ProducerConfig.CLIENT_ID_CONFIG, - "kafka-cruise-control.kafka-persisted-map.producer") - .put(ProducerConfig.COMPRESSION_TYPE_CONFIG, "gzip") - // 1 hour - .put(ProducerConfig.DELIVERY_TIMEOUT_MS_CONFIG, "3600000") - .put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, SERIALIZER_CLASS.getName()) - .put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, "2") - // 1MB - .put(ProducerConfig.MAX_REQUEST_SIZE_CONFIG, "1000000") - .put(ProducerConfig.RECONNECT_BACKOFF_MS_CONFIG, "1000") - .put(ProducerConfig.RETRY_BACKOFF_MS_CONFIG, "1000") - .put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, SERIALIZER_CLASS.getName()) - .build(); - - // The hard-coded consumer config. This is overridable. - private static final Map DEFAULT_CONSUMER_CONFIG = ImmutableMap.builder() - .put(ConsumerConfig.ALLOW_AUTO_CREATE_TOPICS_CONFIG, "false") - .put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") - .put(ConsumerConfig.CLIENT_ID_CONFIG, - "kafka-cruise-control.kafka-persisted-map.consumer") - .put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") - .put(ConsumerConfig.EXCLUDE_INTERNAL_TOPICS_CONFIG, "false") - .put(ConsumerConfig.GROUP_ID_CONFIG, - "kafka-cruise-control.kafka-persisted-map.consumer-group") - .put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed") - .put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, DESERIALIZER_CLASS.getName()) - .put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, Integer.toString(Integer.MAX_VALUE)) - .put(ConsumerConfig.RECONNECT_BACKOFF_MS_CONFIG, "1000") - .put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, DESERIALIZER_CLASS.getName()) - .build(); - - // The configuration which must be set on the data storage kafka topic. - private static final Map REQUIRED_TOPIC_CONFIG = Map.of( - TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT); - // The configured data storage kafka topic name. private final String _topic; @@ -103,8 +53,8 @@ public class KafkaPersistedMap extends PersistedMap implements Closeable { // The configured number of brokers should host each partition of the data storage kafka topic. private final short _topicReplicationFactor; - // Additional topic configuration configured by an administrator. - private final Map _topicAdditionalConfigs; + // Topic configuration configured by an administrator. + private final Map _topicConfig; // The producer instance to send data updates to kafka. private final Supplier> _producer; @@ -127,9 +77,9 @@ public KafkaPersistedMap(KafkaCruiseControlConfig config, AdminClient adminClien this(config.getString(PersistedDataConfig.KAFKA_TOPIC_NAME_CONFIG), config.getInt(PersistedDataConfig.KAFKA_TOPIC_PARTITION_COUNT_CONFIG), config.getShort(PersistedDataConfig.KAFKA_TOPIC_REPLICATION_FACTOR_CONFIG), - config.getMap(PersistedDataConfig.KAFKA_TOPIC_ADDITIONAL_CONFIGS_MAP_CONFIG), - config.getMap(PersistedDataConfig.KAFKA_PRODUCER_ADDITIONAL_CONFIGS_MAP_CONFIG), - config.getMap(PersistedDataConfig.KAFKA_CONSUMER_ADDITIONAL_CONFIGS_MAP_CONFIG), + config.originalsWithPrefix(PersistedDataConfig.KAFKA_TOPIC_CONFIG_PREFIX), + config.originalsWithPrefix(PersistedDataConfig.KAFKA_PRODUCER_CONFIG_PREFIX), + config.originalsWithPrefix(PersistedDataConfig.KAFKA_CONSUMER_CONFIG_PREFIX), KafkaCruiseControlUtils.maybeAddSecurityConfig(config, new HashMap<>()), String.join(",", config.getList(MonitorConfig.BOOTSTRAP_SERVERS_CONFIG)), adminClient); @@ -142,9 +92,9 @@ public KafkaPersistedMap(KafkaCruiseControlConfig config, AdminClient adminClien * @param topic The topic name to use and to ensure exists. * @param topicPartitions The number of partitions to ensure the topic has. * @param topicReplicationFactor Number of partition replicas to use for the topic. - * @param topicAdditionalConfigs The additional topic configuration to apply to the topic. - * @param producerAdditionalConfigs The additional producer configuration to use. - * @param consumerAdditionalConfigs The additional consumer configuration to use. + * @param topicConfigs The configuration to apply to the kafka topic. + * @param producerConfigs The producer configuration to use. + * @param consumerConfigs The consumer configuration to use. * @param commonSecurityConfigs The security configuration to use for the producer and consumer. * e.g. client SSL options. * @param bootstrapServers bootstrap.servers configuration to use for the producer and @@ -154,18 +104,17 @@ public KafkaPersistedMap(KafkaCruiseControlConfig config, AdminClient adminClien public KafkaPersistedMap(String topic, int topicPartitions, short topicReplicationFactor, - Map topicAdditionalConfigs, - Map producerAdditionalConfigs, - Map consumerAdditionalConfigs, + Map topicConfigs, + Map producerConfigs, + Map consumerConfigs, Map commonSecurityConfigs, String bootstrapServers, AdminClient adminClient) { this(new ConcurrentHashMap<>(), topic, topicPartitions, topicReplicationFactor, - topicAdditionalConfigs, + topicConfigs, newTopic -> KafkaCruiseControlUtils.maybeCreateOrUpdateTopic(adminClient, newTopic), - () -> createKafkaProducer(bootstrapServers, producerAdditionalConfigs, - commonSecurityConfigs), - () -> createKafkaConsumer(bootstrapServers, consumerAdditionalConfigs, + () -> createKafkaProducer(bootstrapServers, producerConfigs, commonSecurityConfigs), + () -> createKafkaConsumer(bootstrapServers, consumerConfigs, commonSecurityConfigs)); } @@ -176,7 +125,7 @@ public KafkaPersistedMap(String topic, String topic, int topicPartitions, short topicReplicationFactor, - Map topicAdditionalConfigs, + Map topicConfigs, java.util.function.Consumer topicCreator, Supplier> producer, Supplier> consumerFactory) { @@ -184,7 +133,7 @@ public KafkaPersistedMap(String topic, this._topic = topic; this._topicPartitions = topicPartitions; this._topicReplicationFactor = topicReplicationFactor; - this._topicAdditionalConfigs = topicAdditionalConfigs; + this._topicConfig = topicConfigs; ensureTopicIsPresentAndConfigured(topicCreator); // This odd setup is to lazy-load the producer and also adapt the java Supplier to a @@ -209,26 +158,25 @@ Thread getCacheUpdater() { /** * Configures and creates the requested Kafka client instance. Package private for testing. * + * @param The type of Kafka client to return. * @param bootstrapServers The configured {@code bootstrap.servers} config to use. - * @param defaultConfig The default config values to configure the client with. * @param securityConfig The security options the client should use when connecting to the Kafka * cluster. - * @param additionalConfig Any additional config to override the default and security config - * with. + * @param clientConfig Any config to use. This will override any keys provided in + * securityConfig. * @param clientFactory Function that takes a config map and returns the Kafka client instance. - * @param The type of Kafka client to return. * @return A fully configured Kafka client instance. */ - static C createKafkaClient(String bootstrapServers, Map defaultConfig, - Map securityConfig, Map additionalConfig, + static C createKafkaClient(String bootstrapServers, + Map securityConfig, Map clientConfig, Function, C> clientFactory) { - // Configure the client by combining the default, security and additional config. - Map config = mergeConfig(defaultConfig, securityConfig); + // Configure the client by combining the security and client config. + Map config = new HashMap<>(securityConfig); config.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); - config.putAll(additionalConfig); - LOG.debug("KafkaPersistedMap.createKafkaClient(bootstrapServers={}, defaultConfig={}, " - + "securityConfig={}, additionalConfig={}, clientFactory={})", bootstrapServers, - defaultConfig, securityConfig, additionalConfig, clientFactory); + config.putAll(clientConfig); + LOG.debug("KafkaPersistedMap.createKafkaClient(bootstrapServers={}, securityConfig={}, " + + "clientConfig={}, clientFactory={})", bootstrapServers, + securityConfig, clientConfig, clientFactory); return clientFactory.apply(config); } @@ -236,49 +184,28 @@ static C createKafkaClient(String bootstrapServers, Map defaultCo * Configure and create the KafkaConsumer instance. * * @param bootstrapServers The configured {@code bootstrap.servers} config to use. - * @param consumerAdditionalConfigs Any additional {@link KafkaConsumer} configs to use to - * override the default consumer config. - * @param commonSecurityConfigs The kafka client security configuration options. + * @param consumerConfig Any {@link KafkaConsumer} configs to use to configure the consumer. + * @param securityConfig The kafka client security configuration options. * @return The {@link Consumer} instance. */ private static Consumer createKafkaConsumer(String bootstrapServers, - Map consumerAdditionalConfigs, - Map commonSecurityConfigs) { - return createKafkaClient(bootstrapServers, DEFAULT_CONSUMER_CONFIG, commonSecurityConfigs, - consumerAdditionalConfigs, KafkaConsumer::new); + Map consumerConfig, Map securityConfig) { + return createKafkaClient(bootstrapServers, securityConfig, consumerConfig, + KafkaConsumer::new); } /** * Configure and create the KafkaProducer instance. * * @param bootstrapServers The configured {@code bootstrap.servers} config to use. - * @param producerAdditionalConfigs Any additional {@link KafkaProducer} configs to use to - * override the default producer config. - * @param commonSecurityConfigs The kafka client security configuration options. + * @param producerConfig Any {@link KafkaProducer} configs to use to configure the producer. + * @param securityConfig The kafka client security configuration options. * @return The {@link Producer} instance. */ private static Producer createKafkaProducer(String bootstrapServers, - Map producerAdditionalConfigs, - Map commonSecurityConfigs) { - return createKafkaClient(bootstrapServers, DEFAULT_PRODUCER_CONFIG, commonSecurityConfigs, - producerAdditionalConfigs, KafkaProducer::new); - } - - /** - * Merge the two maps into a new one overwriting any default values with those from overrides. - * - * @param defaults The base key/values to overwrite if necessary. - * @param overrides Any key/values to merge over the values in defaults. - * @param The defaults map value type. - * @param The overrides map value type. - * @return The combined map of key/values. - */ - private static Map mergeConfig( - Map defaults, - Map overrides) { - final Map result = new HashMap<>(defaults); - result.putAll(overrides); - return result; + Map producerConfig, Map securityConfig) { + return createKafkaClient(bootstrapServers, securityConfig, producerConfig, + KafkaProducer::new); } /** @@ -289,8 +216,8 @@ private static Map mergeConfig */ private void ensureTopicIsPresentAndConfigured( java.util.function.Consumer topicCreator) { - Map config = new HashMap<>(this._topicAdditionalConfigs); - config.putAll(REQUIRED_TOPIC_CONFIG); + Map config = new HashMap<>(this._topicConfig.size()); + this._topicConfig.forEach((key, value) -> config.put(key, value.toString())); NewTopic topic = new NewTopic(this._topic, this._topicPartitions, this._topicReplicationFactor); topic.configs(config); diff --git a/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/config/KafkaCruiseControlConfigTest.java b/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/config/KafkaCruiseControlConfigTest.java deleted file mode 100644 index a4022c8ae4..0000000000 --- a/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/config/KafkaCruiseControlConfigTest.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * Copyright 2024 LinkedIn Corp. Licensed under the BSD 2-Clause License (the "License"). See License in the project root for license information. - */ -package com.linkedin.kafka.cruisecontrol.config; - -import java.util.Collections; -import java.util.Map; -import org.apache.kafka.common.config.ConfigDef; -import org.apache.kafka.common.config.ConfigException; -import org.junit.Test; - -import static org.hamcrest.CoreMatchers.is; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.junit.Assert.assertThrows; - -public class KafkaCruiseControlConfigTest { - - private static final String KEY = "key"; - - /** - * getMap() should parse valid values. - */ - @Test - public void getMapParsesValidValues() { - assertGetMapResultMatchesExpected("", Collections.emptyMap()); - assertGetMapResultMatchesExpected(" ", Collections.emptyMap()); - assertGetMapResultMatchesExpected(" ;", Collections.emptyMap()); - assertGetMapResultMatchesExpected("k=", Map.of("k", "")); - assertGetMapResultMatchesExpected(" k = ", Map.of("k", "")); - assertGetMapResultMatchesExpected("k=v", Map.of("k", "v")); - assertGetMapResultMatchesExpected(" k = v ", Map.of("k", "v")); - assertGetMapResultMatchesExpected("k1=v1;k2=v2a,v2b", Map.of("k1", "v1", "k2", "v2a,v2b")); - assertGetMapResultMatchesExpected(" k1 = v1 ; k2 = v2 ", Map.of("k1", "v1", "k2", "v2")); - } - - /** - * getMap() should reject invalid values. - */ - @Test - public void getMapThrowsExceptionForInvalidValues() { - assertGetMapThrowsConfigException("k"); - assertGetMapThrowsConfigException(" k "); - assertGetMapThrowsConfigException("="); - assertGetMapThrowsConfigException(" = "); - assertGetMapThrowsConfigException("=v"); - assertGetMapThrowsConfigException(" = v "); - } - - private void assertGetMapResultMatchesExpected(String rawValue, - Map expectedValue) { - final KafkaCruiseControlConfig config = new KafkaCruiseControlConfig( - PermissiveConfigDef.of(KEY), Map.of(KEY, rawValue)); - assertThat(config.getMap(KEY), is(expectedValue)); - } - - private void assertGetMapThrowsConfigException(String rawValue) { - final KafkaCruiseControlConfig config = new KafkaCruiseControlConfig( - PermissiveConfigDef.of(KEY), Map.of(KEY, rawValue)); - assertThrows(ConfigException.class, () -> config.getMap(KEY)); - } - - private static class PermissiveConfigDef extends ConfigDef { - - /** - * Create an instance with a single key of type STRING. - * @param key The config key name. - * @return The new instance, ready for use with a KafkaCruiseControlConfig instance. - */ - public static PermissiveConfigDef of(String key) { - final PermissiveConfigDef result = new PermissiveConfigDef(); - result.define(key, Type.STRING, Importance.LOW, ""); - return result; - } - - @SuppressWarnings("unchecked") - @Override - public Map parse(Map props) { - return (Map) props; - } - } -} diff --git a/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistedMapFactoryTest.java b/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistedMapFactoryTest.java index db243ed74e..60e37e8af9 100644 --- a/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistedMapFactoryTest.java +++ b/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/PersistedMapFactoryTest.java @@ -3,6 +3,8 @@ */ package com.linkedin.kafka.cruisecontrol.persisteddata; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.StringSerializer; import com.linkedin.kafka.cruisecontrol.config.KafkaCruiseControlConfig; import com.linkedin.kafka.cruisecontrol.config.constants.ExecutorConfig; import com.linkedin.kafka.cruisecontrol.config.constants.PersistedDataConfig; @@ -10,6 +12,8 @@ import java.util.HashMap; import java.util.Map; import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.ProducerConfig; import org.junit.Test; import static org.easymock.EasyMock.mock; @@ -17,17 +21,18 @@ import static org.hamcrest.core.Is.is; import static org.hamcrest.core.Is.isA; - public class PersistedMapFactoryTest { + public static final String SERIALIZER = StringSerializer.class.getName(); + public static final String DESERIALIZER = StringDeserializer.class.getName(); + /** * Ensure setting the backing method config results in the factory producing an instance of the * right type. */ @Test public void instanceReturnsKafkaPersistedMapWhenConfiguredForKafka() { - final PersistedMapFactory factory = configureAndGetPersistedMapFactory( - "kafka"); + final PersistedMapFactory factory = configureAndGetPersistedMapFactory("kafka"); PersistedMap map = factory.instance(); assertThat(map instanceof KafkaPersistedMap, is(true)); } @@ -48,7 +53,16 @@ private static PersistedMapFactory configureAndGetPersistedMapFactory(String bac KafkaCruiseControlConfig config = new KafkaCruiseControlConfig( Map.of(PersistedDataConfig.PERSIST_METHOD_CONFIG, backingMethod, CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "fake", - ExecutorConfig.ZOOKEEPER_CONNECT_CONFIG, "connect:1234")); + ExecutorConfig.ZOOKEEPER_CONNECT_CONFIG, "connect:1234", + PersistedDataConfig.KAFKA_PRODUCER_CONFIG_PREFIX + + ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, SERIALIZER, + PersistedDataConfig.KAFKA_PRODUCER_CONFIG_PREFIX + + ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, SERIALIZER, + PersistedDataConfig.KAFKA_CONSUMER_CONFIG_PREFIX + + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, DESERIALIZER, + PersistedDataConfig.KAFKA_CONSUMER_CONFIG_PREFIX + + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, DESERIALIZER + )); return new PersistedMapFactory(config, () -> mock(KafkaPersistedMap.class), () -> new PersistedMap(new HashMap<>())); diff --git a/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/kafka/KafkaPersistedMapTest.java b/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/kafka/KafkaPersistedMapTest.java index c6424f3bae..3c427767c9 100644 --- a/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/kafka/KafkaPersistedMapTest.java +++ b/cruise-control/src/test/java/com/linkedin/kafka/cruisecontrol/persisteddata/kafka/KafkaPersistedMapTest.java @@ -52,7 +52,7 @@ public class KafkaPersistedMapTest { private static final TopicPartition PARTITION_0 = new TopicPartition(TOPIC, 0); private static final CompletableFuture SUCCESSFUL_PRODUCER_SEND_RESPONSE = CompletableFuture.completedFuture( new RecordMetadata(PARTITION_0, 1, 0, System.currentTimeMillis(), 0, 0)); - private static final Map ADDITIONAL_CONFIGS = Map.of("max.message.bytes", + private static final Map TOPIC_CONFIGS = Map.of("max.message.bytes", "10000"); private static final String KEY_1 = "key1"; private static final String VALUE_1 = "value1"; @@ -79,7 +79,7 @@ private void initPersistedMap() { // Initialize the object under test. this._persistedMap = new KafkaPersistedMap(this._backingMap, TOPIC, NUM_PARTITIONS, - (short) 1, ADDITIONAL_CONFIGS, t -> { + (short) 1, TOPIC_CONFIGS, t -> { }, () -> this._mockProducer, () -> this._mockConsumer); } @@ -160,26 +160,21 @@ public K getKeyFromQueue() throws InterruptedException { @Test public void createKafkaClientAppliesAllTheConfig() { final String bootstrapServers = "bootstrapServers"; - final Map defaultConfig = Map.of( - KEY_1, VALUE_1, - KEY_2, VALUE_1, - KEY_3, VALUE_1); final Map securityConfig = Map.of( - KEY_2, VALUE_2, - KEY_3, VALUE_2); + KEY_1, VALUE_1, + KEY_2, VALUE_1); final Map additionalConfig = Map.of( - KEY_3, VALUE_3); + KEY_2, VALUE_2); final Function, Map> clientFactory = config -> { config.put("called", true); return config; }; assertThat(KafkaPersistedMap.createKafkaClient(bootstrapServers, - defaultConfig, securityConfig, additionalConfig, clientFactory), equalTo(Map.of( + securityConfig, additionalConfig, clientFactory), equalTo(Map.of( CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers, KEY_1, VALUE_1, KEY_2, VALUE_2, - KEY_3, VALUE_3, "called", true))); }