// SetGroupOffsets commits specific offset values along with metadata for a list // of partitions of a particular topic on behalf of the specified group. func (a *T) SetGroupOffsets(group, topic string, offsets []PartitionOffset) error { kafkaClt, err := sarama.NewClient(a.cfg.Kafka.SeedPeers, a.saramaConfig()) if err != nil { return ErrSetup(fmt.Errorf("failed to create sarama.Client: err=(%v)", err)) } defer kafkaClt.Close() coordinator, err := kafkaClt.Coordinator(group) if err != nil { return NewErrQuery(err, "failed to get coordinator") } req := sarama.OffsetCommitRequest{ Version: ProtocolVer1, ConsumerGroup: group, ConsumerGroupGeneration: sarama.GroupGenerationUndefined, } for _, po := range offsets { req.AddBlock(topic, po.Partition, po.Offset, sarama.ReceiveTime, po.Metadata) } res, err := coordinator.CommitOffset(&req) if err != nil { return NewErrQuery(err, "failed to commit offsets") } for p, err := range res.Errors[topic] { if err != sarama.ErrNoError { return NewErrQuery(err, "failed to commit offset: partition=%d", p) } } return nil }
// Spawn creates a producer instance and starts its internal goroutines. func Spawn(cfg *config.T) (*T, error) { saramaCfg := sarama.NewConfig() saramaCfg.ChannelBufferSize = cfg.Producer.ChannelBufferSize saramaCfg.Producer.RequiredAcks = sarama.WaitForAll saramaCfg.Producer.Return.Successes = true saramaCfg.Producer.Return.Errors = true saramaCfg.Producer.Compression = sarama.CompressionSnappy saramaCfg.Producer.Retry.Backoff = 10 * time.Second saramaCfg.Producer.Retry.Max = 6 saramaCfg.Producer.Flush.Frequency = 500 * time.Millisecond saramaCfg.Producer.Flush.Bytes = 1024 * 1024 saramaClient, err := sarama.NewClient(cfg.Kafka.SeedPeers, saramaCfg) if err != nil { return nil, fmt.Errorf("failed to create sarama.Client, err=(%s)", err) } saramaProducer, err := sarama.NewAsyncProducerFromClient(saramaClient) if err != nil { return nil, fmt.Errorf("failed to create sarama.Producer, err=(%s)", err) } p := &T{ baseCID: sarama.RootCID.NewChild("producer"), saramaClient: saramaClient, saramaProducer: saramaProducer, shutdownTimeout: cfg.Producer.ShutdownTimeout, deadMessageCh: cfg.Producer.DeadMessageCh, dispatcherCh: make(chan *sarama.ProducerMessage, cfg.Producer.ChannelBufferSize), resultCh: make(chan produceResult, cfg.Producer.ChannelBufferSize), } spawn(&p.wg, p.merge) spawn(&p.wg, p.dispatch) return p, nil }
func NewTestKafkaClient(brokers []string) *TestKafkaClient { tkc := &TestKafkaClient{} clientCfg := sarama.NewConfig() clientCfg.ClientID = "unittest-runner" err := error(nil) if tkc.client, err = sarama.NewClient(brokers, clientCfg); err != nil { panic(err) } if tkc.consumer, err = sarama.NewConsumerFromClient(tkc.client); err != nil { panic(err) } return tkc }
// Spawn creates a consumer instance with the specified configuration and // starts all its goroutines. func Spawn(cfg *config.T) (*T, error) { saramaCfg := sarama.NewConfig() saramaCfg.ClientID = cfg.ClientID saramaCfg.ChannelBufferSize = cfg.Consumer.ChannelBufferSize saramaCfg.Consumer.Offsets.CommitInterval = 50 * time.Millisecond saramaCfg.Consumer.Retry.Backoff = cfg.Consumer.BackOffTimeout saramaCfg.Consumer.Fetch.Default = 1024 * 1024 kafkaClient, err := sarama.NewClient(cfg.Kafka.SeedPeers, saramaCfg) if err != nil { return nil, ErrSetup(fmt.Errorf("failed to create sarama.Client: err=(%v)", err)) } offsetMgr, err := sarama.NewOffsetManagerFromClient(kafkaClient) if err != nil { return nil, ErrSetup(fmt.Errorf("failed to create sarama.OffsetManager: err=(%v)", err)) } kazooCfg := kazoo.NewConfig() kazooCfg.Chroot = cfg.ZooKeeper.Chroot // ZooKeeper documentation says following about the session timeout: "The // current (ZooKeeper) implementation requires that the timeout be a // minimum of 2 times the tickTime (as set in the server configuration) and // a maximum of 20 times the tickTime". The default tickTime is 2 seconds. // See http://zookeeper.apache.org/doc/trunk/zookeeperProgrammers.html#ch_zkSessions kazooCfg.Timeout = 15 * time.Second kazooConn, err := kazoo.NewKazoo(cfg.ZooKeeper.SeedPeers, kazooCfg) if err != nil { return nil, ErrSetup(fmt.Errorf("failed to create kazoo.Kazoo: err=(%v)", err)) } sc := &T{ baseCID: sarama.RootCID.NewChild("smartConsumer"), cfg: cfg, kafkaClient: kafkaClient, offsetMgr: offsetMgr, kazooConn: kazooConn, } sc.dispatcher = newDispatcher(sc.baseCID, sc, sc.cfg) sc.dispatcher.start() return sc, nil }
func NewKafkaHelper(c *C) *KafkaHelper { kh := &KafkaHelper{c: c} cfg := sarama.NewConfig() cfg.Producer.Return.Successes = true cfg.Producer.Return.Errors = true cfg.Consumer.Offsets.CommitInterval = 50 * time.Millisecond cfg.ClientID = "unittest-runner" err := error(nil) if kh.client, err = sarama.NewClient(KafkaPeers, cfg); err != nil { panic(err) } if kh.consumer, err = sarama.NewConsumerFromClient(kh.client); err != nil { panic(err) } if kh.producer, err = sarama.NewAsyncProducerFromClient(kh.client); err != nil { panic(err) } if kh.offsetMgr, err = sarama.NewOffsetManagerFromClient(kh.client); err != nil { panic(err) } return kh }
// SpawnGracefulProducer creates a `KafkaProducer` instance and starts its internal // goroutines. func SpawnGracefulProducer(config *config.T) (*GracefulProducer, error) { saramaClient, err := sarama.NewClient(config.Kafka.SeedPeers, config.SaramaConfig()) if err != nil { return nil, fmt.Errorf("failed to create sarama.Client, err=(%s)", err) } saramaProducer, err := sarama.NewAsyncProducerFromClient(saramaClient) if err != nil { return nil, fmt.Errorf("failed to create sarama.Producer, err=(%s)", err) } gp := &GracefulProducer{ baseCID: sarama.RootCID.NewChild("producer"), saramaClient: saramaClient, saramaProducer: saramaProducer, shutdownTimeout: config.Producer.ShutdownTimeout, deadMessageCh: config.Producer.DeadMessageCh, dispatcherCh: make(chan *sarama.ProducerMessage, config.ChannelBufferSize), resultCh: make(chan produceResult, config.ChannelBufferSize), } spawn(&gp.wg, gp.merge) spawn(&gp.wg, gp.dispatch) return gp, nil }
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() }
// SpawnSmartConsumer creates a SmartConsumer instance with the specified // configuration and starts all its goroutines. func SpawnSmartConsumer(config *Config) (*SmartConsumer, error) { kafkaClient, err := sarama.NewClient(config.Kafka.SeedPeers, config.saramaConfig()) if err != nil { return nil, ErrConsumerSetup(fmt.Errorf("failed to create sarama.Client: err=(%v)", err)) } offsetMgr, err := sarama.NewOffsetManagerFromClient(kafkaClient) if err != nil { return nil, ErrConsumerSetup(fmt.Errorf("failed to create sarama.OffsetManager: err=(%v)", err)) } kazooConn, err := kazoo.NewKazoo(config.ZooKeeper.SeedPeers, config.kazooConfig()) if err != nil { return nil, ErrConsumerSetup(fmt.Errorf("failed to create kazoo.Kazoo: err=(%v)", err)) } sc := &SmartConsumer{ baseCID: sarama.RootCID.NewChild("smartConsumer"), config: config, kafkaClient: kafkaClient, offsetMgr: offsetMgr, kazooConn: kazooConn, } sc.dispatcher = newDispatcher(sc.baseCID, sc, sc.config) sc.dispatcher.start() return sc, nil }
// GetGroupOffsets for every partition of the specified topic it returns the // current offset range along with the latest offset and metadata committed by // the specified consumer group. func (a *T) GetGroupOffsets(group, topic string) ([]PartitionOffset, error) { kafkaClt, err := sarama.NewClient(a.cfg.Kafka.SeedPeers, a.saramaConfig()) if err != nil { return nil, ErrSetup(fmt.Errorf("failed to create sarama.Client: err=(%v)", err)) } defer kafkaClt.Close() partitions, err := kafkaClt.Partitions(topic) if err != nil { return nil, NewErrQuery(err, "failed to get topic partitions") } // Figure out distribution of partitions among brokers. brokerToPartitions := make(map[*sarama.Broker][]indexedPartition) for i, p := range partitions { broker, err := kafkaClt.Leader(topic, p) if err != nil { return nil, NewErrQuery(err, "failed to get partition leader: partition=%d", p) } brokerToPartitions[broker] = append(brokerToPartitions[broker], indexedPartition{i, p}) } // Query brokers for the oldest and newest offsets of the partitions that // they are leaders for. offsets := make([]PartitionOffset, len(partitions)) var wg sync.WaitGroup errorsCh := make(chan ErrQuery, len(brokerToPartitions)) for broker, brokerPartitions := range brokerToPartitions { broker, brokerPartitions := broker, brokerPartitions var reqNewest sarama.OffsetRequest var reqOldest sarama.OffsetRequest for _, p := range brokerPartitions { reqNewest.AddBlock(topic, p.partition, sarama.OffsetNewest, 1) reqOldest.AddBlock(topic, p.partition, sarama.OffsetOldest, 1) } wg.Add(1) go func() { defer wg.Done() resOldest, err := broker.GetAvailableOffsets(&reqOldest) if err != nil { errorsCh <- NewErrQuery(err, "failed to fetch oldest offset: broker=%v", broker.ID()) return } resNewest, err := broker.GetAvailableOffsets(&reqNewest) if err != nil { errorsCh <- NewErrQuery(err, "failed to fetch newest offset: broker=%v", broker.ID()) return } for _, xp := range brokerPartitions { begin, err := getOffsetResult(resOldest, topic, xp.partition) if err != nil { errorsCh <- NewErrQuery(err, "failed to fetch oldest offset: broker=%v", broker.ID()) return } end, err := getOffsetResult(resNewest, topic, xp.partition) if err != nil { errorsCh <- NewErrQuery(err, "failed to fetch newest offset: broker=%v", broker.ID()) return } offsets[xp.index].Partition = xp.partition offsets[xp.index].Begin = begin offsets[xp.index].End = end } }() } wg.Wait() // If we failed to get offset range for at least one of the partitions then // return the first error that was reported. close(errorsCh) if err, ok := <-errorsCh; ok { return nil, err } // Fetch the last committed offsets for all partitions of the group/topic. coordinator, err := kafkaClt.Coordinator(group) if err != nil { return nil, NewErrQuery(err, "failed to get coordinator") } req := sarama.OffsetFetchRequest{ConsumerGroup: group, Version: ProtocolVer1} for _, p := range partitions { req.AddPartition(topic, p) } res, err := coordinator.FetchOffset(&req) if err != nil { return nil, NewErrQuery(err, "failed to fetch offsets") } for i, p := range partitions { block := res.GetBlock(topic, p) if block == nil { return nil, NewErrQuery(nil, "offset block is missing: partition=%d", p) } offsets[i].Offset = block.Offset offsets[i].Metadata = block.Metadata } return offsets, nil }