franz-go - A complete Apache Kafka client written in Go
Franz-go is an all-encompassing Apache Kafka client fully written Go. This library aims to provide every Kafka feature from Apache Kafka v0.8.0 onward. It has support for transactions, regex topic consuming, the latest partitioning strategies, data loss detection, closest replica fetching, and more. If a client KIP exists, this library aims to support it.
This library attempts to provide an intuitive API while interacting with Kafka the way Kafka expects (timeouts, etc.).
Features
- Feature complete client (Kafka >= 0.8.0 through v3.4+)
- Full Exactly-Once-Semantics (EOS)
- Idempotent & transactional producers
- Simple (legacy) consumer
- Group consumers with eager (roundrobin, range, sticky) and cooperative (cooperative-sticky) balancers
- All compression types supported: gzip, snappy, lz4, zstd
- SSL/TLS provided through custom dialer options
- All SASL mechanisms supported (GSSAPI/Kerberos, PLAIN, SCRAM, and OAUTHBEARER)
- Low-level admin functionality supported through a simple
Request
function - High-level admin package with many helper types to make cluster administration easy.
- Utilizes modern & idiomatic Go (support for contexts, variadic configuration options, ...)
- Highly performant by avoiding channels and goroutines where not necessary
- Written in pure Go (no wrapper lib for a C library or other bindings)
- Ability to add detailed log messages or metrics using hooks
- Plug-in metrics support for prometheus, zap, etc.
- An admin client with many helper functions for easy admin tasks
- A schema registry client and convenience Serde type for encoding and decoding
Works with any Kafka compatible brokers:
- Redpanda: the fastest and most efficient Kafka compatible event streaming platform
- Kafka: the original Java project
- Microsoft Event Hubs
- Event Hubs does not support producing with compression; be sure to use
kgo.ProducerBatchCompression(kgo.NoCompression)
.
- Event Hubs does not support producing with compression; be sure to use
- Amazon MSK
Install
This repo contains multiple tags to allow separate features to be developed and
released independently. The main client is in franz-go. Plugins are released
from plugin/{plugin}
. The raw-protocol package is released from pkg/kmsg
,
and the admin package is released from pkg/kadm
.
The main client is located in the package github.com/twmb/franz-go/pkg/kgo
,
while the root of the project is at github.com/twmb/franz-go
. There are
a few extra packages within the project, as well as a few sub-modules. To
use the main kgo package,
go get github.com/twmb/franz-go
To use a plugin,
go get github.com/twmb/franz-go/plugin/kzap
To use kadm,
go get github.com/twmb/franz-go/pkg/kadm
As an example, your require section in go.mod may look like this:
require (
github.com/twmb/franz-go v1.12.0
github.com/twmb/franz-go/pkg/kmsg v1.4.0
)
Getting started
Here's a basic overview of producing and consuming:
seeds := []string{"localhost:9092"}
// One client can both produce and consume!
// Consuming can either be direct (no consumer group), or through a group. Below, we use a group.
cl, err := kgo.NewClient(
kgo.SeedBrokers(seeds...),
kgo.ConsumerGroup("my-group-identifier"),
kgo.ConsumeTopics("foo"),
)
if err != nil {
panic(err)
}
defer cl.Close()
ctx := context.Background()
// 1.) Producing a message
// All record production goes through Produce, and the callback can be used
// to allow for synchronous or asynchronous production.
var wg sync.WaitGroup
wg.Add(1)
record := &kgo.Record{Topic: "foo", Value: []byte("bar")}
cl.Produce(ctx, record, func(_ *kgo.Record, err error) {
defer wg.Done()
if err != nil {
fmt.Printf("record had a produce error: %v\n", err)
}
})
wg.Wait()
// Alternatively, ProduceSync exists to synchronously produce a batch of records.
if err := cl.ProduceSync(ctx, record).FirstErr(); err != nil {
fmt.Printf("record had a produce error while synchronously producing: %v\n", err)
}
// 2.) Consuming messages from a topic
for {
fetches := cl.PollFetches(ctx)
if errs := fetches.Errors(); len(errs) > 0 {
// All errors are retried internally when fetching, but non-retriable errors are
// returned from polls so that users can notice and take action.
panic(fmt.Sprint(errs))
}
// We can iterate through a record iterator...
iter := fetches.RecordIter()
for !iter.Done() {
record := iter.Next()
fmt.Println(string(record.Value), "from an iterator!")
}
// or a callback function.
fetches.EachPartition(func(p kgo.FetchTopicPartition) {
for _, record := range p.Records {
fmt.Println(string(record.Value), "from range inside a callback!")
}
// We can even use a second callback!
p.EachRecord(func(record *kgo.Record) {
fmt.Println(string(record.Value), "from a second callback!")
})
})
}
This only shows producing and consuming in the most basic sense, and does not show the full list of options to customize how the client runs, nor does it show transactional producing / consuming. Check out the examples directory for more!
API reference documentation can be found on . Supplementary information can be found in the docs directory:
docs βββ admin requests β an overview of how to issue admin requests βββ metrics and logging β a small writeup on how to enable metrics & logging in franz-go, as well as a few thoughts on latency tracking βββ package layout β describes the packages in franz-go βββ producing and consuming β descriptions of producing & consuming & the guarantees βββ transactions β a description of transactions and the safety even in a pre-KIP-447 world
Who uses this?
In alphabetical order,
- Alpaca
- Banyan
- Benthos
- Eoitek
- Mux
- Redpanda Console
- Redpanda Data
- StoneCo
- ThinkingData
- Unistack (Cloud Management System)
- Unity Technologies
If you use this library and want on the list above, please either open a PR or comment on #142!
Version Pinning
By default, the client issues an ApiVersions request on connect to brokers and
defaults to using the maximum supported version for requests that each broker
supports. If you want to pin to an exact version, you can use the MaxVersions
option.
Kafka 0.10.0 introduced the ApiVersions request; if you are working with brokers older than that, you must use the kversions package. Use the MaxVersions option for the client if you do so.
Metrics & logging
Note there exists plug-in packages that allow you to easily add prometheus
metrics, go-metrics, zap logging, etc. to your client! See the plugin
directory for more information! These plugins are provided under dedicated
modules, e.g. github.com/twmb/franz-go/plugin/[email protected]
.
The franz-go client takes a neutral approach to metrics by providing hooks that you can use to plug in your own metrics.
All connections, disconnections, reads, writes, and throttles can be hooked into, as well as per-batch produce & consume metrics. If there is an aspect of the library that you wish you could have insight into, please open an issue and we can discuss adding another hook.
Hooks allow you to log in the event of specific errors, or to trace latencies, count bytes, etc., all with your favorite monitoring systems.
In addition to hooks, logging can be plugged in with a general Logger
interface. A basic logger is provided if you just want to write to a given
file in a simple format. All logs have a message and then key/value pairs of
supplementary information. It is recommended to always use a logger and to use
LogLevelInfo
.
See this example for an expansive example of integrating with prometheus! Alternatively, see this example for how to use the plug-in prometheus package!
Benchmarks
This client is quite fast; it is the fastest and most cpu and memory efficient client in Go.
For 100 byte messages,
-
This client is 4x faster at producing than confluent-kafka-go, and up to 10x-20x faster (at the expense of more memory usage) at consuming.
-
This client is 2.5x faster at producing than sarama, and 1.5x faster at consuming.
-
This client is 2.4x faster at producing than segment's kafka-go, and anywhere from 2x to 6x faster at consuming.
To check benchmarks yourself, see the bench example. This example lets you produce or consume to a cluster and see the byte / record rate. The compare subdirectory shows comparison code.
Supported KIPs
Theoretically, this library supports every (non-Java-specific) client facing KIP. Any KIP that simply adds or modifies a protocol is supported by code generation.
KIP | Kafka release | Status |
---|---|---|
KIP-1 β Disallow acks > 1 | 0.8.3 | Supported & Enforced |
KIP-4 β Request protocol changes | 0.9.0 through 0.10.1 | Supported |
KIP-8 β Flush method on Producer | 0.8.3 | Supported |
KIP-12 β SASL & SSL | 0.9.0 | Supported |
KIP-13 β Throttling (on broker) | 0.9.0 | Supported |
KIP-15 β Close with a timeout | 0.9.0 | Supported (via context) |
KIP-19 β Request timeouts | 0.9.0 | Supported |
KIP-22 β Custom partitioners | 0.9.0 | Supported |
KIP-31 β Relative offsets in message sets | 0.10.0 | Supported |
KIP-32 β Timestamps in message set v1 | 0.10.0 | Supported |
KIP-35 β ApiVersion | 0.10.0 | Supported |
KIP-40 β ListGroups and DescribeGroups | 0.9.0 | Supported |
KIP-41 β max.poll.records | 0.10.0 | Supported (via PollRecords) |
KIP-42 β Producer & consumer interceptors | 0.10.0 | Partial support (hooks) |
KIP-43 β SASL PLAIN & handshake | 0.10.0 | Supported |
KIP-48 β Delegation tokens | 1.1 | Supported |
KIP-54 β Sticky partitioning | 0.11.0 | Supported |
KIP-57 β Fix lz4 | 0.10.0 | Supported |
KIP-62 β background heartbeats & improvements | 0.10.1 | Supported |
KIP-70 β On{Assigned,Revoked} | 0.10.1 | Supported |
KIP-74 β Fetch response size limits | 0.10.1 | Supported |
KIP-78 β ClusterID in Metadata | 0.10.1 | Supported |
KIP-79 β List offsets for times | 0.10.1 | Supported |
KIP-81 β Bound fetch memory usage | WIP | Supported (through a combo of options) |
KIP-82 β Record headers | 0.11.0 | Supported |
KIP-84 β SASL SCRAM | 0.10.2 | Supported |
KIP-86 β SASL Callbacks | 0.10.2 | Supported (through callback fns) |
KIP-88 β OffsetFetch for admins | 0.10.2 | Supported |
KIP-91 β Intuitive producer timeouts | 2.1 | Supported (as a matter of opinion) |
KIP-97 β Backwards compat for old brokers | 0.10.2 | Supported |
KIP-98 β EOS | 0.11.0 | Supported |
KIP-101 β OffsetForLeaderEpoch v0 | 0.11.0 | Supported |
KIP-102 β Consumer close timeouts | 0.10.2 | Supported (via context) |
KIP-107 β DeleteRecords | 0.11.0 | Supported |
KIP-108 β CreateTopic validate only field | 0.10.2 | Supported |
KIP-110 β zstd | 2.1 | Supported |
KIP-112 β Broker request protocol changes | 1.0 | Supported |
KIP-113 β LogDir requests | 1.0 | Supported |
KIP-117 β Admin client | 0.11.0 | Supported (via kmsg) |
KIP-124 β Request rate quotas | 0.11.0 | Supported |
KIP-126 β Ensure proper batch size after compression | 0.11.0 | Supported (avoided entirely) |
KIP-133 β Describe & Alter configs | 0.11.0 | Supported |
KIP-140 β ACLs | 0.11.0 | Supported |
KIP-144 β Broker reconnect backoff | 0.11.0 | Supported |
KIP-152 β More SASL; SASLAuthenticate | 1.0 | Supported |
KIP-183 β Elect preferred leaders | 2.2 | Supported |
KIP-185 β Idempotency is default | 1.0 | Supported |
KIP-192 β Cleaner idempotence semantics | 1.0 | Supported |
KIP-195 β CreatePartitions | 1.0 | Supported |
KIP-204 β DeleteRecords via admin API | 1.1 | Supported |
KIP-207 β New error in ListOffsets | 2.2 | Supported |
KIP-219 β Client-side throttling | 2.0 | Supported |
KIP-222 β Group operations via admin API | 2.0 | Supported |
KIP-226 β Describe configs v1 | 1.1 | Supported |
KIP-227 β Incremental fetch | 1.1 | Supported |
KIP-229 β DeleteGroups | 1.1 | Supported |
KIP-249 β Delegation tokens in admin API | 2.0 | Supported |
KIP-255 β SASL OAUTHBEARER | 2.0 | Supported |
KIP-266 β Fix indefinite consumer timeouts | 2.0 | Supported (via context) |
KIP-279 β OffsetForLeaderEpoch bump | 2.0 | Supported |
KIP-289 β Default group.id to null | 2.2 | Supported |
KIP-294 β TLS verification | 2.0 | Supported (via dialer) |
KIP-302 β Use multiple addrs for resolved hostnames | 2.1 | Supported (via dialer) |
KIP-320 β Fetcher: detect log truncation | 2.1 | Supported |
KIP-322 β DeleteTopics disabled error code | 2.1 | Supported |
KIP-339 β IncrementalAlterConfigs | 2.3 | Supported |
KIP-341 β Sticky group bugfix | ? | Supported |
KIP-342 β OAUTHBEARER extensions | 2.1 | Supported |
KIP-345 β Static group membership | 2.4 | Supported |
KIP-357 β List ACLs per principal via admin API | 2.1 | Supported |
KIP-360 β Safe epoch bumping for UNKNOWN_PRODUCER_ID |
2.5 | Supported |
KIP-361 β Allow disable auto topic creation | 2.3 | Supported |
KIP-368 β Periodically reauthenticate SASL | 2.2 | Supported |
KIP-369 β An always round robin produce partitioner | 2.4 | Supported |
KIP-373 β Users can create delegation tokens for others | 3.3 | Supported |
KIP-380 β Inter-broker protocol changes | 2.2 | Supported |
KIP-389 β Group max size error | 2.2 | Supported |
KIP-392 β Closest replica fetching w/ rack | 2.2 | Supported |
KIP-394 β Require member.id for initial join request | 2.2 | Supported |
KIP-396 β Commit offsets manually | 2.4 | Supported |
KIP-412 β Dynamic log levels w/ IncrementalAlterConfigs | 2.4 | Supported |
KIP-429 β Incremental rebalance (see KAFKA-8179) | 2.4 | Supported |
KIP-430 β Authorized ops in DescribeGroups | 2.3 | Supported |
KIP-447 β Producer scalability for EOS | 2.5 | Supported |
KIP-455 β Replica reassignment API | 2.4 | Supported |
KIP-460 β Leader election API | 2.4 | Supported |
KIP-464 β CreateTopic defaults | 2.4 | Supported |
KIP-467 β Per-record error codes when producing | 2.4 | Supported (and ignored) |
KIP-480 β Sticky partition producing | 2.4 | Supported |
KIP-482 β Tagged fields (KAFKA-8885) | 2.4 | Supported |
KIP-496 β OffsetDelete admin command | 2.4 | Supported |
KIP-497 β New AlterISR API | 2.7 | Supported |
KIP-498 β Max bound on reads | ? | Supported |
KIP-511 β Client name/version in ApiVersions request | 2.4 | Supported |
KIP-514 β Bounded Flush | 2.4 | Supported (via context) |
KIP-516 β Topic IDs | ??? | Supported as it is implemented |
KIP-518 β List groups by state | 2.6 | Supported |
KIP-519 β Configurable SSL "engine" | 2.6 | Supported (via dialer) |
KIP-525 β CreateTopics v5 returns configs | 2.4 | Supported |
KIP-526 β Reduce metadata lookups | 2.5 | Supported |
KIP-533 β Default API timeout (total time, not per request) | 2.5 | Supported (via RetryTimeout) |
KIP-546 β Client Quota APIs | 2.5 | Supported |
KIP-554 β Broker side SCRAM APIs | 2.7 | Supported |
KIP-559 β Protocol info in sync/join | 2.5 | Supported |
KIP-568 β Explicit rebalance triggering on the consumer | 2.6 | Supported |
KIP-569 β Docs & type in DescribeConfigs | 2.6 | Supported |
KIP-570 β Leader epoch in StopReplica | 2.6 | Supported |
KIP-580 β Exponential backoff | 2.6 | Supported |
KIP-584 β Versioning scheme for features | ? | Supported (nothing to do yet) |
KIP-588 β Producer recovery from txn timeout | 2.7 | Supported |
KIP-590 β Envelope (broker only) | 2.7 | Supported |
KIP-595 β New APIs for raft protocol | 2.7 | Supported |
KIP-599 β Throttling on create/delete topic/partition | 2.7 | Supported |
KIP-602 β Use all resolved addrs by default | 2.6 | Supported (via dialer) |
KIP-651 β Support PEM | 2.7 | Supported (via dialer) |
KIP-654 β Aborted txns with unflushed data is not fatal | 2.7 | Supported (default behavior) |
KIP-664 β Describe producers / etc. | 2.8 (mostly) | Supported |
KIP-679 β Strongest producer guarantee by default | 3.0 | Supported (by default always) |
KIP-699 β Batch FindCoordinators | 3.0 | Supported |
KIP-700 β DescribeCluster | 2.8 | Supported |
KIP-704 β AlterISR => AlterPartition | 3.2 | Supported |
KIP-709 β Batch OffsetFetch | 3.0 | Supported |
KIP-730 - AllocateProducerIDs | 3.0 | Supported |
KIP-734 β Support MaxTimestamp in ListOffsets | 3.0 | Supported (simple version bump) |
KIP-735 β Bump default session timeout | ? | Supported |
KIP-778 β KRaft Upgrades (protocol changes only) | 3.2 | Supported |
KIP-784 β Add ErrorCode to DescribeLogDirs response | 3.1 | Supported |
KIP-792 β Generation field in consumer group protocol | 3.4 | Supported |
KIP-794 β Better sticky partitioning | 3.3 | Supported (UniformBytesPartitioner ) |
KIP-800 β Reason in Join/Leave group | 3.1 | Supported |
KIP-814 β SkipAssignment for static group leaders | 3.1 | Supported |
KIP-827 β DescribeLogDirs.{Total,Usable}Bytes |
3.3 | Supported |
KIP-836 β DescribeQuorum voter lag info |
3.3 | Supported |
KIP-841 β AlterPartition.TopicID |
3.3 | Supported |
KIP-866 β ZK to Raft RPC changes | 3.4 | Supported |
KIP-893 β Nullable structs in the protocol | 3.5 | Supported |
KIP-899 β Allow clients to rebootstrap | ? | Supported (UpdateSeedBrokers ) |
Missing from above but included in librdkafka is:
- KIP-85, which does not seem relevant for franz-go
- KIP-92 for consumer lag metrics, which is better suited for an external system via the admin api
- KIP-223 for more metrics
- KIP-235, which is confusing but may be implement via a custom dialer and custom kerberos?
- KIP-359 to verify leader epoch when producing; this is easy to support but actually is not implemented in Kafka yet
- KIP-421 for dynamic values in configs; librdkafka mentions it does not support it, and neither does franz-go for the same reason (we do not use a config file)
- KIP-436 is about yet another metric
- KIP-517, more metrics