...

Text file src/github.com/twmb/franz-go/README.md

Documentation: github.com/twmb/franz-go

     1franz-go - A complete Apache Kafka client written in Go
     2===
     3
     4[![GoDev](https://img.shields.io/static/v1?label=godev&message=reference&color=00add8)][godev]
     5![GitHub](https://img.shields.io/github/license/twmb/franz-go)
     6[![Discord Chat](https://img.shields.io/badge/discord-online-brightgreen.svg)](https://discord.gg/K4R5c8zsMS)
     7
     8[godev]: https://pkg.go.dev/github.com/twmb/franz-go/pkg/kgo
     9
    10Franz-go is an all-encompassing Apache Kafka client fully written Go. This library aims to provide **every Kafka feature** from
    11Apache Kafka v0.8.0 onward. It has support for transactions, regex topic consuming, the latest partitioning strategies,
    12data loss detection, closest replica fetching, and more. If a client KIP exists, this library aims to support it.
    13
    14This library attempts to provide an intuitive API while interacting with Kafka the way Kafka expects (timeouts, etc.).
    15
    16## Features
    17
    18- Feature complete client (Kafka >= 0.8.0 through v3.4+)
    19- Full Exactly-Once-Semantics (EOS)
    20- Idempotent & transactional producers
    21- Simple (legacy) consumer
    22- Group consumers with eager (roundrobin, range, sticky) and cooperative (cooperative-sticky) balancers
    23- All compression types supported: gzip, snappy, lz4, zstd
    24- SSL/TLS provided through custom dialer options
    25- All SASL mechanisms supported (GSSAPI/Kerberos, PLAIN, SCRAM, and OAUTHBEARER)
    26- Low-level admin functionality supported through a simple `Request` function
    27- High-level admin package with many helper types to make cluster administration easy.
    28- Utilizes modern & idiomatic Go (support for contexts, variadic configuration options, ...)
    29- Highly performant by avoiding channels and goroutines where not necessary
    30- Written in pure Go (no wrapper lib for a C library or other bindings)
    31- Ability to add detailed log messages or metrics using hooks
    32- Plug-in metrics support for prometheus, zap, etc.
    33- An [admin client][KADMC] with many helper functions for easy admin tasks
    34- A [schema registry client][SRC] and convenience Serde type for encoding and decoding
    35
    36[KADMC]: https://pkg.go.dev/github.com/twmb/franz-go/pkg/kadm
    37[SRC]: https://pkg.go.dev/github.com/twmb/franz-go/pkg/sr
    38
    39## Works with any Kafka compatible brokers:
    40
    41- [**Redpanda**](https://vectorized.io/): the fastest and most efficient Kafka compatible event streaming platform
    42- **Kafka**: the original Java project
    43- **Microsoft Event Hubs**
    44  - Event Hubs does [not support][MSEH] producing with compression; be sure to use `kgo.ProducerBatchCompression(kgo.NoCompression)`.
    45- **Amazon MSK**
    46
    47[MSEH]: https://docs.microsoft.com/en-us/azure/event-hubs/apache-kafka-troubleshooting-guide#compressionmessage-format-version-issue
    48
    49## Install
    50
    51This repo contains multiple tags to allow separate features to be developed and
    52released independently. The main client is in franz-go. Plugins are released
    53from `plugin/{plugin}`. The raw-protocol package is released from `pkg/kmsg`,
    54and the admin package is released from `pkg/kadm`.
    55
    56The main client is located in the package `github.com/twmb/franz-go/pkg/kgo`,
    57while the root of the project is at `github.com/twmb/franz-go`. There are
    58a few extra packages within the project, as well as a few sub-modules. To
    59use the main kgo package,
    60
    61```
    62go get github.com/twmb/franz-go
    63```
    64
    65To use a plugin,
    66
    67```
    68go get github.com/twmb/franz-go/plugin/kzap
    69```
    70
    71To use kadm,
    72
    73```
    74go get github.com/twmb/franz-go/pkg/kadm
    75```
    76
    77As an example, your require section in go.mod may look like this:
    78
    79
    80```
    81require (
    82	github.com/twmb/franz-go v1.12.0
    83	github.com/twmb/franz-go/pkg/kmsg v1.4.0
    84)
    85```
    86
    87## Getting started
    88
    89Here's a basic overview of producing and consuming:
    90
    91```go
    92seeds := []string{"localhost:9092"}
    93// One client can both produce and consume!
    94// Consuming can either be direct (no consumer group), or through a group. Below, we use a group.
    95cl, err := kgo.NewClient(
    96	kgo.SeedBrokers(seeds...),
    97	kgo.ConsumerGroup("my-group-identifier"),
    98	kgo.ConsumeTopics("foo"),
    99)
   100if err != nil {
   101	panic(err)
   102}
   103defer cl.Close()
   104
   105ctx := context.Background()
   106
   107// 1.) Producing a message
   108// All record production goes through Produce, and the callback can be used
   109// to allow for synchronous or asynchronous production.
   110var wg sync.WaitGroup
   111wg.Add(1)
   112record := &kgo.Record{Topic: "foo", Value: []byte("bar")}
   113cl.Produce(ctx, record, func(_ *kgo.Record, err error) {
   114	defer wg.Done()
   115	if err != nil {
   116		fmt.Printf("record had a produce error: %v\n", err)
   117	}
   118
   119})
   120wg.Wait()
   121
   122// Alternatively, ProduceSync exists to synchronously produce a batch of records.
   123if err := cl.ProduceSync(ctx, record).FirstErr(); err != nil {
   124	fmt.Printf("record had a produce error while synchronously producing: %v\n", err)
   125}
   126
   127// 2.) Consuming messages from a topic
   128for {
   129	fetches := cl.PollFetches(ctx)
   130	if errs := fetches.Errors(); len(errs) > 0 {
   131		// All errors are retried internally when fetching, but non-retriable errors are
   132		// returned from polls so that users can notice and take action.
   133		panic(fmt.Sprint(errs))
   134	}
   135
   136	// We can iterate through a record iterator...
   137	iter := fetches.RecordIter()
   138	for !iter.Done() {
   139		record := iter.Next()
   140		fmt.Println(string(record.Value), "from an iterator!")
   141	}
   142
   143	// or a callback function.
   144	fetches.EachPartition(func(p kgo.FetchTopicPartition) {
   145		for _, record := range p.Records {
   146			fmt.Println(string(record.Value), "from range inside a callback!")
   147		}
   148
   149		// We can even use a second callback!
   150		p.EachRecord(func(record *kgo.Record) {
   151			fmt.Println(string(record.Value), "from a second callback!")
   152		})
   153	})
   154}
   155```
   156
   157This only shows producing and consuming in the most basic sense, and does not
   158show the full list of options to customize how the client runs, nor does it
   159show transactional producing / consuming. Check out the [examples](./examples)
   160directory for more!
   161
   162API reference documentation can be found on
   163[![GoDev](https://img.shields.io/static/v1?label=godev&message=reference&color=00add8)][godev].
   164Supplementary information can be found in the docs directory:
   165
   166<pre>
   167<a href="./docs">docs</a>
   168├── <a href="./docs/admin-requests.md">admin requests</a> — an overview of how to issue admin requests
   169├── <a href="./docs/metrics-and-logging.md">metrics and logging</a> — a small writeup on how to enable metrics & logging in franz-go, as well as a few thoughts on latency tracking
   170├── <a href="./docs/package-layout.md">package layout</a> — describes the packages in franz-go
   171├── <a href="./docs/producing-and-consuming.md">producing and consuming</a> — descriptions of producing & consuming & the guarantees
   172└── <a href="./docs/transactions.md">transactions</a> — a description of transactions and the safety even in a pre-KIP-447 world
   173</pre>
   174
   175## Who uses this?
   176
   177In alphabetical order,
   178
   179* [Alpaca](https://alpaca.markets/blog/alpaca-launches-next-gen-order-management-system/)
   180* [Banyan](https://banyan.com)
   181* [Benthos](https://www.benthos.dev/)
   182* [DeltaStream](https://deltastream.io/)
   183* [Eoitek](https://eoitek.com/)
   184* [Mux](https://mux.com/blog/processing-cdn-logs-exactly-once-with-kafka-transactions/)
   185* [Redpanda Console](https://github.com/redpanda-data/console/)
   186* [Redpanda Data](https://redpanda.com/)
   187* [StoneCo](https://www.stone.com.br/)
   188* [ThinkingData](https://thinkingdata.cn)
   189* [Unistack](https://unistack.org) (Cloud Management System)
   190* [Unity Technologies](https://unity.com)
   191* [Zomato](https://www.zomato.com/)
   192
   193If you use this library and want on the list above, please either open a PR
   194or comment on [#142](https://github.com/twmb/franz-go/issues/142)!
   195
   196## Version Pinning
   197
   198By default, the client issues an ApiVersions request on connect to brokers and
   199defaults to using the maximum supported version for requests that each broker
   200supports. If you want to pin to an exact version, you can use the `MaxVersions`
   201option.
   202
   203Kafka 0.10.0 introduced the ApiVersions request; if you are working with
   204brokers older than that, you must use the kversions package. Use the
   205MaxVersions option for the client if you do so.
   206
   207## Metrics & logging
   208
   209**Note** there exists plug-in packages that allow you to easily add prometheus
   210metrics, go-metrics, zap logging, etc. to your client! See the [plugin](./plugin)
   211directory for more information! These plugins are provided under dedicated
   212modules, e.g. `github.com/twmb/franz-go/plugin/kprom@v1.0.0`.
   213
   214The franz-go client takes a neutral approach to metrics by providing hooks
   215that you can use to plug in your own metrics.
   216
   217All connections, disconnections, reads, writes, and throttles can be hooked
   218into, as well as per-batch produce & consume metrics. If there is an aspect of
   219the library that you wish you could have insight into, please open an issue and
   220we can discuss adding another hook.
   221
   222Hooks allow you to log in the event of specific errors, or to trace latencies,
   223count bytes, etc., all with your favorite monitoring systems.
   224
   225In addition to hooks, logging can be plugged in with a general `Logger`
   226interface.  A basic logger is provided if you just want to write to a given
   227file in a simple format. All logs have a message and then key/value pairs of
   228supplementary information. It is recommended to always use a logger and to use
   229`LogLevelInfo`.
   230
   231See [this example](./examples/hooks_and_logging/expansive_prometheus) for an expansive example of
   232integrating with prometheus! Alternatively, see [this example](./examples/hooks_and_logging/plugin_prometheus)
   233for how to use the plug-in prometheus package!
   234
   235## Benchmarks
   236
   237This client is quite fast; it is the fastest and most cpu and memory efficient
   238client in Go.
   239
   240For 100 byte messages,
   241
   242- This client is 4x faster at producing than confluent-kafka-go, and up to
   243  10x-20x faster (at the expense of more memory usage) at consuming.
   244
   245- This client is 2.5x faster at producing than sarama, and 1.5x faster at
   246  consuming.
   247
   248- This client is 2.4x faster at producing than segment's kafka-go, and anywhere
   249  from 2x to 6x faster at consuming.
   250
   251To check benchmarks yourself, see the [bench](./examples/bench) example. This
   252example lets you produce or consume to a cluster and see the byte / record
   253rate. The [compare](./examples/bench/compare) subdirectory shows comparison
   254code.
   255
   256## Supported KIPs
   257
   258Theoretically, this library supports every (non-Java-specific) client facing
   259KIP. Any KIP that simply adds or modifies a protocol is supported by code
   260generation.
   261
   262| KIP |  Kafka release | Status |
   263|-----|----------------|--------|
   264| [KIP-1](https://cwiki.apache.org/confluence/display/KAFKA/KIP-1+-+Remove+support+of+request.required.acks) — Disallow acks > 1 | 0.8.3 | Supported & Enforced |
   265| [KIP-4](https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations) — Request protocol changes | 0.9.0 through 0.10.1 |  Supported |
   266| [KIP-8](https://cwiki.apache.org/confluence/display/KAFKA/KIP-8+-+Add+a+flush+method+to+the+producer+API) — Flush method on Producer | 0.8.3 | Supported |
   267| [KIP-12](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=51809888) — SASL & SSL | 0.9.0 | Supported |
   268| [KIP-13](https://cwiki.apache.org/confluence/display/KAFKA/KIP-13+-+Quotas) — Throttling (on broker) | 0.9.0 | Supported |
   269| [KIP-15](https://cwiki.apache.org/confluence/display/KAFKA/KIP-15+-+Add+a+close+method+with+a+timeout+in+the+producer) — Close with a timeout | 0.9.0 | Supported (via context) |
   270| [KIP-19](https://cwiki.apache.org/confluence/display/KAFKA/KIP-19+-+Add+a+request+timeout+to+NetworkClient) — Request timeouts | 0.9.0 | Supported |
   271| [KIP-22](https://cwiki.apache.org/confluence/display/KAFKA/KIP-22+-+Expose+a+Partitioner+interface+in+the+new+producer) — Custom partitioners | 0.9.0 | Supported |
   272| [KIP-31](https://cwiki.apache.org/confluence/display/KAFKA/KIP-31+-+Move+to+relative+offsets+in+compressed+message+sets) — Relative offsets in message sets | 0.10.0 | Supported |
   273| [KIP-32](https://cwiki.apache.org/confluence/display/KAFKA/KIP-32+-+Add+timestamps+to+Kafka+message) — Timestamps in message set v1 | 0.10.0 | Supported |
   274| [KIP-35](https://cwiki.apache.org/confluence/display/KAFKA/KIP-35+-+Retrieving+protocol+version) — ApiVersion | 0.10.0 | Supported |
   275| [KIP-40](https://cwiki.apache.org/confluence/display/KAFKA/KIP-40%3A+ListGroups+and+DescribeGroup) — ListGroups and DescribeGroups | 0.9.0 | Supported |
   276| [KIP-41](https://cwiki.apache.org/confluence/display/KAFKA/KIP-41%3A+KafkaConsumer+Max+Records) — max.poll.records | 0.10.0 | Supported (via PollRecords) |
   277| [KIP-42](https://cwiki.apache.org/confluence/display/KAFKA/KIP-42%3A+Add+Producer+and+Consumer+Interceptors) — Producer & consumer interceptors | 0.10.0 | Partial support (hooks) |
   278| [KIP-43](https://cwiki.apache.org/confluence/display/KAFKA/KIP-43%3A+Kafka+SASL+enhancements) — SASL PLAIN & handshake | 0.10.0 | Supported |
   279| [KIP-48](https://cwiki.apache.org/confluence/display/KAFKA/KIP-48+Delegation+token+support+for+Kafka) — Delegation tokens | 1.1 | Supported |
   280| [KIP-54](https://cwiki.apache.org/confluence/display/KAFKA/KIP-54+-+Sticky+Partition+Assignment+Strategy) — Sticky partitioning | 0.11.0 | Supported |
   281| [KIP-57](https://cwiki.apache.org/confluence/display/KAFKA/KIP-57+-+Interoperable+LZ4+Framing) — Fix lz4 | 0.10.0 | Supported |
   282| [KIP-62](https://cwiki.apache.org/confluence/display/KAFKA/KIP-62%3A+Allow+consumer+to+send+heartbeats+from+a+background+thread) — background heartbeats & improvements | 0.10.1 | Supported |
   283| [KIP-70](https://cwiki.apache.org/confluence/display/KAFKA/KIP-70%3A+Revise+Partition+Assignment+Semantics+on+New+Consumer%27s+Subscription+Change) — On{Assigned,Revoked} | 0.10.1 | Supported |
   284| [KIP-74](https://cwiki.apache.org/confluence/display/KAFKA/KIP-74%3A+Add+Fetch+Response+Size+Limit+in+Bytes) — Fetch response size limits | 0.10.1 | Supported |
   285| [KIP-78](https://cwiki.apache.org/confluence/display/KAFKA/KIP-78%3A+Cluster+Id) — ClusterID in Metadata | 0.10.1 | Supported |
   286| [KIP-79](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=65868090) — List offsets for times | 0.10.1 | Supported |
   287| [KIP-81](https://cwiki.apache.org/confluence/display/KAFKA/KIP-81%3A+Bound+Fetch+memory+usage+in+the+consumer) — Bound fetch memory usage | WIP | Supported (through a combo of options) |
   288| [KIP-82](https://cwiki.apache.org/confluence/display/KAFKA/KIP-82+-+Add+Record+Headers) — Record headers | 0.11.0 | Supported |
   289| [KIP-84](https://cwiki.apache.org/confluence/display/KAFKA/KIP-84%3A+Support+SASL+SCRAM+mechanisms) — SASL SCRAM | 0.10.2 | Supported |
   290| [KIP-86](https://cwiki.apache.org/confluence/display/KAFKA/KIP-86%3A+Configurable+SASL+callback+handlers) — SASL Callbacks | 0.10.2 | Supported (through callback fns) |
   291| [KIP-88](https://cwiki.apache.org/confluence/display/KAFKA/KIP-88%3A+OffsetFetch+Protocol+Update) — OffsetFetch for admins | 0.10.2 | Supported
   292| [KIP-91](https://cwiki.apache.org/confluence/display/KAFKA/KIP-91+Provide+Intuitive+User+Timeouts+in+The+Producer) — Intuitive producer timeouts | 2.1 | Supported (as a matter of opinion) |
   293| [KIP-97](https://cwiki.apache.org/confluence/display/KAFKA/KIP-97%3A+Improved+Kafka+Client+RPC+Compatibility+Policy) — Backwards compat for old brokers | 0.10.2 | Supported |
   294| [KIP-98](https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging) — EOS | 0.11.0 | Supported |
   295| [KIP-101](https://cwiki.apache.org/confluence/display/KAFKA/KIP-101+-+Alter+Replication+Protocol+to+use+Leader+Epoch+rather+than+High+Watermark+for+Truncation) — OffsetForLeaderEpoch v0 | 0.11.0 | Supported |
   296| [KIP-102](https://cwiki.apache.org/confluence/display/KAFKA/KIP-102+-+Add+close+with+timeout+for+consumers) — Consumer close timeouts | 0.10.2 | Supported (via context) |
   297| [KIP-107](https://cwiki.apache.org/confluence/display/KAFKA/KIP-107%3A+Add+deleteRecordsBefore%28%29+API+in+AdminClient) — DeleteRecords | 0.11.0 | Supported |
   298| [KIP-108](https://cwiki.apache.org/confluence/display/KAFKA/KIP-108%3A+Create+Topic+Policy) — CreateTopic validate only field | 0.10.2 | Supported |
   299| [KIP-110](https://cwiki.apache.org/confluence/display/KAFKA/KIP-110%3A+Add+Codec+for+ZStandard+Compression) — zstd | 2.1 | Supported |
   300| [KIP-112](https://cwiki.apache.org/confluence/display/KAFKA/KIP-112%3A+Handle+disk+failure+for+JBOD) — Broker request protocol changes | 1.0 | Supported |
   301| [KIP-113](https://cwiki.apache.org/confluence/display/KAFKA/KIP-113%3A+Support+replicas+movement+between+log+directories) — LogDir requests | 1.0 | Supported |
   302| [KIP-117](https://cwiki.apache.org/confluence/display/KAFKA/KIP-117%3A+Add+a+public+AdminClient+API+for+Kafka+admin+operations) — Admin client | 0.11.0 | Supported (via kmsg) |
   303| [KIP-124](https://cwiki.apache.org/confluence/display/KAFKA/KIP-124+-+Request+rate+quotas) — Request rate quotas | 0.11.0 | Supported |
   304| [KIP-126](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=68715855) — Ensure proper batch size after compression | 0.11.0 | Supported (avoided entirely) |
   305| [KIP-133](https://cwiki.apache.org/confluence/display/KAFKA/KIP-133%3A+Describe+and+Alter+Configs+Admin+APIs) — Describe & Alter configs | 0.11.0 | Supported |
   306| [KIP-140](https://cwiki.apache.org/confluence/display/KAFKA/KIP-140%3A+Add+administrative+RPCs+for+adding%2C+deleting%2C+and+listing+ACLs) — ACLs | 0.11.0 | Supported |
   307| [KIP-144](https://cwiki.apache.org/confluence/display/KAFKA/KIP-144%3A+Exponential+backoff+for+broker+reconnect+attempts) — Broker reconnect backoff | 0.11.0 | Supported |
   308| [KIP-152](https://cwiki.apache.org/confluence/display/KAFKA/KIP-152+-+Improve+diagnostics+for+SASL+authentication+failures) — More SASL; SASLAuthenticate | 1.0 | Supported |
   309| [KIP-183](https://cwiki.apache.org/confluence/display/KAFKA/KIP-183+-+Change+PreferredReplicaLeaderElectionCommand+to+use+AdminClient) — Elect preferred leaders | 2.2 | Supported |
   310| [KIP-185](https://cwiki.apache.org/confluence/display/KAFKA/KIP-185%3A+Make+exactly+once+in+order+delivery+per+partition+the+default+Producer+setting) — Idempotency is default | 1.0 | Supported |
   311| [KIP-192](https://cwiki.apache.org/confluence/display/KAFKA/KIP-192+%3A+Provide+cleaner+semantics+when+idempotence+is+enabled) — Cleaner idempotence semantics | 1.0 | Supported |
   312| [KIP-195](https://cwiki.apache.org/confluence/display/KAFKA/KIP-195%3A+AdminClient.createPartitions) — CreatePartitions | 1.0 | Supported |
   313| [KIP-204](https://cwiki.apache.org/confluence/display/KAFKA/KIP-204+%3A+Adding+records+deletion+operation+to+the+new+Admin+Client+API) — DeleteRecords via admin API | 1.1 | Supported |
   314| [KIP-207](https://cwiki.apache.org/confluence/display/KAFKA/KIP-207%3A+Offsets+returned+by+ListOffsetsResponse+should+be+monotonically+increasing+even+during+a+partition+leader+change) — New error in ListOffsets | 2.2 | Supported |
   315| [KIP-219](https://cwiki.apache.org/confluence/display/KAFKA/KIP-219+-+Improve+quota+throttle+communication) — Client-side throttling | 2.0 | Supported |
   316| [KIP-222](https://cwiki.apache.org/confluence/display/KAFKA/KIP-222+-+Add+Consumer+Group+operations+to+Admin+API) — Group operations via admin API | 2.0 | Supported |
   317| [KIP-226](https://cwiki.apache.org/confluence/display/KAFKA/KIP-226+-+Dynamic+Broker+Configuration) — Describe configs v1 | 1.1 | Supported |
   318| [KIP-227](https://cwiki.apache.org/confluence/display/KAFKA/KIP-227%3A+Introduce+Incremental+FetchRequests+to+Increase+Partition+Scalability) — Incremental fetch | 1.1 | Supported |
   319| [KIP-229](https://cwiki.apache.org/confluence/display/KAFKA/KIP-229%3A+DeleteGroups+API) — DeleteGroups | 1.1 | Supported |
   320| [KIP-249](https://cwiki.apache.org/confluence/display/KAFKA/KIP-249%3A+Add+Delegation+Token+Operations+to+KafkaAdminClient) — Delegation tokens in admin API | 2.0 | Supported |
   321| [KIP-255](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=75968876) — SASL OAUTHBEARER | 2.0 | Supported |
   322| [KIP-266](https://cwiki.apache.org/confluence/display/KAFKA/KIP-266%3A+Fix+consumer+indefinite+blocking+behavior) — Fix indefinite consumer timeouts | 2.0 | Supported (via context) |
   323| [KIP-279](https://cwiki.apache.org/confluence/display/KAFKA/KIP-279%3A+Fix+log+divergence+between+leader+and+follower+after+fast+leader+fail+over) — OffsetForLeaderEpoch bump | 2.0 | Supported |
   324| [KIP-289](https://cwiki.apache.org/confluence/display/KAFKA/KIP-289%3A+Improve+the+default+group+id+behavior+in+KafkaConsumer) — Default group.id to null | 2.2 | Supported |
   325| [KIP-294](https://cwiki.apache.org/confluence/display/KAFKA/KIP-294+-+Enable+TLS+hostname+verification+by+default) — TLS verification | 2.0 | Supported (via dialer) |
   326| [KIP-302](https://cwiki.apache.org/confluence/display/KAFKA/KIP-302+-+Enable+Kafka+clients+to+use+all+DNS+resolved+IP+addresses) — Use multiple addrs for resolved hostnames | 2.1 | Supported (via dialer) |
   327| [KIP-320](https://cwiki.apache.org/confluence/display/KAFKA/KIP-320%3A+Allow+fetchers+to+detect+and+handle+log+truncation) — Fetcher: detect log truncation | 2.1 | Supported |
   328| [KIP-322](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=87295558) — DeleteTopics disabled error code | 2.1 | Supported |
   329| [KIP-339](https://cwiki.apache.org/confluence/display/KAFKA/KIP-339%3A+Create+a+new+IncrementalAlterConfigs+API) — IncrementalAlterConfigs | 2.3 | Supported |
   330| [KIP-341](https://cwiki.apache.org/confluence/display/KAFKA/KIP-341%3A+Update+Sticky+Assignor%27s+User+Data+Protocol) — Sticky group bugfix | ? | Supported |
   331| [KIP-342](https://cwiki.apache.org/confluence/display/KAFKA/KIP-342%3A+Add+support+for+Custom+SASL+extensions+in+OAuthBearer+authentication) — OAUTHBEARER extensions | 2.1 | Supported |
   332| [KIP-345](https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A+Introduce+static+membership+protocol+to+reduce+consumer+rebalances) — Static group membership | 2.4 | Supported |
   333| [KIP-357](https://cwiki.apache.org/confluence/display/KAFKA/KIP-357%3A++Add+support+to+list+ACLs+per+principal) — List ACLs per principal via admin API | 2.1 | Supported |
   334| [KIP-360](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=89068820) — Safe epoch bumping for `UNKNOWN_PRODUCER_ID` | 2.5 | Supported |
   335| [KIP-361](https://cwiki.apache.org/confluence/display/KAFKA/KIP-361%3A+Add+Consumer+Configuration+to+Disable+Auto+Topic+Creation) — Allow disable auto topic creation | 2.3 | Supported |
   336| [KIP-368](https://cwiki.apache.org/confluence/display/KAFKA/KIP-368%3A+Allow+SASL+Connections+to+Periodically+Re-Authenticate) — Periodically reauthenticate SASL | 2.2 | Supported |
   337| [KIP-369](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=89070828) — An always round robin produce partitioner | 2.4 | Supported |
   338| [KIP-373](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=93324147) — Users can create delegation tokens for others | 3.3 | Supported |
   339| [KIP-380](https://cwiki.apache.org/confluence/display/KAFKA/KIP-380%3A+Detect+outdated+control+requests+and+bounced+brokers+using+broker+generation) — Inter-broker protocol changes | 2.2 | Supported |
   340| [KIP-389](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=89070828) — Group max size error | 2.2 | Supported |
   341| [KIP-392](https://cwiki.apache.org/confluence/display/KAFKA/KIP-392%3A+Allow+consumers+to+fetch+from+closest+replica) — Closest replica fetching w/ rack | 2.2 | Supported |
   342| [KIP-394](https://cwiki.apache.org/confluence/display/KAFKA/KIP-394%3A+Require+member.id+for+initial+join+group+request) — Require member.id for initial join request |  2.2 | Supported |
   343| [KIP-396](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=97551484) — Commit offsets manually | 2.4 | Supported |
   344| [KIP-405](https://cwiki.apache.org/confluence/display/KAFKA/KIP-405%3A+Kafka+Tiered+Storage) — Kafka Tiered Storage | 3.5 | Supported (protos) |
   345| [KIP-412](https://cwiki.apache.org/confluence/display/KAFKA/KIP-412%3A+Extend+Admin+API+to+support+dynamic+application+log+levels) — Dynamic log levels w/ IncrementalAlterConfigs | 2.4 | Supported |
   346| [KIP-429](https://cwiki.apache.org/confluence/display/KAFKA/KIP-429%3A+Kafka+Consumer+Incremental+Rebalance+Protocol) — Incremental rebalance (see KAFKA-8179) | 2.4 | Supported |
   347| [KIP-430](https://cwiki.apache.org/confluence/display/KAFKA/KIP-430+-+Return+Authorized+Operations+in+Describe+Responses) — Authorized ops in DescribeGroups | 2.3 | Supported |
   348| [KIP-447](https://cwiki.apache.org/confluence/display/KAFKA/KIP-447%3A+Producer+scalability+for+exactly+once+semantics) — Producer scalability for EOS | 2.5 | Supported |
   349| [KIP-455](https://cwiki.apache.org/confluence/display/KAFKA/KIP-455%3A+Create+an+Administrative+API+for+Replica+Reassignment) — Replica reassignment API | 2.4 | Supported |
   350| [KIP-460](https://cwiki.apache.org/confluence/display/KAFKA/KIP-460%3A+Admin+Leader+Election+RPC) — Leader election API | 2.4 | Supported |
   351| [KIP-464](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=113708722) — CreateTopic defaults | 2.4 | Supported |
   352| [KIP-467](https://cwiki.apache.org/confluence/display/KAFKA/KIP-467%3A+Augment+ProduceResponse+error+messaging+for+specific+culprit+records) — Per-record error codes when producing | 2.4 | Supported (and ignored) |
   353| [KIP-480](https://cwiki.apache.org/confluence/display/KAFKA/KIP-480%3A+Sticky+Partitioner) — Sticky partition producing | 2.4 | Supported |
   354| [KIP-482](https://cwiki.apache.org/confluence/display/KAFKA/KIP-482%3A+The+Kafka+Protocol+should+Support+Optional+Tagged+Fields) — Tagged fields (KAFKA-8885) | 2.4 | Supported |
   355| [KIP-496](https://cwiki.apache.org/confluence/display/KAFKA/KIP-496%3A+Administrative+API+to+delete+consumer+offsets) — OffsetDelete admin command | 2.4 | Supported |
   356| [KIP-497](https://cwiki.apache.org/confluence/display/KAFKA/KIP-497%3A+Add+inter-broker+API+to+alter+ISR) — New AlterISR API | 2.7 | Supported |
   357| [KIP-498](https://cwiki.apache.org/confluence/display/KAFKA/KIP-498%3A+Add+client-side+configuration+for+maximum+response+size+to+protect+against+OOM) — Max bound on reads | ? | Supported |
   358| [KIP-511](https://cwiki.apache.org/confluence/display/KAFKA/KIP-511%3A+Collect+and+Expose+Client%27s+Name+and+Version+in+the+Brokers) — Client name/version in ApiVersions request | 2.4 | Supported |
   359| [KIP-514](https://cwiki.apache.org/confluence/display/KAFKA/KIP-514%3A+Add+a+bounded+flush%28%29+API+to+Kafka+Producer) — Bounded Flush | 2.4 | Supported (via context) |
   360| [KIP-516](https://cwiki.apache.org/confluence/display/KAFKA/KIP-516%3A+Topic+Identifiers) — Topic IDs | ??? | Supported as it is implemented |
   361| [KIP-518](https://cwiki.apache.org/confluence/display/KAFKA/KIP-518%3A+Allow+listing+consumer+groups+per+state) — List groups by state | 2.6 | Supported |
   362| [KIP-519](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=128650952) — Configurable SSL "engine" | 2.6 | Supported (via dialer) |
   363| [KIP-525](https://cwiki.apache.org/confluence/display/KAFKA/KIP-525+-+Return+topic+metadata+and+configs+in+CreateTopics+response) — CreateTopics v5 returns configs | 2.4 | Supported |
   364| [KIP-526](https://cwiki.apache.org/confluence/display/KAFKA/KIP-526%3A+Reduce+Producer+Metadata+Lookups+for+Large+Number+of+Topics) — Reduce metadata lookups | 2.5 | Supported |
   365| [KIP-533](https://cwiki.apache.org/confluence/display/KAFKA/KIP-533%3A+Add+default+api+timeout+to+AdminClient) — Default API timeout (total time, not per request) | 2.5 | Supported (via RetryTimeout) |
   366| [KIP-546](https://cwiki.apache.org/confluence/display/KAFKA/KIP-546%3A+Add+Client+Quota+APIs+to+the+Admin+Client) — Client Quota APIs | 2.5 | Supported |
   367| [KIP-554](https://cwiki.apache.org/confluence/display/KAFKA/KIP-554%3A+Add+Broker-side+SCRAM+Config+API) — Broker side SCRAM APIs | 2.7 | Supported |
   368| [KIP-559](https://cwiki.apache.org/confluence/display/KAFKA/KIP-559%3A+Make+the+Kafka+Protocol+Friendlier+with+L7+Proxies) — Protocol info in sync/join | 2.5 | Supported |
   369| [KIP-568](https://cwiki.apache.org/confluence/display/KAFKA/KIP-568%3A+Explicit+rebalance+triggering+on+the+Consumer) — Explicit rebalance triggering on the consumer | 2.6 | Supported |
   370| [KIP-569](https://cwiki.apache.org/confluence/display/KAFKA/KIP-569%3A+DescribeConfigsResponse+-+Update+the+schema+to+include+additional+metadata+information+of+the+field) — Docs & type in DescribeConfigs | 2.6 | Supported |
   371| [KIP-570](https://cwiki.apache.org/confluence/display/KAFKA/KIP-570%3A+Add+leader+epoch+in+StopReplicaRequest) — Leader epoch in StopReplica | 2.6 | Supported |
   372| [KIP-580](https://cwiki.apache.org/confluence/display/KAFKA/KIP-580%3A+Exponential+Backoff+for+Kafka+Clients) — Exponential backoff | 2.6 | Supported |
   373| [KIP-584](https://cwiki.apache.org/confluence/display/KAFKA/KIP-584%3A+Versioning+scheme+for+features) — Versioning scheme for features | ? | Supported (nothing to do yet) |
   374| [KIP-588](https://cwiki.apache.org/confluence/display/KAFKA/KIP-588%3A+Allow+producers+to+recover+gracefully+from+transaction+timeouts) — Producer recovery from txn timeout | 2.7 | Supported |
   375| [KIP-590](https://cwiki.apache.org/confluence/display/KAFKA/KIP-590%3A+Redirect+Zookeeper+Mutation+Protocols+to+The+Controller) — Envelope (broker only) | 2.7 | Supported |
   376| [KIP-595](https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum) — New APIs for raft protocol | 2.7 | Supported |
   377| [KIP-599](https://cwiki.apache.org/confluence/display/KAFKA/KIP-599%3A+Throttle+Create+Topic%2C+Create+Partition+and+Delete+Topic+Operations) — Throttling on create/delete topic/partition | 2.7 | Supported |
   378| [KIP-602](https://cwiki.apache.org/confluence/display/KAFKA/KIP-602%3A+Change+default+value+for+client.dns.lookup) — Use all resolved addrs by default | 2.6 | Supported (via dialer) |
   379| [KIP-651](https://cwiki.apache.org/confluence/display/KAFKA/KIP-651+-+Support+PEM+format+for+SSL+certificates+and+private+key) — Support PEM | 2.7 | Supported (via dialer) |
   380| [KIP-654](https://cwiki.apache.org/confluence/display/KAFKA/KIP-654%3A+Aborted+transaction+with+non-flushed+data+should+throw+a+non-fatal+exception) — Aborted txns with unflushed data is not fatal | 2.7 | Supported (default behavior) |
   381| [KIP-664](https://cwiki.apache.org/confluence/display/KAFKA/KIP-664%3A+Provide+tooling+to+detect+and+abort+hanging+transactions) — Describe producers / etc. | 2.8 (mostly) | Supported |
   382| [KIP-679](https://cwiki.apache.org/confluence/display/KAFKA/KIP-679%3A+Producer+will+enable+the+strongest+delivery+guarantee+by+default) — Strongest producer guarantee by default | 3.0 | Supported (by default always) |
   383| [KIP-699](https://cwiki.apache.org/confluence/display/KAFKA/KIP-699%3A+Update+FindCoordinator+to+resolve+multiple+Coordinators+at+a+time) — Batch FindCoordinators| 3.0 | Supported |
   384| [KIP-700](https://cwiki.apache.org/confluence/display/KAFKA/KIP-700%3A+Add+Describe+Cluster+API) — DescribeCluster | 2.8 | Supported |
   385| [KIP-704](https://cwiki.apache.org/confluence/display/KAFKA/KIP-704%3A+Send+a+hint+to+the+partition+leader+to+recover+the+partition) — AlterISR => AlterPartition | 3.2 | Supported |
   386| [KIP-709](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=173084258) — Batch OffsetFetch | 3.0 | Supported |
   387| [KIP-730](https://cwiki.apache.org/confluence/display/KAFKA/KIP-730%3A+Producer+ID+generation+in+KRaft+mode) - AllocateProducerIDs | 3.0 | Supported |
   388| [KIP-734](https://cwiki.apache.org/confluence/display/KAFKA/KIP-734:+Improve+AdminClient.listOffsets+to+return+timestamp+and+offset+for+the+record+with+the+largest+timestamp) — Support MaxTimestamp in ListOffsets | 3.0 | Supported (simple version bump) |
   389| [KIP-735](https://cwiki.apache.org/confluence/display/KAFKA/KIP-735%3A+Increase+default+consumer+session+timeout) — Bump default session timeout | ? | Supported |
   390| [KIP-778](https://wiki.apache.org/confluence/pages/viewpage.action?pageId=188746840) — KRaft Upgrades (protocol changes only) | 3.2 | Supported |
   391| [KIP-784](https://cwiki.apache.org/confluence/display/KAFKA/KIP-784%3A+Add+top-level+error+code+field+to+DescribeLogDirsResponse) — Add ErrorCode to DescribeLogDirs response | 3.1 | Supported |
   392| [KIP-792](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=191336614) —  Generation field in consumer group protocol | 3.4 | Supported |
   393| [KIP-794](https://cwiki.apache.org/confluence/display/KAFKA/KIP-794%3A+Strictly+Uniform+Sticky+Partitioner) —  Better sticky partitioning | 3.3 | Supported (`UniformBytesPartitioner`) |
   394| [KIP-800](https://cwiki.apache.org/confluence/display/KAFKA/KIP-800%3A+Add+reason+to+JoinGroupRequest+and+LeaveGroupRequest) — Reason in Join/Leave group | 3.1 | Supported |
   395| [KIP-814](https://cwiki.apache.org/confluence/display/KAFKA/KIP-814%3A+Static+membership+protocol+should+let+the+leader+skip+assignment) — SkipAssignment for static group leaders | 3.1 | Supported |
   396| [KIP-827](https://cwiki.apache.org/confluence/display/KAFKA/KIP-827%3A+Expose+logdirs+total+and+usable+space+via+Kafka+API) — `DescribeLogDirs.{Total,Usable}Bytes` | 3.3 | Supported |
   397| [KIP-836](https://cwiki.apache.org/confluence/display/KAFKA/KIP-836%3A+Addition+of+Information+in+DescribeQuorumResponse+about+Voter+Lag) — `DescribeQuorum` voter lag info | 3.3 | Supported |
   398| [KIP-841](https://cwiki.apache.org/confluence/display/KAFKA/KIP-841%3A+Fenced+replicas+should+not+be+allowed+to+join+the+ISR+in+KRaft) — `AlterPartition.TopicID` | 3.3 | Supported |
   399| [KIP-866](https://cwiki.apache.org/confluence/display/KAFKA/KIP-866+ZooKeeper+to+KRaft+Migration) — ZK to Raft RPC changes | 3.4 | Supported |
   400| [KIP-893](https://cwiki.apache.org/confluence/display/KAFKA/KIP-893%3A+The+Kafka+protocol+should+support+nullable+structs) — Nullable structs in the protocol | 3.5 | Supported |
   401| [KIP-899](https://cwiki.apache.org/confluence/display/KAFKA/KIP-899%3A+Allow+clients+to+rebootstrap) — Allow clients to rebootstrap | ? | Supported (`UpdateSeedBrokers`) |
   402| [KIP-903](https://cwiki.apache.org/confluence/display/KAFKA/KIP-903%3A+Replicas+with+stale+broker+epoch+should+not+be+allowed+to+join+the+ISR) — Stale broker epoch fencing| 3.5 | Supported (proto) |
   403
   404Missing from above but included in librdkafka is:
   405
   406- [KIP-85](https://cwiki.apache.org/confluence/display/KAFKA/KIP-85%3A+Dynamic+JAAS+configuration+for+Kafka+clients), which does not seem relevant for franz-go
   407- [KIP-92](https://cwiki.apache.org/confluence/display/KAFKA/KIP-92+-+Add+per+partition+lag+metrics+to+KafkaConsumer) for consumer lag metrics, which is better suited for an external system via the admin api
   408- [KIP-223](https://cwiki.apache.org/confluence/display/KAFKA/KIP-223+-+Add+per-topic+min+lead+and+per-partition+lead+metrics+to+KafkaConsumer) for more metrics
   409- [KIP-235](https://cwiki.apache.org/confluence/display/KAFKA/KIP-235%3A+Add+DNS+alias+support+for+secured+connection), which is confusing but may be implement via a custom dialer and custom kerberos?
   410- [KIP-359](https://cwiki.apache.org/confluence/display/KAFKA/KIP-359%3A+Verify+leader+epoch+in+produce+requests) to verify leader epoch when producing; this is easy to support but actually is not implemented in Kafka yet
   411- [KIP-421](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=100829515) 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)
   412- [KIP-436](https://cwiki.apache.org/confluence/display/KAFKA/KIP-436%3A+Add+a+metric+indicating+start+time) is about yet another metric
   413- [KIP-517](https://cwiki.apache.org/confluence/display/KAFKA/KIP-517%3A+Add+consumer+metrics+to+observe+user+poll+behavior), more metrics

View as plain text