├── .gitignore ├── testdata ├── zookeeper.properties └── server.properties ├── doc.go ├── .travis.yml ├── Gopkg.toml ├── config_test.go ├── cluster.go ├── client_test.go ├── Makefile ├── client.go ├── LICENSE ├── util.go ├── offsets.go ├── README.md.tpl ├── offsets_test.go ├── cmd └── sarama-cluster-cli │ └── main.go ├── examples_test.go ├── Gopkg.lock ├── partitions_test.go ├── README.md ├── balancer_test.go ├── balancer.go ├── config.go ├── cluster_test.go ├── partitions.go ├── consumer_test.go └── consumer.go /.gitignore: -------------------------------------------------------------------------------- 1 | *.log 2 | *.pid 3 | kafka*/ 4 | vendor/ 5 | -------------------------------------------------------------------------------- /testdata/zookeeper.properties: -------------------------------------------------------------------------------- 1 | dataDir=/tmp/sarama-cluster-test/zookeeper 2 | clientPort=22181 3 | maxClientCnxns=0 4 | -------------------------------------------------------------------------------- /doc.go: -------------------------------------------------------------------------------- 1 | /* 2 | Package cluster provides cluster extensions for Sarama, enabing users 3 | to consume topics across from multiple, balanced nodes. 4 | 5 | It requires Kafka v0.9+ and follows the steps guide, described in: 6 | https://cwiki.apache.org/confluence/display/KAFKA/Kafka+0.9+Consumer+Rewrite+Design 7 | */ 8 | package cluster 9 | -------------------------------------------------------------------------------- /.travis.yml: -------------------------------------------------------------------------------- 1 | sudo: false 2 | language: go 3 | go: 4 | - 1.10.x 5 | - 1.9.x 6 | install: 7 | - go get -u github.com/golang/dep/cmd/dep 8 | - dep ensure 9 | env: 10 | - SCALA_VERSION=2.12 KAFKA_VERSION=0.11.0.1 11 | - SCALA_VERSION=2.12 KAFKA_VERSION=1.0.1 12 | - SCALA_VERSION=2.12 KAFKA_VERSION=1.1.0 13 | script: 14 | - make default test-race 15 | addons: 16 | apt: 17 | packages: 18 | - oracle-java8-set-default 19 | -------------------------------------------------------------------------------- /testdata/server.properties: -------------------------------------------------------------------------------- 1 | broker.id=0 2 | port=29092 3 | num.network.threads=2 4 | num.io.threads=8 5 | socket.send.buffer.bytes=1048576 6 | socket.receive.buffer.bytes=1048576 7 | socket.request.max.bytes=104857600 8 | log.dirs=/tmp/sarama-cluster-test/kafka 9 | num.partitions=4 10 | log.flush.interval.ms=10 11 | log.retention.hours=1 12 | log.segment.bytes=536870912 13 | log.retention.check.interval.ms=60000 14 | log.cleaner.enable=true 15 | zookeeper.connect=localhost:22181 16 | zookeeper.connection.timeout.ms=1000000 17 | offsets.topic.replication.factor=1 18 | -------------------------------------------------------------------------------- /Gopkg.toml: -------------------------------------------------------------------------------- 1 | 2 | # Gopkg.toml example 3 | # 4 | # Refer to https://github.com/golang/dep/blob/master/docs/Gopkg.toml.md 5 | # for detailed Gopkg.toml documentation. 6 | # 7 | # required = ["github.com/user/thing/cmd/thing"] 8 | # ignored = ["github.com/user/project/pkgX", "bitbucket.org/user/project/pkgA/pkgY"] 9 | # 10 | # [[constraint]] 11 | # name = "github.com/user/project" 12 | # version = "1.0.0" 13 | # 14 | # [[constraint]] 15 | # name = "github.com/user/project2" 16 | # branch = "dev" 17 | # source = "github.com/myfork/project2" 18 | # 19 | # [[override]] 20 | # name = "github.com/x/y" 21 | # version = "2.4.0" 22 | 23 | 24 | [[constraint]] 25 | name = "github.com/Shopify/sarama" 26 | version = "^1.14.0" 27 | -------------------------------------------------------------------------------- /config_test.go: -------------------------------------------------------------------------------- 1 | package cluster 2 | 3 | import ( 4 | "time" 5 | 6 | . "github.com/onsi/ginkgo" 7 | . "github.com/onsi/gomega" 8 | ) 9 | 10 | var _ = Describe("Config", func() { 11 | var subject *Config 12 | 13 | BeforeEach(func() { 14 | subject = NewConfig() 15 | }) 16 | 17 | It("should init", func() { 18 | Expect(subject.Group.Session.Timeout).To(Equal(30 * time.Second)) 19 | Expect(subject.Group.Heartbeat.Interval).To(Equal(3 * time.Second)) 20 | Expect(subject.Group.Return.Notifications).To(BeFalse()) 21 | Expect(subject.Metadata.Retry.Max).To(Equal(3)) 22 | Expect(subject.Group.Offsets.Synchronization.DwellTime).NotTo(BeZero()) 23 | // Expect(subject.Config.Version).To(Equal(sarama.V0_9_0_0)) 24 | }) 25 | 26 | }) 27 | -------------------------------------------------------------------------------- /cluster.go: -------------------------------------------------------------------------------- 1 | package cluster 2 | 3 | // Strategy for partition to consumer assignement 4 | type Strategy string 5 | 6 | const ( 7 | // StrategyRange is the default and assigns partition ranges to consumers. 8 | // Example with six partitions and two consumers: 9 | // C1: [0, 1, 2] 10 | // C2: [3, 4, 5] 11 | StrategyRange Strategy = "range" 12 | 13 | // StrategyRoundRobin assigns partitions by alternating over consumers. 14 | // Example with six partitions and two consumers: 15 | // C1: [0, 2, 4] 16 | // C2: [1, 3, 5] 17 | StrategyRoundRobin Strategy = "roundrobin" 18 | ) 19 | 20 | // Error instances are wrappers for internal errors with a context and 21 | // may be returned through the consumer's Errors() channel 22 | type Error struct { 23 | Ctx string 24 | error 25 | } 26 | -------------------------------------------------------------------------------- /client_test.go: -------------------------------------------------------------------------------- 1 | package cluster 2 | 3 | import ( 4 | . "github.com/onsi/ginkgo" 5 | . "github.com/onsi/gomega" 6 | ) 7 | 8 | var _ = Describe("Client", func() { 9 | var subject *Client 10 | 11 | BeforeEach(func() { 12 | var err error 13 | subject, err = NewClient(testKafkaAddrs, nil) 14 | Expect(err).NotTo(HaveOccurred()) 15 | }) 16 | 17 | It("should not allow to share clients across multiple consumers", func() { 18 | c1, err := NewConsumerFromClient(subject, testGroup, testTopics) 19 | Expect(err).NotTo(HaveOccurred()) 20 | defer c1.Close() 21 | 22 | _, err = NewConsumerFromClient(subject, testGroup, testTopics) 23 | Expect(err).To(MatchError("cluster: client is already used by another consumer")) 24 | 25 | Expect(c1.Close()).To(Succeed()) 26 | c2, err := NewConsumerFromClient(subject, testGroup, testTopics) 27 | Expect(err).NotTo(HaveOccurred()) 28 | Expect(c2.Close()).To(Succeed()) 29 | }) 30 | 31 | }) 32 | -------------------------------------------------------------------------------- /Makefile: -------------------------------------------------------------------------------- 1 | SCALA_VERSION?= 2.12 2 | KAFKA_VERSION?= 1.1.0 3 | KAFKA_DIR= kafka_$(SCALA_VERSION)-$(KAFKA_VERSION) 4 | KAFKA_SRC= https://archive.apache.org/dist/kafka/$(KAFKA_VERSION)/$(KAFKA_DIR).tgz 5 | KAFKA_ROOT= testdata/$(KAFKA_DIR) 6 | PKG=$(shell go list ./... | grep -v vendor) 7 | 8 | default: vet test 9 | 10 | vet: 11 | go vet $(PKG) 12 | 13 | test: testdeps 14 | KAFKA_DIR=$(KAFKA_DIR) go test $(PKG) -ginkgo.slowSpecThreshold=60 15 | 16 | test-verbose: testdeps 17 | KAFKA_DIR=$(KAFKA_DIR) go test $(PKG) -ginkgo.slowSpecThreshold=60 -v 18 | 19 | test-race: testdeps 20 | KAFKA_DIR=$(KAFKA_DIR) go test $(PKG) -ginkgo.slowSpecThreshold=60 -v -race 21 | 22 | testdeps: $(KAFKA_ROOT) 23 | 24 | doc: README.md 25 | 26 | .PHONY: test testdeps vet doc 27 | 28 | # --------------------------------------------------------------------- 29 | 30 | $(KAFKA_ROOT): 31 | @mkdir -p $(dir $@) 32 | cd $(dir $@) && curl -sSL $(KAFKA_SRC) | tar xz 33 | 34 | README.md: README.md.tpl $(wildcard *.go) 35 | becca -package $(subst $(GOPATH)/src/,,$(PWD)) 36 | -------------------------------------------------------------------------------- /client.go: -------------------------------------------------------------------------------- 1 | package cluster 2 | 3 | import ( 4 | "errors" 5 | "sync/atomic" 6 | 7 | "github.com/Shopify/sarama" 8 | ) 9 | 10 | var errClientInUse = errors.New("cluster: client is already used by another consumer") 11 | 12 | // Client is a group client 13 | type Client struct { 14 | sarama.Client 15 | config Config 16 | 17 | inUse uint32 18 | } 19 | 20 | // NewClient creates a new client instance 21 | func NewClient(addrs []string, config *Config) (*Client, error) { 22 | if config == nil { 23 | config = NewConfig() 24 | } 25 | 26 | if err := config.Validate(); err != nil { 27 | return nil, err 28 | } 29 | 30 | client, err := sarama.NewClient(addrs, &config.Config) 31 | if err != nil { 32 | return nil, err 33 | } 34 | 35 | return &Client{Client: client, config: *config}, nil 36 | } 37 | 38 | // ClusterConfig returns the cluster configuration. 39 | func (c *Client) ClusterConfig() *Config { 40 | cfg := c.config 41 | return &cfg 42 | } 43 | 44 | func (c *Client) claim() bool { 45 | return atomic.CompareAndSwapUint32(&c.inUse, 0, 1) 46 | } 47 | 48 | func (c *Client) release() { 49 | atomic.CompareAndSwapUint32(&c.inUse, 1, 0) 50 | } 51 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | (The MIT License) 2 | 3 | Copyright (c) 2017 Black Square Media Ltd 4 | 5 | Permission is hereby granted, free of charge, to any person obtaining 6 | a copy of this software and associated documentation files (the 7 | 'Software'), to deal in the Software without restriction, including 8 | without limitation the rights to use, copy, modify, merge, publish, 9 | distribute, sublicense, and/or sell copies of the Software, and to 10 | permit persons to whom the Software is furnished to do so, subject to 11 | the following conditions: 12 | 13 | The above copyright notice and this permission notice shall be 14 | included in all copies or substantial portions of the Software. 15 | 16 | THE SOFTWARE IS PROVIDED 'AS IS', WITHOUT WARRANTY OF ANY KIND, 17 | EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF 18 | MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. 19 | IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY 20 | CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, 21 | TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE 22 | SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. 23 | -------------------------------------------------------------------------------- /util.go: -------------------------------------------------------------------------------- 1 | package cluster 2 | 3 | import ( 4 | "fmt" 5 | "sort" 6 | "sync" 7 | ) 8 | 9 | type none struct{} 10 | 11 | type topicPartition struct { 12 | Topic string 13 | Partition int32 14 | } 15 | 16 | func (tp *topicPartition) String() string { 17 | return fmt.Sprintf("%s-%d", tp.Topic, tp.Partition) 18 | } 19 | 20 | type offsetInfo struct { 21 | Offset int64 22 | Metadata string 23 | } 24 | 25 | func (i offsetInfo) NextOffset(fallback int64) int64 { 26 | if i.Offset > -1 { 27 | return i.Offset 28 | } 29 | return fallback 30 | } 31 | 32 | type int32Slice []int32 33 | 34 | func (p int32Slice) Len() int { return len(p) } 35 | func (p int32Slice) Less(i, j int) bool { return p[i] < p[j] } 36 | func (p int32Slice) Swap(i, j int) { p[i], p[j] = p[j], p[i] } 37 | 38 | func (p int32Slice) Diff(o int32Slice) (res []int32) { 39 | on := len(o) 40 | for _, x := range p { 41 | n := sort.Search(on, func(i int) bool { return o[i] >= x }) 42 | if n < on && o[n] == x { 43 | continue 44 | } 45 | res = append(res, x) 46 | } 47 | return 48 | } 49 | 50 | // -------------------------------------------------------------------- 51 | 52 | type loopTomb struct { 53 | c chan none 54 | o sync.Once 55 | w sync.WaitGroup 56 | } 57 | 58 | func newLoopTomb() *loopTomb { 59 | return &loopTomb{c: make(chan none)} 60 | } 61 | 62 | func (t *loopTomb) stop() { t.o.Do(func() { close(t.c) }) } 63 | func (t *loopTomb) Close() { t.stop(); t.w.Wait() } 64 | 65 | func (t *loopTomb) Dying() <-chan none { return t.c } 66 | func (t *loopTomb) Go(f func(<-chan none)) { 67 | t.w.Add(1) 68 | 69 | go func() { 70 | defer t.stop() 71 | defer t.w.Done() 72 | 73 | f(t.c) 74 | }() 75 | } 76 | -------------------------------------------------------------------------------- /offsets.go: -------------------------------------------------------------------------------- 1 | package cluster 2 | 3 | import ( 4 | "sync" 5 | 6 | "github.com/Shopify/sarama" 7 | ) 8 | 9 | // OffsetStash allows to accumulate offsets and 10 | // mark them as processed in a bulk 11 | type OffsetStash struct { 12 | offsets map[topicPartition]offsetInfo 13 | mu sync.Mutex 14 | } 15 | 16 | // NewOffsetStash inits a blank stash 17 | func NewOffsetStash() *OffsetStash { 18 | return &OffsetStash{offsets: make(map[topicPartition]offsetInfo)} 19 | } 20 | 21 | // MarkOffset stashes the provided message offset 22 | func (s *OffsetStash) MarkOffset(msg *sarama.ConsumerMessage, metadata string) { 23 | s.MarkPartitionOffset(msg.Topic, msg.Partition, msg.Offset, metadata) 24 | } 25 | 26 | // MarkPartitionOffset stashes the offset for the provided topic/partition combination 27 | func (s *OffsetStash) MarkPartitionOffset(topic string, partition int32, offset int64, metadata string) { 28 | s.mu.Lock() 29 | defer s.mu.Unlock() 30 | 31 | key := topicPartition{Topic: topic, Partition: partition} 32 | if info := s.offsets[key]; offset >= info.Offset { 33 | info.Offset = offset 34 | info.Metadata = metadata 35 | s.offsets[key] = info 36 | } 37 | } 38 | 39 | // ResetPartitionOffset stashes the offset for the provided topic/partition combination. 40 | // Difference between ResetPartitionOffset and MarkPartitionOffset is that, ResetPartitionOffset supports earlier offsets 41 | func (s *OffsetStash) ResetPartitionOffset(topic string, partition int32, offset int64, metadata string) { 42 | s.mu.Lock() 43 | defer s.mu.Unlock() 44 | 45 | key := topicPartition{Topic: topic, Partition: partition} 46 | if info := s.offsets[key]; offset <= info.Offset { 47 | info.Offset = offset 48 | info.Metadata = metadata 49 | s.offsets[key] = info 50 | } 51 | } 52 | 53 | // ResetOffset stashes the provided message offset 54 | // See ResetPartitionOffset for explanation 55 | func (s *OffsetStash) ResetOffset(msg *sarama.ConsumerMessage, metadata string) { 56 | s.ResetPartitionOffset(msg.Topic, msg.Partition, msg.Offset, metadata) 57 | } 58 | 59 | // Offsets returns the latest stashed offsets by topic-partition 60 | func (s *OffsetStash) Offsets() map[string]int64 { 61 | s.mu.Lock() 62 | defer s.mu.Unlock() 63 | 64 | res := make(map[string]int64, len(s.offsets)) 65 | for tp, info := range s.offsets { 66 | res[tp.String()] = info.Offset 67 | } 68 | return res 69 | } 70 | -------------------------------------------------------------------------------- /README.md.tpl: -------------------------------------------------------------------------------- 1 | # Sarama Cluster 2 | 3 | [![GoDoc](https://godoc.org/github.com/bsm/sarama-cluster?status.svg)](https://godoc.org/github.com/bsm/sarama-cluster) 4 | [![Build Status](https://travis-ci.org/bsm/sarama-cluster.svg?branch=master)](https://travis-ci.org/bsm/sarama-cluster) 5 | [![Go Report Card](https://goreportcard.com/badge/github.com/bsm/sarama-cluster)](https://goreportcard.com/report/github.com/bsm/sarama-cluster) 6 | [![License](https://img.shields.io/badge/License-MIT-blue.svg)](https://opensource.org/licenses/MIT) 7 | 8 | Cluster extensions for [Sarama](https://github.com/Shopify/sarama), the Go client library for Apache Kafka 0.9 (and later). 9 | 10 | ## Documentation 11 | 12 | Documentation and example are available via godoc at http://godoc.org/github.com/bsm/sarama-cluster 13 | 14 | ## Examples 15 | 16 | Consumers have two modes of operation. In the default multiplexed mode messages (and errors) of multiple 17 | topics and partitions are all passed to the single channel: 18 | 19 | ```go 20 | package main 21 | 22 | import ( 23 | "fmt" 24 | "log" 25 | "os" 26 | "os/signal" 27 | 28 | cluster "github.com/bsm/sarama-cluster" 29 | ) 30 | 31 | func main() {{ "ExampleConsumer" | code }} 32 | ``` 33 | 34 | Users who require access to individual partitions can use the partitioned mode which exposes access to partition-level 35 | consumers: 36 | 37 | ```go 38 | package main 39 | 40 | import ( 41 | "fmt" 42 | "log" 43 | "os" 44 | "os/signal" 45 | 46 | cluster "github.com/bsm/sarama-cluster" 47 | ) 48 | 49 | func main() {{ "ExampleConsumer_Partitions" | code }} 50 | ``` 51 | 52 | ## Running tests 53 | 54 | You need to install Ginkgo & Gomega to run tests. Please see 55 | http://onsi.github.io/ginkgo for more details. 56 | 57 | To run tests, call: 58 | 59 | $ make test 60 | 61 | ## Troubleshooting 62 | 63 | ### Consumer not receiving any messages? 64 | 65 | By default, sarama's `Config.Consumer.Offsets.Initial` is set to `sarama.OffsetNewest`. This means that in the event that a brand new consumer is created, and it has never committed any offsets to kafka, it will only receive messages starting from the message after the current one that was written. 66 | 67 | If you wish to receive all messages (from the start of all messages in the topic) in the event that a consumer does not have any offsets committed to kafka, you need to set `Config.Consumer.Offsets.Initial` to `sarama.OffsetOldest`. 68 | -------------------------------------------------------------------------------- /offsets_test.go: -------------------------------------------------------------------------------- 1 | package cluster 2 | 3 | import ( 4 | . "github.com/onsi/ginkgo" 5 | . "github.com/onsi/gomega" 6 | ) 7 | 8 | var _ = Describe("OffsetStash", func() { 9 | var subject *OffsetStash 10 | 11 | BeforeEach(func() { 12 | subject = NewOffsetStash() 13 | }) 14 | 15 | It("should update", func() { 16 | Expect(subject.offsets).To(HaveLen(0)) 17 | 18 | subject.MarkPartitionOffset("topic", 0, 0, "m3ta") 19 | Expect(subject.offsets).To(HaveLen(1)) 20 | Expect(subject.offsets).To(HaveKeyWithValue( 21 | topicPartition{Topic: "topic", Partition: 0}, 22 | offsetInfo{Offset: 0, Metadata: "m3ta"}, 23 | )) 24 | 25 | subject.MarkPartitionOffset("topic", 0, 200, "m3ta") 26 | Expect(subject.offsets).To(HaveLen(1)) 27 | Expect(subject.offsets).To(HaveKeyWithValue( 28 | topicPartition{Topic: "topic", Partition: 0}, 29 | offsetInfo{Offset: 200, Metadata: "m3ta"}, 30 | )) 31 | 32 | subject.MarkPartitionOffset("topic", 0, 199, "m3t@") 33 | Expect(subject.offsets).To(HaveLen(1)) 34 | Expect(subject.offsets).To(HaveKeyWithValue( 35 | topicPartition{Topic: "topic", Partition: 0}, 36 | offsetInfo{Offset: 200, Metadata: "m3ta"}, 37 | )) 38 | 39 | subject.MarkPartitionOffset("topic", 1, 300, "") 40 | Expect(subject.offsets).To(HaveLen(2)) 41 | Expect(subject.offsets).To(HaveKeyWithValue( 42 | topicPartition{Topic: "topic", Partition: 1}, 43 | offsetInfo{Offset: 300, Metadata: ""}, 44 | )) 45 | }) 46 | 47 | It("should reset", func() { 48 | Expect(subject.offsets).To(HaveLen(0)) 49 | 50 | subject.MarkPartitionOffset("topic", 0, 0, "m3ta") 51 | Expect(subject.offsets).To(HaveLen(1)) 52 | Expect(subject.offsets).To(HaveKeyWithValue( 53 | topicPartition{Topic: "topic", Partition: 0}, 54 | offsetInfo{Offset: 0, Metadata: "m3ta"}, 55 | )) 56 | 57 | subject.MarkPartitionOffset("topic", 0, 200, "m3ta") 58 | Expect(subject.offsets).To(HaveLen(1)) 59 | Expect(subject.offsets).To(HaveKeyWithValue( 60 | topicPartition{Topic: "topic", Partition: 0}, 61 | offsetInfo{Offset: 200, Metadata: "m3ta"}, 62 | )) 63 | 64 | subject.ResetPartitionOffset("topic", 0, 199, "m3t@") 65 | Expect(subject.offsets).To(HaveLen(1)) 66 | Expect(subject.offsets).To(HaveKeyWithValue( 67 | topicPartition{Topic: "topic", Partition: 0}, 68 | offsetInfo{Offset: 199, Metadata: "m3t@"}, 69 | )) 70 | 71 | subject.MarkPartitionOffset("topic", 1, 300, "") 72 | Expect(subject.offsets).To(HaveLen(2)) 73 | Expect(subject.offsets).To(HaveKeyWithValue( 74 | topicPartition{Topic: "topic", Partition: 1}, 75 | offsetInfo{Offset: 300, Metadata: ""}, 76 | )) 77 | 78 | subject.ResetPartitionOffset("topic", 1, 200, "m3t@") 79 | Expect(subject.offsets).To(HaveLen(2)) 80 | Expect(subject.offsets).To(HaveKeyWithValue( 81 | topicPartition{Topic: "topic", Partition: 1}, 82 | offsetInfo{Offset: 200, Metadata: "m3t@"}, 83 | )) 84 | 85 | }) 86 | 87 | }) 88 | -------------------------------------------------------------------------------- /cmd/sarama-cluster-cli/main.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import ( 4 | "flag" 5 | "fmt" 6 | "log" 7 | "os" 8 | "os/signal" 9 | "strings" 10 | "syscall" 11 | 12 | "github.com/Shopify/sarama" 13 | "github.com/bsm/sarama-cluster" 14 | ) 15 | 16 | var ( 17 | groupID = flag.String("group", "", "REQUIRED: The shared consumer group name") 18 | brokerList = flag.String("brokers", os.Getenv("KAFKA_PEERS"), "The comma separated list of brokers in the Kafka cluster") 19 | topicList = flag.String("topics", "", "REQUIRED: The comma separated list of topics to consume") 20 | offset = flag.String("offset", "newest", "The offset to start with. Can be `oldest`, `newest`") 21 | verbose = flag.Bool("verbose", false, "Whether to turn on sarama logging") 22 | 23 | logger = log.New(os.Stderr, "", log.LstdFlags) 24 | ) 25 | 26 | func main() { 27 | flag.Parse() 28 | 29 | if *groupID == "" { 30 | printUsageErrorAndExit("You have to provide a -group name.") 31 | } else if *brokerList == "" { 32 | printUsageErrorAndExit("You have to provide -brokers as a comma-separated list, or set the KAFKA_PEERS environment variable.") 33 | } else if *topicList == "" { 34 | printUsageErrorAndExit("You have to provide -topics as a comma-separated list.") 35 | } 36 | 37 | // Init config 38 | config := cluster.NewConfig() 39 | if *verbose { 40 | sarama.Logger = logger 41 | } else { 42 | config.Consumer.Return.Errors = true 43 | config.Group.Return.Notifications = true 44 | } 45 | 46 | switch *offset { 47 | case "oldest": 48 | config.Consumer.Offsets.Initial = sarama.OffsetOldest 49 | case "newest": 50 | config.Consumer.Offsets.Initial = sarama.OffsetNewest 51 | default: 52 | printUsageErrorAndExit("-offset should be `oldest` or `newest`") 53 | } 54 | 55 | // Init consumer, consume errors & messages 56 | consumer, err := cluster.NewConsumer(strings.Split(*brokerList, ","), *groupID, strings.Split(*topicList, ","), config) 57 | if err != nil { 58 | printErrorAndExit(69, "Failed to start consumer: %s", err) 59 | } 60 | defer consumer.Close() 61 | 62 | // Create signal channel 63 | sigchan := make(chan os.Signal, 1) 64 | signal.Notify(sigchan, syscall.SIGHUP, syscall.SIGINT, syscall.SIGTERM) 65 | 66 | // Consume all channels, wait for signal to exit 67 | for { 68 | select { 69 | case msg, more := <-consumer.Messages(): 70 | if more { 71 | fmt.Fprintf(os.Stdout, "%s/%d/%d\t%s\n", msg.Topic, msg.Partition, msg.Offset, msg.Value) 72 | consumer.MarkOffset(msg, "") 73 | } 74 | case ntf, more := <-consumer.Notifications(): 75 | if more { 76 | logger.Printf("Rebalanced: %+v\n", ntf) 77 | } 78 | case err, more := <-consumer.Errors(): 79 | if more { 80 | logger.Printf("Error: %s\n", err.Error()) 81 | } 82 | case <-sigchan: 83 | return 84 | } 85 | } 86 | } 87 | 88 | func printErrorAndExit(code int, format string, values ...interface{}) { 89 | fmt.Fprintf(os.Stderr, "ERROR: "+format+"\n\n", values...) 90 | os.Exit(code) 91 | } 92 | 93 | func printUsageErrorAndExit(format string, values ...interface{}) { 94 | fmt.Fprintf(os.Stderr, "ERROR: "+format+"\n\n", values...) 95 | flag.Usage() 96 | os.Exit(64) 97 | } 98 | -------------------------------------------------------------------------------- /examples_test.go: -------------------------------------------------------------------------------- 1 | package cluster_test 2 | 3 | import ( 4 | "fmt" 5 | "log" 6 | "os" 7 | "os/signal" 8 | "regexp" 9 | 10 | cluster "github.com/bsm/sarama-cluster" 11 | ) 12 | 13 | // This example shows how to use the consumer to read messages 14 | // from a multiple topics through a multiplexed channel. 15 | func ExampleConsumer() { 16 | 17 | // init (custom) config, enable errors and notifications 18 | config := cluster.NewConfig() 19 | config.Consumer.Return.Errors = true 20 | config.Group.Return.Notifications = true 21 | 22 | // init consumer 23 | brokers := []string{"127.0.0.1:9092"} 24 | topics := []string{"my_topic", "other_topic"} 25 | consumer, err := cluster.NewConsumer(brokers, "my-consumer-group", topics, config) 26 | if err != nil { 27 | panic(err) 28 | } 29 | defer consumer.Close() 30 | 31 | // trap SIGINT to trigger a shutdown. 32 | signals := make(chan os.Signal, 1) 33 | signal.Notify(signals, os.Interrupt) 34 | 35 | // consume errors 36 | go func() { 37 | for err := range consumer.Errors() { 38 | log.Printf("Error: %s\n", err.Error()) 39 | } 40 | }() 41 | 42 | // consume notifications 43 | go func() { 44 | for ntf := range consumer.Notifications() { 45 | log.Printf("Rebalanced: %+v\n", ntf) 46 | } 47 | }() 48 | 49 | // consume messages, watch signals 50 | for { 51 | select { 52 | case msg, ok := <-consumer.Messages(): 53 | if ok { 54 | fmt.Fprintf(os.Stdout, "%s/%d/%d\t%s\t%s\n", msg.Topic, msg.Partition, msg.Offset, msg.Key, msg.Value) 55 | consumer.MarkOffset(msg, "") // mark message as processed 56 | } 57 | case <-signals: 58 | return 59 | } 60 | } 61 | } 62 | 63 | // This example shows how to use the consumer to read messages 64 | // through individual partitions. 65 | func ExampleConsumer_Partitions() { 66 | 67 | // init (custom) config, set mode to ConsumerModePartitions 68 | config := cluster.NewConfig() 69 | config.Group.Mode = cluster.ConsumerModePartitions 70 | 71 | // init consumer 72 | brokers := []string{"127.0.0.1:9092"} 73 | topics := []string{"my_topic", "other_topic"} 74 | consumer, err := cluster.NewConsumer(brokers, "my-consumer-group", topics, config) 75 | if err != nil { 76 | panic(err) 77 | } 78 | defer consumer.Close() 79 | 80 | // trap SIGINT to trigger a shutdown. 81 | signals := make(chan os.Signal, 1) 82 | signal.Notify(signals, os.Interrupt) 83 | 84 | // consume partitions 85 | for { 86 | select { 87 | case part, ok := <-consumer.Partitions(): 88 | if !ok { 89 | return 90 | } 91 | 92 | // start a separate goroutine to consume messages 93 | go func(pc cluster.PartitionConsumer) { 94 | for msg := range pc.Messages() { 95 | fmt.Fprintf(os.Stdout, "%s/%d/%d\t%s\t%s\n", msg.Topic, msg.Partition, msg.Offset, msg.Key, msg.Value) 96 | consumer.MarkOffset(msg, "") // mark message as processed 97 | } 98 | }(part) 99 | case <-signals: 100 | return 101 | } 102 | } 103 | } 104 | 105 | // This example shows how to use the consumer with 106 | // topic whitelists. 107 | func ExampleConfig_whitelist() { 108 | 109 | // init (custom) config, enable errors and notifications 110 | config := cluster.NewConfig() 111 | config.Group.Topics.Whitelist = regexp.MustCompile(`myservice.*`) 112 | 113 | // init consumer 114 | consumer, err := cluster.NewConsumer([]string{"127.0.0.1:9092"}, "my-consumer-group", nil, config) 115 | if err != nil { 116 | panic(err) 117 | } 118 | defer consumer.Close() 119 | 120 | // consume messages 121 | msg := <-consumer.Messages() 122 | fmt.Fprintf(os.Stdout, "%s/%d/%d\t%s\t%s\n", msg.Topic, msg.Partition, msg.Offset, msg.Key, msg.Value) 123 | } 124 | -------------------------------------------------------------------------------- /Gopkg.lock: -------------------------------------------------------------------------------- 1 | # This file is autogenerated, do not edit; changes may be undone by the next 'dep ensure'. 2 | 3 | 4 | [[projects]] 5 | name = "github.com/Shopify/sarama" 6 | packages = ["."] 7 | revision = "35324cf48e33d8260e1c7c18854465a904ade249" 8 | version = "v1.17.0" 9 | 10 | [[projects]] 11 | name = "github.com/davecgh/go-spew" 12 | packages = ["spew"] 13 | revision = "346938d642f2ec3594ed81d874461961cd0faa76" 14 | version = "v1.1.0" 15 | 16 | [[projects]] 17 | name = "github.com/eapache/go-resiliency" 18 | packages = ["breaker"] 19 | revision = "ea41b0fad31007accc7f806884dcdf3da98b79ce" 20 | version = "v1.1.0" 21 | 22 | [[projects]] 23 | branch = "master" 24 | name = "github.com/eapache/go-xerial-snappy" 25 | packages = ["."] 26 | revision = "bb955e01b9346ac19dc29eb16586c90ded99a98c" 27 | 28 | [[projects]] 29 | name = "github.com/eapache/queue" 30 | packages = ["."] 31 | revision = "44cc805cf13205b55f69e14bcb69867d1ae92f98" 32 | version = "v1.1.0" 33 | 34 | [[projects]] 35 | branch = "master" 36 | name = "github.com/golang/snappy" 37 | packages = ["."] 38 | revision = "2e65f85255dbc3072edf28d6b5b8efc472979f5a" 39 | 40 | [[projects]] 41 | name = "github.com/onsi/ginkgo" 42 | packages = [ 43 | ".", 44 | "config", 45 | "extensions/table", 46 | "internal/codelocation", 47 | "internal/containernode", 48 | "internal/failer", 49 | "internal/leafnodes", 50 | "internal/remote", 51 | "internal/spec", 52 | "internal/spec_iterator", 53 | "internal/specrunner", 54 | "internal/suite", 55 | "internal/testingtproxy", 56 | "internal/writer", 57 | "reporters", 58 | "reporters/stenographer", 59 | "reporters/stenographer/support/go-colorable", 60 | "reporters/stenographer/support/go-isatty", 61 | "types" 62 | ] 63 | revision = "fa5fabab2a1bfbd924faf4c067d07ae414e2aedf" 64 | version = "v1.5.0" 65 | 66 | [[projects]] 67 | name = "github.com/onsi/gomega" 68 | packages = [ 69 | ".", 70 | "format", 71 | "internal/assertion", 72 | "internal/asyncassertion", 73 | "internal/oraclematcher", 74 | "internal/testingtsupport", 75 | "matchers", 76 | "matchers/support/goraph/bipartitegraph", 77 | "matchers/support/goraph/edge", 78 | "matchers/support/goraph/node", 79 | "matchers/support/goraph/util", 80 | "types" 81 | ] 82 | revision = "62bff4df71bdbc266561a0caee19f0594b17c240" 83 | version = "v1.4.0" 84 | 85 | [[projects]] 86 | name = "github.com/pierrec/lz4" 87 | packages = [ 88 | ".", 89 | "internal/xxh32" 90 | ] 91 | revision = "6b9367c9ff401dbc54fabce3fb8d972e799b702d" 92 | version = "v2.0.2" 93 | 94 | [[projects]] 95 | branch = "master" 96 | name = "github.com/rcrowley/go-metrics" 97 | packages = ["."] 98 | revision = "e2704e165165ec55d062f5919b4b29494e9fa790" 99 | 100 | [[projects]] 101 | branch = "master" 102 | name = "golang.org/x/net" 103 | packages = [ 104 | "html", 105 | "html/atom", 106 | "html/charset" 107 | ] 108 | revision = "afe8f62b1d6bbd81f31868121a50b06d8188e1f9" 109 | 110 | [[projects]] 111 | branch = "master" 112 | name = "golang.org/x/sys" 113 | packages = ["unix"] 114 | revision = "63fc586f45fe72d95d5240a5d5eb95e6503907d3" 115 | 116 | [[projects]] 117 | name = "golang.org/x/text" 118 | packages = [ 119 | "encoding", 120 | "encoding/charmap", 121 | "encoding/htmlindex", 122 | "encoding/internal", 123 | "encoding/internal/identifier", 124 | "encoding/japanese", 125 | "encoding/korean", 126 | "encoding/simplifiedchinese", 127 | "encoding/traditionalchinese", 128 | "encoding/unicode", 129 | "internal/gen", 130 | "internal/tag", 131 | "internal/utf8internal", 132 | "language", 133 | "runes", 134 | "transform", 135 | "unicode/cldr" 136 | ] 137 | revision = "f21a4dfb5e38f5895301dc265a8def02365cc3d0" 138 | version = "v0.3.0" 139 | 140 | [[projects]] 141 | name = "gopkg.in/yaml.v2" 142 | packages = ["."] 143 | revision = "5420a8b6744d3b0345ab293f6fcba19c978f1183" 144 | version = "v2.2.1" 145 | 146 | [solve-meta] 147 | analyzer-name = "dep" 148 | analyzer-version = 1 149 | inputs-digest = "2fa33a2d1ae87e0905ef09332bb4b3fda29179f6bcd48fd3b94070774b9e458b" 150 | solver-name = "gps-cdcl" 151 | solver-version = 1 152 | -------------------------------------------------------------------------------- /partitions_test.go: -------------------------------------------------------------------------------- 1 | package cluster 2 | 3 | import ( 4 | "github.com/Shopify/sarama" 5 | . "github.com/onsi/ginkgo" 6 | . "github.com/onsi/gomega" 7 | ) 8 | 9 | var _ = Describe("partitionConsumer", func() { 10 | var subject *partitionConsumer 11 | 12 | BeforeEach(func() { 13 | var err error 14 | subject, err = newPartitionConsumer(&mockConsumer{}, "topic", 0, offsetInfo{2000, "m3ta"}, sarama.OffsetOldest) 15 | Expect(err).NotTo(HaveOccurred()) 16 | }) 17 | 18 | AfterEach(func() { 19 | close(subject.dead) 20 | Expect(subject.Close()).NotTo(HaveOccurred()) 21 | }) 22 | 23 | It("should set state", func() { 24 | Expect(subject.getState()).To(Equal(partitionState{ 25 | Info: offsetInfo{2000, "m3ta"}, 26 | })) 27 | }) 28 | 29 | It("should recover from default offset if requested offset is out of bounds", func() { 30 | pc, err := newPartitionConsumer(&mockConsumer{}, "topic", 0, offsetInfo{200, "m3ta"}, sarama.OffsetOldest) 31 | Expect(err).NotTo(HaveOccurred()) 32 | defer pc.Close() 33 | close(pc.dead) 34 | 35 | state := pc.getState() 36 | Expect(state.Info.Offset).To(Equal(int64(-1))) 37 | Expect(state.Info.Metadata).To(Equal("m3ta")) 38 | }) 39 | 40 | It("should update state", func() { 41 | subject.MarkOffset(2001, "met@") // should set state 42 | Expect(subject.getState()).To(Equal(partitionState{ 43 | Info: offsetInfo{2002, "met@"}, 44 | Dirty: true, 45 | })) 46 | 47 | subject.markCommitted(2002) // should reset dirty status 48 | Expect(subject.getState()).To(Equal(partitionState{ 49 | Info: offsetInfo{2002, "met@"}, 50 | })) 51 | 52 | subject.MarkOffset(2001, "me7a") // should not update state 53 | Expect(subject.getState()).To(Equal(partitionState{ 54 | Info: offsetInfo{2002, "met@"}, 55 | })) 56 | 57 | subject.MarkOffset(2002, "me7a") // should bump state 58 | Expect(subject.getState()).To(Equal(partitionState{ 59 | Info: offsetInfo{2003, "me7a"}, 60 | Dirty: true, 61 | })) 62 | 63 | // After committing a later offset, try rewinding back to earlier offset with new metadata. 64 | subject.ResetOffset(2001, "met@") 65 | Expect(subject.getState()).To(Equal(partitionState{ 66 | Info: offsetInfo{2002, "met@"}, 67 | Dirty: true, 68 | })) 69 | 70 | subject.markCommitted(2002) // should not unset state 71 | Expect(subject.getState()).To(Equal(partitionState{ 72 | Info: offsetInfo{2002, "met@"}, 73 | })) 74 | 75 | subject.MarkOffset(2002, "me7a") // should bump state 76 | Expect(subject.getState()).To(Equal(partitionState{ 77 | Info: offsetInfo{2003, "me7a"}, 78 | Dirty: true, 79 | })) 80 | 81 | subject.markCommitted(2003) 82 | Expect(subject.getState()).To(Equal(partitionState{ 83 | Info: offsetInfo{2003, "me7a"}, 84 | })) 85 | }) 86 | 87 | }) 88 | 89 | var _ = Describe("partitionMap", func() { 90 | var subject *partitionMap 91 | 92 | BeforeEach(func() { 93 | subject = newPartitionMap() 94 | }) 95 | 96 | It("should fetch/store", func() { 97 | Expect(subject.Fetch("topic", 0)).To(BeNil()) 98 | 99 | pc, err := newPartitionConsumer(&mockConsumer{}, "topic", 0, offsetInfo{2000, "m3ta"}, sarama.OffsetNewest) 100 | Expect(err).NotTo(HaveOccurred()) 101 | 102 | subject.Store("topic", 0, pc) 103 | Expect(subject.Fetch("topic", 0)).To(Equal(pc)) 104 | Expect(subject.Fetch("topic", 1)).To(BeNil()) 105 | Expect(subject.Fetch("other", 0)).To(BeNil()) 106 | }) 107 | 108 | It("should return info", func() { 109 | pc0, err := newPartitionConsumer(&mockConsumer{}, "topic", 0, offsetInfo{2000, "m3ta"}, sarama.OffsetNewest) 110 | Expect(err).NotTo(HaveOccurred()) 111 | pc1, err := newPartitionConsumer(&mockConsumer{}, "topic", 1, offsetInfo{2000, "m3ta"}, sarama.OffsetNewest) 112 | Expect(err).NotTo(HaveOccurred()) 113 | subject.Store("topic", 0, pc0) 114 | subject.Store("topic", 1, pc1) 115 | 116 | info := subject.Info() 117 | Expect(info).To(HaveLen(1)) 118 | Expect(info).To(HaveKeyWithValue("topic", []int32{0, 1})) 119 | }) 120 | 121 | It("should create snapshots", func() { 122 | pc0, err := newPartitionConsumer(&mockConsumer{}, "topic", 0, offsetInfo{2000, "m3ta"}, sarama.OffsetNewest) 123 | Expect(err).NotTo(HaveOccurred()) 124 | pc1, err := newPartitionConsumer(&mockConsumer{}, "topic", 1, offsetInfo{2000, "m3ta"}, sarama.OffsetNewest) 125 | Expect(err).NotTo(HaveOccurred()) 126 | 127 | subject.Store("topic", 0, pc0) 128 | subject.Store("topic", 1, pc1) 129 | subject.Fetch("topic", 1).MarkOffset(2000, "met@") 130 | 131 | Expect(subject.Snapshot()).To(Equal(map[topicPartition]partitionState{ 132 | {"topic", 0}: {Info: offsetInfo{2000, "m3ta"}, Dirty: false}, 133 | {"topic", 1}: {Info: offsetInfo{2001, "met@"}, Dirty: true}, 134 | })) 135 | }) 136 | 137 | }) 138 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # Sarama Cluster 2 | 3 | [![GoDoc](https://godoc.org/github.com/bsm/sarama-cluster?status.svg)](https://godoc.org/github.com/bsm/sarama-cluster) 4 | [![Build Status](https://travis-ci.org/bsm/sarama-cluster.svg?branch=master)](https://travis-ci.org/bsm/sarama-cluster) 5 | [![Go Report Card](https://goreportcard.com/badge/github.com/bsm/sarama-cluster)](https://goreportcard.com/report/github.com/bsm/sarama-cluster) 6 | [![License](https://img.shields.io/badge/License-MIT-blue.svg)](https://opensource.org/licenses/MIT) 7 | 8 | Cluster extensions for [Sarama](https://github.com/Shopify/sarama), the Go client library for Apache Kafka 0.9 (and later). 9 | 10 | ## DEPRECATION NOTICE 11 | 12 | Please note that since https://github.com/Shopify/sarama/pull/1099 was merged and released (>= v1.19.0) this library is officially deprecated. The native implementation supports a variety of use cases that are not available through this library. 13 | 14 | ## Documentation 15 | 16 | Documentation and example are available via godoc at http://godoc.org/github.com/bsm/sarama-cluster 17 | 18 | ## Examples 19 | 20 | Consumers have two modes of operation. In the default multiplexed mode messages (and errors) of multiple 21 | topics and partitions are all passed to the single channel: 22 | 23 | ```go 24 | package main 25 | 26 | import ( 27 | "fmt" 28 | "log" 29 | "os" 30 | "os/signal" 31 | 32 | cluster "github.com/bsm/sarama-cluster" 33 | ) 34 | 35 | func main() { 36 | 37 | // init (custom) config, enable errors and notifications 38 | config := cluster.NewConfig() 39 | config.Consumer.Return.Errors = true 40 | config.Group.Return.Notifications = true 41 | 42 | // init consumer 43 | brokers := []string{"127.0.0.1:9092"} 44 | topics := []string{"my_topic", "other_topic"} 45 | consumer, err := cluster.NewConsumer(brokers, "my-consumer-group", topics, config) 46 | if err != nil { 47 | panic(err) 48 | } 49 | defer consumer.Close() 50 | 51 | // trap SIGINT to trigger a shutdown. 52 | signals := make(chan os.Signal, 1) 53 | signal.Notify(signals, os.Interrupt) 54 | 55 | // consume errors 56 | go func() { 57 | for err := range consumer.Errors() { 58 | log.Printf("Error: %s\n", err.Error()) 59 | } 60 | }() 61 | 62 | // consume notifications 63 | go func() { 64 | for ntf := range consumer.Notifications() { 65 | log.Printf("Rebalanced: %+v\n", ntf) 66 | } 67 | }() 68 | 69 | // consume messages, watch signals 70 | for { 71 | select { 72 | case msg, ok := <-consumer.Messages(): 73 | if ok { 74 | fmt.Fprintf(os.Stdout, "%s/%d/%d\t%s\t%s\n", msg.Topic, msg.Partition, msg.Offset, msg.Key, msg.Value) 75 | consumer.MarkOffset(msg, "") // mark message as processed 76 | } 77 | case <-signals: 78 | return 79 | } 80 | } 81 | } 82 | ``` 83 | 84 | Users who require access to individual partitions can use the partitioned mode which exposes access to partition-level 85 | consumers: 86 | 87 | ```go 88 | package main 89 | 90 | import ( 91 | "fmt" 92 | "log" 93 | "os" 94 | "os/signal" 95 | 96 | cluster "github.com/bsm/sarama-cluster" 97 | ) 98 | 99 | func main() { 100 | 101 | // init (custom) config, set mode to ConsumerModePartitions 102 | config := cluster.NewConfig() 103 | config.Group.Mode = cluster.ConsumerModePartitions 104 | 105 | // init consumer 106 | brokers := []string{"127.0.0.1:9092"} 107 | topics := []string{"my_topic", "other_topic"} 108 | consumer, err := cluster.NewConsumer(brokers, "my-consumer-group", topics, config) 109 | if err != nil { 110 | panic(err) 111 | } 112 | defer consumer.Close() 113 | 114 | // trap SIGINT to trigger a shutdown. 115 | signals := make(chan os.Signal, 1) 116 | signal.Notify(signals, os.Interrupt) 117 | 118 | // consume partitions 119 | for { 120 | select { 121 | case part, ok := <-consumer.Partitions(): 122 | if !ok { 123 | return 124 | } 125 | 126 | // start a separate goroutine to consume messages 127 | go func(pc cluster.PartitionConsumer) { 128 | for msg := range pc.Messages() { 129 | fmt.Fprintf(os.Stdout, "%s/%d/%d\t%s\t%s\n", msg.Topic, msg.Partition, msg.Offset, msg.Key, msg.Value) 130 | consumer.MarkOffset(msg, "") // mark message as processed 131 | } 132 | }(part) 133 | case <-signals: 134 | return 135 | } 136 | } 137 | } 138 | ``` 139 | 140 | ## Running tests 141 | 142 | You need to install Ginkgo & Gomega to run tests. Please see 143 | http://onsi.github.io/ginkgo for more details. 144 | 145 | To run tests, call: 146 | 147 | $ make test 148 | 149 | ## Troubleshooting 150 | 151 | ### Consumer not receiving any messages? 152 | 153 | By default, sarama's `Config.Consumer.Offsets.Initial` is set to `sarama.OffsetNewest`. This means that in the event that a brand new consumer is created, and it has never committed any offsets to kafka, it will only receive messages starting from the message after the current one that was written. 154 | 155 | If you wish to receive all messages (from the start of all messages in the topic) in the event that a consumer does not have any offsets committed to kafka, you need to set `Config.Consumer.Offsets.Initial` to `sarama.OffsetOldest`. 156 | -------------------------------------------------------------------------------- /balancer_test.go: -------------------------------------------------------------------------------- 1 | package cluster 2 | 3 | import ( 4 | "github.com/Shopify/sarama" 5 | . "github.com/onsi/ginkgo" 6 | . "github.com/onsi/ginkgo/extensions/table" 7 | . "github.com/onsi/gomega" 8 | ) 9 | 10 | var _ = Describe("Notification", func() { 11 | 12 | It("should init and convert", func() { 13 | n := newNotification(map[string][]int32{ 14 | "a": {1, 2, 3}, 15 | "b": {4, 5}, 16 | "c": {1, 2}, 17 | }) 18 | Expect(n).To(Equal(&Notification{ 19 | Type: RebalanceStart, 20 | Current: map[string][]int32{"a": {1, 2, 3}, "b": {4, 5}, "c": {1, 2}}, 21 | })) 22 | 23 | o := n.success(map[string][]int32{ 24 | "a": {3, 4}, 25 | "b": {1, 2, 3, 4}, 26 | "d": {3, 4}, 27 | }) 28 | Expect(o).To(Equal(&Notification{ 29 | Type: RebalanceOK, 30 | Claimed: map[string][]int32{"a": {4}, "b": {1, 2, 3}, "d": {3, 4}}, 31 | Released: map[string][]int32{"a": {1, 2}, "b": {5}, "c": {1, 2}}, 32 | Current: map[string][]int32{"a": {3, 4}, "b": {1, 2, 3, 4}, "d": {3, 4}}, 33 | })) 34 | }) 35 | 36 | It("should copy on error", func() { 37 | n := newNotification(map[string][]int32{ 38 | "a": {1, 2, 3}, 39 | "b": {4, 5}, 40 | "c": {1, 2}, 41 | }) 42 | o := n.error() 43 | 44 | Expect(n).To(Equal(&Notification{ 45 | Type: RebalanceStart, 46 | Current: map[string][]int32{"a": {1, 2, 3}, "b": {4, 5}, "c": {1, 2}}, 47 | })) 48 | 49 | Expect(o).To(Equal(&Notification{ 50 | Type: RebalanceError, 51 | Claimed: map[string][]int32{}, 52 | Released: map[string][]int32{}, 53 | Current: map[string][]int32{"a": {1, 2, 3}, "b": {4, 5}, "c": {1, 2}}, 54 | })) 55 | }) 56 | 57 | }) 58 | 59 | var _ = Describe("balancer", func() { 60 | var subject *balancer 61 | 62 | BeforeEach(func() { 63 | client := &mockClient{ 64 | topics: map[string][]int32{ 65 | "one": {0, 1, 2, 3}, 66 | "two": {0, 1, 2}, 67 | "three": {0, 1}, 68 | }, 69 | } 70 | 71 | var err error 72 | subject, err = newBalancerFromMeta(client, StrategyRange, map[string]sarama.ConsumerGroupMemberMetadata{ 73 | "b": {Topics: []string{"three", "one"}}, 74 | "a": {Topics: []string{"one", "two"}}, 75 | }) 76 | Expect(err).NotTo(HaveOccurred()) 77 | }) 78 | 79 | It("should parse from meta data", func() { 80 | Expect(subject.topics).To(HaveLen(3)) 81 | }) 82 | 83 | It("should perform", func() { 84 | Expect(subject.Perform()).To(Equal(map[string]map[string][]int32{ 85 | "a": {"one": {0, 1}, "two": {0, 1, 2}}, 86 | "b": {"one": {2, 3}, "three": {0, 1}}, 87 | })) 88 | 89 | subject.strategy = StrategyRoundRobin 90 | Expect(subject.Perform()).To(Equal(map[string]map[string][]int32{ 91 | "a": {"one": {0, 2}, "two": {0, 1, 2}}, 92 | "b": {"one": {1, 3}, "three": {0, 1}}, 93 | })) 94 | }) 95 | 96 | }) 97 | 98 | var _ = Describe("topicInfo", func() { 99 | 100 | DescribeTable("Ranges", 101 | func(memberIDs []string, partitions []int32, expected map[string][]int32) { 102 | info := topicInfo{MemberIDs: memberIDs, Partitions: partitions} 103 | Expect(info.Ranges()).To(Equal(expected)) 104 | }, 105 | 106 | Entry("three members, three partitions", []string{"M1", "M2", "M3"}, []int32{0, 1, 2}, map[string][]int32{ 107 | "M1": {0}, "M2": {1}, "M3": {2}, 108 | }), 109 | Entry("member ID order", []string{"M3", "M1", "M2"}, []int32{0, 1, 2}, map[string][]int32{ 110 | "M1": {0}, "M2": {1}, "M3": {2}, 111 | }), 112 | Entry("more members than partitions", []string{"M1", "M2", "M3"}, []int32{0, 1}, map[string][]int32{ 113 | "M1": {0}, "M3": {1}, 114 | }), 115 | Entry("far more members than partitions", []string{"M1", "M2", "M3"}, []int32{0}, map[string][]int32{ 116 | "M2": {0}, 117 | }), 118 | Entry("fewer members than partitions", []string{"M1", "M2", "M3"}, []int32{0, 1, 2, 3}, map[string][]int32{ 119 | "M1": {0}, "M2": {1, 2}, "M3": {3}, 120 | }), 121 | Entry("uneven members/partitions ratio", []string{"M1", "M2", "M3"}, []int32{0, 2, 4, 6, 8}, map[string][]int32{ 122 | "M1": {0, 2}, "M2": {4}, "M3": {6, 8}, 123 | }), 124 | ) 125 | 126 | DescribeTable("RoundRobin", 127 | func(memberIDs []string, partitions []int32, expected map[string][]int32) { 128 | info := topicInfo{MemberIDs: memberIDs, Partitions: partitions} 129 | Expect(info.RoundRobin()).To(Equal(expected)) 130 | }, 131 | 132 | Entry("three members, three partitions", []string{"M1", "M2", "M3"}, []int32{0, 1, 2}, map[string][]int32{ 133 | "M1": {0}, "M2": {1}, "M3": {2}, 134 | }), 135 | Entry("member ID order", []string{"M3", "M1", "M2"}, []int32{0, 1, 2}, map[string][]int32{ 136 | "M1": {0}, "M2": {1}, "M3": {2}, 137 | }), 138 | Entry("more members than partitions", []string{"M1", "M2", "M3"}, []int32{0, 1}, map[string][]int32{ 139 | "M1": {0}, "M2": {1}, 140 | }), 141 | Entry("far more members than partitions", []string{"M1", "M2", "M3"}, []int32{0}, map[string][]int32{ 142 | "M1": {0}, 143 | }), 144 | Entry("fewer members than partitions", []string{"M1", "M2", "M3"}, []int32{0, 1, 2, 3}, map[string][]int32{ 145 | "M1": {0, 3}, "M2": {1}, "M3": {2}, 146 | }), 147 | Entry("uneven members/partitions ratio", []string{"M1", "M2", "M3"}, []int32{0, 2, 4, 6, 8}, map[string][]int32{ 148 | "M1": {0, 6}, "M2": {2, 8}, "M3": {4}, 149 | }), 150 | ) 151 | 152 | }) 153 | -------------------------------------------------------------------------------- /balancer.go: -------------------------------------------------------------------------------- 1 | package cluster 2 | 3 | import ( 4 | "math" 5 | "sort" 6 | 7 | "github.com/Shopify/sarama" 8 | ) 9 | 10 | // NotificationType defines the type of notification 11 | type NotificationType uint8 12 | 13 | // String describes the notification type 14 | func (t NotificationType) String() string { 15 | switch t { 16 | case RebalanceStart: 17 | return "rebalance start" 18 | case RebalanceOK: 19 | return "rebalance OK" 20 | case RebalanceError: 21 | return "rebalance error" 22 | } 23 | return "unknown" 24 | } 25 | 26 | const ( 27 | UnknownNotification NotificationType = iota 28 | RebalanceStart 29 | RebalanceOK 30 | RebalanceError 31 | ) 32 | 33 | // Notification are state events emitted by the consumers on rebalance 34 | type Notification struct { 35 | // Type exposes the notification type 36 | Type NotificationType 37 | 38 | // Claimed contains topic/partitions that were claimed by this rebalance cycle 39 | Claimed map[string][]int32 40 | 41 | // Released contains topic/partitions that were released as part of this rebalance cycle 42 | Released map[string][]int32 43 | 44 | // Current are topic/partitions that are currently claimed to the consumer 45 | Current map[string][]int32 46 | } 47 | 48 | func newNotification(current map[string][]int32) *Notification { 49 | return &Notification{ 50 | Type: RebalanceStart, 51 | Current: current, 52 | } 53 | } 54 | 55 | func (n *Notification) success(current map[string][]int32) *Notification { 56 | o := &Notification{ 57 | Type: RebalanceOK, 58 | Claimed: make(map[string][]int32), 59 | Released: make(map[string][]int32), 60 | Current: current, 61 | } 62 | for topic, partitions := range current { 63 | o.Claimed[topic] = int32Slice(partitions).Diff(int32Slice(n.Current[topic])) 64 | } 65 | for topic, partitions := range n.Current { 66 | o.Released[topic] = int32Slice(partitions).Diff(int32Slice(current[topic])) 67 | } 68 | return o 69 | } 70 | 71 | func (n *Notification) error() *Notification { 72 | o := &Notification{ 73 | Type: RebalanceError, 74 | Claimed: make(map[string][]int32), 75 | Released: make(map[string][]int32), 76 | Current: make(map[string][]int32), 77 | } 78 | for topic, partitions := range n.Claimed { 79 | o.Claimed[topic] = append(make([]int32, 0, len(partitions)), partitions...) 80 | } 81 | for topic, partitions := range n.Released { 82 | o.Released[topic] = append(make([]int32, 0, len(partitions)), partitions...) 83 | } 84 | for topic, partitions := range n.Current { 85 | o.Current[topic] = append(make([]int32, 0, len(partitions)), partitions...) 86 | } 87 | return o 88 | } 89 | 90 | // -------------------------------------------------------------------- 91 | 92 | type topicInfo struct { 93 | Partitions []int32 94 | MemberIDs []string 95 | } 96 | 97 | func (info topicInfo) Perform(s Strategy) map[string][]int32 { 98 | if s == StrategyRoundRobin { 99 | return info.RoundRobin() 100 | } 101 | return info.Ranges() 102 | } 103 | 104 | func (info topicInfo) Ranges() map[string][]int32 { 105 | sort.Strings(info.MemberIDs) 106 | 107 | mlen := len(info.MemberIDs) 108 | plen := len(info.Partitions) 109 | res := make(map[string][]int32, mlen) 110 | 111 | for pos, memberID := range info.MemberIDs { 112 | n, i := float64(plen)/float64(mlen), float64(pos) 113 | min := int(math.Floor(i*n + 0.5)) 114 | max := int(math.Floor((i+1)*n + 0.5)) 115 | sub := info.Partitions[min:max] 116 | if len(sub) > 0 { 117 | res[memberID] = sub 118 | } 119 | } 120 | return res 121 | } 122 | 123 | func (info topicInfo) RoundRobin() map[string][]int32 { 124 | sort.Strings(info.MemberIDs) 125 | 126 | mlen := len(info.MemberIDs) 127 | res := make(map[string][]int32, mlen) 128 | for i, pnum := range info.Partitions { 129 | memberID := info.MemberIDs[i%mlen] 130 | res[memberID] = append(res[memberID], pnum) 131 | } 132 | return res 133 | } 134 | 135 | // -------------------------------------------------------------------- 136 | 137 | type balancer struct { 138 | client sarama.Client 139 | topics map[string]topicInfo 140 | strategy Strategy 141 | } 142 | 143 | func newBalancerFromMeta(client sarama.Client, strategy Strategy, members map[string]sarama.ConsumerGroupMemberMetadata) (*balancer, error) { 144 | balancer := newBalancer(client, strategy) 145 | for memberID, meta := range members { 146 | for _, topic := range meta.Topics { 147 | if err := balancer.Topic(topic, memberID); err != nil { 148 | return nil, err 149 | } 150 | } 151 | } 152 | return balancer, nil 153 | } 154 | 155 | func newBalancer(client sarama.Client, strategy Strategy) *balancer { 156 | return &balancer{ 157 | client: client, 158 | topics: make(map[string]topicInfo), 159 | strategy: strategy, 160 | } 161 | } 162 | 163 | func (r *balancer) Topic(name string, memberID string) error { 164 | topic, ok := r.topics[name] 165 | if !ok { 166 | nums, err := r.client.Partitions(name) 167 | if err != nil { 168 | return err 169 | } 170 | topic = topicInfo{ 171 | Partitions: nums, 172 | MemberIDs: make([]string, 0, 1), 173 | } 174 | } 175 | topic.MemberIDs = append(topic.MemberIDs, memberID) 176 | r.topics[name] = topic 177 | return nil 178 | } 179 | 180 | func (r *balancer) Perform() map[string]map[string][]int32 { 181 | res := make(map[string]map[string][]int32, 1) 182 | for topic, info := range r.topics { 183 | for memberID, partitions := range info.Perform(r.strategy) { 184 | if _, ok := res[memberID]; !ok { 185 | res[memberID] = make(map[string][]int32, 1) 186 | } 187 | res[memberID][topic] = partitions 188 | } 189 | } 190 | return res 191 | } 192 | -------------------------------------------------------------------------------- /config.go: -------------------------------------------------------------------------------- 1 | package cluster 2 | 3 | import ( 4 | "regexp" 5 | "time" 6 | 7 | "github.com/Shopify/sarama" 8 | ) 9 | 10 | var minVersion = sarama.V0_9_0_0 11 | 12 | type ConsumerMode uint8 13 | 14 | const ( 15 | ConsumerModeMultiplex ConsumerMode = iota 16 | ConsumerModePartitions 17 | ) 18 | 19 | // Config extends sarama.Config with Group specific namespace 20 | type Config struct { 21 | sarama.Config 22 | 23 | // Group is the namespace for group management properties 24 | Group struct { 25 | 26 | // The strategy to use for the allocation of partitions to consumers (defaults to StrategyRange) 27 | PartitionStrategy Strategy 28 | 29 | // By default, messages and errors from the subscribed topics and partitions are all multiplexed and 30 | // made available through the consumer's Messages() and Errors() channels. 31 | // 32 | // Users who require low-level access can enable ConsumerModePartitions where individual partitions 33 | // are exposed on the Partitions() channel. Messages and errors must then be consumed on the partitions 34 | // themselves. 35 | Mode ConsumerMode 36 | 37 | Offsets struct { 38 | Retry struct { 39 | // The number retries when committing offsets (defaults to 3). 40 | Max int 41 | } 42 | Synchronization struct { 43 | // The duration allowed for other clients to commit their offsets before resumption in this client, e.g. during a rebalance 44 | // NewConfig sets this to the Consumer.MaxProcessingTime duration of the Sarama configuration 45 | DwellTime time.Duration 46 | } 47 | } 48 | 49 | Session struct { 50 | // The allowed session timeout for registered consumers (defaults to 30s). 51 | // Must be within the allowed server range. 52 | Timeout time.Duration 53 | } 54 | 55 | Heartbeat struct { 56 | // Interval between each heartbeat (defaults to 3s). It should be no more 57 | // than 1/3rd of the Group.Session.Timeout setting 58 | Interval time.Duration 59 | } 60 | 61 | // Return specifies which group channels will be populated. If they are set to true, 62 | // you must read from the respective channels to prevent deadlock. 63 | Return struct { 64 | // If enabled, rebalance notification will be returned on the 65 | // Notifications channel (default disabled). 66 | Notifications bool 67 | } 68 | 69 | Topics struct { 70 | // An additional whitelist of topics to subscribe to. 71 | Whitelist *regexp.Regexp 72 | // An additional blacklist of topics to avoid. If set, this will precede over 73 | // the Whitelist setting. 74 | Blacklist *regexp.Regexp 75 | } 76 | 77 | Member struct { 78 | // Custom metadata to include when joining the group. The user data for all joined members 79 | // can be retrieved by sending a DescribeGroupRequest to the broker that is the 80 | // coordinator for the group. 81 | UserData []byte 82 | } 83 | } 84 | } 85 | 86 | // NewConfig returns a new configuration instance with sane defaults. 87 | func NewConfig() *Config { 88 | c := &Config{ 89 | Config: *sarama.NewConfig(), 90 | } 91 | c.Group.PartitionStrategy = StrategyRange 92 | c.Group.Offsets.Retry.Max = 3 93 | c.Group.Offsets.Synchronization.DwellTime = c.Consumer.MaxProcessingTime 94 | c.Group.Session.Timeout = 30 * time.Second 95 | c.Group.Heartbeat.Interval = 3 * time.Second 96 | c.Config.Version = minVersion 97 | return c 98 | } 99 | 100 | // Validate checks a Config instance. It will return a 101 | // sarama.ConfigurationError if the specified values don't make sense. 102 | func (c *Config) Validate() error { 103 | if c.Group.Heartbeat.Interval%time.Millisecond != 0 { 104 | sarama.Logger.Println("Group.Heartbeat.Interval only supports millisecond precision; nanoseconds will be truncated.") 105 | } 106 | if c.Group.Session.Timeout%time.Millisecond != 0 { 107 | sarama.Logger.Println("Group.Session.Timeout only supports millisecond precision; nanoseconds will be truncated.") 108 | } 109 | if c.Group.PartitionStrategy != StrategyRange && c.Group.PartitionStrategy != StrategyRoundRobin { 110 | sarama.Logger.Println("Group.PartitionStrategy is not supported; range will be assumed.") 111 | } 112 | if !c.Version.IsAtLeast(minVersion) { 113 | sarama.Logger.Println("Version is not supported; 0.9. will be assumed.") 114 | c.Version = minVersion 115 | } 116 | if err := c.Config.Validate(); err != nil { 117 | return err 118 | } 119 | 120 | // validate the Group values 121 | switch { 122 | case c.Group.Offsets.Retry.Max < 0: 123 | return sarama.ConfigurationError("Group.Offsets.Retry.Max must be >= 0") 124 | case c.Group.Offsets.Synchronization.DwellTime <= 0: 125 | return sarama.ConfigurationError("Group.Offsets.Synchronization.DwellTime must be > 0") 126 | case c.Group.Offsets.Synchronization.DwellTime > 10*time.Minute: 127 | return sarama.ConfigurationError("Group.Offsets.Synchronization.DwellTime must be <= 10m") 128 | case c.Group.Heartbeat.Interval <= 0: 129 | return sarama.ConfigurationError("Group.Heartbeat.Interval must be > 0") 130 | case c.Group.Session.Timeout <= 0: 131 | return sarama.ConfigurationError("Group.Session.Timeout must be > 0") 132 | case !c.Metadata.Full && c.Group.Topics.Whitelist != nil: 133 | return sarama.ConfigurationError("Metadata.Full must be enabled when Group.Topics.Whitelist is used") 134 | case !c.Metadata.Full && c.Group.Topics.Blacklist != nil: 135 | return sarama.ConfigurationError("Metadata.Full must be enabled when Group.Topics.Blacklist is used") 136 | } 137 | 138 | // ensure offset is correct 139 | switch c.Consumer.Offsets.Initial { 140 | case sarama.OffsetOldest, sarama.OffsetNewest: 141 | default: 142 | return sarama.ConfigurationError("Consumer.Offsets.Initial must be either OffsetOldest or OffsetNewest") 143 | } 144 | 145 | return nil 146 | } 147 | -------------------------------------------------------------------------------- /cluster_test.go: -------------------------------------------------------------------------------- 1 | package cluster 2 | 3 | import ( 4 | "fmt" 5 | "os" 6 | "os/exec" 7 | "path/filepath" 8 | "testing" 9 | 10 | "github.com/Shopify/sarama" 11 | . "github.com/onsi/ginkgo" 12 | . "github.com/onsi/gomega" 13 | ) 14 | 15 | const ( 16 | testGroup = "sarama-cluster-group" 17 | testKafkaData = "/tmp/sarama-cluster-test" 18 | ) 19 | 20 | var ( 21 | testKafkaRoot = "kafka_2.12-1.1.0" 22 | testKafkaAddrs = []string{"127.0.0.1:29092"} 23 | testTopics = []string{"topic-a", "topic-b"} 24 | 25 | testClient sarama.Client 26 | testKafkaCmd, testZkCmd *exec.Cmd 27 | ) 28 | 29 | func init() { 30 | if dir := os.Getenv("KAFKA_DIR"); dir != "" { 31 | testKafkaRoot = dir 32 | } 33 | } 34 | 35 | var _ = Describe("offsetInfo", func() { 36 | 37 | It("should calculate next offset", func() { 38 | Expect(offsetInfo{-2, ""}.NextOffset(sarama.OffsetOldest)).To(Equal(sarama.OffsetOldest)) 39 | Expect(offsetInfo{-2, ""}.NextOffset(sarama.OffsetNewest)).To(Equal(sarama.OffsetNewest)) 40 | Expect(offsetInfo{-1, ""}.NextOffset(sarama.OffsetOldest)).To(Equal(sarama.OffsetOldest)) 41 | Expect(offsetInfo{-1, ""}.NextOffset(sarama.OffsetNewest)).To(Equal(sarama.OffsetNewest)) 42 | Expect(offsetInfo{0, ""}.NextOffset(sarama.OffsetOldest)).To(Equal(int64(0))) 43 | Expect(offsetInfo{100, ""}.NextOffset(sarama.OffsetOldest)).To(Equal(int64(100))) 44 | }) 45 | 46 | }) 47 | 48 | var _ = Describe("int32Slice", func() { 49 | 50 | It("should diff", func() { 51 | Expect(((int32Slice)(nil)).Diff(int32Slice{1, 3, 5})).To(BeNil()) 52 | Expect(int32Slice{1, 3, 5}.Diff((int32Slice)(nil))).To(Equal([]int32{1, 3, 5})) 53 | Expect(int32Slice{1, 3, 5}.Diff(int32Slice{1, 3, 5})).To(BeNil()) 54 | Expect(int32Slice{1, 3, 5}.Diff(int32Slice{1, 2, 3, 4, 5})).To(BeNil()) 55 | Expect(int32Slice{1, 3, 5}.Diff(int32Slice{2, 3, 4})).To(Equal([]int32{1, 5})) 56 | Expect(int32Slice{1, 3, 5}.Diff(int32Slice{1, 4})).To(Equal([]int32{3, 5})) 57 | Expect(int32Slice{1, 3, 5}.Diff(int32Slice{2, 5})).To(Equal([]int32{1, 3})) 58 | }) 59 | 60 | }) 61 | 62 | // -------------------------------------------------------------------- 63 | 64 | var _ = BeforeSuite(func() { 65 | testZkCmd = testCmd( 66 | testDataDir(testKafkaRoot, "bin", "kafka-run-class.sh"), 67 | "org.apache.zookeeper.server.quorum.QuorumPeerMain", 68 | testDataDir("zookeeper.properties"), 69 | ) 70 | 71 | testKafkaCmd = testCmd( 72 | testDataDir(testKafkaRoot, "bin", "kafka-run-class.sh"), 73 | "-name", "kafkaServer", "kafka.Kafka", 74 | testDataDir("server.properties"), 75 | ) 76 | 77 | // Remove old test data before starting 78 | Expect(os.RemoveAll(testKafkaData)).NotTo(HaveOccurred()) 79 | 80 | Expect(os.MkdirAll(testKafkaData, 0777)).To(Succeed()) 81 | Expect(testZkCmd.Start()).To(Succeed()) 82 | Expect(testKafkaCmd.Start()).To(Succeed()) 83 | 84 | // Wait for client 85 | Eventually(func() error { 86 | var err error 87 | 88 | // sync-producer requires Return.Successes set to true 89 | testConf := sarama.NewConfig() 90 | testConf.Producer.Return.Successes = true 91 | testClient, err = sarama.NewClient(testKafkaAddrs, testConf) 92 | return err 93 | }, "30s", "1s").Should(Succeed()) 94 | 95 | // Ensure we can retrieve partition info 96 | Eventually(func() error { 97 | _, err := testClient.Partitions(testTopics[0]) 98 | return err 99 | }, "30s", "1s").Should(Succeed()) 100 | 101 | // Seed a few messages 102 | Expect(testSeed(1000, testTopics)).To(Succeed()) 103 | }) 104 | 105 | var _ = AfterSuite(func() { 106 | if testClient != nil { 107 | _ = testClient.Close() 108 | } 109 | 110 | _ = testKafkaCmd.Process.Kill() 111 | _ = testZkCmd.Process.Kill() 112 | _ = testKafkaCmd.Wait() 113 | _ = testZkCmd.Wait() 114 | _ = os.RemoveAll(testKafkaData) 115 | }) 116 | 117 | // -------------------------------------------------------------------- 118 | 119 | func TestSuite(t *testing.T) { 120 | RegisterFailHandler(Fail) 121 | RunSpecs(t, "sarama/cluster") 122 | } 123 | 124 | func testDataDir(tokens ...string) string { 125 | tokens = append([]string{"testdata"}, tokens...) 126 | return filepath.Join(tokens...) 127 | } 128 | 129 | func testSeed(n int, testTopics []string) error { 130 | producer, err := sarama.NewSyncProducerFromClient(testClient) 131 | if err != nil { 132 | return err 133 | } 134 | defer producer.Close() 135 | 136 | for i := 0; i < n; i++ { 137 | kv := sarama.StringEncoder(fmt.Sprintf("PLAINDATA-%08d", i)) 138 | for _, t := range testTopics { 139 | msg := &sarama.ProducerMessage{Topic: t, Key: kv, Value: kv} 140 | if _, _, err := producer.SendMessage(msg); err != nil { 141 | return err 142 | } 143 | } 144 | } 145 | return nil 146 | } 147 | 148 | func testCmd(name string, arg ...string) *exec.Cmd { 149 | cmd := exec.Command(name, arg...) 150 | if testing.Verbose() || os.Getenv("CI") != "" { 151 | cmd.Stderr = os.Stderr 152 | cmd.Stdout = os.Stdout 153 | } 154 | cmd.Env = []string{"KAFKA_HEAP_OPTS=-Xmx1G -Xms1G"} 155 | return cmd 156 | } 157 | 158 | type testConsumerMessage struct { 159 | sarama.ConsumerMessage 160 | ConsumerID string 161 | } 162 | 163 | // -------------------------------------------------------------------- 164 | 165 | var _ sarama.Consumer = &mockConsumer{} 166 | var _ sarama.PartitionConsumer = &mockPartitionConsumer{} 167 | 168 | type mockClient struct { 169 | sarama.Client 170 | 171 | topics map[string][]int32 172 | } 173 | type mockConsumer struct{ sarama.Consumer } 174 | type mockPartitionConsumer struct { 175 | sarama.PartitionConsumer 176 | 177 | Topic string 178 | Partition int32 179 | Offset int64 180 | } 181 | 182 | func (m *mockClient) Partitions(t string) ([]int32, error) { 183 | pts, ok := m.topics[t] 184 | if !ok { 185 | return nil, sarama.ErrInvalidTopic 186 | } 187 | return pts, nil 188 | } 189 | 190 | func (*mockConsumer) ConsumePartition(topic string, partition int32, offset int64) (sarama.PartitionConsumer, error) { 191 | if offset > -1 && offset < 1000 { 192 | return nil, sarama.ErrOffsetOutOfRange 193 | } 194 | return &mockPartitionConsumer{ 195 | Topic: topic, 196 | Partition: partition, 197 | Offset: offset, 198 | }, nil 199 | } 200 | 201 | func (*mockPartitionConsumer) Close() error { return nil } 202 | -------------------------------------------------------------------------------- /partitions.go: -------------------------------------------------------------------------------- 1 | package cluster 2 | 3 | import ( 4 | "sort" 5 | "sync" 6 | "time" 7 | 8 | "github.com/Shopify/sarama" 9 | ) 10 | 11 | // PartitionConsumer allows code to consume individual partitions from the cluster. 12 | // 13 | // See docs for Consumer.Partitions() for more on how to implement this. 14 | type PartitionConsumer interface { 15 | sarama.PartitionConsumer 16 | 17 | // Topic returns the consumed topic name 18 | Topic() string 19 | 20 | // Partition returns the consumed partition 21 | Partition() int32 22 | 23 | // InitialOffset returns the offset used for creating the PartitionConsumer instance. 24 | // The returned offset can be a literal offset, or OffsetNewest, or OffsetOldest 25 | InitialOffset() int64 26 | 27 | // MarkOffset marks the offset of a message as preocessed. 28 | MarkOffset(offset int64, metadata string) 29 | 30 | // ResetOffset resets the offset to a previously processed message. 31 | ResetOffset(offset int64, metadata string) 32 | } 33 | 34 | type partitionConsumer struct { 35 | sarama.PartitionConsumer 36 | 37 | state partitionState 38 | mu sync.Mutex 39 | 40 | topic string 41 | partition int32 42 | initialOffset int64 43 | 44 | closeOnce sync.Once 45 | closeErr error 46 | 47 | dying, dead chan none 48 | } 49 | 50 | func newPartitionConsumer(manager sarama.Consumer, topic string, partition int32, info offsetInfo, defaultOffset int64) (*partitionConsumer, error) { 51 | offset := info.NextOffset(defaultOffset) 52 | pcm, err := manager.ConsumePartition(topic, partition, offset) 53 | 54 | // Resume from default offset, if requested offset is out-of-range 55 | if err == sarama.ErrOffsetOutOfRange { 56 | info.Offset = -1 57 | offset = defaultOffset 58 | pcm, err = manager.ConsumePartition(topic, partition, offset) 59 | } 60 | if err != nil { 61 | return nil, err 62 | } 63 | 64 | return &partitionConsumer{ 65 | PartitionConsumer: pcm, 66 | state: partitionState{Info: info}, 67 | 68 | topic: topic, 69 | partition: partition, 70 | initialOffset: offset, 71 | 72 | dying: make(chan none), 73 | dead: make(chan none), 74 | }, nil 75 | } 76 | 77 | // Topic implements PartitionConsumer 78 | func (c *partitionConsumer) Topic() string { return c.topic } 79 | 80 | // Partition implements PartitionConsumer 81 | func (c *partitionConsumer) Partition() int32 { return c.partition } 82 | 83 | // InitialOffset implements PartitionConsumer 84 | func (c *partitionConsumer) InitialOffset() int64 { return c.initialOffset } 85 | 86 | // AsyncClose implements PartitionConsumer 87 | func (c *partitionConsumer) AsyncClose() { 88 | c.closeOnce.Do(func() { 89 | c.closeErr = c.PartitionConsumer.Close() 90 | close(c.dying) 91 | }) 92 | } 93 | 94 | // Close implements PartitionConsumer 95 | func (c *partitionConsumer) Close() error { 96 | c.AsyncClose() 97 | <-c.dead 98 | return c.closeErr 99 | } 100 | 101 | func (c *partitionConsumer) waitFor(stopper <-chan none) { 102 | select { 103 | case <-stopper: 104 | case <-c.dying: 105 | } 106 | close(c.dead) 107 | } 108 | 109 | func (c *partitionConsumer) multiplex(stopper <-chan none, messages chan<- *sarama.ConsumerMessage, errors chan<- error) { 110 | defer close(c.dead) 111 | 112 | for { 113 | select { 114 | case msg, ok := <-c.Messages(): 115 | if !ok { 116 | return 117 | } 118 | select { 119 | case messages <- msg: 120 | case <-stopper: 121 | return 122 | case <-c.dying: 123 | return 124 | } 125 | case err, ok := <-c.Errors(): 126 | if !ok { 127 | return 128 | } 129 | select { 130 | case errors <- err: 131 | case <-stopper: 132 | return 133 | case <-c.dying: 134 | return 135 | } 136 | case <-stopper: 137 | return 138 | case <-c.dying: 139 | return 140 | } 141 | } 142 | } 143 | 144 | func (c *partitionConsumer) getState() partitionState { 145 | c.mu.Lock() 146 | state := c.state 147 | c.mu.Unlock() 148 | 149 | return state 150 | } 151 | 152 | func (c *partitionConsumer) markCommitted(offset int64) { 153 | c.mu.Lock() 154 | if offset == c.state.Info.Offset { 155 | c.state.Dirty = false 156 | } 157 | c.mu.Unlock() 158 | } 159 | 160 | // MarkOffset implements PartitionConsumer 161 | func (c *partitionConsumer) MarkOffset(offset int64, metadata string) { 162 | c.mu.Lock() 163 | if next := offset + 1; next > c.state.Info.Offset { 164 | c.state.Info.Offset = next 165 | c.state.Info.Metadata = metadata 166 | c.state.Dirty = true 167 | } 168 | c.mu.Unlock() 169 | } 170 | 171 | // ResetOffset implements PartitionConsumer 172 | func (c *partitionConsumer) ResetOffset(offset int64, metadata string) { 173 | c.mu.Lock() 174 | if next := offset + 1; next <= c.state.Info.Offset { 175 | c.state.Info.Offset = next 176 | c.state.Info.Metadata = metadata 177 | c.state.Dirty = true 178 | } 179 | c.mu.Unlock() 180 | } 181 | 182 | // -------------------------------------------------------------------- 183 | 184 | type partitionState struct { 185 | Info offsetInfo 186 | Dirty bool 187 | LastCommit time.Time 188 | } 189 | 190 | // -------------------------------------------------------------------- 191 | 192 | type partitionMap struct { 193 | data map[topicPartition]*partitionConsumer 194 | mu sync.RWMutex 195 | } 196 | 197 | func newPartitionMap() *partitionMap { 198 | return &partitionMap{ 199 | data: make(map[topicPartition]*partitionConsumer), 200 | } 201 | } 202 | 203 | func (m *partitionMap) IsSubscribedTo(topic string) bool { 204 | m.mu.RLock() 205 | defer m.mu.RUnlock() 206 | 207 | for tp := range m.data { 208 | if tp.Topic == topic { 209 | return true 210 | } 211 | } 212 | return false 213 | } 214 | 215 | func (m *partitionMap) Fetch(topic string, partition int32) *partitionConsumer { 216 | m.mu.RLock() 217 | pc, _ := m.data[topicPartition{topic, partition}] 218 | m.mu.RUnlock() 219 | return pc 220 | } 221 | 222 | func (m *partitionMap) Store(topic string, partition int32, pc *partitionConsumer) { 223 | m.mu.Lock() 224 | m.data[topicPartition{topic, partition}] = pc 225 | m.mu.Unlock() 226 | } 227 | 228 | func (m *partitionMap) Snapshot() map[topicPartition]partitionState { 229 | m.mu.RLock() 230 | defer m.mu.RUnlock() 231 | 232 | snap := make(map[topicPartition]partitionState, len(m.data)) 233 | for tp, pc := range m.data { 234 | snap[tp] = pc.getState() 235 | } 236 | return snap 237 | } 238 | 239 | func (m *partitionMap) Stop() { 240 | m.mu.RLock() 241 | defer m.mu.RUnlock() 242 | 243 | var wg sync.WaitGroup 244 | for tp := range m.data { 245 | wg.Add(1) 246 | go func(p *partitionConsumer) { 247 | _ = p.Close() 248 | wg.Done() 249 | }(m.data[tp]) 250 | } 251 | wg.Wait() 252 | } 253 | 254 | func (m *partitionMap) Clear() { 255 | m.mu.Lock() 256 | for tp := range m.data { 257 | delete(m.data, tp) 258 | } 259 | m.mu.Unlock() 260 | } 261 | 262 | func (m *partitionMap) Info() map[string][]int32 { 263 | info := make(map[string][]int32) 264 | m.mu.RLock() 265 | for tp := range m.data { 266 | info[tp.Topic] = append(info[tp.Topic], tp.Partition) 267 | } 268 | m.mu.RUnlock() 269 | 270 | for topic := range info { 271 | sort.Sort(int32Slice(info[topic])) 272 | } 273 | return info 274 | } 275 | -------------------------------------------------------------------------------- /consumer_test.go: -------------------------------------------------------------------------------- 1 | package cluster 2 | 3 | import ( 4 | "fmt" 5 | "regexp" 6 | "sync/atomic" 7 | "time" 8 | 9 | "github.com/Shopify/sarama" 10 | . "github.com/onsi/ginkgo" 11 | . "github.com/onsi/gomega" 12 | ) 13 | 14 | var _ = Describe("Consumer", func() { 15 | 16 | var newConsumerOf = func(group string, topics ...string) (*Consumer, error) { 17 | config := NewConfig() 18 | config.Consumer.Return.Errors = true 19 | config.Consumer.Offsets.Initial = sarama.OffsetOldest 20 | return NewConsumer(testKafkaAddrs, group, topics, config) 21 | } 22 | 23 | var subscriptionsOf = func(c *Consumer) GomegaAsyncAssertion { 24 | return Eventually(func() map[string][]int32 { 25 | return c.Subscriptions() 26 | }, "10s", "100ms") 27 | } 28 | 29 | It("should init and share", func() { 30 | // start CS1 31 | cs1, err := newConsumerOf(testGroup, testTopics...) 32 | Expect(err).NotTo(HaveOccurred()) 33 | 34 | // CS1 should consume all 8 partitions 35 | subscriptionsOf(cs1).Should(Equal(map[string][]int32{ 36 | "topic-a": {0, 1, 2, 3}, 37 | "topic-b": {0, 1, 2, 3}, 38 | })) 39 | 40 | // start CS2 41 | cs2, err := newConsumerOf(testGroup, testTopics...) 42 | Expect(err).NotTo(HaveOccurred()) 43 | defer cs2.Close() 44 | 45 | // CS1 and CS2 should consume 4 partitions each 46 | subscriptionsOf(cs1).Should(HaveLen(2)) 47 | subscriptionsOf(cs1).Should(HaveKeyWithValue("topic-a", HaveLen(2))) 48 | subscriptionsOf(cs1).Should(HaveKeyWithValue("topic-b", HaveLen(2))) 49 | 50 | subscriptionsOf(cs2).Should(HaveLen(2)) 51 | subscriptionsOf(cs2).Should(HaveKeyWithValue("topic-a", HaveLen(2))) 52 | subscriptionsOf(cs2).Should(HaveKeyWithValue("topic-b", HaveLen(2))) 53 | 54 | // shutdown CS1, now CS2 should consume all 8 partitions 55 | Expect(cs1.Close()).NotTo(HaveOccurred()) 56 | subscriptionsOf(cs2).Should(Equal(map[string][]int32{ 57 | "topic-a": {0, 1, 2, 3}, 58 | "topic-b": {0, 1, 2, 3}, 59 | })) 60 | }) 61 | 62 | It("should allow more consumers than partitions", func() { 63 | cs1, err := newConsumerOf(testGroup, "topic-a") 64 | Expect(err).NotTo(HaveOccurred()) 65 | defer cs1.Close() 66 | cs2, err := newConsumerOf(testGroup, "topic-a") 67 | Expect(err).NotTo(HaveOccurred()) 68 | defer cs2.Close() 69 | cs3, err := newConsumerOf(testGroup, "topic-a") 70 | Expect(err).NotTo(HaveOccurred()) 71 | defer cs3.Close() 72 | cs4, err := newConsumerOf(testGroup, "topic-a") 73 | Expect(err).NotTo(HaveOccurred()) 74 | 75 | // start 4 consumers, one for each partition 76 | subscriptionsOf(cs1).Should(HaveKeyWithValue("topic-a", HaveLen(1))) 77 | subscriptionsOf(cs2).Should(HaveKeyWithValue("topic-a", HaveLen(1))) 78 | subscriptionsOf(cs3).Should(HaveKeyWithValue("topic-a", HaveLen(1))) 79 | subscriptionsOf(cs4).Should(HaveKeyWithValue("topic-a", HaveLen(1))) 80 | 81 | // add a 5th consumer 82 | cs5, err := newConsumerOf(testGroup, "topic-a") 83 | Expect(err).NotTo(HaveOccurred()) 84 | defer cs5.Close() 85 | 86 | // make sure no errors occurred 87 | Expect(cs1.Errors()).ShouldNot(Receive()) 88 | Expect(cs2.Errors()).ShouldNot(Receive()) 89 | Expect(cs3.Errors()).ShouldNot(Receive()) 90 | Expect(cs4.Errors()).ShouldNot(Receive()) 91 | Expect(cs5.Errors()).ShouldNot(Receive()) 92 | 93 | // close 4th, make sure the 5th takes over 94 | Expect(cs4.Close()).To(Succeed()) 95 | subscriptionsOf(cs1).Should(HaveKeyWithValue("topic-a", HaveLen(1))) 96 | subscriptionsOf(cs2).Should(HaveKeyWithValue("topic-a", HaveLen(1))) 97 | subscriptionsOf(cs3).Should(HaveKeyWithValue("topic-a", HaveLen(1))) 98 | subscriptionsOf(cs4).Should(BeEmpty()) 99 | subscriptionsOf(cs5).Should(HaveKeyWithValue("topic-a", HaveLen(1))) 100 | 101 | // there should still be no errors 102 | Expect(cs1.Errors()).ShouldNot(Receive()) 103 | Expect(cs2.Errors()).ShouldNot(Receive()) 104 | Expect(cs3.Errors()).ShouldNot(Receive()) 105 | Expect(cs4.Errors()).ShouldNot(Receive()) 106 | Expect(cs5.Errors()).ShouldNot(Receive()) 107 | }) 108 | 109 | It("should be allowed to subscribe to partitions via white/black-lists", func() { 110 | config := NewConfig() 111 | config.Consumer.Return.Errors = true 112 | config.Group.Topics.Whitelist = regexp.MustCompile(`topic-\w+`) 113 | config.Group.Topics.Blacklist = regexp.MustCompile(`[bcd]$`) 114 | 115 | cs, err := NewConsumer(testKafkaAddrs, testGroup, nil, config) 116 | Expect(err).NotTo(HaveOccurred()) 117 | defer cs.Close() 118 | 119 | subscriptionsOf(cs).Should(Equal(map[string][]int32{ 120 | "topic-a": {0, 1, 2, 3}, 121 | })) 122 | }) 123 | 124 | It("should receive rebalance notifications", func() { 125 | config := NewConfig() 126 | config.Consumer.Return.Errors = true 127 | config.Group.Return.Notifications = true 128 | 129 | cs, err := NewConsumer(testKafkaAddrs, testGroup, testTopics, config) 130 | Expect(err).NotTo(HaveOccurred()) 131 | defer cs.Close() 132 | 133 | select { 134 | case n := <-cs.Notifications(): 135 | Expect(n).To(Equal(&Notification{ 136 | Type: RebalanceStart, 137 | Current: map[string][]int32{}, 138 | })) 139 | case err := <-cs.Errors(): 140 | Expect(err).NotTo(HaveOccurred()) 141 | case <-cs.Messages(): 142 | Fail("expected notification to arrive before message") 143 | } 144 | 145 | select { 146 | case n := <-cs.Notifications(): 147 | Expect(n).To(Equal(&Notification{ 148 | Type: RebalanceOK, 149 | Claimed: map[string][]int32{ 150 | "topic-a": {0, 1, 2, 3}, 151 | "topic-b": {0, 1, 2, 3}, 152 | }, 153 | Released: map[string][]int32{}, 154 | Current: map[string][]int32{ 155 | "topic-a": {0, 1, 2, 3}, 156 | "topic-b": {0, 1, 2, 3}, 157 | }, 158 | })) 159 | case err := <-cs.Errors(): 160 | Expect(err).NotTo(HaveOccurred()) 161 | case <-cs.Messages(): 162 | Fail("expected notification to arrive before message") 163 | } 164 | }) 165 | 166 | It("should support manual mark/commit", func() { 167 | cs, err := newConsumerOf(testGroup, "topic-a") 168 | Expect(err).NotTo(HaveOccurred()) 169 | defer cs.Close() 170 | 171 | subscriptionsOf(cs).Should(Equal(map[string][]int32{ 172 | "topic-a": {0, 1, 2, 3}}, 173 | )) 174 | 175 | cs.MarkPartitionOffset("topic-a", 1, 3, "") 176 | cs.MarkPartitionOffset("topic-a", 2, 4, "") 177 | Expect(cs.CommitOffsets()).NotTo(HaveOccurred()) 178 | 179 | offsets, err := cs.fetchOffsets(cs.Subscriptions()) 180 | Expect(err).NotTo(HaveOccurred()) 181 | Expect(offsets).To(Equal(map[string]map[int32]offsetInfo{ 182 | "topic-a": {0: {Offset: -1}, 1: {Offset: 4}, 2: {Offset: 5}, 3: {Offset: -1}}, 183 | })) 184 | }) 185 | 186 | It("should support manual mark/commit, reset/commit", func() { 187 | cs, err := newConsumerOf(testGroup, "topic-a") 188 | Expect(err).NotTo(HaveOccurred()) 189 | defer cs.Close() 190 | 191 | subscriptionsOf(cs).Should(Equal(map[string][]int32{ 192 | "topic-a": {0, 1, 2, 3}}, 193 | )) 194 | 195 | cs.MarkPartitionOffset("topic-a", 1, 3, "") 196 | cs.MarkPartitionOffset("topic-a", 2, 4, "") 197 | cs.MarkPartitionOffset("topic-b", 1, 2, "") // should not throw NPE 198 | Expect(cs.CommitOffsets()).NotTo(HaveOccurred()) 199 | 200 | cs.ResetPartitionOffset("topic-a", 1, 2, "") 201 | cs.ResetPartitionOffset("topic-a", 2, 3, "") 202 | cs.ResetPartitionOffset("topic-b", 1, 2, "") // should not throw NPE 203 | Expect(cs.CommitOffsets()).NotTo(HaveOccurred()) 204 | 205 | offsets, err := cs.fetchOffsets(cs.Subscriptions()) 206 | Expect(err).NotTo(HaveOccurred()) 207 | Expect(offsets).To(Equal(map[string]map[int32]offsetInfo{ 208 | "topic-a": {0: {Offset: -1}, 1: {Offset: 3}, 2: {Offset: 4}, 3: {Offset: -1}}, 209 | })) 210 | }) 211 | 212 | It("should not commit unprocessed offsets", func() { 213 | const groupID = "panicking" 214 | 215 | cs, err := newConsumerOf(groupID, "topic-a") 216 | Expect(err).NotTo(HaveOccurred()) 217 | 218 | subscriptionsOf(cs).Should(Equal(map[string][]int32{ 219 | "topic-a": {0, 1, 2, 3}, 220 | })) 221 | 222 | n := 0 223 | Expect(func() { 224 | for range cs.Messages() { 225 | n++ 226 | panic("stop here!") 227 | } 228 | }).To(Panic()) 229 | Expect(cs.Close()).To(Succeed()) 230 | Expect(n).To(Equal(1)) 231 | 232 | bk, err := testClient.Coordinator(groupID) 233 | Expect(err).NotTo(HaveOccurred()) 234 | 235 | req := &sarama.OffsetFetchRequest{ 236 | Version: 1, 237 | ConsumerGroup: groupID, 238 | } 239 | req.AddPartition("topic-a", 0) 240 | req.AddPartition("topic-a", 1) 241 | req.AddPartition("topic-a", 2) 242 | req.AddPartition("topic-a", 3) 243 | Expect(bk.FetchOffset(req)).To(Equal(&sarama.OffsetFetchResponse{ 244 | Blocks: map[string]map[int32]*sarama.OffsetFetchResponseBlock{ 245 | "topic-a": {0: {Offset: -1}, 1: {Offset: -1}, 2: {Offset: -1}, 3: {Offset: -1}}, 246 | }, 247 | })) 248 | }) 249 | 250 | It("should consume partitions", func() { 251 | count := int32(0) 252 | consume := func(consumerID string) { 253 | defer GinkgoRecover() 254 | 255 | config := NewConfig() 256 | config.Group.Mode = ConsumerModePartitions 257 | config.Consumer.Offsets.Initial = sarama.OffsetOldest 258 | 259 | cs, err := NewConsumer(testKafkaAddrs, "partitions", testTopics, config) 260 | Expect(err).NotTo(HaveOccurred()) 261 | defer cs.Close() 262 | 263 | for pc := range cs.Partitions() { 264 | go func(pc PartitionConsumer) { 265 | defer pc.Close() 266 | 267 | for msg := range pc.Messages() { 268 | atomic.AddInt32(&count, 1) 269 | cs.MarkOffset(msg, "") 270 | } 271 | }(pc) 272 | } 273 | } 274 | 275 | go consume("A") 276 | go consume("B") 277 | go consume("C") 278 | 279 | Eventually(func() int32 { 280 | return atomic.LoadInt32(&count) 281 | }, "30s", "100ms").Should(BeNumerically(">=", 2000)) 282 | }) 283 | 284 | It("should not lock on rebalance while dying", func() { 285 | count := int32(0) 286 | consume := func(consumerID string, closeChan, doneCh chan none) { 287 | defer GinkgoRecover() 288 | 289 | config := NewConfig() 290 | config.Group.Mode = ConsumerModePartitions 291 | config.Consumer.Offsets.Initial = sarama.OffsetOldest 292 | config.Group.Offsets.Synchronization.DwellTime = time.Millisecond * 10000 293 | 294 | cs, err := NewConsumer(testKafkaAddrs, "partitions-no-lock", testTopics, config) 295 | Expect(err).NotTo(HaveOccurred()) 296 | defer func() { 297 | cs.Close() 298 | close(doneCh) 299 | }() 300 | 301 | for { 302 | select { 303 | case pc := <-cs.Partitions(): 304 | go func(pc PartitionConsumer) { 305 | defer func() { 306 | pc.Close() 307 | }() 308 | for { 309 | select { 310 | case msg, ok := <-pc.Messages(): 311 | if !ok { 312 | return 313 | } 314 | atomic.AddInt32(&count, 1) 315 | cs.MarkOffset(msg, "") 316 | cs.CommitOffsets() 317 | case <-closeChan: 318 | return 319 | } 320 | } 321 | }(pc) 322 | case <-closeChan: 323 | return 324 | } 325 | } 326 | } 327 | 328 | chanA := make(chan none) 329 | chanADone := make(chan none) 330 | go consume("A", chanA, chanADone) 331 | chanB := make(chan none) 332 | chanBDone := make(chan none) 333 | go consume("B", chanB, chanBDone) 334 | 335 | Eventually(func() int32 { 336 | return atomic.LoadInt32(&count) 337 | }, "30s", "100ms").Should(BeNumerically(">=", 100)) 338 | 339 | close(chanA) 340 | time.Sleep(time.Millisecond * 5000) 341 | close(chanB) 342 | 343 | Eventually(func() chan none { 344 | return chanADone 345 | }, "30s", "100ms").Should(BeClosed()) 346 | 347 | Eventually(func() chan none { 348 | return chanBDone 349 | }, "30s", "100ms").Should(BeClosed()) 350 | }) 351 | 352 | It("should consume/commit/resume", func() { 353 | acc := make(chan *testConsumerMessage, 20000) 354 | consume := func(consumerID string, max int32) { 355 | defer GinkgoRecover() 356 | 357 | cs, err := NewConsumer(testKafkaAddrs, "fuzzing", testTopics, nil) 358 | Expect(err).NotTo(HaveOccurred()) 359 | defer cs.Close() 360 | cs.consumerID = consumerID 361 | 362 | for msg := range cs.Messages() { 363 | acc <- &testConsumerMessage{*msg, consumerID} 364 | cs.MarkOffset(msg, "") 365 | 366 | if atomic.AddInt32(&max, -1) <= 0 { 367 | return 368 | } 369 | } 370 | } 371 | 372 | go consume("A", 1500) 373 | go consume("B", 2000) 374 | go consume("C", 1500) 375 | go consume("D", 200) 376 | go consume("E", 100) 377 | time.Sleep(10 * time.Second) // wait for consumers to subscribe to topics 378 | Expect(testSeed(5000, testTopics)).NotTo(HaveOccurred()) 379 | Eventually(func() int { return len(acc) }, "30s", "100ms").Should(BeNumerically(">=", 5000)) 380 | 381 | go consume("F", 300) 382 | go consume("G", 400) 383 | go consume("H", 1000) 384 | go consume("I", 2000) 385 | Expect(testSeed(5000, testTopics)).NotTo(HaveOccurred()) 386 | Eventually(func() int { return len(acc) }, "30s", "100ms").Should(BeNumerically(">=", 8000)) 387 | 388 | go consume("J", 1000) 389 | Expect(testSeed(5000, testTopics)).NotTo(HaveOccurred()) 390 | Eventually(func() int { return len(acc) }, "30s", "100ms").Should(BeNumerically(">=", 9000)) 391 | 392 | go consume("K", 1000) 393 | go consume("L", 3000) 394 | Expect(testSeed(5000, testTopics)).NotTo(HaveOccurred()) 395 | Eventually(func() int { return len(acc) }, "30s", "100ms").Should(BeNumerically(">=", 12000)) 396 | 397 | go consume("M", 1000) 398 | Expect(testSeed(5000, testTopics)).NotTo(HaveOccurred()) 399 | Eventually(func() int { return len(acc) }, "30s", "100ms").Should(BeNumerically(">=", 15000)) 400 | 401 | close(acc) 402 | 403 | uniques := make(map[string][]string) 404 | for msg := range acc { 405 | key := fmt.Sprintf("%s/%d/%d", msg.Topic, msg.Partition, msg.Offset) 406 | uniques[key] = append(uniques[key], msg.ConsumerID) 407 | } 408 | Expect(uniques).To(HaveLen(15000)) 409 | }) 410 | 411 | It("should allow close to be called multiple times", func() { 412 | cs, err := newConsumerOf(testGroup, testTopics...) 413 | Expect(err).NotTo(HaveOccurred()) 414 | Expect(cs.Close()).NotTo(HaveOccurred()) 415 | Expect(cs.Close()).NotTo(HaveOccurred()) 416 | }) 417 | 418 | }) 419 | -------------------------------------------------------------------------------- /consumer.go: -------------------------------------------------------------------------------- 1 | package cluster 2 | 3 | import ( 4 | "sort" 5 | "sync" 6 | "sync/atomic" 7 | "time" 8 | 9 | "github.com/Shopify/sarama" 10 | ) 11 | 12 | // Consumer is a cluster group consumer 13 | type Consumer struct { 14 | client *Client 15 | ownClient bool 16 | 17 | consumer sarama.Consumer 18 | subs *partitionMap 19 | 20 | consumerID string 21 | groupID string 22 | 23 | memberID string 24 | generationID int32 25 | membershipMu sync.RWMutex 26 | 27 | coreTopics []string 28 | extraTopics []string 29 | 30 | dying, dead chan none 31 | closeOnce sync.Once 32 | 33 | consuming int32 34 | messages chan *sarama.ConsumerMessage 35 | errors chan error 36 | partitions chan PartitionConsumer 37 | notifications chan *Notification 38 | 39 | commitMu sync.Mutex 40 | } 41 | 42 | // NewConsumer initializes a new consumer 43 | func NewConsumer(addrs []string, groupID string, topics []string, config *Config) (*Consumer, error) { 44 | client, err := NewClient(addrs, config) 45 | if err != nil { 46 | return nil, err 47 | } 48 | 49 | consumer, err := NewConsumerFromClient(client, groupID, topics) 50 | if err != nil { 51 | return nil, err 52 | } 53 | consumer.ownClient = true 54 | return consumer, nil 55 | } 56 | 57 | // NewConsumerFromClient initializes a new consumer from an existing client. 58 | // 59 | // Please note that clients cannot be shared between consumers (due to Kafka internals), 60 | // they can only be re-used which requires the user to call Close() on the first consumer 61 | // before using this method again to initialize another one. Attempts to use a client with 62 | // more than one consumer at a time will return errors. 63 | func NewConsumerFromClient(client *Client, groupID string, topics []string) (*Consumer, error) { 64 | if !client.claim() { 65 | return nil, errClientInUse 66 | } 67 | 68 | consumer, err := sarama.NewConsumerFromClient(client.Client) 69 | if err != nil { 70 | client.release() 71 | return nil, err 72 | } 73 | 74 | sort.Strings(topics) 75 | c := &Consumer{ 76 | client: client, 77 | consumer: consumer, 78 | subs: newPartitionMap(), 79 | groupID: groupID, 80 | 81 | coreTopics: topics, 82 | 83 | dying: make(chan none), 84 | dead: make(chan none), 85 | 86 | messages: make(chan *sarama.ConsumerMessage), 87 | errors: make(chan error, client.config.ChannelBufferSize), 88 | partitions: make(chan PartitionConsumer, 1), 89 | notifications: make(chan *Notification), 90 | } 91 | if err := c.client.RefreshCoordinator(groupID); err != nil { 92 | client.release() 93 | return nil, err 94 | } 95 | 96 | go c.mainLoop() 97 | return c, nil 98 | } 99 | 100 | // Messages returns the read channel for the messages that are returned by 101 | // the broker. 102 | // 103 | // This channel will only return if Config.Group.Mode option is set to 104 | // ConsumerModeMultiplex (default). 105 | func (c *Consumer) Messages() <-chan *sarama.ConsumerMessage { return c.messages } 106 | 107 | // Partitions returns the read channels for individual partitions of this broker. 108 | // 109 | // This channel will only return if Config.Group.Mode option is set to 110 | // ConsumerModePartitions. 111 | // 112 | // The Partitions() channel must be listened to for the life of this consumer; 113 | // when a rebalance happens old partitions will be closed (naturally come to 114 | // completion) and new ones will be emitted. The returned channel will only close 115 | // when the consumer is completely shut down. 116 | func (c *Consumer) Partitions() <-chan PartitionConsumer { return c.partitions } 117 | 118 | // Errors returns a read channel of errors that occur during offset management, if 119 | // enabled. By default, errors are logged and not returned over this channel. If 120 | // you want to implement any custom error handling, set your config's 121 | // Consumer.Return.Errors setting to true, and read from this channel. 122 | func (c *Consumer) Errors() <-chan error { return c.errors } 123 | 124 | // Notifications returns a channel of Notifications that occur during consumer 125 | // rebalancing. Notifications will only be emitted over this channel, if your config's 126 | // Group.Return.Notifications setting to true. 127 | func (c *Consumer) Notifications() <-chan *Notification { return c.notifications } 128 | 129 | // HighWaterMarks returns the current high water marks for each topic and partition 130 | // Consistency between partitions is not guaranteed since high water marks are updated separately. 131 | func (c *Consumer) HighWaterMarks() map[string]map[int32]int64 { return c.consumer.HighWaterMarks() } 132 | 133 | // MarkOffset marks the provided message as processed, alongside a metadata string 134 | // that represents the state of the partition consumer at that point in time. The 135 | // metadata string can be used by another consumer to restore that state, so it 136 | // can resume consumption. 137 | // 138 | // Note: calling MarkOffset does not necessarily commit the offset to the backend 139 | // store immediately for efficiency reasons, and it may never be committed if 140 | // your application crashes. This means that you may end up processing the same 141 | // message twice, and your processing should ideally be idempotent. 142 | func (c *Consumer) MarkOffset(msg *sarama.ConsumerMessage, metadata string) { 143 | if sub := c.subs.Fetch(msg.Topic, msg.Partition); sub != nil { 144 | sub.MarkOffset(msg.Offset, metadata) 145 | } 146 | } 147 | 148 | // MarkPartitionOffset marks an offset of the provided topic/partition as processed. 149 | // See MarkOffset for additional explanation. 150 | func (c *Consumer) MarkPartitionOffset(topic string, partition int32, offset int64, metadata string) { 151 | if sub := c.subs.Fetch(topic, partition); sub != nil { 152 | sub.MarkOffset(offset, metadata) 153 | } 154 | } 155 | 156 | // MarkOffsets marks stashed offsets as processed. 157 | // See MarkOffset for additional explanation. 158 | func (c *Consumer) MarkOffsets(s *OffsetStash) { 159 | s.mu.Lock() 160 | defer s.mu.Unlock() 161 | 162 | for tp, info := range s.offsets { 163 | if sub := c.subs.Fetch(tp.Topic, tp.Partition); sub != nil { 164 | sub.MarkOffset(info.Offset, info.Metadata) 165 | } 166 | delete(s.offsets, tp) 167 | } 168 | } 169 | 170 | // ResetOffset marks the provided message as processed, alongside a metadata string 171 | // that represents the state of the partition consumer at that point in time. The 172 | // metadata string can be used by another consumer to restore that state, so it 173 | // can resume consumption. 174 | // 175 | // Difference between ResetOffset and MarkOffset is that it allows to rewind to an earlier offset 176 | func (c *Consumer) ResetOffset(msg *sarama.ConsumerMessage, metadata string) { 177 | if sub := c.subs.Fetch(msg.Topic, msg.Partition); sub != nil { 178 | sub.ResetOffset(msg.Offset, metadata) 179 | } 180 | } 181 | 182 | // ResetPartitionOffset marks an offset of the provided topic/partition as processed. 183 | // See ResetOffset for additional explanation. 184 | func (c *Consumer) ResetPartitionOffset(topic string, partition int32, offset int64, metadata string) { 185 | sub := c.subs.Fetch(topic, partition) 186 | if sub != nil { 187 | sub.ResetOffset(offset, metadata) 188 | } 189 | } 190 | 191 | // ResetOffsets marks stashed offsets as processed. 192 | // See ResetOffset for additional explanation. 193 | func (c *Consumer) ResetOffsets(s *OffsetStash) { 194 | s.mu.Lock() 195 | defer s.mu.Unlock() 196 | 197 | for tp, info := range s.offsets { 198 | if sub := c.subs.Fetch(tp.Topic, tp.Partition); sub != nil { 199 | sub.ResetOffset(info.Offset, info.Metadata) 200 | } 201 | delete(s.offsets, tp) 202 | } 203 | } 204 | 205 | // Subscriptions returns the consumed topics and partitions 206 | func (c *Consumer) Subscriptions() map[string][]int32 { 207 | return c.subs.Info() 208 | } 209 | 210 | // CommitOffsets allows to manually commit previously marked offsets. By default there is no 211 | // need to call this function as the consumer will commit offsets automatically 212 | // using the Config.Consumer.Offsets.CommitInterval setting. 213 | // 214 | // Please be aware that calling this function during an internal rebalance cycle may return 215 | // broker errors (e.g. sarama.ErrUnknownMemberId or sarama.ErrIllegalGeneration). 216 | func (c *Consumer) CommitOffsets() error { 217 | c.commitMu.Lock() 218 | defer c.commitMu.Unlock() 219 | 220 | memberID, generationID := c.membership() 221 | req := &sarama.OffsetCommitRequest{ 222 | Version: 2, 223 | ConsumerGroup: c.groupID, 224 | ConsumerGroupGeneration: generationID, 225 | ConsumerID: memberID, 226 | RetentionTime: -1, 227 | } 228 | 229 | if ns := c.client.config.Consumer.Offsets.Retention; ns != 0 { 230 | req.RetentionTime = int64(ns / time.Millisecond) 231 | } 232 | 233 | snap := c.subs.Snapshot() 234 | dirty := false 235 | for tp, state := range snap { 236 | if state.Dirty { 237 | dirty = true 238 | req.AddBlock(tp.Topic, tp.Partition, state.Info.Offset, 0, state.Info.Metadata) 239 | } 240 | } 241 | if !dirty { 242 | return nil 243 | } 244 | 245 | broker, err := c.client.Coordinator(c.groupID) 246 | if err != nil { 247 | c.closeCoordinator(broker, err) 248 | return err 249 | } 250 | 251 | resp, err := broker.CommitOffset(req) 252 | if err != nil { 253 | c.closeCoordinator(broker, err) 254 | return err 255 | } 256 | 257 | for topic, errs := range resp.Errors { 258 | for partition, kerr := range errs { 259 | if kerr != sarama.ErrNoError { 260 | err = kerr 261 | } else if state, ok := snap[topicPartition{topic, partition}]; ok { 262 | if sub := c.subs.Fetch(topic, partition); sub != nil { 263 | sub.markCommitted(state.Info.Offset) 264 | } 265 | } 266 | } 267 | } 268 | return err 269 | } 270 | 271 | // Close safely closes the consumer and releases all resources 272 | func (c *Consumer) Close() (err error) { 273 | c.closeOnce.Do(func() { 274 | close(c.dying) 275 | <-c.dead 276 | 277 | if e := c.release(); e != nil { 278 | err = e 279 | } 280 | if e := c.consumer.Close(); e != nil { 281 | err = e 282 | } 283 | close(c.messages) 284 | close(c.errors) 285 | 286 | if e := c.leaveGroup(); e != nil { 287 | err = e 288 | } 289 | close(c.partitions) 290 | close(c.notifications) 291 | 292 | // drain 293 | for range c.messages { 294 | } 295 | for range c.errors { 296 | } 297 | for p := range c.partitions { 298 | _ = p.Close() 299 | } 300 | for range c.notifications { 301 | } 302 | 303 | c.client.release() 304 | if c.ownClient { 305 | if e := c.client.Close(); e != nil { 306 | err = e 307 | } 308 | } 309 | }) 310 | return 311 | } 312 | 313 | func (c *Consumer) mainLoop() { 314 | defer close(c.dead) 315 | defer atomic.StoreInt32(&c.consuming, 0) 316 | 317 | for { 318 | atomic.StoreInt32(&c.consuming, 0) 319 | 320 | // Check if close was requested 321 | select { 322 | case <-c.dying: 323 | return 324 | default: 325 | } 326 | 327 | // Start next consume cycle 328 | c.nextTick() 329 | } 330 | } 331 | 332 | func (c *Consumer) nextTick() { 333 | // Remember previous subscriptions 334 | var notification *Notification 335 | if c.client.config.Group.Return.Notifications { 336 | notification = newNotification(c.subs.Info()) 337 | } 338 | 339 | // Refresh coordinator 340 | if err := c.refreshCoordinator(); err != nil { 341 | c.rebalanceError(err, nil) 342 | return 343 | } 344 | 345 | // Release subscriptions 346 | if err := c.release(); err != nil { 347 | c.rebalanceError(err, nil) 348 | return 349 | } 350 | 351 | // Issue rebalance start notification 352 | if c.client.config.Group.Return.Notifications { 353 | c.handleNotification(newNotification(c.subs.Info())) 354 | } 355 | 356 | // Rebalance, fetch new subscriptions 357 | subs, err := c.rebalance() 358 | if err != nil { 359 | c.rebalanceError(err, notification) 360 | return 361 | } 362 | 363 | // Coordinate loops, make sure everything is 364 | // stopped on exit 365 | tomb := newLoopTomb() 366 | defer tomb.Close() 367 | 368 | // Start the heartbeat 369 | tomb.Go(c.hbLoop) 370 | 371 | // Subscribe to topic/partitions 372 | if err := c.subscribe(tomb, subs); err != nil { 373 | c.rebalanceError(err, notification) 374 | return 375 | } 376 | 377 | // Update/issue notification with new claims 378 | if c.client.config.Group.Return.Notifications { 379 | notification = notification.success(subs) 380 | c.handleNotification(notification) 381 | } 382 | 383 | // Start topic watcher loop 384 | tomb.Go(c.twLoop) 385 | 386 | // Start consuming and committing offsets 387 | tomb.Go(c.cmLoop) 388 | atomic.StoreInt32(&c.consuming, 1) 389 | 390 | // Wait for signals 391 | select { 392 | case <-tomb.Dying(): 393 | case <-c.dying: 394 | } 395 | } 396 | 397 | // heartbeat loop, triggered by the mainLoop 398 | func (c *Consumer) hbLoop(stopped <-chan none) { 399 | ticker := time.NewTicker(c.client.config.Group.Heartbeat.Interval) 400 | defer ticker.Stop() 401 | 402 | for { 403 | select { 404 | case <-ticker.C: 405 | switch err := c.heartbeat(); err { 406 | case nil, sarama.ErrNoError: 407 | case sarama.ErrNotCoordinatorForConsumer, sarama.ErrRebalanceInProgress: 408 | return 409 | default: 410 | c.handleError(&Error{Ctx: "heartbeat", error: err}) 411 | return 412 | } 413 | case <-stopped: 414 | return 415 | case <-c.dying: 416 | return 417 | } 418 | } 419 | } 420 | 421 | // topic watcher loop, triggered by the mainLoop 422 | func (c *Consumer) twLoop(stopped <-chan none) { 423 | ticker := time.NewTicker(c.client.config.Metadata.RefreshFrequency / 2) 424 | defer ticker.Stop() 425 | 426 | for { 427 | select { 428 | case <-ticker.C: 429 | topics, err := c.client.Topics() 430 | if err != nil { 431 | c.handleError(&Error{Ctx: "topics", error: err}) 432 | return 433 | } 434 | 435 | for _, topic := range topics { 436 | if !c.isKnownCoreTopic(topic) && 437 | !c.isKnownExtraTopic(topic) && 438 | c.isPotentialExtraTopic(topic) { 439 | return 440 | } 441 | } 442 | case <-stopped: 443 | return 444 | case <-c.dying: 445 | return 446 | } 447 | } 448 | } 449 | 450 | // commit loop, triggered by the mainLoop 451 | func (c *Consumer) cmLoop(stopped <-chan none) { 452 | ticker := time.NewTicker(c.client.config.Consumer.Offsets.CommitInterval) 453 | defer ticker.Stop() 454 | 455 | for { 456 | select { 457 | case <-ticker.C: 458 | if err := c.commitOffsetsWithRetry(c.client.config.Group.Offsets.Retry.Max); err != nil { 459 | c.handleError(&Error{Ctx: "commit", error: err}) 460 | return 461 | } 462 | case <-stopped: 463 | return 464 | case <-c.dying: 465 | return 466 | } 467 | } 468 | } 469 | 470 | func (c *Consumer) rebalanceError(err error, n *Notification) { 471 | if n != nil { 472 | // Get a copy of the notification that represents the notification's error state 473 | n = n.error() 474 | c.handleNotification(n) 475 | } 476 | 477 | switch err { 478 | case sarama.ErrRebalanceInProgress: 479 | default: 480 | c.handleError(&Error{Ctx: "rebalance", error: err}) 481 | } 482 | 483 | select { 484 | case <-c.dying: 485 | case <-time.After(c.client.config.Metadata.Retry.Backoff): 486 | } 487 | } 488 | 489 | func (c *Consumer) handleNotification(n *Notification) { 490 | if c.client.config.Group.Return.Notifications { 491 | select { 492 | case c.notifications <- n: 493 | case <-c.dying: 494 | return 495 | } 496 | } 497 | } 498 | 499 | func (c *Consumer) handleError(e *Error) { 500 | if c.client.config.Consumer.Return.Errors { 501 | select { 502 | case c.errors <- e: 503 | case <-c.dying: 504 | return 505 | } 506 | } else { 507 | sarama.Logger.Printf("%s error: %s\n", e.Ctx, e.Error()) 508 | } 509 | } 510 | 511 | // Releases the consumer and commits offsets, called from rebalance() and Close() 512 | func (c *Consumer) release() (err error) { 513 | // Stop all consumers 514 | c.subs.Stop() 515 | 516 | // Clear subscriptions on exit 517 | defer c.subs.Clear() 518 | 519 | // Wait for messages to be processed 520 | timeout := time.NewTimer(c.client.config.Group.Offsets.Synchronization.DwellTime) 521 | defer timeout.Stop() 522 | 523 | select { 524 | case <-c.dying: 525 | case <-timeout.C: 526 | } 527 | 528 | // Commit offsets, continue on errors 529 | if e := c.commitOffsetsWithRetry(c.client.config.Group.Offsets.Retry.Max); e != nil { 530 | err = e 531 | } 532 | 533 | return 534 | } 535 | 536 | // -------------------------------------------------------------------- 537 | 538 | // Performs a heartbeat, part of the mainLoop() 539 | func (c *Consumer) heartbeat() error { 540 | broker, err := c.client.Coordinator(c.groupID) 541 | if err != nil { 542 | c.closeCoordinator(broker, err) 543 | return err 544 | } 545 | 546 | memberID, generationID := c.membership() 547 | resp, err := broker.Heartbeat(&sarama.HeartbeatRequest{ 548 | GroupId: c.groupID, 549 | MemberId: memberID, 550 | GenerationId: generationID, 551 | }) 552 | if err != nil { 553 | c.closeCoordinator(broker, err) 554 | return err 555 | } 556 | return resp.Err 557 | } 558 | 559 | // Performs a rebalance, part of the mainLoop() 560 | func (c *Consumer) rebalance() (map[string][]int32, error) { 561 | memberID, _ := c.membership() 562 | sarama.Logger.Printf("cluster/consumer %s rebalance\n", memberID) 563 | 564 | allTopics, err := c.client.Topics() 565 | if err != nil { 566 | return nil, err 567 | } 568 | c.extraTopics = c.selectExtraTopics(allTopics) 569 | sort.Strings(c.extraTopics) 570 | 571 | // Re-join consumer group 572 | strategy, err := c.joinGroup() 573 | switch { 574 | case err == sarama.ErrUnknownMemberId: 575 | c.membershipMu.Lock() 576 | c.memberID = "" 577 | c.membershipMu.Unlock() 578 | return nil, err 579 | case err != nil: 580 | return nil, err 581 | } 582 | 583 | // Sync consumer group state, fetch subscriptions 584 | subs, err := c.syncGroup(strategy) 585 | switch { 586 | case err == sarama.ErrRebalanceInProgress: 587 | return nil, err 588 | case err != nil: 589 | _ = c.leaveGroup() 590 | return nil, err 591 | } 592 | return subs, nil 593 | } 594 | 595 | // Performs the subscription, part of the mainLoop() 596 | func (c *Consumer) subscribe(tomb *loopTomb, subs map[string][]int32) error { 597 | // fetch offsets 598 | offsets, err := c.fetchOffsets(subs) 599 | if err != nil { 600 | _ = c.leaveGroup() 601 | return err 602 | } 603 | 604 | // create consumers in parallel 605 | var mu sync.Mutex 606 | var wg sync.WaitGroup 607 | 608 | for topic, partitions := range subs { 609 | for _, partition := range partitions { 610 | wg.Add(1) 611 | 612 | info := offsets[topic][partition] 613 | go func(topic string, partition int32) { 614 | if e := c.createConsumer(tomb, topic, partition, info); e != nil { 615 | mu.Lock() 616 | err = e 617 | mu.Unlock() 618 | } 619 | wg.Done() 620 | }(topic, partition) 621 | } 622 | } 623 | wg.Wait() 624 | 625 | if err != nil { 626 | _ = c.release() 627 | _ = c.leaveGroup() 628 | } 629 | return err 630 | } 631 | 632 | // -------------------------------------------------------------------- 633 | 634 | // Send a request to the broker to join group on rebalance() 635 | func (c *Consumer) joinGroup() (*balancer, error) { 636 | memberID, _ := c.membership() 637 | req := &sarama.JoinGroupRequest{ 638 | GroupId: c.groupID, 639 | MemberId: memberID, 640 | SessionTimeout: int32(c.client.config.Group.Session.Timeout / time.Millisecond), 641 | ProtocolType: "consumer", 642 | } 643 | 644 | meta := &sarama.ConsumerGroupMemberMetadata{ 645 | Version: 1, 646 | Topics: append(c.coreTopics, c.extraTopics...), 647 | UserData: c.client.config.Group.Member.UserData, 648 | } 649 | err := req.AddGroupProtocolMetadata(string(StrategyRange), meta) 650 | if err != nil { 651 | return nil, err 652 | } 653 | err = req.AddGroupProtocolMetadata(string(StrategyRoundRobin), meta) 654 | if err != nil { 655 | return nil, err 656 | } 657 | 658 | broker, err := c.client.Coordinator(c.groupID) 659 | if err != nil { 660 | c.closeCoordinator(broker, err) 661 | return nil, err 662 | } 663 | 664 | resp, err := broker.JoinGroup(req) 665 | if err != nil { 666 | c.closeCoordinator(broker, err) 667 | return nil, err 668 | } else if resp.Err != sarama.ErrNoError { 669 | c.closeCoordinator(broker, resp.Err) 670 | return nil, resp.Err 671 | } 672 | 673 | var strategy *balancer 674 | if resp.LeaderId == resp.MemberId { 675 | members, err := resp.GetMembers() 676 | if err != nil { 677 | return nil, err 678 | } 679 | 680 | strategy, err = newBalancerFromMeta(c.client, Strategy(resp.GroupProtocol), members) 681 | if err != nil { 682 | return nil, err 683 | } 684 | } 685 | 686 | c.membershipMu.Lock() 687 | c.memberID = resp.MemberId 688 | c.generationID = resp.GenerationId 689 | c.membershipMu.Unlock() 690 | 691 | return strategy, nil 692 | } 693 | 694 | // Send a request to the broker to sync the group on rebalance(). 695 | // Returns a list of topics and partitions to consume. 696 | func (c *Consumer) syncGroup(strategy *balancer) (map[string][]int32, error) { 697 | memberID, generationID := c.membership() 698 | req := &sarama.SyncGroupRequest{ 699 | GroupId: c.groupID, 700 | MemberId: memberID, 701 | GenerationId: generationID, 702 | } 703 | 704 | if strategy != nil { 705 | for memberID, topics := range strategy.Perform() { 706 | if err := req.AddGroupAssignmentMember(memberID, &sarama.ConsumerGroupMemberAssignment{ 707 | Topics: topics, 708 | }); err != nil { 709 | return nil, err 710 | } 711 | } 712 | } 713 | 714 | broker, err := c.client.Coordinator(c.groupID) 715 | if err != nil { 716 | c.closeCoordinator(broker, err) 717 | return nil, err 718 | } 719 | 720 | resp, err := broker.SyncGroup(req) 721 | if err != nil { 722 | c.closeCoordinator(broker, err) 723 | return nil, err 724 | } else if resp.Err != sarama.ErrNoError { 725 | c.closeCoordinator(broker, resp.Err) 726 | return nil, resp.Err 727 | } 728 | 729 | // Return if there is nothing to subscribe to 730 | if len(resp.MemberAssignment) == 0 { 731 | return nil, nil 732 | } 733 | 734 | // Get assigned subscriptions 735 | members, err := resp.GetMemberAssignment() 736 | if err != nil { 737 | return nil, err 738 | } 739 | 740 | // Sort partitions, for each topic 741 | for topic := range members.Topics { 742 | sort.Sort(int32Slice(members.Topics[topic])) 743 | } 744 | return members.Topics, nil 745 | } 746 | 747 | // Fetches latest committed offsets for all subscriptions 748 | func (c *Consumer) fetchOffsets(subs map[string][]int32) (map[string]map[int32]offsetInfo, error) { 749 | offsets := make(map[string]map[int32]offsetInfo, len(subs)) 750 | req := &sarama.OffsetFetchRequest{ 751 | Version: 1, 752 | ConsumerGroup: c.groupID, 753 | } 754 | 755 | for topic, partitions := range subs { 756 | offsets[topic] = make(map[int32]offsetInfo, len(partitions)) 757 | for _, partition := range partitions { 758 | offsets[topic][partition] = offsetInfo{Offset: -1} 759 | req.AddPartition(topic, partition) 760 | } 761 | } 762 | 763 | broker, err := c.client.Coordinator(c.groupID) 764 | if err != nil { 765 | c.closeCoordinator(broker, err) 766 | return nil, err 767 | } 768 | 769 | resp, err := broker.FetchOffset(req) 770 | if err != nil { 771 | c.closeCoordinator(broker, err) 772 | return nil, err 773 | } 774 | 775 | for topic, partitions := range subs { 776 | for _, partition := range partitions { 777 | block := resp.GetBlock(topic, partition) 778 | if block == nil { 779 | return nil, sarama.ErrIncompleteResponse 780 | } 781 | 782 | if block.Err == sarama.ErrNoError { 783 | offsets[topic][partition] = offsetInfo{Offset: block.Offset, Metadata: block.Metadata} 784 | } else { 785 | return nil, block.Err 786 | } 787 | } 788 | } 789 | return offsets, nil 790 | } 791 | 792 | // Send a request to the broker to leave the group on failes rebalance() and on Close() 793 | func (c *Consumer) leaveGroup() error { 794 | broker, err := c.client.Coordinator(c.groupID) 795 | if err != nil { 796 | c.closeCoordinator(broker, err) 797 | return err 798 | } 799 | 800 | memberID, _ := c.membership() 801 | if _, err = broker.LeaveGroup(&sarama.LeaveGroupRequest{ 802 | GroupId: c.groupID, 803 | MemberId: memberID, 804 | }); err != nil { 805 | c.closeCoordinator(broker, err) 806 | } 807 | return err 808 | } 809 | 810 | // -------------------------------------------------------------------- 811 | 812 | func (c *Consumer) createConsumer(tomb *loopTomb, topic string, partition int32, info offsetInfo) error { 813 | memberID, _ := c.membership() 814 | sarama.Logger.Printf("cluster/consumer %s consume %s/%d from %d\n", memberID, topic, partition, info.NextOffset(c.client.config.Consumer.Offsets.Initial)) 815 | 816 | // Create partitionConsumer 817 | pc, err := newPartitionConsumer(c.consumer, topic, partition, info, c.client.config.Consumer.Offsets.Initial) 818 | if err != nil { 819 | return err 820 | } 821 | 822 | // Store in subscriptions 823 | c.subs.Store(topic, partition, pc) 824 | 825 | // Start partition consumer goroutine 826 | tomb.Go(func(stopper <-chan none) { 827 | if c.client.config.Group.Mode == ConsumerModePartitions { 828 | pc.waitFor(stopper) 829 | } else { 830 | pc.multiplex(stopper, c.messages, c.errors) 831 | } 832 | }) 833 | 834 | if c.client.config.Group.Mode == ConsumerModePartitions { 835 | select { 836 | case c.partitions <- pc: 837 | case <-c.dying: 838 | pc.Close() 839 | } 840 | } 841 | return nil 842 | } 843 | 844 | func (c *Consumer) commitOffsetsWithRetry(retries int) error { 845 | err := c.CommitOffsets() 846 | if err != nil && retries > 0 { 847 | return c.commitOffsetsWithRetry(retries - 1) 848 | } 849 | return err 850 | } 851 | 852 | func (c *Consumer) closeCoordinator(broker *sarama.Broker, err error) { 853 | if broker != nil { 854 | _ = broker.Close() 855 | } 856 | 857 | switch err { 858 | case sarama.ErrConsumerCoordinatorNotAvailable, sarama.ErrNotCoordinatorForConsumer: 859 | _ = c.client.RefreshCoordinator(c.groupID) 860 | } 861 | } 862 | 863 | func (c *Consumer) selectExtraTopics(allTopics []string) []string { 864 | extra := allTopics[:0] 865 | for _, topic := range allTopics { 866 | if !c.isKnownCoreTopic(topic) && c.isPotentialExtraTopic(topic) { 867 | extra = append(extra, topic) 868 | } 869 | } 870 | return extra 871 | } 872 | 873 | func (c *Consumer) isKnownCoreTopic(topic string) bool { 874 | pos := sort.SearchStrings(c.coreTopics, topic) 875 | return pos < len(c.coreTopics) && c.coreTopics[pos] == topic 876 | } 877 | 878 | func (c *Consumer) isKnownExtraTopic(topic string) bool { 879 | pos := sort.SearchStrings(c.extraTopics, topic) 880 | return pos < len(c.extraTopics) && c.extraTopics[pos] == topic 881 | } 882 | 883 | func (c *Consumer) isPotentialExtraTopic(topic string) bool { 884 | rx := c.client.config.Group.Topics 885 | if rx.Blacklist != nil && rx.Blacklist.MatchString(topic) { 886 | return false 887 | } 888 | if rx.Whitelist != nil && rx.Whitelist.MatchString(topic) { 889 | return true 890 | } 891 | return false 892 | } 893 | 894 | func (c *Consumer) refreshCoordinator() error { 895 | if err := c.refreshMetadata(); err != nil { 896 | return err 897 | } 898 | return c.client.RefreshCoordinator(c.groupID) 899 | } 900 | 901 | func (c *Consumer) refreshMetadata() (err error) { 902 | if c.client.config.Metadata.Full { 903 | err = c.client.RefreshMetadata() 904 | } else { 905 | var topics []string 906 | if topics, err = c.client.Topics(); err == nil && len(topics) != 0 { 907 | err = c.client.RefreshMetadata(topics...) 908 | } 909 | } 910 | 911 | // maybe we didn't have authorization to describe all topics 912 | switch err { 913 | case sarama.ErrTopicAuthorizationFailed: 914 | err = c.client.RefreshMetadata(c.coreTopics...) 915 | } 916 | return 917 | } 918 | 919 | func (c *Consumer) membership() (memberID string, generationID int32) { 920 | c.membershipMu.RLock() 921 | memberID, generationID = c.memberID, c.generationID 922 | c.membershipMu.RUnlock() 923 | return 924 | } 925 | --------------------------------------------------------------------------------