Skip to content
Kafka library in Go
Go
Branch: master
Clone or download
stevevls Fix behavior of Conn.ReadPartitions() when no topics specified (#383)
Ensure all topic partitions are returned when the connection does
not have a configured topic and the caller does not pass any
topics as arguments.

Fixes #381
Latest commit 3b34bbf Dec 5, 2019
Permalink
Type Name Latest commit message Commit time
Failed to load latest commit information.
.circleci Add ztsd compression Apr 9, 2019
.github/ISSUE_TEMPLATE Created bug report and feature request templates (#218) Feb 27, 2019
examples auto split kafkaURL to array (#280) Jul 22, 2019
gzip Stream compression (#306) Jul 22, 2019
lz4 Stream compression (#306) Jul 22, 2019
sasl Make sasl.Mechanism safe for concurrent use (#323) Jul 24, 2019
snappy
testing Pad shorter version with zeroes (#289) Jun 11, 2019
zstd Stream compression (#306) Jul 22, 2019
.gitignore Adding MaxMessageBytes into the ProducerConfig (#208) Jun 17, 2019
LICENSE update README May 31, 2017
README.md Add CRC32 and Murmur2 balancers (#334) Aug 9, 2019
balancer.go Fixed compatibility with the sarama hash partitioner (#336) Aug 9, 2019
balancer_test.go Fixed compatibility with the sarama hash partitioner (#336) Aug 9, 2019
batch.go Handle end of fetched batches of messages gracefully (#353) Sep 21, 2019
batch_test.go Add ApiVersions request (#160) Jan 9, 2019
buffer.go recycle compression buffers (#320) Aug 9, 2019
client.go Introduce Client API (#272) May 14, 2019
client_test.go Introduce Client API (#272) May 14, 2019
commit.go Internally, kafka-go offset is consistent with itself. Unfortunately,… Jan 10, 2018
commit_test.go Internally, kafka-go offset is consistent with itself. Unfortunately,… Jan 10, 2018
compression.go recycle compression buffers (#320) Aug 9, 2019
compression_test.go Stream compression (#306) Jul 22, 2019
conn.go Fix behavior of Conn.ReadPartitions() when no topics specified (#383) Dec 5, 2019
conn_test.go Fix behavior of Conn.ReadPartitions() when no topics specified (#383) Dec 5, 2019
consumergroup.go Introduced new pluggable logging API (#301) Sep 17, 2019
consumergroup_test.go Introduced new pluggable logging API (#301) Sep 17, 2019
crc32.go don't use io.StringWriter (#337) Aug 14, 2019
crc32_test.go kafka.writeBuffer (#318) Jul 22, 2019
createtopics.go kafka.writeBuffer (#318) Jul 22, 2019
createtopics_test.go kafka.writeBuffer (#318) Jul 22, 2019
deletetopics.go kafka.writeBuffer (#318) Jul 22, 2019
deletetopics_test.go kafka.writeBuffer (#318) Jul 22, 2019
describegroups.go kafka.writeBuffer (#318) Jul 22, 2019
describegroups_test.go kafka.writeBuffer (#318) Jul 22, 2019
dialer.go Make sasl.Mechanism safe for concurrent use (#323) Jul 24, 2019
dialer_test.go Adding MaxMessageBytes into the ProducerConfig (#208) Jun 17, 2019
discard.go Updated discardN to return errShortRead if N bytes not available (#143) Nov 15, 2018
discard_test.go Updated discardN to return errShortRead if N bytes not available (#143) Nov 15, 2018
docker-compose.yml Added support for SASL PLAIN and SCRAM mechanisms (#223) Mar 18, 2019
error.go Update `Error` consts and `Error.Temporary()` (#372) Nov 1, 2019
error_test.go Standardized id capitalization (#177) Jan 9, 2019
example_consumergroup_test.go Introduced top-level ConsumerGroup construct (#277) Aug 9, 2019
example_writer_test.go now typo free (#45) Dec 26, 2017
export_test.go Added support for message set compression (#135) Nov 5, 2018
fetch.go kafka.writeBuffer (#318) Jul 22, 2019
findcoordinator.go kafka.writeBuffer (#318) Jul 22, 2019
findcoordinator_test.go kafka.writeBuffer (#318) Jul 22, 2019
go.mod Stream compression (#306) Jul 22, 2019
go.sum Stream compression (#306) Jul 22, 2019
groupbalancer.go Exported Consumer GroupBalancer and made it pluggable (#124) Oct 20, 2018
groupbalancer_test.go Exported Consumer GroupBalancer and made it pluggable (#124) Oct 20, 2018
heartbeat.go kafka.writeBuffer (#318) Jul 22, 2019
heartbeat_test.go kafka.writeBuffer (#318) Jul 22, 2019
joingroup.go kafka.writeBuffer (#318) Jul 22, 2019
joingroup_test.go kafka.writeBuffer (#318) Jul 22, 2019
leavegroup.go kafka.writeBuffer (#318) Jul 22, 2019
leavegroup_test.go kafka.writeBuffer (#318) Jul 22, 2019
listgroups.go kafka.writeBuffer (#318) Jul 22, 2019
listgroups_test.go kafka.writeBuffer (#318) Jul 22, 2019
listoffset.go kafka.writeBuffer (#318) Jul 22, 2019
logger.go Introduced new pluggable logging API (#301) Sep 17, 2019
message.go Optimize message size calculation (#329) Sep 9, 2019
message_test.go Optimize message size calculation (#329) Sep 9, 2019
metadata.go Fix behavior of Conn.ReadPartitions() when no topics specified (#383) Dec 5, 2019
offsetcommit.go kafka.writeBuffer (#318) Jul 22, 2019
offsetcommit_test.go kafka.writeBuffer (#318) Jul 22, 2019
offsetfetch.go kafka.writeBuffer (#318) Jul 22, 2019
offsetfetch_test.go kafka.writeBuffer (#318) Jul 22, 2019
produce.go kafka.writeBuffer (#318) Jul 22, 2019
protocol.go kafka.writeBuffer (#318) Jul 22, 2019
protocol_test.go kafka.writeBuffer (#318) Jul 22, 2019
read.go fix and optimize varint encoding (#305) Jul 8, 2019
read_test.go kafka.writeBuffer (#318) Jul 22, 2019
reader.go Updated constant for `FirstOffset` with correct type (#368) Oct 21, 2019
reader_test.go Introduced new pluggable logging API (#301) Sep 17, 2019
recordbatch.go Extract record batch structure (#314) Sep 10, 2019
saslauthenticate.go kafka.writeBuffer (#318) Jul 22, 2019
saslauthenticate_test.go kafka.writeBuffer (#318) Jul 22, 2019
saslhandshake.go kafka.writeBuffer (#318) Jul 22, 2019
saslhandshake_test.go kafka.writeBuffer (#318) Jul 22, 2019
sizeof.go Add transactional id to producers (#235) Apr 7, 2019
stats.go fix aligment issues with stats on 32-bit systems (#83) Mar 20, 2018
syncgroup.go kafka.writeBuffer (#318) Jul 22, 2019
syncgroup_test.go kafka.writeBuffer (#318) Jul 22, 2019
time.go writing messages now doesn't require dynamic memory allocations anymore May 31, 2017
write.go Extract record batch structure (#314) Sep 10, 2019
write_test.go kafka.writeBuffer (#318) Jul 22, 2019
writer.go Introduced new pluggable logging API (#301) Sep 17, 2019
writer_test.go

README.md

kafka-go CircleCI Go Report Card GoDoc

Motivations

We rely on both Go and Kafka a lot at Segment. Unfortunately, the state of the Go client libraries for Kafka at the time of this writing was not ideal. The available options were:

  • sarama, which is by far the most popular but is quite difficult to work with. It is poorly documented, the API exposes low level concepts of the Kafka protocol, and it doesn't support recent Go features like contexts. It also passes all values as pointers which causes large numbers of dynamic memory allocations, more frequent garbage collections, and higher memory usage.

  • confluent-kafka-go is a cgo based wrapper around librdkafka, which means it introduces a dependency to a C library on all Go code that uses the package. It has much better documentation than sarama but still lacks support for Go contexts.

  • goka is a more recent Kafka client for Go which focuses on a specific usage pattern. It provides abstractions for using Kafka as a message passing bus between services rather than an ordered log of events, but this is not the typical use case of Kafka for us at Segment. The package also depends on sarama for all interactions with Kafka.

This is where kafka-go comes into play. It provides both low and high level APIs for interacting with Kafka, mirroring concepts and implementing interfaces of the Go standard library to make it easy to use and integrate with existing software.

Kafka versions

kafka-go is currently compatible with Kafka versions from 0.10.1.0 to 2.1.0. While latest versions will be working, some features available from the Kafka API may not be implemented yet.

Golang version

kafka-go is currently compatible with golang version from 1.12+. To use with older versions of golang use release v0.2.5.

Connection GoDoc

The Conn type is the core of the kafka-go package. It wraps around a raw network connection to expose a low-level API to a Kafka server.

Here are some examples showing typical use of a connection object:

// to produce messages
topic := "my-topic"
partition := 0

conn, _ := kafka.DialLeader(context.Background(), "tcp", "localhost:9092", topic, partition)

conn.SetWriteDeadline(time.Now().Add(10*time.Second))
conn.WriteMessages(
    kafka.Message{Value: []byte("one!")},
    kafka.Message{Value: []byte("two!")},
    kafka.Message{Value: []byte("three!")},
)

conn.Close()
// to consume messages
topic := "my-topic"
partition := 0

conn, _ := kafka.DialLeader(context.Background(), "tcp", "localhost:9092", topic, partition)

conn.SetReadDeadline(time.Now().Add(10*time.Second))
batch := conn.ReadBatch(10e3, 1e6) // fetch 10KB min, 1MB max

b := make([]byte, 10e3) // 10KB max per message
for {
    _, err := batch.Read(b)
    if err != nil {
        break
    }
    fmt.Println(string(b))
}

batch.Close()
conn.Close()

Because it is low level, the Conn type turns out to be a great building block for higher level abstractions, like the Reader for example.

Reader GoDoc

A Reader is another concept exposed by the kafka-go package, which intends to make it simpler to implement the typical use case of consuming from a single topic-partition pair. A Reader also automatically handles reconnections and offset management, and exposes an API that supports asynchronous cancellations and timeouts using Go contexts.

// make a new reader that consumes from topic-A, partition 0, at offset 42
r := kafka.NewReader(kafka.ReaderConfig{
    Brokers:   []string{"localhost:9092"},
    Topic:     "topic-A",
    Partition: 0,
    MinBytes:  10e3, // 10KB
    MaxBytes:  10e6, // 10MB
})
r.SetOffset(42)

for {
    m, err := r.ReadMessage(context.Background())
    if err != nil {
        break
    }
    fmt.Printf("message at offset %d: %s = %s\n", m.Offset, string(m.Key), string(m.Value))
}

r.Close()

Consumer Groups

kafka-go also supports Kafka consumer groups including broker managed offsets. To enable consumer groups, simply specify the GroupID in the ReaderConfig.

ReadMessage automatically commits offsets when using consumer groups.

// make a new reader that consumes from topic-A
r := kafka.NewReader(kafka.ReaderConfig{
    Brokers:   []string{"localhost:9092"},
    GroupID:   "consumer-group-id",
    Topic:     "topic-A",
    MinBytes:  10e3, // 10KB
    MaxBytes:  10e6, // 10MB
})

for {
    m, err := r.ReadMessage(context.Background())
    if err != nil {
        break
    }
    fmt.Printf("message at topic/partition/offset %v/%v/%v: %s = %s\n", m.Topic, m.Partition, m.Offset, string(m.Key), string(m.Value))
}

r.Close()

There are a number of limitations when using consumer groups:

  • (*Reader).SetOffset will return an error when GroupID is set
  • (*Reader).Offset will always return -1 when GroupID is set
  • (*Reader).Lag will always return -1 when GroupID is set
  • (*Reader).ReadLag will return an error when GroupID is set
  • (*Reader).Stats will return a partition of -1 when GroupID is set

Explicit Commits

kafka-go also supports explicit commits. Instead of calling ReadMessage, call FetchMessage followed by CommitMessages.

ctx := context.Background()
for {
    m, err := r.FetchMessage(ctx)
    if err != nil {
        break
    }
    fmt.Printf("message at topic/partition/offset %v/%v/%v: %s = %s\n", m.Topic, m.Partition, m.Offset, string(m.Key), string(m.Value))
    r.CommitMessages(ctx, m)
}

Managing Commits

By default, CommitMessages will synchronously commit offsets to Kafka. For improved performance, you can instead periodically commit offsets to Kafka by setting CommitInterval on the ReaderConfig.

// make a new reader that consumes from topic-A
r := kafka.NewReader(kafka.ReaderConfig{
    Brokers:        []string{"localhost:9092"},
    GroupID:        "consumer-group-id",
    Topic:          "topic-A",
    MinBytes:       10e3, // 10KB
    MaxBytes:       10e6, // 10MB
    CommitInterval: time.Second, // flushes commits to Kafka every second
})

Writer GoDoc

To produce messages to Kafka, a program may use the low-level Conn API, but the package also provides a higher level Writer type which is more appropriate to use in most cases as it provides additional features:

  • Automatic retries and reconnections on errors.
  • Configurable distribution of messages across available partitions.
  • Synchronous or asynchronous writes of messages to Kafka.
  • Asynchronous cancellation using contexts.
  • Flushing of pending messages on close to support graceful shutdowns.
// make a writer that produces to topic-A, using the least-bytes distribution
w := kafka.NewWriter(kafka.WriterConfig{
	Brokers: []string{"localhost:9092"},
	Topic:   "topic-A",
	Balancer: &kafka.LeastBytes{},
})

w.WriteMessages(context.Background(),
	kafka.Message{
		Key:   []byte("Key-A"),
		Value: []byte("Hello World!"),
	},
	kafka.Message{
		Key:   []byte("Key-B"),
		Value: []byte("One!"),
	},
	kafka.Message{
		Key:   []byte("Key-C"),
		Value: []byte("Two!"),
	},
)

w.Close()

Note: Even though kafka.Message contain Topic and Partition fields, they MUST NOT be set when writing messages. They are intended for read use only.

Compatibility with other clients

Sarama

If you're switching from Sarama and need/want to use the same algorithm for message partitioning, you can use the kafka.Hash balancer. kafka.Hash routes messages to the same partitions that Sarama's default partitioner would route to.

w := kafka.NewWriter(kafka.WriterConfig{
	Brokers:  []string{"localhost:9092"},
	Topic:    "topic-A",
	Balancer: &kafka.Hash{},
})

librdkafka and confluent-kafka-go

Use the kafka.CRC32Balancer balancer to get the same behaviour as librdkafka's default consistent_random partition strategy.

w := kafka.NewWriter(kafka.WriterConfig{
	Brokers:  []string{"localhost:9092"},
	Topic:    "topic-A",
	Balancer: kafka.CRC32Balancer{},
})

Java

Use the kafka.Murmur2Balancer balancer to get the same behaviour as the canonical Java client's default partitioner. Note: the Java class allows you to directly specify the partition which is not permitted.

w := kafka.NewWriter(kafka.WriterConfig{
	Brokers:  []string{"localhost:9092"},
	Topic:    "topic-A",
	Balancer: kafka.Murmur2Balancer{},
})

Compression

Compression can be enabled on the Writer by configuring the CompressionCodec:

w := kafka.NewWriter(kafka.WriterConfig{
	Brokers: []string{"localhost:9092"},
	Topic:   "topic-A",
	CompressionCodec: snappy.NewCompressionCodec(),
})

The Reader will by determine if the consumed messages are compressed by examining the message attributes. However, the package(s) for all expected codecs must be imported so that they get loaded correctly. For example, if you are going to be receiving messages compressed with Snappy, add the following import:

import _ "github.com/segmentio/kafka-go/snappy"

TLS Support

For a bare bones Conn type or in the Reader/Writer configs you can specify a dialer option for TLS support. If the TLS field is nil, it will not connect with TLS.

Connection

dialer := &kafka.Dialer{
    Timeout:   10 * time.Second,
    DualStack: true,
    TLS:       &tls.Config{...tls config...},
}

conn, err := dialer.DialContext(ctx, "tcp", "localhost:9093")

Reader

dialer := &kafka.Dialer{
    Timeout:   10 * time.Second,
    DualStack: true,
    TLS:       &tls.Config{...tls config...},
}

r := kafka.NewReader(kafka.ReaderConfig{
    Brokers:        []string{"localhost:9093"},
    GroupID:        "consumer-group-id",
    Topic:          "topic-A",
    Dialer:         dialer,
})

Writer

dialer := &kafka.Dialer{
    Timeout:   10 * time.Second,
    DualStack: true,
    TLS:       &tls.Config{...tls config...},
}

w := kafka.NewWriter(kafka.WriterConfig{
	Brokers: []string{"localhost:9093"},
	Topic:   "topic-A",
	Balancer: &kafka.Hash{},
	Dialer:   dialer,
})
You can’t perform that action at this time.