Skip to content

Commit a99717d

Browse files
authored
kafkareceiver: Opt-in to use franz-go client (#40716)
<!--Ex. Fixing a bug - Describe the bug and how this fixes the issue. Ex. Adding a feature - Explain what this achieves.--> #### Description Adds the option to use the franz-go client in the `kafkareceiver`. The main rationale for this change is to allow users to opt-in to use a more performant Kafka client, as the default Sarama client has been known to have performance issues in high-throughput scenarios. The change _should_ be backward compatible, and significantly increases the consumption performance of the `kafkareceiver` when in use. In all cases, but particularly if AutoCommit isn't enabled. The benchmarks I've run (and committed) show a significant performance improvement: - Default (AutoCommit enabled): `+0-29%` - AutoCommit enabled, OnError=rue: `+20-38%` - After=true OnError=false: `+1000-4000%` - After=true_OnError=true: `+1100-4100%` - After=false_OnError=false: `+1100-3900%` - After=false_OnError=true: `+1100-3900%` ``` │ sarama.txt │ franz.txt │ │ spans/s │ spans/s vs base │ TracesReceiver/AutoCommit_OnError=false_(default)/batch_1/partitions_1-12 367.4k ± 3% 475.1k ± 1% +29.32% (p=0.000 n=10) TracesReceiver/AutoCommit_OnError=false_(default)/batch_1/partitions_2-12 557.2k ± 4% 620.8k ± 22% ~ (p=0.143 n=10) TracesReceiver/AutoCommit_OnError=false_(default)/batch_10/partitions_1-12 1.382M ± 3% 1.706M ± 4% +23.42% (p=0.000 n=10) TracesReceiver/AutoCommit_OnError=false_(default)/batch_10/partitions_2-12 2.104M ± 4% 1.926M ± 13% ~ (p=0.353 n=10) TracesReceiver/AutoCommit_OnError=true/batch_1/partitions_1-12 361.9k ± 2% 502.4k ± 2% +38.80% (p=0.000 n=10) TracesReceiver/AutoCommit_OnError=true/batch_1/partitions_2-12 561.7k ± 5% 675.8k ± 22% +20.31% (p=0.019 n=10) TracesReceiver/AutoCommit_OnError=true/batch_10/partitions_1-12 1.368M ± 3% 1.746M ± 3% +27.61% (p=0.000 n=10) TracesReceiver/AutoCommit_OnError=true/batch_10/partitions_2-12 2.157M ± 4% 2.210M ± 18% ~ (p=0.684 n=10) TracesReceiver/After=true_OnError=false/batch_1/partitions_1-12 9.918k ± 1% 410.510k ± 2% +4038.83% (p=0.000 n=10) TracesReceiver/After=true_OnError=false/batch_1/partitions_2-12 16.54k ± 4% 515.87k ± 16% +3019.11% (p=0.000 n=10) TracesReceiver/After=true_OnError=false/batch_10/partitions_1-12 100.1k ± 0% 1619.3k ± 3% +1518.39% (p=0.000 n=10) TracesReceiver/After=true_OnError=false/batch_10/partitions_2-12 171.5k ± 15% 1969.4k ± 19% +1048.12% (p=0.000 n=10) TracesReceiver/After=true_OnError=true/batch_1/partitions_1-12 9.807k ± 1% 414.966k ± 1% +4131.32% (p=0.000 n=10) TracesReceiver/After=true_OnError=true/batch_1/partitions_2-12 17.02k ± 7% 445.86k ± 31% +2518.85% (p=0.000 n=10) TracesReceiver/After=true_OnError=true/batch_10/partitions_1-12 99.46k ± 1% 1620.77k ± 3% +1529.50% (p=0.000 n=10) TracesReceiver/After=true_OnError=true/batch_10/partitions_2-12 147.3k ± 17% 1603.6k ± 23% +988.30% (p=0.000 n=10) TracesReceiver/After=false_OnError=false/batch_1/partitions_1-12 9.775k ± 1% 400.567k ± 2% +3997.66% (p=0.000 n=10) TracesReceiver/After=false_OnError=false/batch_1/partitions_2-12 15.73k ± 7% 464.95k ± 15% +2854.86% (p=0.000 n=10) TracesReceiver/After=false_OnError=false/batch_10/partitions_1-12 99.48k ± 0% 1586.74k ± 2% +1494.98% (p=0.000 n=10) TracesReceiver/After=false_OnError=false/batch_10/partitions_2-12 146.7k ± 1% 1868.7k ± 16% +1174.18% (p=0.000 n=10) TracesReceiver/After=false_OnError=true/batch_1/partitions_1-12 9.759k ± 1% 395.539k ± 1% +3952.87% (p=0.000 n=10) TracesReceiver/After=false_OnError=true/batch_1/partitions_2-12 15.68k ± 2% 413.63k ± 26% +2538.10% (p=0.000 n=10) TracesReceiver/After=false_OnError=true/batch_10/partitions_1-12 99.39k ± 1% 1589.17k ± 3% +1498.95% (p=0.000 n=10) TracesReceiver/After=false_OnError=true/batch_10/partitions_2-12 146.6k ± 15% 1894.1k ± 18% +1192.17% (p=0.000 n=10) geomean 111.0k 912.9k +722.19% ``` <!-- Issue number (e.g. #1234) or full URL to issue, if applicable. --> #### Link to tracking issue Closes #40628 <!--Describe what testing was performed and which tests were added.--> #### Testing Unit tested and benchmarked. <!--Describe the documentation added.--> #### Documentation Updated readme --------- Signed-off-by: Marc Lopez Rubio <[email protected]>
1 parent dfc894e commit a99717d

17 files changed

+1719
-413
lines changed
Lines changed: 31 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,31 @@
1+
# Use this changelog template to create an entry for release notes.
2+
3+
# One of 'breaking', 'deprecation', 'new_component', 'enhancement', 'bug_fix'
4+
change_type: enhancement
5+
6+
# The name of the component, or a single word describing the area of concern, (e.g. filelogreceiver)
7+
component: kafkareceiver
8+
9+
# A brief description of the change. Surround your text with quotes ("") if it needs to start with a backtick (`).
10+
note: "Add an Alpha feature gate `receiver.kafkareceiver.UseFranzGo` to use franz-go in the Kafka receiver for better performance."
11+
12+
# Mandatory: One or more tracking issues related to the change. You can use the PR number here if no issue exists.
13+
issues: [40628]
14+
15+
# (Optional) One or more lines of additional information to render under the primary note.
16+
# These lines will be padded with 2 spaces and then inserted directly into the document.
17+
# Use pipe (|) for multiline entries.
18+
subtext: |
19+
Adds an experimental opt-in support to use the franz-go client in the Kafka receiver.
20+
The franz-go client is a high-performance Kafka client that can improve the performance of the Kafka receiver.
21+
The default client remains sarama, which is used by the Kafka exporter and other components.
22+
Enable the franz-go client by setting the `receiver.kafkareceiver.UseFranzGo` feature gate.
23+
24+
# If your change doesn't affect end users or the exported elements of any package,
25+
# you should instead start your pull request title with [chore] or use the "Skip Changelog" label.
26+
# Optional: The change log or logs in which this entry should be included.
27+
# e.g. '[user]' or '[user, api]'
28+
# Include 'user' if the change is relevant to end users.
29+
# Include 'api' if there is a change to a library API.
30+
# Default: '[user]'
31+
change_logs: [user]

exporter/kafkaexporter/kafka_exporter.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -87,7 +87,7 @@ func (e *kafkaExporter[T]) Start(ctx context.Context, host component.Host) (err
8787
}
8888

8989
if franzGoClientFeatureGate.IsEnabled() {
90-
producer, ferr := kafka.NewFranzSyncProducer(e.cfg.ClientConfig,
90+
producer, ferr := kafka.NewFranzSyncProducer(ctx, e.cfg.ClientConfig,
9191
e.cfg.Producer, e.cfg.TimeoutSettings.Timeout, e.logger,
9292
)
9393
if ferr != nil {

internal/kafka/franz_client.go

Lines changed: 120 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -7,6 +7,7 @@ import (
77
"context"
88
"fmt"
99
"hash/fnv"
10+
"strings"
1011
"time"
1112

1213
"github.com/aws/aws-msk-iam-sasl-signer-go/signer"
@@ -34,7 +35,7 @@ const (
3435
)
3536

3637
// NewFranzSyncProducer creates a new Kafka client using the franz-go library.
37-
func NewFranzSyncProducer(clientCfg configkafka.ClientConfig,
38+
func NewFranzSyncProducer(ctx context.Context, clientCfg configkafka.ClientConfig,
3839
cfg configkafka.ProducerConfig,
3940
timeout time.Duration,
4041
logger *zap.Logger,
@@ -45,15 +46,16 @@ func NewFranzSyncProducer(clientCfg configkafka.ClientConfig,
4546
default:
4647
codec = codec.WithLevel(int(cfg.CompressionParams.Level))
4748
}
48-
opts := []kgo.Opt{
49-
kgo.SeedBrokers(clientCfg.Brokers...),
50-
kgo.WithLogger(kzap.New(logger.Named("kafka"))),
49+
opts, err := commonOpts(ctx, clientCfg, logger,
5150
kgo.ProduceRequestTimeout(timeout),
5251
kgo.ProducerBatchCompression(codec),
5352
// Use the UniformBytesPartitioner that is the default in franz-go with
5453
// the legacy compatibility sarama hashing to avoid hashing to different
5554
// partitions in case partitioning is enabled.
5655
kgo.RecordPartitioner(newSaramaCompatPartitioner()),
56+
)
57+
if err != nil {
58+
return nil, err
5759
}
5860
// Configure required acks
5961
switch cfg.RequiredAcks {
@@ -68,17 +70,128 @@ func NewFranzSyncProducer(clientCfg configkafka.ClientConfig,
6870
opts = append(opts, kgo.DisableIdempotentWrite())
6971
opts = append(opts, kgo.RequiredAcks(kgo.LeaderAck()))
7072
}
73+
74+
// Configure max message size
75+
if cfg.MaxMessageBytes > 0 {
76+
opts = append(opts, kgo.ProducerBatchMaxBytes(
77+
int32(cfg.MaxMessageBytes),
78+
))
79+
}
80+
// Configure batch size
81+
if cfg.FlushMaxMessages > 0 {
82+
opts = append(opts, kgo.MaxBufferedRecords(cfg.FlushMaxMessages))
83+
}
84+
85+
return kgo.NewClient(opts...)
86+
}
87+
88+
// NewFranzConsumerGroup creates a new Kafka consumer client using the franz-go library.
89+
func NewFranzConsumerGroup(ctx context.Context, clientCfg configkafka.ClientConfig,
90+
consumerCfg configkafka.ConsumerConfig,
91+
topics []string,
92+
logger *zap.Logger,
93+
opts ...kgo.Opt,
94+
) (*kgo.Client, error) {
95+
opts, err := commonOpts(ctx, clientCfg, logger, append([]kgo.Opt{
96+
kgo.ConsumeTopics(topics...),
97+
kgo.ConsumerGroup(consumerCfg.GroupID),
98+
}, opts...)...)
99+
if err != nil {
100+
return nil, err
101+
}
102+
103+
for _, t := range topics {
104+
// Similar to librdkafka, if the topic starts with `^`, it is a regex topic:
105+
// https://github.com/confluentinc/librdkafka/blob/b871fdabab84b2ea1be3866a2ded4def7e31b006/src/rdkafka.h#L3899-L3938
106+
if strings.HasPrefix(t, "^") {
107+
opts = append(opts, kgo.ConsumeRegex())
108+
break
109+
}
110+
}
111+
112+
// Configure session timeout
113+
if consumerCfg.SessionTimeout > 0 {
114+
opts = append(opts, kgo.SessionTimeout(consumerCfg.SessionTimeout))
115+
}
116+
117+
// Configure heartbeat interval
118+
if consumerCfg.HeartbeatInterval > 0 {
119+
opts = append(opts, kgo.HeartbeatInterval(consumerCfg.HeartbeatInterval))
120+
}
121+
122+
// Configure fetch sizes
123+
if consumerCfg.MinFetchSize > 0 {
124+
opts = append(opts, kgo.FetchMinBytes(consumerCfg.MinFetchSize))
125+
}
126+
if consumerCfg.DefaultFetchSize > 0 {
127+
opts = append(opts, kgo.FetchMaxBytes(consumerCfg.DefaultFetchSize))
128+
}
129+
130+
// Configure max fetch wait
131+
if consumerCfg.MaxFetchWait > 0 {
132+
opts = append(opts, kgo.FetchMaxWait(consumerCfg.MaxFetchWait))
133+
}
134+
135+
interval := consumerCfg.AutoCommit.Interval
136+
if !consumerCfg.AutoCommit.Enable {
137+
// Set auto-commit interval to a very high value to "disable" it, but
138+
// still allow using marks.
139+
interval = time.Hour
140+
}
141+
// Configure auto-commit to use marks, this simplifies the committing
142+
// logic and makes it more consistent with the Sarama client.
143+
opts = append(opts, kgo.AutoCommitMarks(),
144+
kgo.AutoCommitInterval(interval),
145+
)
146+
147+
// Configure the offset to reset to if an exception is found (or no current
148+
// partition offset is found.
149+
switch consumerCfg.InitialOffset {
150+
case configkafka.EarliestOffset:
151+
opts = append(opts, kgo.ConsumeResetOffset(kgo.NewOffset().AtStart()))
152+
case configkafka.LatestOffset:
153+
opts = append(opts, kgo.ConsumeResetOffset(kgo.NewOffset().AtEnd()))
154+
}
155+
156+
// Configure group instance ID if provided
157+
if consumerCfg.GroupInstanceID != "" {
158+
opts = append(opts, kgo.InstanceID(consumerCfg.GroupInstanceID))
159+
}
160+
161+
// Configure rebalance strategy
162+
switch consumerCfg.GroupRebalanceStrategy {
163+
case "range": // Sarama default.
164+
opts = append(opts, kgo.Balancers(kgo.RangeBalancer()))
165+
case "roundrobin":
166+
opts = append(opts, kgo.Balancers(kgo.RoundRobinBalancer()))
167+
case "sticky":
168+
opts = append(opts, kgo.Balancers(kgo.StickyBalancer()))
169+
// NOTE(marclop): This is a new type of balancer, document accordingly.
170+
case "cooperative-sticky":
171+
opts = append(opts, kgo.Balancers(kgo.CooperativeStickyBalancer()))
172+
}
173+
return kgo.NewClient(opts...)
174+
}
175+
176+
func commonOpts(ctx context.Context, clientCfg configkafka.ClientConfig,
177+
logger *zap.Logger,
178+
opts ...kgo.Opt,
179+
) ([]kgo.Opt, error) {
180+
opts = append(opts,
181+
kgo.WithLogger(kzap.New(logger.Named("franz"))),
182+
kgo.SeedBrokers(clientCfg.Brokers...),
183+
)
71184
// Configure TLS if needed
72185
if clientCfg.TLS != nil {
73-
tlsCfg, err := clientCfg.TLS.LoadTLSConfig(context.Background())
186+
tlsCfg, err := clientCfg.TLS.LoadTLSConfig(ctx)
74187
if err != nil {
75188
return nil, fmt.Errorf("failed to load TLS config: %w", err)
76189
}
77190
if tlsCfg != nil {
78191
opts = append(opts, kgo.DialTLSConfig(tlsCfg))
79192
}
80193
}
81-
// Configure Auth
194+
// Configure authentication
82195
if clientCfg.Authentication.PlainText != nil {
83196
auth := plain.Auth{
84197
User: clientCfg.Authentication.PlainText.Username,
@@ -104,18 +217,7 @@ func NewFranzSyncProducer(clientCfg configkafka.ClientConfig,
104217
if clientCfg.ClientID != "" {
105218
opts = append(opts, kgo.ClientID(clientCfg.ClientID))
106219
}
107-
// Configure max message size
108-
if cfg.MaxMessageBytes > 0 {
109-
opts = append(opts, kgo.ProducerBatchMaxBytes(
110-
int32(cfg.MaxMessageBytes),
111-
))
112-
}
113-
// Configure batch size
114-
if cfg.FlushMaxMessages > 0 {
115-
opts = append(opts, kgo.MaxBufferedRecords(cfg.FlushMaxMessages))
116-
}
117-
118-
return kgo.NewClient(opts...)
220+
return opts, nil
119221
}
120222

121223
func configureKgoSASL(cfg *configkafka.SASLConfig) (kgo.Opt, error) {

0 commit comments

Comments
 (0)