mz_kafka_util/
admin.rs

1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
// Copyright Materialize, Inc. and contributors. All rights reserved.
//
// Use of this software is governed by the Business Source License
// included in the LICENSE file.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0.

//! Helpers for working with Kafka's admin API.

use std::collections::BTreeMap;
use std::iter;
use std::time::Duration;

use anyhow::{anyhow, bail};
use itertools::Itertools;
use mz_ore::collections::CollectionExt;
use mz_ore::retry::Retry;
use mz_ore::str::separated;
use rdkafka::admin::{
    AdminClient, AdminOptions, AlterConfig, ConfigEntry, ConfigResource, ConfigSource, NewTopic,
    OwnedResourceSpecifier, ResourceSpecifier,
};
use rdkafka::client::ClientContext;
use rdkafka::error::{KafkaError, RDKafkaErrorCode};
use tracing::{info, warn};

/// Get the current configuration for a particular topic.
///
/// Materialize may not have permission to list configs for the topic, so callers of this method
/// should "fail open" if the configs are not available.
pub async fn get_topic_config<'a, C>(
    client: &'a AdminClient<C>,
    admin_opts: &AdminOptions,
    topic_name: &str,
) -> anyhow::Result<Vec<ConfigEntry>>
where
    C: ClientContext,
{
    let ConfigResource { specifier, entries } = client
        .describe_configs([&ResourceSpecifier::Topic(topic_name)], admin_opts)
        .await?
        .into_iter()
        .exactly_one()??;

    match specifier {
        OwnedResourceSpecifier::Topic(name) if name.as_str() == topic_name => {}
        unexpected => {
            bail!("describe configs returned unexpected resource specifier: {unexpected:?}")
        }
    };

    Ok(entries)
}

/// Alter the current configuration for a particular topic.
///
/// Materialize may not have permission to alter configs for the topic, so callers of this method
/// should "fail open" if the configs are not available.
pub async fn alter_topic_config<'a, C>(
    client: &'a AdminClient<C>,
    admin_opts: &AdminOptions,
    topic_name: &str,
    new_config: impl IntoIterator<Item = (&str, &str)>,
) -> anyhow::Result<()>
where
    C: ClientContext,
{
    let mut alter_config = AlterConfig::new(ResourceSpecifier::Topic(topic_name));
    for (key, val) in new_config {
        alter_config = alter_config.set(key, val);
    }
    let result = client
        .alter_configs([&alter_config], admin_opts)
        .await?
        .into_iter()
        .exactly_one()?;

    let (specifier, result) = match result {
        Ok(specifier) => (specifier, Ok(())),
        Err((specifier, err)) => (specifier, Err(KafkaError::AdminOp(err))),
    };

    match specifier {
        OwnedResourceSpecifier::Topic(name) if name.as_str() == topic_name => {}
        unexpected => {
            bail!("alter configs returned unexpected resource specifier: {unexpected:?}")
        }
    };

    Ok(result?)
}

/// When the topic we want to ensure already exists, what happens?
#[derive(Debug, Copy, Clone)]
pub enum EnsureTopicConfig {
    /// Do nothing: assume whatever config is there is fine.
    Skip,
    /// Check and log the existing config, but don't try and change it.
    Check,
    /// If the remote config doesn't match, try and change it to match.
    Alter,
}

