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
// 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::iter;
use std::time::Duration;

use mz_ore::collections::CollectionExt;
use mz_ore::retry::Retry;
use rdkafka::admin::{AdminClient, AdminOptions, NewTopic};
use rdkafka::client::ClientContext;
use rdkafka::error::{KafkaError, RDKafkaErrorCode};

/// Creates a Kafka topic 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 create_new_topic<'a, C>(
    client: &'a AdminClient<C>,
    admin_opts: &AdminOptions,
    new_topic: &'a NewTopic<'a>,
) -> Result<bool, CreateTopicError>
where
    C: ClientContext,
{
    create_topic_helper(client, admin_opts, new_topic, false).await
}

/// Like `create_new_topic` but allow topic to already exist
pub async fn ensure_topic<'a, C>(
    client: &'a AdminClient<C>,
    admin_opts: &AdminOptions,
    new_topic: &'a NewTopic<'a>,
) -> Result<bool, CreateTopicError>
where
    C: ClientContext,
{
    create_topic_helper(client, admin_opts, new_topic, true).await
}

async fn create_topic_helper<'a, C>(
    client: &'a AdminClient<C>,
    admin_opts: &AdminOptions,
    new_topic: &'a NewTopic<'a>,
    allow_existing: bool,
) -> Result<bool, CreateTopicError>
where
    C: ClientContext,
{
    let res = client
        .create_topics(iter::once(new_topic), admin_opts)
        .await?;
    if res.len() != 1 {
        return Err(CreateTopicError::TopicCountMismatch(res.len()));
    }
    let already_exists = match res.into_element() {
        Ok(_) => Ok(false),
        Err((_, RDKafkaErrorCode::TopicAlreadyExists)) if allow_existing => Ok(true),
        Err((_, e)) => Err(CreateTopicError::Kafka(KafkaError::AdminOp(e))),
    }?;

    // We don't need to read in metadata / do any validation if the topic already exists.
    if already_exists {
        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(CreateTopicError::MissingMetadata)?;
            // 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 num_partitions = i32::try_from(topic.partitions().len())
                    .map_err(|_| CreateTopicError::TooManyPartitions)?;
                if num_partitions != new_topic.num_partitions {
                    return Err(CreateTopicError::PartitionCountMismatch {
                        expected: new_topic.num_partitions,
                        actual: num_partitions,
                    });
                }
            }
            Ok(false)
        })
        .await
}

/// An error while creating a Kafka topic.
#[derive(Debug, thiserror::Error)]
pub enum CreateTopicError {
    /// 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 metadata could not be fetched after the topic was created.
    #[error("unable to fetch topic metadata after creation")]
    MissingMetadata,
    /// The topic reported more than the maximum allowable number of partitions.
    #[error("the topic reported more than {} partitions", i32::MAX)]
    TooManyPartitions,
    /// The topic metadata reported a number of partitions that did not match
    /// the number of partitions in the topic creation request.
    #[error("topic reports {actual} partitions, but expected {expected} partitions")]
    PartitionCountMismatch {
        /// The requested number of partitions.
        expected: i32,
        /// The reported number of partitions.
        actual: i32,
    },
}

/// 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,
}