From 081592f9ee04206be662b4fe0daf6320d80c063a Mon Sep 17 00:00:00 2001 From: Mihai Todor Date: Thu, 21 Nov 2024 02:00:09 +0000 Subject: [PATCH] Add Redpanda Migrator offset metadata - New `redpanda_migrator_offsets` input. - Field `kafka_offset_metadata` added to the `redpanda_migrator_offsets` output. Signed-off-by: Mihai Todor --- CHANGELOG.md | 4 +- .../inputs/redpanda_migrator_offsets.adoc | 558 ++++++++++++++++++ .../outputs/redpanda_migrator_offsets.adoc | 12 + .../kafka/enterprise/redpanda_common_input.go | 2 +- .../redpanda_migrator_bundle_input.tmpl.yaml | 17 +- .../redpanda_migrator_offsets_input.go | 149 +++++ .../redpanda_migrator_offsets_output.go | 52 +- internal/impl/kafka/franz_reader.go | 9 +- internal/impl/kafka/franz_reader_ordered.go | 28 +- internal/impl/kafka/input_redpanda.go | 2 +- internal/plugins/info.csv | 1 + 11 files changed, 793 insertions(+), 41 deletions(-) create mode 100644 docs/modules/components/pages/inputs/redpanda_migrator_offsets.adoc create mode 100644 internal/impl/kafka/enterprise/redpanda_migrator_offsets_input.go diff --git a/CHANGELOG.md b/CHANGELOG.md index e46ce85f6c..be4ba9b40a 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -7,9 +7,11 @@ All notable changes to this project will be documented in this file. ### Added -- New `pg_stream` input supporting change data capture (CDC) from PostgreSQL (@le-vlad) +- New `pg_stream` input supporting change data capture (CDC) from PostgreSQL. (@le-vlad) - Field `metadata_max_age` added to the `redpanda_migrator_offsets` output. (@mihaitodor) - Field `kafka_timestamp_ms` added to the `kafka`, `kafka_franz`, `redpanda`, `redpanda_common` and `redpanda_migrator` outputs. (@mihaitodor) +- New `redpanda_migrator_offsets` input. (@mihaitodor) +- Field `kafka_offset_metadata` added to the `redpanda_migrator_offsets` output. (@mihaitodor) ### Changed diff --git a/docs/modules/components/pages/inputs/redpanda_migrator_offsets.adoc b/docs/modules/components/pages/inputs/redpanda_migrator_offsets.adoc new file mode 100644 index 0000000000..a5a8459f36 --- /dev/null +++ b/docs/modules/components/pages/inputs/redpanda_migrator_offsets.adoc @@ -0,0 +1,558 @@ += redpanda_migrator_offsets +:type: input +:status: beta +:categories: ["Services"] + + + +//// + THIS FILE IS AUTOGENERATED! + + To make changes, edit the corresponding source file under: + + https://github.com/redpanda-data/connect/tree/main/internal/impl/. + + And: + + https://github.com/redpanda-data/connect/tree/main/cmd/tools/docs_gen/templates/plugin.adoc.tmpl +//// + +// © 2024 Redpanda Data Inc. + + +component_type_dropdown::[] + + +Redpanda Migrator consumer group offsets output using the https://github.com/twmb/franz-go[Franz Kafka client library^]. + +Introduced in version 4.40.0. + + +[tabs] +====== +Common:: ++ +-- + +```yml +# Common config fields, showing default values +input: + label: "" + redpanda_migrator_offsets: + seed_brokers: [] # No default (required) + consumer_group: "" # No default (optional) + auto_replay_nacks: true +``` + +-- +Advanced:: ++ +-- + +```yml +# All config fields, showing default values +input: + label: "" + redpanda_migrator_offsets: + seed_brokers: [] # No default (required) + client_id: benthos + tls: + enabled: false + skip_cert_verify: false + enable_renegotiation: false + root_cas: "" + root_cas_file: "" + client_certs: [] + sasl: [] # No default (optional) + metadata_max_age: 5m + rack_id: "" + fetch_max_bytes: 50MiB + fetch_min_bytes: 1B + fetch_max_partition_bytes: 1MiB + consumer_group: "" # No default (optional) + commit_period: 5s + partition_buffer_bytes: 1MB + auto_replay_nacks: true +``` + +-- +====== + +TODO: Description + +== Metadata + +This input adds the following metadata fields to each message: + +```text +- kafka_key +- kafka_topic +- kafka_partition +- kafka_offset +- kafka_timestamp_unix +- kafka_timestamp_ms +- kafka_tombstone_message +- kafka_offset_metadata +``` + + +== Fields + +=== `seed_brokers` + +A list of broker addresses to connect to in order to establish connections. If an item of the list contains commas it will be expanded into multiple addresses. + + +*Type*: `array` + + +```yml +# Examples + +seed_brokers: + - localhost:9092 + +seed_brokers: + - foo:9092 + - bar:9092 + +seed_brokers: + - foo:9092,bar:9092 +``` + +=== `client_id` + +An identifier for the client connection. + + +*Type*: `string` + +*Default*: `"benthos"` + +=== `tls` + +Custom TLS settings can be used to override system defaults. + + +*Type*: `object` + + +=== `tls.enabled` + +Whether custom TLS settings are enabled. + + +*Type*: `bool` + +*Default*: `false` + +=== `tls.skip_cert_verify` + +Whether to skip server side certificate verification. + + +*Type*: `bool` + +*Default*: `false` + +=== `tls.enable_renegotiation` + +Whether to allow the remote server to repeatedly request renegotiation. Enable this option if you're seeing the error message `local error: tls: no renegotiation`. + + +*Type*: `bool` + +*Default*: `false` +Requires version 3.45.0 or newer + +=== `tls.root_cas` + +An optional root certificate authority to use. This is a string, representing a certificate chain from the parent trusted root certificate, to possible intermediate signing certificates, to the host certificate. +[CAUTION] +==== +This field contains sensitive information that usually shouldn't be added to a config directly, read our xref:configuration:secrets.adoc[secrets page for more info]. +==== + + + +*Type*: `string` + +*Default*: `""` + +```yml +# Examples + +root_cas: |- + -----BEGIN CERTIFICATE----- + ... + -----END CERTIFICATE----- +``` + +=== `tls.root_cas_file` + +An optional path of a root certificate authority file to use. This is a file, often with a .pem extension, containing a certificate chain from the parent trusted root certificate, to possible intermediate signing certificates, to the host certificate. + + +*Type*: `string` + +*Default*: `""` + +```yml +# Examples + +root_cas_file: ./root_cas.pem +``` + +=== `tls.client_certs` + +A list of client certificates to use. For each certificate either the fields `cert` and `key`, or `cert_file` and `key_file` should be specified, but not both. + + +*Type*: `array` + +*Default*: `[]` + +```yml +# Examples + +client_certs: + - cert: foo + key: bar + +client_certs: + - cert_file: ./example.pem + key_file: ./example.key +``` + +=== `tls.client_certs[].cert` + +A plain text certificate to use. + + +*Type*: `string` + +*Default*: `""` + +=== `tls.client_certs[].key` + +A plain text certificate key to use. +[CAUTION] +==== +This field contains sensitive information that usually shouldn't be added to a config directly, read our xref:configuration:secrets.adoc[secrets page for more info]. +==== + + + +*Type*: `string` + +*Default*: `""` + +=== `tls.client_certs[].cert_file` + +The path of a certificate to use. + + +*Type*: `string` + +*Default*: `""` + +=== `tls.client_certs[].key_file` + +The path of a certificate key to use. + + +*Type*: `string` + +*Default*: `""` + +=== `tls.client_certs[].password` + +A plain text password for when the private key is password encrypted in PKCS#1 or PKCS#8 format. The obsolete `pbeWithMD5AndDES-CBC` algorithm is not supported for the PKCS#8 format. + +Because the obsolete pbeWithMD5AndDES-CBC algorithm does not authenticate the ciphertext, it is vulnerable to padding oracle attacks that can let an attacker recover the plaintext. +[CAUTION] +==== +This field contains sensitive information that usually shouldn't be added to a config directly, read our xref:configuration:secrets.adoc[secrets page for more info]. +==== + + + +*Type*: `string` + +*Default*: `""` + +```yml +# Examples + +password: foo + +password: ${KEY_PASSWORD} +``` + +=== `sasl` + +Specify one or more methods of SASL authentication. SASL is tried in order; if the broker supports the first mechanism, all connections will use that mechanism. If the first mechanism fails, the client will pick the first supported mechanism. If the broker does not support any client mechanisms, connections will fail. + + +*Type*: `array` + + +```yml +# Examples + +sasl: + - mechanism: SCRAM-SHA-512 + password: bar + username: foo +``` + +=== `sasl[].mechanism` + +The SASL mechanism to use. + + +*Type*: `string` + + +|=== +| Option | Summary + +| `AWS_MSK_IAM` +| AWS IAM based authentication as specified by the 'aws-msk-iam-auth' java library. +| `OAUTHBEARER` +| OAuth Bearer based authentication. +| `PLAIN` +| Plain text authentication. +| `SCRAM-SHA-256` +| SCRAM based authentication as specified in RFC5802. +| `SCRAM-SHA-512` +| SCRAM based authentication as specified in RFC5802. +| `none` +| Disable sasl authentication + +|=== + +=== `sasl[].username` + +A username to provide for PLAIN or SCRAM-* authentication. + + +*Type*: `string` + +*Default*: `""` + +=== `sasl[].password` + +A password to provide for PLAIN or SCRAM-* authentication. +[CAUTION] +==== +This field contains sensitive information that usually shouldn't be added to a config directly, read our xref:configuration:secrets.adoc[secrets page for more info]. +==== + + + +*Type*: `string` + +*Default*: `""` + +=== `sasl[].token` + +The token to use for a single session's OAUTHBEARER authentication. + + +*Type*: `string` + +*Default*: `""` + +=== `sasl[].extensions` + +Key/value pairs to add to OAUTHBEARER authentication requests. + + +*Type*: `object` + + +=== `sasl[].aws` + +Contains AWS specific fields for when the `mechanism` is set to `AWS_MSK_IAM`. + + +*Type*: `object` + + +=== `sasl[].aws.region` + +The AWS region to target. + + +*Type*: `string` + +*Default*: `""` + +=== `sasl[].aws.endpoint` + +Allows you to specify a custom endpoint for the AWS API. + + +*Type*: `string` + +*Default*: `""` + +=== `sasl[].aws.credentials` + +Optional manual configuration of AWS credentials to use. More information can be found in xref:guides:cloud/aws.adoc[]. + + +*Type*: `object` + + +=== `sasl[].aws.credentials.profile` + +A profile from `~/.aws/credentials` to use. + + +*Type*: `string` + +*Default*: `""` + +=== `sasl[].aws.credentials.id` + +The ID of credentials to use. + + +*Type*: `string` + +*Default*: `""` + +=== `sasl[].aws.credentials.secret` + +The secret for the credentials being used. +[CAUTION] +==== +This field contains sensitive information that usually shouldn't be added to a config directly, read our xref:configuration:secrets.adoc[secrets page for more info]. +==== + + + +*Type*: `string` + +*Default*: `""` + +=== `sasl[].aws.credentials.token` + +The token for the credentials being used, required when using short term credentials. + + +*Type*: `string` + +*Default*: `""` + +=== `sasl[].aws.credentials.from_ec2_role` + +Use the credentials of a host EC2 machine configured to assume https://docs.aws.amazon.com/IAM/latest/UserGuide/id_roles_use_switch-role-ec2.html[an IAM role associated with the instance^]. + + +*Type*: `bool` + +*Default*: `false` +Requires version 4.2.0 or newer + +=== `sasl[].aws.credentials.role` + +A role ARN to assume. + + +*Type*: `string` + +*Default*: `""` + +=== `sasl[].aws.credentials.role_external_id` + +An external ID to provide when assuming a role. + + +*Type*: `string` + +*Default*: `""` + +=== `metadata_max_age` + +The maximum age of metadata before it is refreshed. + + +*Type*: `string` + +*Default*: `"5m"` + +=== `rack_id` + +A rack specifies where the client is physically located and changes fetch requests to consume from the closest replica as opposed to the leader replica. + + +*Type*: `string` + +*Default*: `""` + +=== `fetch_max_bytes` + +Sets the maximum amount of bytes a broker will try to send during a fetch. Note that brokers may not obey this limit if it has records larger than this limit. This is the equivalent to the Java fetch.max.bytes setting. + + +*Type*: `string` + +*Default*: `"50MiB"` + +=== `fetch_min_bytes` + +Sets the minimum amount of bytes a broker will try to send during a fetch. This is the equivalent to the Java fetch.min.bytes setting. + + +*Type*: `string` + +*Default*: `"1B"` + +=== `fetch_max_partition_bytes` + +Sets the maximum amount of bytes that will be consumed for a single partition in a fetch request. Note that if a single batch is larger than this number, that batch will still be returned so the client can make progress. This is the equivalent to the Java fetch.max.partition.bytes setting. + + +*Type*: `string` + +*Default*: `"1MiB"` + +=== `consumer_group` + +An optional consumer group to consume as. When specified the partitions of specified topics are automatically distributed across consumers sharing a consumer group, and partition offsets are automatically committed and resumed under this name. Consumer groups are not supported when specifying explicit partitions to consume from in the `topics` field. + + +*Type*: `string` + + +=== `commit_period` + +The period of time between each commit of the current partition offsets. Offsets are always committed during shutdown. + + +*Type*: `string` + +*Default*: `"5s"` + +=== `partition_buffer_bytes` + +A buffer size (in bytes) for each consumed partition, allowing records to be queued internally before flushing. Increasing this may improve throughput at the cost of higher memory utilisation. Note that each buffer can grow slightly beyond this value. + + +*Type*: `string` + +*Default*: `"1MB"` + +=== `auto_replay_nacks` + +Whether messages that are rejected (nacked) at the output level should be automatically replayed indefinitely, eventually resulting in back pressure if the cause of the rejections is persistent. If set to `false` these messages will instead be deleted. Disabling auto replays can greatly improve memory efficiency of high throughput streams as the original shape of the data can be discarded immediately upon consumption and mutation. + + +*Type*: `bool` + +*Default*: `true` + + diff --git a/docs/modules/components/pages/outputs/redpanda_migrator_offsets.adoc b/docs/modules/components/pages/outputs/redpanda_migrator_offsets.adoc index b76f32f6fb..f8a95e065f 100644 --- a/docs/modules/components/pages/outputs/redpanda_migrator_offsets.adoc +++ b/docs/modules/components/pages/outputs/redpanda_migrator_offsets.adoc @@ -41,6 +41,7 @@ output: redpanda_migrator_offsets: seed_brokers: [] # No default (required) kafka_key: ${! @kafka_key } + offset_metadata: ${! @kafka_offset_metadata } max_in_flight: 1 ``` @@ -66,6 +67,7 @@ output: sasl: [] # No default (optional) metadata_max_age: 5m kafka_key: ${! @kafka_key } + offset_metadata: ${! @kafka_offset_metadata } max_in_flight: 1 timeout: 10s max_message_bytes: 1MB @@ -480,6 +482,16 @@ This field supports xref:configuration:interpolation.adoc#bloblang-queries[inter *Default*: `"${! @kafka_key }"` +=== `offset_metadata` + +The offset metadata value. +This field supports xref:configuration:interpolation.adoc#bloblang-queries[interpolation functions]. + + +*Type*: `string` + +*Default*: `"${! @kafka_offset_metadata }"` + === `max_in_flight` The maximum number of batches to be sending in parallel at any given time. diff --git a/internal/impl/kafka/enterprise/redpanda_common_input.go b/internal/impl/kafka/enterprise/redpanda_common_input.go index 1798940c71..19cea54f85 100644 --- a/internal/impl/kafka/enterprise/redpanda_common_input.go +++ b/internal/impl/kafka/enterprise/redpanda_common_input.go @@ -114,7 +114,7 @@ func init() { time.Sleep(time.Millisecond * 100) } return - }) + }, nil) if err != nil { return nil, err } diff --git a/internal/impl/kafka/enterprise/redpanda_migrator_bundle_input.tmpl.yaml b/internal/impl/kafka/enterprise/redpanda_migrator_bundle_input.tmpl.yaml index b21046833d..5ff71fea17 100644 --- a/internal/impl/kafka/enterprise/redpanda_migrator_bundle_input.tmpl.yaml +++ b/internal/impl/kafka/enterprise/redpanda_migrator_bundle_input.tmpl.yaml @@ -38,7 +38,7 @@ fields: mapping: | #!blobl - let redpandaMigratorOffsets = this.redpanda_migrator.with("seed_brokers", "consumer_group", "client_id", "rack_id", "tls", "sasl").assign({"topics": ["__consumer_offsets"]}) + let redpandaMigratorOffsets = this.redpanda_migrator.with("seed_brokers", "consumer_group", "client_id", "rack_id", "fetch_max_bytes", "fetch_min_bytes", "fetch_max_partition_bytes", "tls", "sasl") root = if this.redpanda_migrator.length() == 0 { throw("the redpanda_migrator input must be configured") @@ -63,7 +63,7 @@ mapping: | - redpanda_migrator: %s processors: - mapping: meta input_label = "redpanda_migrator" - - kafka_franz: %s + - redpanda_migrator_offsets: %s processors: - mapping: meta input_label = "redpanda_migrator_offsets" """.format(this.schema_registry.string(), this.redpanda_migrator.string(), $redpandaMigratorOffsets.string()).parse_yaml() @@ -86,7 +86,7 @@ mapping: | - redpanda_migrator: %s processors: - mapping: meta input_label = "redpanda_migrator" - - kafka_franz: %s + - redpanda_migrator_offsets: %s processors: - mapping: meta input_label = "redpanda_migrator_offsets" """.format(this.schema_registry.string(), this.redpanda_migrator.string(), $redpandaMigratorOffsets.string()).parse_yaml() @@ -97,7 +97,7 @@ mapping: | - redpanda_migrator: %s processors: - mapping: meta input_label = "redpanda_migrator" - - kafka_franz: %s + - redpanda_migrator_offsets: %s processors: - mapping: meta input_label = "redpanda_migrator_offsets" """.format(this.redpanda_migrator.string(), $redpandaMigratorOffsets.string()).parse_yaml() @@ -137,9 +137,8 @@ tests: consumer_group: "migrator" processors: - mapping: meta input_label = "redpanda_migrator" - - kafka_franz: + - redpanda_migrator_offsets: seed_brokers: [ "127.0.0.1:9092" ] - topics: [ "__consumer_offsets" ] consumer_group: "migrator" processors: - mapping: meta input_label = "redpanda_migrator_offsets" @@ -177,9 +176,8 @@ tests: consumer_group: "migrator" processors: - mapping: meta input_label = "redpanda_migrator" - - kafka_franz: + - redpanda_migrator_offsets: seed_brokers: [ "127.0.0.1:9092" ] - topics: [ "__consumer_offsets" ] consumer_group: "migrator" processors: - mapping: meta input_label = "redpanda_migrator_offsets" @@ -200,9 +198,8 @@ tests: consumer_group: "migrator" processors: - mapping: meta input_label = "redpanda_migrator" - - kafka_franz: + - redpanda_migrator_offsets: seed_brokers: [ "127.0.0.1:9092" ] - topics: [ "__consumer_offsets" ] consumer_group: "migrator" processors: - mapping: meta input_label = "redpanda_migrator_offsets" diff --git a/internal/impl/kafka/enterprise/redpanda_migrator_offsets_input.go b/internal/impl/kafka/enterprise/redpanda_migrator_offsets_input.go new file mode 100644 index 0000000000..ebe426efcd --- /dev/null +++ b/internal/impl/kafka/enterprise/redpanda_migrator_offsets_input.go @@ -0,0 +1,149 @@ +// Copyright 2024 Redpanda Data, Inc. +// +// Licensed as a Redpanda Enterprise file under the Redpanda Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/redpanda-data/connect/blob/main/licenses/rcl.md + +package enterprise + +import ( + "slices" + + "github.com/twmb/franz-go/pkg/kgo" + "github.com/twmb/franz-go/pkg/kmsg" + + "github.com/redpanda-data/benthos/v4/public/service" + "github.com/redpanda-data/connect/v4/internal/impl/kafka" +) + +const ( + // Consumer fields + rmoiFieldRackID = "rack_id" + rmoiFieldFetchMaxBytes = "fetch_max_bytes" + rmoiFieldFetchMinBytes = "fetch_min_bytes" + rmoiFieldFetchMaxPartitionBytes = "fetch_max_partition_bytes" +) + +func redpandaInputConfig() *service.ConfigSpec { + return service.NewConfigSpec(). + Beta(). + Categories("Services"). + Version("4.40.0"). + Summary(`Redpanda Migrator consumer group offsets output using the https://github.com/twmb/franz-go[Franz Kafka client library^].`). + Description(` +TODO: Description + +== Metadata + +This input adds the following metadata fields to each message: + +` + "```text" + ` +- kafka_key +- kafka_topic +- kafka_partition +- kafka_offset +- kafka_timestamp_unix +- kafka_timestamp_ms +- kafka_tombstone_message +- kafka_offset_metadata +` + "```" + ` +`). + Fields(redpandaInputConfigFields()...) +} + +func redpandaInputConfigFields() []*service.ConfigField { + return slices.Concat( + kafka.FranzConnectionFields(), + []*service.ConfigField{ + service.NewStringField(rmoiFieldRackID). + Description("A rack specifies where the client is physically located and changes fetch requests to consume from the closest replica as opposed to the leader replica."). + Default(""). + Advanced(), + service.NewStringField(rmoiFieldFetchMaxBytes). + Description("Sets the maximum amount of bytes a broker will try to send during a fetch. Note that brokers may not obey this limit if it has records larger than this limit. This is the equivalent to the Java fetch.max.bytes setting."). + Advanced(). + Default("50MiB"), + service.NewStringField(rmoiFieldFetchMinBytes). + Description("Sets the minimum amount of bytes a broker will try to send during a fetch. This is the equivalent to the Java fetch.min.bytes setting."). + Advanced(). + Default("1B"), + service.NewStringField(rmoiFieldFetchMaxPartitionBytes). + Description("Sets the maximum amount of bytes that will be consumed for a single partition in a fetch request. Note that if a single batch is larger than this number, that batch will still be returned so the client can make progress. This is the equivalent to the Java fetch.max.partition.bytes setting."). + Advanced(). + Default("1MiB"), + }, + kafka.FranzReaderOrderedConfigFields(), + []*service.ConfigField{ + service.NewAutoRetryNacksToggleField(), + }, + ) +} + +func init() { + err := service.RegisterBatchInput("redpanda_migrator_offsets", redpandaInputConfig(), + func(conf *service.ParsedConfig, mgr *service.Resources) (service.BatchInput, error) { + tmpOpts, err := kafka.FranzConnectionOptsFromConfig(conf, mgr.Logger()) + if err != nil { + return nil, err + } + clientOpts := append([]kgo.Opt{}, tmpOpts...) + + d := kafka.FranzConsumerDetails{} + + if d.RackID, err = conf.FieldString(rmoiFieldRackID); err != nil { + return nil, err + } + + d.Topics = []string{`__consumer_offsets`} + + if d.FetchMaxBytes, err = kafka.BytesFromStrFieldAsInt32(rmoiFieldFetchMaxBytes, conf); err != nil { + return nil, err + } + if d.FetchMinBytes, err = kafka.BytesFromStrFieldAsInt32(rmoiFieldFetchMinBytes, conf); err != nil { + return nil, err + } + if d.FetchMaxPartitionBytes, err = kafka.BytesFromStrFieldAsInt32(rmoiFieldFetchMaxPartitionBytes, conf); err != nil { + return nil, err + } + + clientOpts = append(clientOpts, d.FranzOpts()...) + + rdr, err := kafka.NewFranzReaderOrderedFromConfig(conf, mgr, func() ([]kgo.Opt, error) { + return clientOpts, nil + }, func(record *kgo.Record) *service.Message { + key := kmsg.NewOffsetCommitKey() + // Check the version to ensure that we process only offset commit keys + if err := key.ReadFrom(record.Key); err != nil || (key.Version != 0 && key.Version != 1) { + return nil + } + + offsetCommitValue := kmsg.NewOffsetCommitValue() + if err := offsetCommitValue.ReadFrom(record.Value); err != nil { + // Omit records we can't decode + return nil + } + + msg := service.NewMessage(record.Value) + msg.MetaSetMut("kafka_key", record.Key) + msg.MetaSetMut("kafka_topic", record.Topic) + msg.MetaSetMut("kafka_partition", int(record.Partition)) + msg.MetaSetMut("kafka_offset", int(record.Offset)) + msg.MetaSetMut("kafka_timestamp_unix", record.Timestamp.Unix()) + msg.MetaSetMut("kafka_timestamp_ms", record.Timestamp.UnixMilli()) + msg.MetaSetMut("kafka_tombstone_message", record.Value == nil) + msg.MetaSetMut("kafka_offset_metadata", offsetCommitValue.Metadata) + + return msg + }) + if err != nil { + return nil, err + } + + return service.AutoRetryNacksBatchedToggled(conf, rdr) + }) + if err != nil { + panic(err) + } +} diff --git a/internal/impl/kafka/enterprise/redpanda_migrator_offsets_output.go b/internal/impl/kafka/enterprise/redpanda_migrator_offsets_output.go index 524e24788c..2cd4fc1b8c 100644 --- a/internal/impl/kafka/enterprise/redpanda_migrator_offsets_output.go +++ b/internal/impl/kafka/enterprise/redpanda_migrator_offsets_output.go @@ -27,8 +27,9 @@ import ( ) const ( - rmooFieldMaxInFlight = "max_in_flight" - rmooFieldKafkaKey = "kafka_key" + rmooFieldKafkaKey = "kafka_key" + rmooFieldOffsetMetadata = "offset_metadata" + rmooFieldMaxInFlight = "max_in_flight" ) func redpandaMigratorOffsetsOutputConfig() *service.ConfigSpec { @@ -49,6 +50,9 @@ func RedpandaMigratorOffsetsOutputConfigFields() []*service.ConfigField { []*service.ConfigField{ service.NewInterpolatedStringField(rmooFieldKafkaKey). Description("Kafka key.").Default("${! @kafka_key }"), + service.NewInterpolatedStringField(rmooFieldOffsetMetadata). + Description("The offset metadata value."). + Default(`${! @kafka_offset_metadata }`), service.NewIntField(rmooFieldMaxInFlight). Description("The maximum number of batches to be sending in parallel at any given time."). Default(1), @@ -80,10 +84,11 @@ func init() { // RedpandaMigratorOffsetsWriter implements a Redpanda Migrator offsets writer using the franz-go library. type RedpandaMigratorOffsetsWriter struct { - clientDetails *kafka.FranzConnectionDetails - clientOpts []kgo.Opt - kafkaKey *service.InterpolatedString - backoffCtor func() backoff.BackOff + clientDetails *kafka.FranzConnectionDetails + clientOpts []kgo.Opt + kafkaKey *service.InterpolatedString + offsetMetadata *service.InterpolatedString + backoffCtor func() backoff.BackOff connMut sync.Mutex client *kadm.Client @@ -106,6 +111,10 @@ func NewRedpandaMigratorOffsetsWriterFromConfig(conf *service.ParsedConfig, mgr return nil, err } + if w.offsetMetadata, err = conf.FieldInterpolatedString(rmooFieldOffsetMetadata); err != nil { + return nil, err + } + if w.clientOpts, err = kafka.FranzProducerLimitsOptsFromConfig(conf); err != nil { return nil, err } @@ -167,7 +176,6 @@ func (w *RedpandaMigratorOffsetsWriter) Write(ctx context.Context, msg *service. var kafkaKey []byte var err error - // TODO: The `kafka_key` metadata field is cast from `[]byte` to string in the `kafka_franz` input, which is wrong. if kafkaKey, err = w.kafkaKey.TryBytes(msg); err != nil { return fmt.Errorf("failed to extract kafka key: %w", err) } @@ -183,13 +191,20 @@ func (w *RedpandaMigratorOffsetsWriter) Write(ctx context.Context, msg *service. return fmt.Errorf("failed to get message bytes: %s", err) } - val := kmsg.NewOffsetCommitValue() - if err := val.ReadFrom(msgBytes); err != nil { + offsetCommitValue := kmsg.NewOffsetCommitValue() + if err := offsetCommitValue.ReadFrom(msgBytes); err != nil { return fmt.Errorf("failed to decode offset commit value: %s", err) } + var offsetMetadata string + if w.offsetMetadata != nil { + if offsetMetadata, err = w.offsetMetadata.TryString(msg); err != nil { + return fmt.Errorf("failed to extract offset metadata: %w", err) + } + } + updateConsumerOffsets := func() error { - listedOffsets, err := w.client.ListOffsetsAfterMilli(ctx, val.CommitTimestamp, key.Topic) + listedOffsets, err := w.client.ListOffsetsAfterMilli(ctx, offsetCommitValue.CommitTimestamp, key.Topic) if err != nil { return fmt.Errorf("failed to translate consumer offsets: %s", err) } @@ -198,11 +213,20 @@ func (w *RedpandaMigratorOffsetsWriter) Write(ctx context.Context, msg *service. return fmt.Errorf("listed offsets returned and error: %s", err) } - // TODO: Add metadata to offsets! offsets := listedOffsets.Offsets() - offsets.KeepFunc(func(o kadm.Offset) bool { - return o.Partition == key.Partition - }) + // Logic extracted from offsets.KeepFunc() and adjusted to set the metadata. + for topic, partitionOffsets := range offsets { + for partition, offset := range partitionOffsets { + if offset.Partition != key.Partition { + delete(partitionOffsets, partition) + } + offset.Metadata = offsetMetadata + partitionOffsets[partition] = offset + } + if len(partitionOffsets) == 0 { + delete(offsets, topic) + } + } offsetResponses, err := w.client.CommitOffsets(ctx, key.Group, offsets) if err != nil { diff --git a/internal/impl/kafka/franz_reader.go b/internal/impl/kafka/franz_reader.go index 98894805c1..63a0d21e78 100644 --- a/internal/impl/kafka/franz_reader.go +++ b/internal/impl/kafka/franz_reader.go @@ -36,7 +36,8 @@ func bytesFromStrField(name string, pConf *service.ParsedConfig) (uint64, error) return fieldAsBytes, nil } -func bytesFromStrFieldAsInt32(name string, pConf *service.ParsedConfig) (int32, error) { +// BytesFromStrFieldAsInt32 attempts to parse string field containing a human-readable byte size +func BytesFromStrFieldAsInt32(name string, pConf *service.ParsedConfig) (int32, error) { ui64, err := bytesFromStrField(name, pConf) if err != nil { return 0, err @@ -161,13 +162,13 @@ func FranzConsumerDetailsFromConfig(conf *service.ParsedConfig) (*FranzConsumerD return nil, err } - if d.FetchMaxBytes, err = bytesFromStrFieldAsInt32(kfrFieldFetchMaxBytes, conf); err != nil { + if d.FetchMaxBytes, err = BytesFromStrFieldAsInt32(kfrFieldFetchMaxBytes, conf); err != nil { return nil, err } - if d.FetchMinBytes, err = bytesFromStrFieldAsInt32(kfrFieldFetchMinBytes, conf); err != nil { + if d.FetchMinBytes, err = BytesFromStrFieldAsInt32(kfrFieldFetchMinBytes, conf); err != nil { return nil, err } - if d.FetchMaxPartitionBytes, err = bytesFromStrFieldAsInt32(kfrFieldFetchMaxPartitionBytes, conf); err != nil { + if d.FetchMaxPartitionBytes, err = BytesFromStrFieldAsInt32(kfrFieldFetchMaxPartitionBytes, conf); err != nil { return nil, err } diff --git a/internal/impl/kafka/franz_reader_ordered.go b/internal/impl/kafka/franz_reader_ordered.go index d32f57a08e..0505ba9014 100644 --- a/internal/impl/kafka/franz_reader_ordered.go +++ b/internal/impl/kafka/franz_reader_ordered.go @@ -63,9 +63,10 @@ type FranzReaderOrdered struct { partState *partitionState - consumerGroup string - commitPeriod time.Duration - cacheLimit uint64 + consumerGroup string + commitPeriod time.Duration + cacheLimit uint64 + recordToMessageFn func(record *kgo.Record) *service.Message readBackOff backoff.BackOff @@ -76,18 +77,19 @@ type FranzReaderOrdered struct { // NewFranzReaderOrderedFromConfig attempts to instantiate a new // FranzReaderOrdered reader from a parsed config. -func NewFranzReaderOrderedFromConfig(conf *service.ParsedConfig, res *service.Resources, optsFn func() ([]kgo.Opt, error)) (*FranzReaderOrdered, error) { +func NewFranzReaderOrderedFromConfig(conf *service.ParsedConfig, res *service.Resources, optsFn func() ([]kgo.Opt, error), recordToMessageFn func(record *kgo.Record) *service.Message) (*FranzReaderOrdered, error) { readBackOff := backoff.NewExponentialBackOff() readBackOff.InitialInterval = time.Millisecond readBackOff.MaxInterval = time.Millisecond * 100 readBackOff.MaxElapsedTime = 0 f := FranzReaderOrdered{ - readBackOff: readBackOff, - res: res, - log: res.Logger(), - shutSig: shutdown.NewSignaller(), - clientOpts: optsFn, + readBackOff: readBackOff, + res: res, + log: res.Logger(), + shutSig: shutdown.NewSignaller(), + clientOpts: optsFn, + recordToMessageFn: recordToMessageFn, } f.consumerGroup, _ = conf.FieldString(kroFieldConsumerGroup) @@ -115,7 +117,13 @@ func (f *FranzReaderOrdered) recordsToBatch(records []*kgo.Record) *batchWithRec var batch service.MessageBatch for _, r := range records { length += uint64(len(r.Value) + len(r.Key)) - batch = append(batch, FranzRecordToMessageV1(r)) + if f.recordToMessageFn != nil { + if msg := f.recordToMessageFn(r); msg != nil { + batch = append(batch, msg) + } + } else { + batch = append(batch, FranzRecordToMessageV1(r)) + } // The record lives on for checkpointing, but we don't need the contents // going forward so discard these. This looked fine to me but could // potentially be a source of problems so treat this as sus. diff --git a/internal/impl/kafka/input_redpanda.go b/internal/impl/kafka/input_redpanda.go index 5c8bd94664..e1aecbec5b 100644 --- a/internal/impl/kafka/input_redpanda.go +++ b/internal/impl/kafka/input_redpanda.go @@ -119,7 +119,7 @@ func init() { rdr, err := NewFranzReaderOrderedFromConfig(conf, mgr, func() ([]kgo.Opt, error) { return clientOpts, nil - }) + }, nil) if err != nil { return nil, err } diff --git a/internal/plugins/info.csv b/internal/plugins/info.csv index 1727bc002f..1f195b95da 100644 --- a/internal/plugins/info.csv +++ b/internal/plugins/info.csv @@ -202,6 +202,7 @@ redpanda_migrator ,input ,redpanda_migrator ,4.37.0 ,enterp redpanda_migrator ,output ,redpanda_migrator ,4.37.0 ,enterprise ,n ,y ,y redpanda_migrator_bundle ,input ,redpanda_migrator_bundle ,4.37.0 ,enterprise ,n ,y ,y redpanda_migrator_bundle ,output ,redpanda_migrator_bundle ,4.37.0 ,enterprise ,n ,y ,y +redpanda_migrator_offsets ,input ,redpanda_migrator_offsets ,4.40.0 ,enterprise ,n ,y ,y redpanda_migrator_offsets ,output ,redpanda_migrator_offsets ,4.37.0 ,enterprise ,n ,y ,y reject ,output ,reject ,0.0.0 ,certified ,n ,y ,y reject_errored ,output ,reject_errored ,0.0.0 ,certified ,n ,y ,y