/// Expect the current configuration for a particular topic to match a provided set of values.
/// If there are other configs set, we will ignore them.
///
/// Materialize may not have permission to alter configs for the topic, so callers of this method
/// should "fail open" if the configs are not available.
pub async fn ensure_topic_config<'a, C>(
    client: &'a AdminClient<C>,
    admin_opts: &AdminOptions,
    new_topic: &'a NewTopic<'a>,
    expect: EnsureTopicConfig,
) -> anyhow::Result<bool>
where
    C: ClientContext,
{
    let try_alter = match expect {
        EnsureTopicConfig::Skip => return Ok(true),
        EnsureTopicConfig::Check => false,
        EnsureTopicConfig::Alter => true,
    };

    let mut expected_configs: BTreeMap<_, _> = new_topic.config.iter().copied().collect();

    let actual_configs = get_topic_config(client, admin_opts, new_topic.name).await?;
    info!(
        topic = new_topic.name,
        "got configuration for existing topic: [{}]",
        separated(
            ", ",
            actual_configs.iter().map(|e| {
                let kv = [&*e.name, e.value.as_ref().map_or("<none>", |v| &*v)];
                separated(": ", kv)
            })
        )
    );

    let actual_config_values: BTreeMap<_, _> = actual_configs
        .iter()
        .filter_map(|e| e.value.as_ref().map(|v| (e.name.as_str(), v.as_str())))
        .collect();
    for (config, expected) in &expected_configs {
        match actual_config_values.get(config) {
            Some(actual) => {
                if actual != expected {
                    warn!(
                        topic = new_topic.name,
                        config, expected, actual, "unexpected value for config entry"
                    )
                }
            }
            None => {
                warn!(
                    topic = new_topic.name,
                    config, expected, "missing expected value for config entry"
                )
            }
        }
    }

    if try_alter {
        // rdkafka does not support the newer incremental-alter APIs. Instead, we copy over values
        // from the fetched dynamic config that we don't explicitly want to overwrite.
        for entry in &actual_configs {
            if entry.source != ConfigSource::DynamicTopic {
                continue;
            }
            let Some(value) = entry.value.as_ref() else {
                continue;
            };
            expected_configs.entry(&entry.name).or_insert(value);
        }
        alter_topic_config(client, admin_opts, new_topic.name, expected_configs).await?;
        Ok(true)
    } else {
        Ok(false)
    }
}

/// Creates a Kafka topic if it does not exist, and waits for it to be reported in the broker
/// metadata.
///
/// This function is a wrapper around [`AdminClient::create_topics`] that
/// attempts to ensure the topic creation has propagated throughout the Kafka
/// cluster before returning. Kafka topic creation is asynchronous, so
/// attempting to consume from or produce to a topic immediately after its
/// creation can result in "unknown topic" errors.
///
/// This function does not return successfully unless it can find the metadata
/// for the newly-created topic in a call to [`rdkafka::client::Client::fetch_metadata`] and
/// verify that the metadata reports the topic has the number of partitions
/// requested in `new_topic`. Empirically, this seems to be the condition that
/// guarantees that future attempts to consume from or produce to the topic will
/// succeed.
///
/// Returns a boolean indicating whether the topic already existed.
pub async fn ensure_topic<'a, C>(
    client: &'a AdminClient<C>,
    admin_opts: &AdminOptions,
    new_topic: &'a NewTopic<'a>,
    on_existing: EnsureTopicConfig,
) -> anyhow::Result<bool>
where
    C: ClientContext,
{
    let res = client
        .create_topics(iter::once(new_topic), admin_opts)
        .await?;

    let already_exists = match res.as_slice() {
        &[Ok(_)] => false,
        &[Err((_, RDKafkaErrorCode::TopicAlreadyExists))] => true,
        &[Err((_, e))] => bail!(KafkaError::AdminOp(e)),
        other => bail!(
            "kafka topic creation returned {} results, but exactly one result was expected",
            other.len()
        ),
    };

    // We don't need to read in metadata / do any validation if the topic already exists.
    if already_exists {
        match ensure_topic_config(client, admin_opts, new_topic, on_existing).await {
            Ok(true) => {}
            Ok(false) => {
                info!(
                    topic = new_topic.name,
                    "did not sync topic config; configs may not match expected values"
                );
            }
            Err(error) => {
                warn!(
                    topic=new_topic.name,
                    "unable to enforce topic config; configs may not match expected values: {error:#}"
                )
            }
        }

        return Ok(true);
    }

    // Topic creation is asynchronous, and if we don't wait for it to complete,
    // we might produce a message (below) that causes it to get automatically
    // created with the default number partitions, and not the number of
    // partitions requested in `new_topic`.
    Retry::default()
        .max_duration(Duration::from_secs(30))
        .retry_async(|_| async {
            let metadata = client
                .inner()
                // N.B. It is extremely important not to ask specifically
                // about the topic here, even though the API supports it!
                // Asking about the topic will create it automatically...
                // with the wrong number of partitions. Yes, this is
                // unbelievably horrible.
                .fetch_metadata(None, Some(Duration::from_secs(10)))?;
            let topic = metadata
                .topics()
                .iter()
                .find(|t| t.name() == new_topic.name)
                .ok_or(anyhow!("unable to fetch topic metadata after creation"))?;
            // If the desired number of partitions is not "use the broker
            // default", wait for the topic to have the correct number of
            // partitions.
            if new_topic.num_partitions != -1 {
                let actual = i32::try_from(topic.partitions().len())?;
                if actual != new_topic.num_partitions {
                    bail!(
                        "topic reports {actual} partitions, but expected {} partitions",
                        new_topic.num_partitions
                    );
                }
            }
            Ok(false)
        })
        .await
}

