Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

sql,storage: enable configurable group and transactional IDs for Kafka #23792

Merged
merged 5 commits into from
Dec 14, 2023

Conversation

benesch
Copy link
Member

@benesch benesch commented Dec 10, 2023

Allow users to configure the consumer group and transactional IDs when
creating Kafka sources and sinks. Specifically:

  • The stabilized GROUP ID PREFIX option for Kafka sources allows
    setting the prefix of the consumer group ID.

  • The renamed and stabilized PROGRESS GROUP ID PREFIX option for
    Kafka sinks allows setting the prefix of the consumer group ID used
    when Materialize reads records from the sink's progress topic.

  • The new and instastabilized TRANSACTIONAL ID option for Kafka
    sinks allows setting the transacitonal ID used when Materialize
    produces records to the sink's data topic.

Also change the default progress group and transactional ID to be

materialize-{REGION-ID}-{CONNECTION-ID}-{SINK-ID}

to be in line with the default group ID for Kafka sources.

Importantly, the new default includes the region ID, which makes it
possible to point multiple Materialize environments at the same Kafka
cluster without conflicts.

This is a backwards incompatible change, so this commit adds a migration
to keep the group and transacitonal IDs for existing Kafka sinks the
same.

Fix https://github.com/MaterializeInc/database-issues/issues/3308.
Fix https://github.com/MaterializeInc/database-issues/issues/6987.

Motivation

  • This PR adds known-desirable features.

Tips for reviewer

Go commit by commit. Commit messages are fairly detailed.

@sploiselle: could use your close review on all the SQL options handling refactoring. Unfortunately each and every commit includes some options handling changes. Commits 1 and 3 are particularly option handling heavy.

@jkosh44 and/or @mjibson: could use your close review on the migration to lock in the old transactional and group IDs for existing sinks.

@bkirwi and/or @petrosagg: you can probably skip the SQL options handling and focus on the new format of the group and transactional IDs.

Documentation PR: #23793.

Also, I know there's a desire to bring back the PARTITION COUNT, REPLICATION FACTOR, etc. options that are removed in this PR. I wrote up an issue describing the design questions we need to answer in order to bring them back: https://github.com/MaterializeInc/database-issues/issues/7143. There were enough open questions that it seemed worth simplifying the code by removing the existing implementation. I think it's likely that the new implementation would have to rewrite much of that code anyway.

Checklist

  • This PR has adequate test coverage / QA involvement has been duly considered.
  • This PR has an associated up-to-date design doc, is a design doc (template), or is sufficiently small to not require a design.
  • If this PR evolves an existing $T ⇔ Proto$T mapping (possibly in a backwards-incompatible way), then it is tagged with a T-proto label.
  • If this PR will require changes to cloud orchestration or tests, there is a companion cloud PR to account for those changes that is tagged with the release-blocker label (example).
  • This PR includes the following user-facing behavior changes:
    • Allow configuring the group ID for Kafka sources, and the group ID and transactional ID for Kafka sinks.

@benesch benesch requested a review from a team December 10, 2023 23:21
@benesch benesch requested a review from a team as a code owner December 10, 2023 23:21
@benesch benesch requested review from a team and maddyblue December 10, 2023 23:21
benesch added a commit to benesch/materialize that referenced this pull request Dec 10, 2023
This commit adds documentation for the features added in MaterializeInc#23792. See
that PR for details.
benesch added a commit to benesch/materialize that referenced this pull request Dec 10, 2023
This commit adds documentation for the features added in MaterializeInc#23792. See
that PR for details.
@benesch benesch force-pushed the topic-metadata-refresh-interval branch from 661116b to 35e05ee Compare December 11, 2023 00:49
@benesch benesch requested a review from a team as a code owner December 11, 2023 00:49
Copy link
Contributor

@petrosagg petrosagg left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Kafka sinks getting in shape 🚀

// to disable the consumer group protocol entirely.
"group.id" => group_id.clone(),
// Allow Kafka monitoring tools to identify this consumer.
"client.id" => group_id.clone(),
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This should be the value from connection.client_id(), right?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Oof, yes, fixed.

