Cruise-control is the first of its kind to fully automate the dynamic workload rebalance and self-healing of a Kafka cluster. It provides great value to Kafka users by simplifying the operation of Kafka clusters.
Why: During the first startup of Kafka brokers and Cruise Control, it could happen that one of the broker is lagging behind, or the cluster is not ready (not enough broker is available) to create the metrics topic (cruise.control.metrics.topic.auto.create=true) with the desired guaranties (replica count). If we (startup script/automation) did not wait enough with the CC start command, then it could fail.
There is now retry logic in the CruiseControlMetricsReporterSampler initialization to handle this edge case.
2 things could happen depending on the Kafka's auto.create.topics.enable config
auto.create.topics.enable=false
CC startup fails with the following error after the unsuccessful refreshPartitionAssignment() as the topic does not exist:
java.lang.IllegalStateException: Cruise Control cannot find partitions for the metrics reporter that topic matches __CruiseControlMetrics in the target cluster.
at com.linkedin.kafka.cruisecontrol.monitor.sampling.CruiseControlMetricsReporterSampler.configure(CruiseControlMetricsReporterSampler.java:195) ~[classes/:?]
at com.linkedin.kafka.cruisecontrol.config.KafkaCruiseControlConfigUtils.getConfiguredInstance(KafkaCruiseControlConfigUtils.java:49) ~[classes/:?]
at com.linkedin.kafka.cruisecontrol.config.KafkaCruiseControlConfig.getConfiguredInstance(KafkaCruiseControlConfig.java:98) ~[classes/:?]
auto.create.topics.enable=true
During the refreshPartitionAssignment() the Kafka Consumer send a topicMetadata request to the brokers. This request triggers the auto topic creation mechanism in Kafka. Based on which broker actually creates the topic, the topic creation can be sync or async, meaning the refreshPartitionAssignment() could fail or succeed. In either case, the topic creation won't use the desired topic properties we defined in the CruiseControlMetricsReporter config cruise.control.metrics.topic.num.partitions, cruise.control.metrics.topic.replication.factor. It will use a default topic configuration silently.
If the refreshPartitionAssignment() succeed, we did not even notice anything.
If the refreshPartitionAssignment() fails, the CC won't start. However, the topic auto creation got triggered, and the brokers will create the topic. If we check the logs, we will see the error above, but we will have a hard time to figure out why the error happened since the topic exist in Kafka (with the default topic config). If we start the CC again, then it starts without any issue, and will use the topic created by the topic auto creation mechanism.
What:
Changed the Kafka consumer creation to disable the topic auto creation during the metadata request.
Introduced a mechanism to explicitly wait for the metrics topic to appear during the CruiseControlMetricsReporterSampler configuration.
Expected Behavior
Metrics topic got created with the defined configuration.
CC tolerate lagging, slow broker startup
Actual Behavior
Metrics topic got created with the default topic config
CC fails instantly if the topic does not exist during the startup
Summary
cruise.control.metrics.topic.auto.create=true
) with the desired guaranties (replica count). If we (startup script/automation) did not wait enough with the CC start command, then it could fail. There is now retry logic in theCruiseControlMetricsReporterSampler
initialization to handle this edge case. 2 things could happen depending on the Kafka'sauto.create.topics.enable
configauto.create.topics.enable=false
CC startup fails with the following error after the unsuccessful
refreshPartitionAssignment()
as the topic does not exist:auto.create.topics.enable=true
During the
refreshPartitionAssignment()
the Kafka Consumer send atopicMetadata
request to the brokers. This request triggers the auto topic creation mechanism in Kafka. Based on which broker actually creates the topic, the topic creation can be sync or async, meaning therefreshPartitionAssignment()
could fail or succeed. In either case, the topic creation won't use the desired topic properties we defined in theCruiseControlMetricsReporter
configcruise.control.metrics.topic.num.partitions
,cruise.control.metrics.topic.replication.factor
. It will use a default topic configuration silently. If therefreshPartitionAssignment()
succeed, we did not even notice anything. If therefreshPartitionAssignment()
fails, the CC won't start. However, the topic auto creation got triggered, and the brokers will create the topic. If we check the logs, we will see the error above, but we will have a hard time to figure out why the error happened since the topic exist in Kafka (with the default topic config). If we start the CC again, then it starts without any issue, and will use the topic created by the topic auto creation mechanism.CruiseControlMetricsReporterSampler
configuration.Expected Behavior
Metrics topic got created with the defined configuration. CC tolerate lagging, slow broker startup
Actual Behavior
Metrics topic got created with the default topic config CC fails instantly if the topic does not exist during the startup
Steps to Reproduce
Known Workarounds
No
Categorization