/// Deletes a Kafka topic and waits for it to be reported absent in the broker metadata.
///
/// This function is a wrapper around [`AdminClient::delete_topics`] that attempts to ensure the
/// topic deletion has propagated throughout the Kafka cluster before returning.
///
/// This function does not return successfully unless it can observe the metadata not containing
/// the newly-created topic in a call to [`rdkafka::client::Client::fetch_metadata`]
pub async fn delete_existing_topic<'a, C>(
    client: &'a AdminClient<C>,
    admin_opts: &AdminOptions,
    topic: &'a str,
) -> Result<(), DeleteTopicError>
where
    C: ClientContext,
{
    delete_topic_helper(client, admin_opts, topic, false).await
}

/// Like `delete_existing_topic` but allow topic to be already deleted
pub async fn delete_topic<'a, C>(
    client: &'a AdminClient<C>,
    admin_opts: &AdminOptions,
    topic: &'a str,
) -> Result<(), DeleteTopicError>
where
    C: ClientContext,
{
    delete_topic_helper(client, admin_opts, topic, true).await
}

async fn delete_topic_helper<'a, C>(
    client: &'a AdminClient<C>,
    admin_opts: &AdminOptions,
    topic: &'a str,
    allow_missing: bool,
) -> Result<(), DeleteTopicError>
where
    C: ClientContext,
{
    let res = client.delete_topics(&[topic], admin_opts).await?;
    if res.len() != 1 {
        return Err(DeleteTopicError::TopicCountMismatch(res.len()));
    }
    let already_missing = match res.into_element() {
        Ok(_) => Ok(false),
        Err((_, RDKafkaErrorCode::UnknownTopic)) if allow_missing => Ok(true),
        Err((_, e)) => Err(DeleteTopicError::Kafka(KafkaError::AdminOp(e))),
    }?;

    // We don't need to read in metadata / do any validation if the topic already exists.
    if already_missing {
        return Ok(());
    }

    // Topic deletion is asynchronous, and if we don't wait for it to complete,
    // we might produce a message (below) that causes it to get automatically
    // created with the default number partitions, and not the number of
    // partitions requested in `new_topic`.
    Retry::default()
        .max_duration(Duration::from_secs(30))
        .retry_async(|_| async {
            let metadata = client
                .inner()
                // N.B. It is extremely important not to ask specifically
                // about the topic here, even though the API supports it!
                // Asking about the topic will create it automatically...
                // with the wrong number of partitions. Yes, this is
                // unbelievably horrible.
                .fetch_metadata(None, Some(Duration::from_secs(10)))?;
            let topic_exists = metadata.topics().iter().any(|t| t.name() == topic);
            if topic_exists {
                Err(DeleteTopicError::TopicRessurected)
            } else {
                Ok(())
            }
        })
        .await
}

/// An error while creating a Kafka topic.
#[derive(Debug, thiserror::Error)]
pub enum DeleteTopicError {
    /// An error from the underlying Kafka library.
    #[error(transparent)]
    Kafka(#[from] KafkaError),
    /// Topic creation returned the wrong number of results.
    #[error("kafka topic creation returned {0} results, but exactly one result was expected")]
    TopicCountMismatch(usize),
    /// The topic remained in metadata after being deleted.
    #[error("topic was recreated after being deleted")]
    TopicRessurected,
}