src/storage-client/src/sink.rs Show resolved Hide resolved
partition_count: -1,
replication_factor: -1,
cleanup_policy: TopicCleanupPolicy::Retention {
ms: Some(-1),
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this a behavior change? If I followed the code on main correctly we're currently setting the broker defaults when creating the data topic. If we plan to eventually expose this in SQL it would be better to leave this the same so when we run the future migration we can safely set all existing sinks to use the broker default explicitly

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Eek! Yes, thanks. Total oversight due to the confusing API for this function. Fixed now.

Copy link
Contributor

@sploiselle sploiselle left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Read all the code but paid much closer attention to the option handling changes in the SQL crate.


// We want a fairly low ceiling on our polling frequency, since we rely
// on this heartbeat to determine the health of our Kafka connection.
let metadata_refresh_frequency =
metadata_refresh_interval.min(Duration::from_secs(60));
let topic_metadata_refresh_interval =
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: this variable name confused me. Felt unclear why this is the same as the setting albeit with a ceiling applied. maybe something like topic_metadata_refresh_polling_interval?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Changed to poll_interval!

src/storage-client/src/sink.rs Show resolved Hide resolved
@@ -819,39 +819,36 @@ impl<T: AstInfo> AstDisplay for CreateConnectionOption<T> {
impl_display_t!(CreateConnectionOption);

#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Ord, Hash)]
pub enum KafkaConfigOptionName {
CompressionType,
pub enum KafkaSourceConfigOptionName {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

🙇 –– I've been meaning to fix this for weeks and just couldn't get around to it. tysm.

@@ -1793,13 +1793,6 @@ feature_flags!(
internal: true,
enable_for_item_parsing: true,
},
{
name: enable_kafka_config_denylist_options,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

🎉

.await
.check_ssh_status(fence_producer.context())?;
let client_id = connection.client_id(connection_context, sink_id);
let transactional_id = connection
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Allowing users to specify a particular id here introduces the risk of collisions, in case they specify the same transactional id for multiple sinks.

Can we get away with the <X> PREFIX-type options we've provided elsewhere?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Also: we're going to need multiple transactional ids if we ever get around to horizontally scaling Kafka sinks, so we may want to reserve some flexibility for that reason too.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As discussed on Slack: done!

) -> Cow<str> {
match &self.progress_group_id_prefix {
None => Cow::Owned(self.client_id(connection_context, sink_id)),
Some(progress_group_id_prefix) => Cow::Borrowed(progress_group_id_prefix),
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We want to add a suffix to the prefix here instead of just piping it through, yeah?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As discussed on Slack: done!

) -> Cow<str> {
match &self.group_id_prefix {
None => Cow::Owned(self.client_id(connection_context, source_id)),
Some(group_id_prefix) => Cow::Borrowed(group_id_prefix),
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ditto - seems like either we need to add a suffix or rename this method.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As discussed on Slack: done!

Copy link
Contributor

@jkosh44 jkosh44 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Migration LGTM, though I think it would be re-assuring to add an upgrade test somewhere.

@sploiselle
Copy link
Contributor

@jkosh44 Disregard last message––too many open PRs :D.

@benesch benesch force-pushed the topic-metadata-refresh-interval branch from 35e05ee to c5c51d9 Compare December 13, 2023 07:43
@benesch benesch requested a review from a team as a code owner December 13, 2023 07:43
@benesch benesch force-pushed the topic-metadata-refresh-interval branch from c5c51d9 to 9b98774 Compare December 13, 2023 07:59
Copy link
Member Author

@benesch benesch left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ok, I think this is plausibly mergeable. I addressed the bugs @petrosagg spotted (many thanks!), and adjusted the prefixing behavior per my conversation with @bkirwi.

@benesch benesch force-pushed the topic-metadata-refresh-interval branch from 9b98774 to acea2ad Compare December 13, 2023 08:13
// meaning. For example, `partition_count` could have type
// `Option<NonNeg<i32>>`, where `-1` is prohibited as a value and the Rustier
// `None` value represents the broker default (n.b.: `u32` is not a good choice
// because the maximum number of Kafka partitions is `i32::MAX`, not
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

True, but I think a user would have an equally bad time even if i32::MAX was accepted :D

@petrosagg
Copy link
Contributor

I pushed a commit that adds support for LEGACY TRANSACTIONAL ID and migrates existing sinks to that. I didn't bother to ban that syntax from users, I don't think anyone will type it if we don't document it

@petrosagg petrosagg force-pushed the topic-metadata-refresh-interval branch 2 times, most recently from c4e9d7c to 274e113 Compare December 13, 2023 16:21
Copy link
Contributor

@bkirwi bkirwi left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks good! (I have not reviewed the SQL-parsey stuff in detail, but the new ids and the migration look reasonable to me.)

(One commit message mentions that the new id format includes the region ID, but we include the environment id instead. Might be worth editing the message if we're not squashing the PR, but that's very minor and also seems fine to punt if you prefer.)

// Disable Kafka auto commit. We manually commit offsets
// to Kafka once we have reclocked those offsets, so
// that users can use standard Kafka tools for progress
// tracking.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

🙏

WHERE mz_sinks.name = 'working'
starting
running
paused
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

🥇

@petrosagg petrosagg force-pushed the topic-metadata-refresh-interval branch from 274e113 to 39ad82d Compare December 13, 2023 16:59
@benesch
Copy link
Member Author

benesch commented Dec 13, 2023

Thanks for doing the LEGACY IDS bit, @petrosagg! I'm comfortable shipping without a LD flag to facilitate the migration, but since that means the flag will be relatively long lived (vs something we're trying to quickly migrate sinks off of), I do think we should ensure that users can't type it. Just to be sure that we don't get new sinks using the legacy ID format. Even if we don't document it, users will be able to see the LEGACY IDS via SHOW CREATE SINK, and possibly use that as the basis for new sinks, which would make doing the migration to remove this option that much harder.

I think it's pretty easy to prevent users from typing LEGACY IDS while still allowing the migration to plan that option—just add a check to purification that prohibits that option!

@petrosagg
Copy link
Contributor

I think it's pretty easy to prevent users from typing LEGACY IDS while still allowing the migration to plan that option—just add a check to purification that prohibits that option!

Indeed, in fact I had mostly written this when I thought "Nah, no sane user will actually type it out by looking at our PRs", but yeah a SHOW CREATE SOURCE is another avenue. I'll add it tomorrow unless someone beats me to it

@benesch benesch force-pushed the topic-metadata-refresh-interval branch from 39ad82d to 2ee7f6e Compare December 14, 2023 07:32
@benesch
Copy link
Member Author

benesch commented Dec 14, 2023

I think it's pretty easy to prevent users from typing LEGACY IDS while still allowing the migration to plan that option—just add a check to purification that prohibits that option!

Indeed, in fact I had mostly written this when I thought "Nah, no sane user will actually type it out by looking at our PRs", but yeah a SHOW CREATE SOURCE is another avenue. I'll add it tomorrow unless someone beats me to it

Added, and marked for automerge! 🤞🏽

@benesch benesch enabled auto-merge December 14, 2023 07:33
benesch and others added 5 commits December 14, 2023 02:36
Drop the `MS` from `TOPIC METADATA REFRESH INTERVAL MS`. The new option
is spelled `TOPIC METADATA REFRESH INTERVAL` and takes a SQL interval
rather than a raw number of milliseconds.

This option is unsafe, so this is not a backwards incompatible change.

This commit also adjusts the implementation so that
`topic_metadata_refresh_interval` is a dedicated field of a
`KafkaSourceConnection`.

Part of #14656.
This commit removes the following options that configure the topic
created for the Kafka sink:

  * `PARTITION COUNT`
  * `REPLICATION FACTOR`
  * `RETENTION MS`
  * `RETENTION BYTES`

These options are all behind unsafe mode right now, so this is a
backwards compatible change.

The rationale: there are semantic questions about how these options
should behave if the Kafka topic already exists. They also should be
named `TOPIC PARTITION COUNT`, `TOPIC REPLICATION FACTOR`, etc., and
equally available for the progress topic on the Kafka connection. Rather
than spending the time trying to fix them now, just remove them, to keep
the code simpler.

Fix #14656.
Splitting the parsing and planning of Kafka sources and sinks is a net
simplification for the code. Even though a few options are shared
between sources and sinks (TOPIC, GROUP ID PREFIX), most are not. Having
separate types for sources and sinks avoids a separate validation step
to determine whether each option is valid for the context.

This is a pure refactoring commit.
Allow users to configure the consumer group and transactional IDs when
creating Kafka sources and sinks. Specifically:

  * The stabilized `GROUP ID PREFIX` option for Kafka sources allows
    setting the prefix of the consumer group ID.

  * The renamed and stabilized `PROGRESS GROUP ID PREFIX` option for
    Kafka sinks allows setting the prefix of the consumer group ID used
    when Materialize reads records from the sink's progress topic.

  * The new and instastabilized `TRANSACTIONAL ID PREFIX` option for
    Kafka sinks allows setting the transactional ID prefix used when
    Materialize produces records to the sink's data topic.

Also change the default progress group and transactional ID to be

    materialize-{REGION-ID}-{CONNECTION-ID}-{SINK-ID}

to be in line with the default group ID for Kafka sources.

Importantly, the new default includes the region ID, which makes it
possible to point multiple Materialize environments at the same Kafka
cluster without conflicts.

This is a backwards incompatible change, so this commit adds a migration
to keep the group and transactional IDs for existing Kafka sinks the
same.

Fix #11357.
Fix #23198.
@benesch benesch force-pushed the topic-metadata-refresh-interval branch from 2ee7f6e to b61fc78 Compare December 14, 2023 07:36
@benesch benesch merged commit d68a5de into MaterializeInc:main Dec 14, 2023
66 checks passed
benesch added a commit to benesch/materialize that referenced this pull request Dec 19, 2023
This commit adds documentation for the features added in MaterializeInc#23792. See
that PR for details.
benesch added a commit to benesch/materialize that referenced this pull request Dec 19, 2023
This commit adds documentation for the features added in MaterializeInc#23792. See
that PR for details.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants