// Different topics can be consumed at the same time. func (s *SmartConsumerSuite) TestMultipleTopics(c *C) { // Given s.kh.ResetOffsets("group-1", "test.1") s.kh.ResetOffsets("group-1", "test.4") produced1 := s.kh.PutMessages("multiple.topics", "test.1", map[string]int{"A": 1}) produced4 := s.kh.PutMessages("multiple.topics", "test.4", map[string]int{"B": 1, "C": 1}) log.Infof("*** GIVEN 1") sc, err := Spawn(testhelpers.NewTestConfig("consumer-1")) c.Assert(err, IsNil) // When log.Infof("*** WHEN") consumed := s.consume(c, sc, "group-1", "test.4", 1) consumed = s.consume(c, sc, "group-1", "test.1", 1, consumed) consumed = s.consume(c, sc, "group-1", "test.4", 1, consumed) // Then log.Infof("*** THEN") assertMsg(c, consumed["A"][0], produced1["A"][0]) assertMsg(c, consumed["B"][0], produced4["B"][0]) assertMsg(c, consumed["C"][0], produced4["C"][0]) sc.Stop() }
// register listens for topic subscription updates on the `topicsCh` channel // and updates the member registration in ZooKeeper accordingly. func (cgr *consumerGroupRegistry) register() { cid := cgr.baseCID.NewChild("register") defer cid.LogScope()() defer cgr.retry(cgr.groupMemberZNode.Deregister, func(err error) bool { return err != nil && err != kazoo.ErrInstanceNotRegistered }, fmt.Sprintf("<%s> failed to deregister", cid)) for { var topics []string select { case topics = <-cgr.topicsCh: case <-cgr.stoppingCh: return } sort.Sort(sort.StringSlice(topics)) log.Infof("<%s> registering...: id=%s, topics=%v", cid, cgr.groupMemberZNode.ID, topics) if cgr.retry( func() error { if err := cgr.groupMemberZNode.Deregister(); err != nil && err != kazoo.ErrInstanceNotRegistered { return fmt.Errorf("could not deregister: err=(%s)", err) } return cgr.groupMemberZNode.Register(topics) }, nil, fmt.Sprintf("<%s> failed to register", cid), ) { return } log.Infof("<%s> registered: id=%s, topics=%v", cid, cgr.groupMemberZNode.ID, topics) } }
// When there are more consumers in a group then partitions in a topic then // some consumers get assigned no partitions and their consume requests timeout. func (s *SmartConsumerSuite) TestTooFewPartitions(c *C) { // Given s.kh.ResetOffsets("group-1", "test.1") produced := s.kh.PutMessages("few", "test.1", map[string]int{"": 3}) sc1, err := Spawn(testhelpers.NewTestConfig("consumer-1")) c.Assert(err, IsNil) log.Infof("*** GIVEN 1") // Consume first message to make `consumer-1` subscribe for `test.1` consumed := s.consume(c, sc1, "group-1", "test.1", 2) assertMsg(c, consumed[""][0], produced[""][0]) // When: log.Infof("*** WHEN") sc2, err := Spawn(testhelpers.NewTestConfig("consumer-2")) c.Assert(err, IsNil) _, err = sc2.Consume("group-1", "test.1") // Then: `consumer-2` request times out, when `consumer-1` requests keep // return messages. log.Infof("*** THEN") if _, ok := err.(ErrRequestTimeout); !ok { c.Errorf("Expected ErrConsumerRequestTimeout, got %s", err) } s.consume(c, sc1, "group-1", "test.1", 1, consumed) assertMsg(c, consumed[""][1], produced[""][1]) sc1.Stop() sc2.Stop() }
// sendMembershipUpdate retrieves registration records for the specified members // from ZooKeeper and sends current list of members along with topics they are // subscribed to down the `membershipChangesCh`. The method can be interrupted // any time by the stop signal. // // FIXME: It is assumed that all members of the group are registered with the // FIXME: `static` pattern. If a member that pattern is either `white_list` or // FIXME: `black_list` joins the group the result will be unpredictable. func (cgr *consumerGroupRegistry) sendMembershipUpdate(cid *sarama.ContextID, members []*kazoo.ConsumergroupInstance) { log.Infof("<%s> fetching group subscriptions...", cid) subscriptions := make(map[string][]string, len(members)) for _, member := range members { var registration *kazoo.Registration if cgr.retry( func() error { var err error registration, err = member.Registration() return err }, nil, fmt.Sprintf("<%s> failed to get member registration", cid), ) { return } // Sort topics to ensure deterministic output. topics := make([]string, 0, len(registration.Subscription)) for topic := range registration.Subscription { topics = append(topics, topic) } sort.Sort(sort.StringSlice(topics)) subscriptions[member.ID] = topics } log.Infof("<%s> group subscriptions changed: %v", cid, subscriptions) select { case cgr.membershipChangesCh <- subscriptions: case <-cgr.stoppingCh: return } }
// This test makes an attempt to exercise the code path where a message is // received when a down stream dispatch tier is being stopped due to // registration timeout, in that case a successor tier is created that will be // started as soon as the original one is completely shutdown. // // It is impossible to see from the service behavior if the expected code path // has been exercised by the test. The only way to check that is through the // code coverage reports. func (s *SmartConsumerSuite) TestRequestDuringTimeout(c *C) { // Given s.kh.ResetOffsets("group-1", "test.4") s.kh.PutMessages("join", "test.4", map[string]int{"A": 30}) cfg := testhelpers.NewTestConfig("consumer-1") cfg.Consumer.RegistrationTimeout = 200 * time.Millisecond cfg.Consumer.ChannelBufferSize = 1 sc, err := Spawn(cfg) c.Assert(err, IsNil) // When/Then for i := 0; i < 10; i++ { for j := 0; j < 3; j++ { begin := time.Now() log.Infof("*** consuming...") consMsg, err := sc.Consume("group-1", "test.4") c.Assert(err, IsNil) log.Infof("*** consumed: in=%s, by=%s, topic=%s, partition=%d, offset=%d, message=%s", time.Now().Sub(begin), sc.baseCID.String(), consMsg.Topic, consMsg.Partition, consMsg.Offset, consMsg.Value) } time.Sleep(200 * time.Millisecond) } sc.Stop() }
// If we consume from a topic that has several partitions then partitions are // selected for consumption in random order. func (s *SmartConsumerSuite) TestMultiplePartitions(c *C) { // Given s.kh.ResetOffsets("group-1", "test.4") s.kh.PutMessages("multiple.partitions", "test.4", map[string]int{"A": 100, "B": 100}) log.Infof("*** GIVEN 1") sc, err := Spawn(testhelpers.NewTestConfig("consumer-1")) c.Assert(err, IsNil) // When: exactly one half of all produced events is consumed. log.Infof("*** WHEN") consumed := s.consume(c, sc, "group-1", "test.4", 1) // Wait until first messages from partitions `A` and `B` are fetched. waitFirstFetched(sc, 2) // Consume 100 messages total consumed = s.consume(c, sc, "group-1", "test.4", 99, consumed) // Then: we have events consumed from both partitions more or less evenly. log.Infof("*** THEN") if len(consumed["A"]) < 25 || len(consumed["A"]) > 75 { c.Errorf("Consumption disbalance: consumed[A]=%d, consumed[B]=%d", len(consumed["A"]), len(consumed["B"])) } sc.Stop() }
// If we stop one consumer and start another, the new one picks up where the // previous one left off. func (s *SmartConsumerSuite) TestSequentialConsume(c *C) { // Given s.kh.ResetOffsets("group-1", "test.1") produced := s.kh.PutMessages("sequencial", "test.1", map[string]int{"": 3}) cfg := testhelpers.NewTestConfig("consumer-1") sc1, err := Spawn(cfg) c.Assert(err, IsNil) log.Infof("*** GIVEN 1") consumed := s.consume(c, sc1, "group-1", "test.1", 2) assertMsg(c, consumed[""][0], produced[""][0]) assertMsg(c, consumed[""][1], produced[""][1]) // When: one consumer stopped and another one takes its place. log.Infof("*** WHEN") sc1.Stop() sc2, err := Spawn(cfg) c.Assert(err, IsNil) // Then: the second message is consumed. log.Infof("*** THEN") consumed = s.consume(c, sc2, "group-1", "test.1", 1, consumed) assertMsg(c, consumed[""][2], produced[""][2]) sc2.Stop() }
// A topic that has a lot of partitions can be consumed. func (s *SmartConsumerSuite) TestLotsOfPartitions(c *C) { // Given s.kh.ResetOffsets("group-1", "test.64") cfg := testhelpers.NewTestConfig("consumer-1") sc, err := Spawn(cfg) c.Assert(err, IsNil) // Consume should stop by timeout and nothing should be consumed. msg, err := sc.Consume("group-1", "test.64") if _, ok := err.(ErrRequestTimeout); !ok { c.Fatalf("Unexpected message consumed: %v", msg) } s.kh.PutMessages("lots", "test.64", map[string]int{"A": 7, "B": 13, "C": 169}) // When log.Infof("*** WHEN") consumed := s.consume(c, sc, "group-1", "test.64", consumeAll) // Then log.Infof("*** THEN") c.Assert(7, Equals, len(consumed["A"])) c.Assert(13, Equals, len(consumed["B"])) c.Assert(169, Equals, len(consumed["C"])) sc.Stop() }
// When a consumer registration times out the partitions that used to be // assigned to it are redistributed among active consumers. func (s *SmartConsumerSuite) TestRebalanceOnTimeout(c *C) { // Given s.kh.ResetOffsets("group-1", "test.4") s.kh.PutMessages("join", "test.4", map[string]int{"A": 10, "B": 10}) sc1, err := Spawn(testhelpers.NewTestConfig("consumer-1")) c.Assert(err, IsNil) cfg2 := testhelpers.NewTestConfig("consumer-2") cfg2.Consumer.RegistrationTimeout = 300 * time.Millisecond sc2, err := Spawn(cfg2) c.Assert(err, IsNil) // Consume the first message to make the consumers join the group and // subscribe to the topic. log.Infof("*** GIVEN 1") consumed1 := s.consume(c, sc1, "group-1", "test.4", 1) consumed2 := s.consume(c, sc2, "group-1", "test.4", 1) if len(consumed1["B"]) == 0 { c.Assert(len(consumed1["A"]), Equals, 1) } else { c.Assert(len(consumed1["A"]), Equals, 0) } c.Assert(len(consumed2["A"]), Equals, 0) c.Assert(len(consumed2["B"]), Equals, 1) // Consume 4 more messages to make sure that each consumer pulls from a // particular assigned to it. log.Infof("*** GIVEN 2") consumed1 = s.consume(c, sc1, "group-1", "test.4", 4, consumed1) consumed2 = s.consume(c, sc2, "group-1", "test.4", 4, consumed2) if len(consumed1["B"]) == 1 { c.Assert(len(consumed1["A"]), Equals, 4) } else { c.Assert(len(consumed1["A"]), Equals, 5) } c.Assert(len(consumed2["A"]), Equals, 0) c.Assert(len(consumed2["B"]), Equals, 5) drainFirstFetched(sc1) // When: `consumer-2` registration timeout elapses, the partitions get // rebalanced so that `consumer-1` becomes assigned to all of them... log.Infof("*** WHEN") // Wait for partition `B` reassigned back to sc1. waitFirstFetched(sc1, 1) // ...and consumes the remaining messages from all partitions. log.Infof("*** THEN") consumed1 = s.consume(c, sc1, "group-1", "test.4", 10, consumed1) c.Assert(len(consumed1["A"]), Equals, 10) c.Assert(len(consumed1["B"]), Equals, 5) c.Assert(len(consumed2["A"]), Equals, 0) c.Assert(len(consumed2["B"]), Equals, 5) sc2.Stop() sc1.Stop() }
// dispatch implements message processing and graceful shutdown. It receives // messages from `dispatchedCh` where they are send to by `Produce` method and // submits them to the embedded `sarama.AsyncProducer`. The dispatcher main // purpose is to prevent loss of messages during shutdown. It achieves that by // allowing some graceful period after it stops receiving messages and stopping // the embedded `sarama.AsyncProducer`. func (gp *GracefulProducer) dispatch() { cid := gp.baseCID.NewChild("dispatch") defer cid.LogScope()() nilOrDispatcherCh := gp.dispatcherCh var nilOrProdInputCh chan<- *sarama.ProducerMessage pendingMsgCount := 0 // The normal operation loop is implemented as two-stroke machine. On the // first stroke a message is received from `dispatchCh`, and on the second // it is sent to `prodInputCh`. Note that producer results can be received // at any time. prodMsg := (*sarama.ProducerMessage)(nil) channelOpened := true for { select { case prodMsg, channelOpened = <-nilOrDispatcherCh: if !channelOpened { goto gracefulShutdown } pendingMsgCount += 1 nilOrDispatcherCh = nil nilOrProdInputCh = gp.saramaProducer.Input() case nilOrProdInputCh <- prodMsg: nilOrDispatcherCh = gp.dispatcherCh nilOrProdInputCh = nil case prodResult := <-gp.resultCh: pendingMsgCount -= 1 gp.handleProduceResult(cid, prodResult) } } gracefulShutdown: // Give the `sarama.AsyncProducer` some time to commit buffered messages. log.Infof("<%v> About to stop producer: pendingMsgCount=%d", cid, pendingMsgCount) shutdownTimeoutCh := time.After(gp.shutdownTimeout) for pendingMsgCount > 0 { select { case <-shutdownTimeoutCh: goto shutdownNow case prodResult := <-gp.resultCh: pendingMsgCount -= 1 gp.handleProduceResult(cid, prodResult) } } shutdownNow: log.Infof("<%v> Stopping producer: pendingMsgCount=%d", cid, pendingMsgCount) gp.saramaProducer.AsyncClose() for prodResult := range gp.resultCh { gp.handleProduceResult(cid, prodResult) } }
func (cgr *consumerGroupRegistry) claimPartition(cid *sarama.ContextID, topic string, partition int32, cancelCh <-chan none) func() { if !retry(func() error { return cgr.groupMemberZNode.ClaimPartition(topic, partition) }, nil, fmt.Sprintf("<%s> failed to claim partition", cid), cgr.config.Consumer.BackOffTimeout, cancelCh, ) { log.Infof("<%s> partition claimed", cid) } return func() { if !retry(func() error { return cgr.groupMemberZNode.ReleasePartition(topic, partition) }, func(err error) bool { return err != nil && err != kazoo.ErrPartitionNotClaimed }, fmt.Sprintf("<%s> failed to release partition", cid), cgr.config.Consumer.BackOffTimeout, cancelCh, ) { log.Infof("<%s> partition released", cid) } } }
func GenMessages(c *C, prefix, topic string, keys map[string]int) map[string][]*sarama.ProducerMessage { config := NewConfig() config.ClientID = "producer" config.Kafka.SeedPeers = testKafkaPeers producer, err := SpawnGracefulProducer(config) c.Assert(err, IsNil) messages := make(map[string][]*sarama.ProducerMessage) var wg sync.WaitGroup var lock sync.Mutex for key, count := range keys { for i := 0; i < count; i++ { key := key message := fmt.Sprintf("%s:%s:%d", prefix, key, i) spawn(&wg, func() { keyEncoder := sarama.StringEncoder(key) msgEncoder := sarama.StringEncoder(message) prodMsg, err := producer.Produce(topic, keyEncoder, msgEncoder) c.Assert(err, IsNil) log.Infof("*** produced: topic=%s, partition=%d, offset=%d, message=%s", topic, prodMsg.Partition, prodMsg.Offset, message) lock.Lock() messages[key] = append(messages[key], prodMsg) lock.Unlock() }) } } wg.Wait() // Sort the produced messages in ascending order of their offsets. for _, keyMessages := range messages { sort.Sort(MessageSlice(keyMessages)) } return messages }
// A `ErrConsumerBufferOverflow` error can be returned if internal buffers are // filled with in-flight consume requests. func (s *SmartConsumerSuite) TestBufferOverflowError(c *C) { // Given s.kh.ResetOffsets("group-1", "test.1") s.kh.PutMessages("join", "test.1", map[string]int{"A": 30}) cfg := testhelpers.NewTestConfig("consumer-1") cfg.Consumer.ChannelBufferSize = 1 sc, err := Spawn(cfg) c.Assert(err, IsNil) // When var overflowErrorCount int32 var wg sync.WaitGroup for i := 0; i < 3; i++ { spawn(&wg, func() { for i := 0; i < 10; i++ { _, err := sc.Consume("group-1", "test.1") if _, ok := err.(ErrBufferOverflow); ok { atomic.AddInt32(&overflowErrorCount, 1) } } }) } wg.Wait() // Then c.Assert(overflowErrorCount, Not(Equals), 0) log.Infof("*** overflow was hit %d times", overflowErrorCount) sc.Stop() }
func waitFirstFetched(sc *T, count int) { var partitions []int32 for i := 0; i < count; i++ { ec := <-firstMessageFetchedCh partitions = append(partitions, ec.partition) } log.Infof("*** first messages fetched: partitions=%v", partitions) }
// When a new consumer joins a group the partitions get evenly redistributed // among all consumers. func (s *SmartConsumerSuite) TestRebalanceOnJoin(c *C) { // Given s.kh.ResetOffsets("group-1", "test.4") s.kh.PutMessages("join", "test.4", map[string]int{"A": 10, "B": 10}) sc1, err := Spawn(testhelpers.NewTestConfig("consumer-1")) c.Assert(err, IsNil) // Consume the first message to make the consumer join the group and // subscribe to the topic. log.Infof("*** GIVEN 1") consumed1 := s.consume(c, sc1, "group-1", "test.4", 1) // Wait until first messages from partitions `A` and `B` are fetched. waitFirstFetched(sc1, 2) // Consume 4 messages and make sure that there are messages from both // partitions among them. log.Infof("*** GIVEN 2") consumed1 = s.consume(c, sc1, "group-1", "test.4", 4, consumed1) c.Assert(len(consumed1["A"]), Not(Equals), 0) c.Assert(len(consumed1["B"]), Not(Equals), 0) consumedBeforeJoin := len(consumed1["B"]) // When: another consumer joins the group rebalancing occurs. log.Infof("*** WHEN") sc2, err := Spawn(testhelpers.NewTestConfig("consumer-2")) c.Assert(err, IsNil) // Then: log.Infof("*** THEN") consumed2 := s.consume(c, sc2, "group-1", "test.4", consumeAll) consumed1 = s.consume(c, sc1, "group-1", "test.4", consumeAll, consumed1) // Partition "A" has been consumed by `consumer-1` only c.Assert(len(consumed1["A"]), Equals, 10) c.Assert(len(consumed2["A"]), Equals, 0) // Partition "B" has been consumed by both consumers, but ever since // `consumer-2` joined the group the first one have not got any new messages. c.Assert(len(consumed1["B"]), Equals, consumedBeforeJoin) c.Assert(len(consumed2["B"]), Not(Equals), 0) c.Assert(len(consumed1["B"])+len(consumed2["B"]), Equals, 10) // `consumer-2` started consumer from where `consumer-1` left off. c.Assert(consumed2["B"][0].Offset, Equals, consumed1["B"][len(consumed1["B"])-1].Offset+1) sc2.Stop() sc1.Stop() }
func waitFirstFetched(sc *SmartConsumer, count int) { var partitions []int32 for i := 0; i < count; i++ { ec := <-sc.config.testing.firstMessageFetchedCh partitions = append(partitions, ec.partition) } log.Infof("*** first messages fetched: partitions=%v", partitions) }
// handleExpired marks the respective dispatch tier as expired and triggers its // asynchronous stop. When the tier is stopped it will notify about that via the // `stoppedChildrenCh` channel. func (d *dispatcher) handleExpired(dt dispatchTier) { log.Infof("<%s> child expired: %s", d.contextID, dt) edt := d.children[dt.key()] if edt == nil || edt.instance != dt || edt.expired { return } edt.expired = true go edt.instance.stop() }
// handleStopped if the specified dispatch tier has a successor then it is // started and takes over the tier's spot among the downstream dispatch tiers, // otherwise the tier is deleted. func (d *dispatcher) handleStopped(dt dispatchTier) dispatchTier { log.Infof("<%s> child stopped: %s", d.contextID, dt) edt := d.children[dt.key()] if edt == nil { return nil } successor := edt.successor if successor == nil { delete(d.children, dt.key()) return nil } log.Infof("<%s> starting successor: %s", d.contextID, successor) edt.expired = false edt.instance = successor edt.successor = nil successor.start(edt.d.stoppedChildrenCh) timeout := edt.d.config.Consumer.RegistrationTimeout edt.timer = time.AfterFunc(timeout, func() { edt.d.expiredChildrenCh <- successor }) return edt.instance }
func (kh *KafkaHelper) ResetOffsets(group, topic string) { partitions, err := kh.client.Partitions(topic) kh.c.Assert(err, IsNil) for _, p := range partitions { offset, err := kh.client.GetOffset(topic, p, sarama.OffsetNewest) kh.c.Assert(err, IsNil) pom, err := kh.offsetMgr.ManagePartition(group, topic, p) kh.c.Assert(err, IsNil) pom.SubmitOffset(offset, "dummy") log.Infof("Set initial offset %s/%s/%d=%d", group, topic, p, offset) pom.Close() } }
// If the same topic is consumed by different consumer groups, then consumption // by one group does not affect the consumption by another. func (s *SmartConsumerSuite) TestMultipleGroups(c *C) { // Given s.kh.ResetOffsets("group-1", "test.4") s.kh.ResetOffsets("group-2", "test.4") s.kh.PutMessages("multi", "test.4", map[string]int{"A": 10, "B": 10, "C": 10}) log.Infof("*** GIVEN 1") sc, err := Spawn(testhelpers.NewTestConfig("consumer-1")) c.Assert(err, IsNil) // When log.Infof("*** WHEN") consumed1 := s.consume(c, sc, "group-1", "test.4", 10) consumed2 := s.consume(c, sc, "group-2", "test.4", 20) consumed1 = s.consume(c, sc, "group-1", "test.4", 20, consumed1) consumed2 = s.consume(c, sc, "group-2", "test.4", 10, consumed2) // Then: both groups consumed the same events log.Infof("*** THEN") c.Assert(consumed1, DeepEquals, consumed2) sc.Stop() }
func (gc *groupConsumer) rebalance(topicConsumers map[string]*topicConsumer, memberSubscriptions map[string][]string, rebalanceResultCh chan<- error, ) { cid := gc.baseCID.NewChild("rebalance") defer cid.LogScope(topicConsumers, memberSubscriptions)() assignedPartitions, err := gc.resolvePartitions(memberSubscriptions) if err != nil { rebalanceResultCh <- err return } log.Infof("<%s> assigned: %v", cid, assignedPartitions) var wg sync.WaitGroup // Stop consuming partitions that are no longer assigned to this group // and start consuming newly assigned partitions for topics that has been // consumed already. for topic, tg := range gc.topicGears { tg := tg assignedTopicPartition := assignedPartitions[topic] spawn(&wg, func() { gc.rewireMultiplexer(tg, assignedTopicPartition) }) } // Start consuming partitions for topics that we has not been consumed before. for topic, assignedTopicPartitions := range assignedPartitions { tc := topicConsumers[topic] tg := gc.topicGears[topic] if tc == nil || tg != nil { continue } tg = &topicGear{ topicConsumer: tc, exclusiveConsumers: make(map[int32]*exclusiveConsumer, len(assignedTopicPartitions)), } assignedTopicPartitions := assignedTopicPartitions spawn(&wg, func() { gc.rewireMultiplexer(tg, assignedTopicPartitions) }) gc.topicGears[topic] = tg } wg.Wait() // Clean up gears for topics that are not consumed anymore. for topic, tg := range gc.topicGears { if tg.multiplexer == nil { delete(gc.topicGears, topic) } } // Notify the caller that rebalancing has completed successfully. rebalanceResultCh <- nil return }
func (gc *groupConsumer) rebalance(topicConsumers map[string]*topicConsumer, subscriptions map[string][]string, rebalanceResultCh chan<- error, ) { cid := gc.baseCID.NewChild("rebalance") defer cid.LogScope(topicConsumers, subscriptions)() assignedPartitions, err := gc.resolvePartitions(subscriptions) if err != nil { rebalanceResultCh <- err return } log.Infof("<%s> assigned partitions: %v", cid, assignedPartitions) var wg sync.WaitGroup // Stop consuming partitions that are no longer assigned to this group // and start consuming newly assigned partitions for topics that has been // consumed already. for topic, tcg := range gc.topicConsumerGears { tcg.muxInputsAsync(&wg, topicConsumers[topic], assignedPartitions[topic]) } // Start consuming partitions for topics that has not been consumed before. for topic, assignedTopicPartitions := range assignedPartitions { tc := topicConsumers[topic] tcg := gc.topicConsumerGears[topic] if tc == nil || tcg != nil { continue } tcg = newTopicConsumerGear(gc.spawnTopicInput) tcg.muxInputsAsync(&wg, tc, assignedTopicPartitions) gc.topicConsumerGears[topic] = tcg } wg.Wait() // Clean up gears for topics that do not have assigned partitions anymore. for topic, tcg := range gc.topicConsumerGears { if tcg.isIdle() { delete(gc.topicConsumerGears, topic) } } // Notify the caller that rebalancing has completed successfully. rebalanceResultCh <- nil return }
func (kh *KafkaHelper) PutMessages(prefix, topic string, keys map[string]int) map[string][]*sarama.ProducerMessage { messages := make(map[string][]*sarama.ProducerMessage) var wg sync.WaitGroup total := 0 for key, count := range keys { total += count for i := 0; i < count; i++ { key := key message := fmt.Sprintf("%s:%s:%d", prefix, key, i) wg.Add(1) go func() { defer wg.Done() keyEncoder := sarama.StringEncoder(key) msgEncoder := sarama.StringEncoder(message) prodMsg := &sarama.ProducerMessage{ Topic: topic, Key: keyEncoder, Value: msgEncoder, } kh.producer.Input() <- prodMsg }() } } for i := 0; i < total; i++ { select { case prodMsg := <-kh.producer.Successes(): key := string(prodMsg.Key.(sarama.StringEncoder)) messages[key] = append(messages[key], prodMsg) log.Infof("*** produced: topic=%s, partition=%d, offset=%d, message=%s", topic, prodMsg.Partition, prodMsg.Offset, prodMsg.Value) case prodErr := <-kh.producer.Errors(): kh.c.Error(prodErr) } } // Sort the produced messages in ascending order of their offsets. for _, keyMessages := range messages { sort.Sort(messageSlice(keyMessages)) } wg.Wait() return messages }
func main() { // Make go runtime execute in parallel as many goroutines as there are CPUs. runtime.GOMAXPROCS(runtime.NumCPU()) if err := initLogging(); err != nil { fmt.Printf("Failed to initialize logger: err=(%s)\n", err) os.Exit(1) } if pidFile != "" { if err := writePID(pidFile); err != nil { log.Errorf("Failed to write PID file: err=(%s)", err) os.Exit(1) } } // Clean up the unix domain socket file in case we failed to clean up on // shutdown the last time. Otherwise the service won't be able to listen // on this address and as a result will fail to start up. if config.UnixAddr != "" { if err := os.Remove(config.UnixAddr); err != nil && !os.IsNotExist(err) { log.Errorf("Cannot remove %s: err=(%s)", config.UnixAddr, err) } } log.Infof("Starting with config: %+v", config) svc, err := pixy.SpawnService(config) if err != nil { log.Errorf("Failed to start service: err=(%s)", err) os.Exit(1) } // Spawn OS signal listener to ensure graceful stop. osSigCh := make(chan os.Signal, 1) signal.Notify(osSigCh, syscall.SIGINT, syscall.SIGQUIT, syscall.SIGTERM) // Wait for a quit signal and terminate the service when it is received. <-osSigCh svc.Stop() }
func ResetOffsets(c *C, group, topic string) { config := NewConfig() config.Kafka.SeedPeers = testKafkaPeers config.ZooKeeper.SeedPeers = testZookeeperPeers kafkaClient, err := sarama.NewClient(config.Kafka.SeedPeers, config.saramaConfig()) c.Assert(err, IsNil) defer kafkaClient.Close() offsetManager, err := sarama.NewOffsetManagerFromClient(kafkaClient) c.Assert(err, IsNil) partitions, err := kafkaClient.Partitions(topic) c.Assert(err, IsNil) for _, p := range partitions { offset, err := kafkaClient.GetOffset(topic, p, sarama.OffsetNewest) c.Assert(err, IsNil) pom, err := offsetManager.ManagePartition(group, topic, p) c.Assert(err, IsNil) pom.SubmitOffset(offset, "dummy") log.Infof("Set initial offset %s/%s/%d=%d", group, topic, p, offset) pom.Close() } offsetManager.Close() }
func (ec *exclusiveConsumer) run() { defer ec.contextID.LogScope()() defer ec.registry.claimPartition(ec.contextID, ec.topic, ec.partition, ec.stoppingCh)() pom, err := ec.offsetMgr.ManagePartition(ec.group, ec.topic, ec.partition) if err != nil { panic(fmt.Errorf("<%s> failed to spawn partition manager: err=(%s)", ec.contextID, err)) } defer pom.Close() // Wait for the initial offset to be retrieved. var initialOffset sarama.DecoratedOffset select { case initialOffset = <-pom.InitialOffset(): case <-ec.stoppingCh: return } pc, concreteOffset, err := ec.dumbConsumer.ConsumePartition(ec.topic, ec.partition, initialOffset.Offset) if err != nil { panic(fmt.Errorf("<%s> failed to start partition consumer: err=(%s)", ec.contextID, err)) } defer pc.Close() log.Infof("<%s> initialized: initialOffset=%d, concreteOffset=%d", ec.contextID, initialOffset.Offset, concreteOffset) var lastSubmittedOffset, lastCommittedOffset int64 // Initialize the Kafka offset storage for a group on first consumption. if initialOffset.Offset == sarama.OffsetNewest { pom.SubmitOffset(concreteOffset, "") lastSubmittedOffset = concreteOffset } firstMessageFetched := false for { var msg *sarama.ConsumerMessage // Wait for a fetched message to to provided by the controlled // partition consumer. for { select { case msg = <-pc.Messages(): // Notify tests when the very first message is fetched. if !firstMessageFetched && firstMessageFetchedCh != nil { firstMessageFetched = true firstMessageFetchedCh <- ec } goto offerAndAck case committedOffset := <-pom.CommittedOffsets(): lastCommittedOffset = committedOffset.Offset continue case <-ec.stoppingCh: goto done } } offerAndAck: // Offer the fetched message to the upstream consumer and wait for it // to be acknowledged. for { select { case ec.messagesCh <- msg: // Keep offering the same message until it is acknowledged. case <-ec.acksCh: lastSubmittedOffset = msg.Offset + 1 pom.SubmitOffset(lastSubmittedOffset, "") break offerAndAck case committedOffset := <-pom.CommittedOffsets(): lastCommittedOffset = committedOffset.Offset continue case <-ec.stoppingCh: goto done } } } done: if lastCommittedOffset == lastSubmittedOffset { return } // It is necessary to wait for the offset of the last consumed message to // be committed to Kafka before releasing ownership over the partition, // otherwise the message can be consumed by the new partition owner again. log.Infof("<%s> waiting for the last offset to be committed: submitted=%d, committed=%d", ec.contextID, lastSubmittedOffset, lastCommittedOffset) for committedOffset := range pom.CommittedOffsets() { if committedOffset.Offset == lastSubmittedOffset { return } log.Infof("<%s> waiting for the last offset to be committed: submitted=%d, committed=%d", ec.contextID, lastSubmittedOffset, committedOffset.Offset) } }
func logConsumed(sc *T, consMsg *sarama.ConsumerMessage) { log.Infof("*** consumed: by=%s, topic=%s, partition=%d, offset=%d, message=%s", sc.baseCID.String(), consMsg.Topic, consMsg.Partition, consMsg.Offset, consMsg.Value) }
// When a consumer leaves a group the partitions get evenly redistributed // among the remaining consumers. func (s *SmartConsumerSuite) TestRebalanceOnLeave(c *C) { // Given s.kh.ResetOffsets("group-1", "test.4") produced := s.kh.PutMessages("leave", "test.4", map[string]int{"A": 10, "B": 10, "C": 10}) var err error consumers := make([]*T, 3) for i := 0; i < 3; i++ { consumers[i], err = Spawn(testhelpers.NewTestConfig(fmt.Sprintf("consumer-%d", i))) c.Assert(err, IsNil) } log.Infof("*** GIVEN 1") // Consume the first message to make the consumer join the group and // subscribe to the topic. consumed := make([]map[string][]*sarama.ConsumerMessage, 3) for i := 0; i < 3; i++ { consumed[i] = s.consume(c, consumers[i], "group-1", "test.4", 1) } // consumer[0] can consume the first message from all partitions and // consumer[1] can consume the first message from either `B` or `C`. log.Infof("*** GIVEN 2") if len(consumed[0]["A"]) == 1 { if len(consumed[1]["B"]) == 1 { assertMsg(c, consumed[2]["B"][0], produced["B"][1]) } else { // if len(consumed[1]["C"]) == 1 { assertMsg(c, consumed[2]["B"][0], produced["B"][0]) } } else if len(consumed[0]["B"]) == 1 { if len(consumed[1]["B"]) == 1 { assertMsg(c, consumed[2]["B"][0], produced["B"][2]) } else { // if len(consumed[1]["C"]) == 1 { assertMsg(c, consumed[2]["B"][0], produced["B"][1]) } } else { // if len(consumed[0]["C"]) == 1 { if len(consumed[1]["B"]) == 1 { assertMsg(c, consumed[2]["B"][0], produced["B"][1]) } else { // if len(consumed[1]["C"]) == 1 { assertMsg(c, consumed[2]["B"][0], produced["B"][0]) } } s.consume(c, consumers[2], "group-1", "test.4", 4, consumed[2]) c.Assert(len(consumed[2]["B"]), Equals, 5) lastConsumedFromBby2 := consumed[2]["B"][4] for _, consumer := range consumers { drainFirstFetched(consumer) } // When log.Infof("*** WHEN") consumers[2].Stop() // Wait for partition `C` reassign back to consumer[1] waitFirstFetched(consumers[1], 1) // Then: partition `B` is reassigned to `consumer[1]` and it picks up where // `consumer[2]` left off. log.Infof("*** THEN") consumedSoFar := make(map[string]int) for _, consumedByOne := range consumed { for key, consumedWithKey := range consumedByOne { consumedSoFar[key] = consumedSoFar[key] + len(consumedWithKey) } } leftToBeConsumedBy1 := 20 - (consumedSoFar["B"] + consumedSoFar["C"]) consumedBy1 := s.consume(c, consumers[1], "group-1", "test.4", leftToBeConsumedBy1) c.Assert(len(consumedBy1["B"]), Equals, 10-consumedSoFar["B"]) c.Assert(consumedBy1["B"][0].Offset, Equals, lastConsumedFromBby2.Offset+1) consumers[0].Stop() consumers[1].Stop() }