diff --git a/async_producer.go b/async_producer.go index 7574665bc..ea1cb3f5e 100644 --- a/async_producer.go +++ b/async_producer.go @@ -49,65 +49,27 @@ type AsyncProducer interface { // you can set Producer.Return.Errors in your config to false, which prevents // errors to be returned. Errors() <-chan *ProducerError -} - -// transactionManager keeps the state necessary to ensure idempotent production -type transactionManager struct { - producerID int64 - producerEpoch int16 - sequenceNumbers map[string]int32 - mutex sync.Mutex -} - -const ( - noProducerID = -1 - noProducerEpoch = -1 -) -func (t *transactionManager) getAndIncrementSequenceNumber(topic string, partition int32) (int32, int16) { - key := fmt.Sprintf("%s-%d", topic, partition) - t.mutex.Lock() - defer t.mutex.Unlock() - sequence := t.sequenceNumbers[key] - t.sequenceNumbers[key] = sequence + 1 - return sequence, t.producerEpoch -} + // IsTransactional return true when current producer is is transactional. + IsTransactional() bool -func (t *transactionManager) bumpEpoch() { - t.mutex.Lock() - defer t.mutex.Unlock() - t.producerEpoch++ - for k := range t.sequenceNumbers { - t.sequenceNumbers[k] = 0 - } -} + // TxnStatus return current producer transaction status. + TxnStatus() ProducerTxnStatusFlag -func (t *transactionManager) getProducerID() (int64, int16) { - t.mutex.Lock() - defer t.mutex.Unlock() - return t.producerID, t.producerEpoch -} + // BeginTxn mark current transaction as ready. + BeginTxn() error -func newTransactionManager(conf *Config, client Client) (*transactionManager, error) { - txnmgr := &transactionManager{ - producerID: noProducerID, - producerEpoch: noProducerEpoch, - } + // CommitTxn commit current transaction. + CommitTxn() error - if conf.Producer.Idempotent { - initProducerIDResponse, err := client.InitProducerID() - if err != nil { - return nil, err - } - txnmgr.producerID = initProducerIDResponse.ProducerID - txnmgr.producerEpoch = initProducerIDResponse.ProducerEpoch - txnmgr.sequenceNumbers = make(map[string]int32) - txnmgr.mutex = sync.Mutex{} + // AbortTxn abort current transaction. + AbortTxn() error - Logger.Printf("Obtained a ProducerId: %d and ProducerEpoch: %d\n", txnmgr.producerID, txnmgr.producerEpoch) - } + // AddOffsetsToTxn add associated offsets to current transaction. + AddOffsetsToTxn(offsets map[string][]*PartitionOffsetMetadata, groupId string) error - return txnmgr, nil + // AddMessageToTxn add message offsets to current transaction. + AddMessageToTxn(msg *ConsumerMessage, groupId string, metadata *string) error } type asyncProducer struct { @@ -123,6 +85,7 @@ type asyncProducer struct { brokerLock sync.Mutex txnmgr *transactionManager + txLock sync.Mutex metricsRegistry metrics.Registry } @@ -179,9 +142,12 @@ func newAsyncProducer(client Client) (AsyncProducer, error) { type flagSet int8 const ( - syn flagSet = 1 << iota // first message from partitionProducer to brokerProducer - fin // final message from partitionProducer to brokerProducer and back - shutdown // start the shutdown process + syn flagSet = 1 << iota // first message from partitionProducer to brokerProducer + fin // final message from partitionProducer to brokerProducer and back + shutdown // start the shutdown process + endtxn // endtxn + committxn // endtxn + aborttxn // endtxn ) // ProducerMessage is the collection of elements passed to the Producer in order to send a message. @@ -287,6 +253,97 @@ func (pe ProducerErrors) Error() string { return fmt.Sprintf("kafka: Failed to deliver %d messages.", len(pe)) } +func (p *asyncProducer) IsTransactional() bool { + return p.txnmgr.isTransactional() +} + +func (p *asyncProducer) AddMessageToTxn(msg *ConsumerMessage, groupId string, metadata *string) error { + offsets := make(map[string][]*PartitionOffsetMetadata) + offsets[msg.Topic] = []*PartitionOffsetMetadata{ + { + Partition: msg.Partition, + Offset: msg.Offset + 1, + Metadata: metadata, + }, + } + return p.AddOffsetsToTxn(offsets, groupId) +} + +func (p *asyncProducer) AddOffsetsToTxn(offsets map[string][]*PartitionOffsetMetadata, groupId string) error { + p.txLock.Lock() + defer p.txLock.Unlock() + + if !p.IsTransactional() { + DebugLogger.Printf("producer/txnmgr [%s] attempt to call AddOffsetsToTxn on a non-transactional producer\n", p.txnmgr.transactionalID) + return ErrNonTransactedProducer + } + + DebugLogger.Printf("producer/txnmgr [%s] add offsets to transaction\n", p.txnmgr.transactionalID) + return p.txnmgr.addOffsetsToTxn(offsets, groupId) +} + +func (p *asyncProducer) TxnStatus() ProducerTxnStatusFlag { + return p.txnmgr.currentTxnStatus() +} + +func (p *asyncProducer) BeginTxn() error { + p.txLock.Lock() + defer p.txLock.Unlock() + + if !p.IsTransactional() { + DebugLogger.Println("producer/txnmgr attempt to call BeginTxn on a non-transactional producer") + return ErrNonTransactedProducer + } + + return p.txnmgr.transitionTo(ProducerTxnFlagInTransaction, nil) +} + +func (p *asyncProducer) CommitTxn() error { + p.txLock.Lock() + defer p.txLock.Unlock() + + if !p.IsTransactional() { + DebugLogger.Printf("producer/txnmgr [%s] attempt to call CommitTxn on a non-transactional producer\n", p.txnmgr.transactionalID) + return ErrNonTransactedProducer + } + + DebugLogger.Printf("producer/txnmgr [%s] committing transaction\n", p.txnmgr.transactionalID) + err := p.finishTransaction(true) + if err != nil { + return err + } + DebugLogger.Printf("producer/txnmgr [%s] transaction committed\n", p.txnmgr.transactionalID) + return nil +} + +func (p *asyncProducer) AbortTxn() error { + p.txLock.Lock() + defer p.txLock.Unlock() + + if !p.IsTransactional() { + DebugLogger.Printf("producer/txnmgr [%s] attempt to call AbortTxn on a non-transactional producer\n", p.txnmgr.transactionalID) + return ErrNonTransactedProducer + } + DebugLogger.Printf("producer/txnmgr [%s] aborting transaction\n", p.txnmgr.transactionalID) + err := p.finishTransaction(false) + if err != nil { + return err + } + DebugLogger.Printf("producer/txnmgr [%s] transaction aborted\n", p.txnmgr.transactionalID) + return nil +} + +func (p *asyncProducer) finishTransaction(commit bool) error { + p.inFlight.Add(1) + if commit { + p.input <- &ProducerMessage{flags: endtxn | committxn} + } else { + p.input <- &ProducerMessage{flags: endtxn | aborttxn} + } + p.inFlight.Wait() + return p.txnmgr.finishTransaction(commit) +} + func (p *asyncProducer) Errors() <-chan *ProducerError { return p.errors } @@ -340,11 +397,27 @@ func (p *asyncProducer) dispatcher() { continue } + if msg.flags&endtxn != 0 { + var err error + if msg.flags&committxn != 0 { + err = p.txnmgr.transitionTo(ProducerTxnFlagEndTransaction|ProducerTxnFlagCommittingTransaction, nil) + } else { + err = p.txnmgr.transitionTo(ProducerTxnFlagEndTransaction|ProducerTxnFlagAbortingTransaction, nil) + } + if err != nil { + Logger.Printf("producer/txnmgr unable to end transaction %s", err) + } + p.inFlight.Done() + continue + } + if msg.flags&shutdown != 0 { shuttingDown = true p.inFlight.Done() continue - } else if msg.retries == 0 { + } + + if msg.retries == 0 { if shuttingDown { // we can't just call returnError here because that decrements the wait group, // which hasn't been incremented yet for this message, and shouldn't be @@ -357,6 +430,13 @@ func (p *asyncProducer) dispatcher() { continue } p.inFlight.Add(1) + // Ignore retried msg, there are already in txn. + // Can't produce new record when transaction is not started. + if p.IsTransactional() && p.txnmgr.currentTxnStatus()&ProducerTxnFlagInTransaction == 0 { + Logger.Printf("attempt to send message when transaction is not started or is in ending state, got %d, expect %d\n", p.txnmgr.currentTxnStatus(), ProducerTxnFlagInTransaction) + p.returnError(msg, ErrTransactionNotReady) + continue + } } for _, interceptor := range p.conf.Producer.Interceptors { @@ -609,6 +689,10 @@ func (pp *partitionProducer) dispatch() { msg.hasSequence = true } + if pp.parent.IsTransactional() { + pp.parent.txnmgr.maybeAddPartitionToCurrentTxn(pp.topic, pp.partition) + } + pp.brokerProducer.input <- msg } } @@ -719,6 +803,16 @@ func (p *asyncProducer) newBrokerProducer(broker *Broker) *brokerProducer { } }(set) + if p.IsTransactional() { + // Add partition to tx before sending current batch + err := p.txnmgr.publishTxnPartitions() + if err != nil { + // Request failed to be sent + sendResponse(nil, err) + continue + } + } + // Use AsyncProduce vs Produce to not block waiting for the response // so that we can pipeline multiple produce requests and achieve higher throughput, see: // https://kafka.apache.org/protocol#protocol_network @@ -1158,10 +1252,26 @@ func (p *asyncProducer) bumpIdempotentProducerEpoch() { } } +func (p *asyncProducer) maybeTransitionToErrorState(err error) error { + if errors.Is(err, ErrClusterAuthorizationFailed) || + errors.Is(err, ErrProducerFenced) || + errors.Is(err, ErrUnsupportedVersion) || + errors.Is(err, ErrTransactionalIDAuthorizationFailed) { + return p.txnmgr.transitionTo(ProducerTxnFlagInError|ProducerTxnFlagFatalError, err) + } + if p.txnmgr.coordinatorSupportsBumpingEpoch && p.txnmgr.currentTxnStatus()&ProducerTxnFlagEndTransaction == 0 { + p.txnmgr.epochBumpRequired = true + } + return p.txnmgr.transitionTo(ProducerTxnFlagInError|ProducerTxnFlagAbortableError, err) +} + func (p *asyncProducer) returnError(msg *ProducerMessage, err error) { + if p.IsTransactional() { + _ = p.maybeTransitionToErrorState(err) + } // We need to reset the producer ID epoch if we set a sequence number on it, because the broker // will never see a message with this number, so we can never continue the sequence. - if msg.hasSequence { + if !p.IsTransactional() && msg.hasSequence { Logger.Printf("producer/txnmanager rolling over epoch due to publish failure on %s/%d", msg.Topic, msg.Partition) p.bumpIdempotentProducerEpoch() } diff --git a/async_producer_test.go b/async_producer_test.go index 1c0abb786..41fb88f96 100644 --- a/async_producer_test.go +++ b/async_producer_test.go @@ -14,6 +14,7 @@ import ( "github.com/fortytw2/leaktest" "github.com/rcrowley/go-metrics" + "github.com/stretchr/testify/require" ) const TestMessage = "ABC THE MESSAGE" @@ -1662,6 +1663,507 @@ func TestProducerError(t *testing.T) { } } +func TestTxmngInitProducerId(t *testing.T) { + broker := NewMockBroker(t, 1) + defer broker.Close() + + metadataLeader := new(MetadataResponse) + metadataLeader.Version = 1 + metadataLeader.AddBroker(broker.Addr(), broker.BrokerID()) + broker.Returns(metadataLeader) + + config := NewTestConfig() + config.Producer.Idempotent = true + config.Version = V0_11_0_0 + config.Producer.RequiredAcks = WaitForAll + config.Net.MaxOpenRequests = 1 + + client, err := NewClient([]string{broker.Addr()}, config) + require.NoError(t, err) + defer client.Close() + + producerIdResponse := &InitProducerIDResponse{ + Err: ErrNoError, + ProducerID: 1, + ProducerEpoch: 0, + } + broker.Returns(producerIdResponse) + + txmng, err := newTransactionManager(config, client) + require.NoError(t, err) + + require.Equal(t, int64(1), txmng.producerID) + require.Equal(t, int16(0), txmng.producerEpoch) +} + +func TestTxnProduceBumpEpoch(t *testing.T) { + broker := NewMockBroker(t, 1) + defer broker.Close() + + config := NewTestConfig() + config.Producer.Idempotent = true + config.Producer.Transaction.ID = "test" + config.Version = V2_6_0_0 + config.Producer.RequiredAcks = WaitForAll + config.Net.MaxOpenRequests = 1 + config.Producer.Return.Errors = false + + config.ApiVersionsRequest = false + + metadataLeader := new(MetadataResponse) + metadataLeader.Version = 5 + metadataLeader.ControllerID = broker.brokerID + metadataLeader.AddBroker(broker.Addr(), broker.BrokerID()) + metadataLeader.AddTopic("test-topic", ErrNoError) + metadataLeader.AddTopicPartition("test-topic", 0, broker.BrokerID(), nil, nil, nil, ErrNoError) + broker.Returns(metadataLeader) + + client, err := NewClient([]string{broker.Addr()}, config) + require.NoError(t, err) + defer client.Close() + + findCoordinatorResponse := FindCoordinatorResponse{ + Coordinator: client.Brokers()[0], + Err: ErrNoError, + Version: 1, + } + broker.Returns(&findCoordinatorResponse) + + producerIdResponse := &InitProducerIDResponse{ + Err: ErrNoError, + ProducerID: 1000, + ProducerEpoch: 0, + Version: 3, + } + broker.Returns(producerIdResponse) + + ap, err := NewAsyncProducerFromClient(client) + producer := ap.(*asyncProducer) + require.NoError(t, err) + defer ap.Close() + require.Equal(t, int64(1000), producer.txnmgr.producerID) + require.Equal(t, int16(0), producer.txnmgr.producerEpoch) + + addPartitionsToTxnResponse := &AddPartitionsToTxnResponse{ + Errors: map[string][]*PartitionError{ + "test-topic": { + { + Partition: 0, + }, + }, + }, + } + broker.Returns(addPartitionsToTxnResponse) + + produceResponse := new(ProduceResponse) + produceResponse.Version = 3 + produceResponse.AddTopicPartition("test-topic", 0, ErrOutOfOrderSequenceNumber) + broker.Returns(produceResponse) + + err = producer.BeginTxn() + require.NoError(t, err) + + producer.Input() <- &ProducerMessage{Topic: "test-topic", Key: nil, Value: StringEncoder(TestMessage)} + + // Force send + producer.inFlight.Add(1) + producer.Input() <- &ProducerMessage{flags: shutdown} + producer.inFlight.Wait() + + err = producer.CommitTxn() + require.Error(t, err) + require.Equal(t, ProducerTxnFlagInError|ProducerTxnFlagAbortableError, producer.txnmgr.status) + + err = producer.CommitTxn() + require.Error(t, err) + require.Equal(t, ProducerTxnFlagInError|ProducerTxnFlagAbortableError, producer.txnmgr.status) + + endTxnResponse := &EndTxnResponse{ + Err: ErrNoError, + } + broker.Returns(endTxnResponse) + + producerBumpIdResponse := &InitProducerIDResponse{ + Err: ErrNoError, + ProducerID: 1000, + ProducerEpoch: 1, + Version: 3, + } + broker.Returns(producerBumpIdResponse) + + err = producer.AbortTxn() + require.NoError(t, err) + require.Equal(t, ProducerTxnFlagReady, producer.txnmgr.status) + require.Equal(t, int64(1000), producer.txnmgr.producerID) + require.Equal(t, int16(1), producer.txnmgr.producerEpoch) +} + +func TestTxnProduceRecordWithCommit(t *testing.T) { + broker := NewMockBroker(t, 1) + defer broker.Close() + + config := NewTestConfig() + config.Producer.Idempotent = true + config.Producer.Transaction.ID = "test" + config.Version = V0_11_0_0 + config.Producer.RequiredAcks = WaitForAll + config.Net.MaxOpenRequests = 1 + + metadataLeader := new(MetadataResponse) + metadataLeader.Version = 1 + metadataLeader.ControllerID = broker.brokerID + metadataLeader.AddBroker(broker.Addr(), broker.BrokerID()) + metadataLeader.AddTopic("test-topic", ErrNoError) + metadataLeader.AddTopicPartition("test-topic", 0, broker.BrokerID(), nil, nil, nil, ErrNoError) + broker.Returns(metadataLeader) + + client, err := NewClient([]string{broker.Addr()}, config) + require.NoError(t, err) + defer client.Close() + + findCoordinatorResponse := FindCoordinatorResponse{ + Coordinator: client.Brokers()[0], + Err: ErrNoError, + Version: 1, + } + broker.Returns(&findCoordinatorResponse) + + producerIdResponse := &InitProducerIDResponse{ + Err: ErrNoError, + ProducerID: 1, + ProducerEpoch: 0, + } + broker.Returns(producerIdResponse) + + ap, err := NewAsyncProducerFromClient(client) + producer := ap.(*asyncProducer) + require.NoError(t, err) + defer ap.Close() + + addPartitionsToTxnResponse := &AddPartitionsToTxnResponse{ + Errors: map[string][]*PartitionError{ + "test-topic": { + { + Partition: 0, + }, + }, + }, + } + broker.Returns(addPartitionsToTxnResponse) + + produceResponse := new(ProduceResponse) + produceResponse.Version = 3 + produceResponse.AddTopicPartition("test-topic", 0, ErrNoError) + broker.Returns(produceResponse) + + endTxnResponse := &EndTxnResponse{ + Err: ErrNoError, + } + broker.Returns(endTxnResponse) + + require.Equal(t, ProducerTxnFlagReady, producer.txnmgr.status) + + err = producer.BeginTxn() + require.NoError(t, err) + require.Equal(t, ProducerTxnFlagInTransaction, producer.txnmgr.status) + + producer.Input() <- &ProducerMessage{Topic: "test-topic", Key: nil, Value: StringEncoder(TestMessage)} + err = producer.CommitTxn() + require.NoError(t, err) + require.Equal(t, ProducerTxnFlagReady, producer.txnmgr.status) +} + +func TestTxnProduceBatchAddPartition(t *testing.T) { + broker := NewMockBroker(t, 1) + defer broker.Close() + + config := NewTestConfig() + config.Producer.Idempotent = true + config.Producer.Transaction.ID = "test" + config.Version = V0_11_0_0 + config.Producer.RequiredAcks = WaitForAll + config.Net.MaxOpenRequests = 1 + + config.Producer.Retry.Max = 1 + config.Producer.Flush.Messages = 3 + config.Producer.Flush.Frequency = 30 * time.Second + config.Producer.Flush.Bytes = 1 << 12 + config.Producer.Partitioner = NewManualPartitioner + + metadataLeader := new(MetadataResponse) + metadataLeader.Version = 1 + metadataLeader.ControllerID = broker.brokerID + metadataLeader.AddBroker(broker.Addr(), broker.BrokerID()) + metadataLeader.AddTopic("test-topic", ErrNoError) + metadataLeader.AddTopicPartition("test-topic", 0, broker.BrokerID(), nil, nil, nil, ErrNoError) + metadataLeader.AddTopicPartition("test-topic", 1, broker.BrokerID(), nil, nil, nil, ErrNoError) + metadataLeader.AddTopicPartition("test-topic", 2, broker.BrokerID(), nil, nil, nil, ErrNoError) + broker.Returns(metadataLeader) + + client, err := NewClient([]string{broker.Addr()}, config) + require.NoError(t, err) + defer client.Close() + + findCoordinatorResponse := FindCoordinatorResponse{ + Coordinator: client.Brokers()[0], + Err: ErrNoError, + Version: 1, + } + broker.Returns(&findCoordinatorResponse) + + producerIdResponse := &InitProducerIDResponse{ + Err: ErrNoError, + ProducerID: 1, + ProducerEpoch: 0, + } + broker.Returns(producerIdResponse) + + ap, err := NewAsyncProducerFromClient(client) + producer := ap.(*asyncProducer) + require.NoError(t, err) + defer ap.Close() + + go func() { + for err := range producer.Errors() { + require.NoError(t, err) + } + }() + + broker.Returns(&AddPartitionsToTxnResponse{ + Errors: map[string][]*PartitionError{ + "test-topic": { + { + Partition: 0, + Err: ErrNoError, + }, + { + Partition: 1, + Err: ErrNoError, + }, + { + Partition: 2, + Err: ErrNoError, + }, + }, + }, + }) + + produceResponse := new(ProduceResponse) + produceResponse.Version = 3 + produceResponse.AddTopicPartition("test-topic", 0, ErrNoError) + produceResponse.AddTopicPartition("test-topic", 1, ErrNoError) + produceResponse.AddTopicPartition("test-topic", 2, ErrNoError) + broker.Returns(produceResponse) + + endTxnResponse := &EndTxnResponse{ + Err: ErrNoError, + } + broker.Returns(endTxnResponse) + + require.Equal(t, ProducerTxnFlagReady, producer.txnmgr.status) + + err = producer.BeginTxn() + require.NoError(t, err) + require.Equal(t, ProducerTxnFlagInTransaction, producer.txnmgr.status) + + producer.Input() <- &ProducerMessage{Topic: "test-topic", Partition: 0, Key: nil, Value: StringEncoder("partition-0")} + producer.Input() <- &ProducerMessage{Topic: "test-topic", Partition: 1, Key: nil, Value: StringEncoder("partition-1")} + producer.Input() <- &ProducerMessage{Topic: "test-topic", Partition: 2, Key: nil, Value: StringEncoder("partition-2")} + + err = producer.CommitTxn() + require.NoError(t, err) + require.Equal(t, ProducerTxnFlagReady, producer.txnmgr.status) + + produceExchange := broker.History()[len(broker.History())-2] + produceRequest := produceExchange.Request.(*ProduceRequest) + require.Equal(t, 3, len(produceRequest.records["test-topic"])) + + addPartitionExchange := broker.History()[len(broker.History())-3] + addpartitionRequest := addPartitionExchange.Request.(*AddPartitionsToTxnRequest) + require.Equal(t, 3, len(addpartitionRequest.TopicPartitions["test-topic"])) + require.Contains(t, addpartitionRequest.TopicPartitions["test-topic"], int32(0)) + require.Contains(t, addpartitionRequest.TopicPartitions["test-topic"], int32(1)) + require.Contains(t, addpartitionRequest.TopicPartitions["test-topic"], int32(2)) +} + +func TestTxnProduceRecordWithAbort(t *testing.T) { + broker := NewMockBroker(t, 1) + defer broker.Close() + + config := NewTestConfig() + config.Producer.Idempotent = true + config.Producer.Transaction.ID = "test" + config.Version = V0_11_0_0 + config.Producer.RequiredAcks = WaitForAll + config.Net.MaxOpenRequests = 1 + + metadataLeader := new(MetadataResponse) + metadataLeader.Version = 1 + metadataLeader.ControllerID = broker.brokerID + metadataLeader.AddBroker(broker.Addr(), broker.BrokerID()) + metadataLeader.AddTopic("test-topic", ErrNoError) + metadataLeader.AddTopicPartition("test-topic", 0, broker.BrokerID(), nil, nil, nil, ErrNoError) + broker.Returns(metadataLeader) + + client, err := NewClient([]string{broker.Addr()}, config) + require.NoError(t, err) + defer client.Close() + + findCoordinatorResponse := FindCoordinatorResponse{ + Coordinator: client.Brokers()[0], + Err: ErrNoError, + Version: 1, + } + broker.Returns(&findCoordinatorResponse) + + producerIdResponse := &InitProducerIDResponse{ + Err: ErrNoError, + ProducerID: 1, + ProducerEpoch: 0, + } + broker.Returns(producerIdResponse) + + ap, err := NewAsyncProducerFromClient(client) + producer := ap.(*asyncProducer) + require.NoError(t, err) + defer ap.Close() + + broker.Returns(&AddPartitionsToTxnResponse{ + Errors: map[string][]*PartitionError{ + "test-topic": { + { + Partition: 0, + Err: ErrNoError, + }, + }, + }, + }) + + produceResponse := new(ProduceResponse) + produceResponse.Version = 3 + produceResponse.AddTopicPartition("test-topic", 0, ErrNoError) + broker.Returns(produceResponse) + + endTxnResponse := &EndTxnResponse{ + Err: ErrNoError, + } + broker.Returns(endTxnResponse) + + require.Equal(t, ProducerTxnFlagReady, producer.txnmgr.status) + + err = producer.BeginTxn() + require.NoError(t, err) + require.Equal(t, ProducerTxnFlagInTransaction, producer.txnmgr.status) + + producer.Input() <- &ProducerMessage{Topic: "test-topic", Key: nil, Value: StringEncoder(TestMessage)} + err = producer.AbortTxn() + require.NoError(t, err) + require.Equal(t, ProducerTxnFlagReady, producer.txnmgr.status) +} + +func TestTxnCanAbort(t *testing.T) { + broker := NewMockBroker(t, 1) + defer broker.Close() + + config := NewTestConfig() + config.Producer.Idempotent = true + config.Producer.Transaction.ID = "test" + config.Version = V0_11_0_0 + config.Producer.RequiredAcks = WaitForAll + config.Producer.Return.Errors = false + config.Producer.Return.Successes = true + config.Producer.Retry.Backoff = 0 + config.Producer.Flush.Messages = 1 + config.Producer.Retry.Max = 1 + config.Net.MaxOpenRequests = 1 + + metadataLeader := new(MetadataResponse) + metadataLeader.Version = 1 + metadataLeader.ControllerID = broker.brokerID + metadataLeader.AddBroker(broker.Addr(), broker.BrokerID()) + metadataLeader.AddTopic("test-topic", ErrNoError) + metadataLeader.AddTopic("test-topic-2", ErrNoError) + metadataLeader.AddTopicPartition("test-topic", 0, broker.BrokerID(), nil, nil, nil, ErrNoError) + metadataLeader.AddTopicPartition("test-topic-2", 0, broker.BrokerID(), nil, nil, nil, ErrNoError) + broker.Returns(metadataLeader) + + client, err := NewClient([]string{broker.Addr()}, config) + require.NoError(t, err) + defer client.Close() + + findCoordinatorResponse := FindCoordinatorResponse{ + Coordinator: client.Brokers()[0], + Err: ErrNoError, + Version: 1, + } + broker.Returns(&findCoordinatorResponse) + + producerIdResponse := &InitProducerIDResponse{ + Err: ErrNoError, + ProducerID: 1, + ProducerEpoch: 0, + } + broker.Returns(producerIdResponse) + + ap, err := NewAsyncProducerFromClient(client) + producer := ap.(*asyncProducer) + require.NoError(t, err) + defer ap.Close() + + broker.Returns(&AddPartitionsToTxnResponse{ + Errors: map[string][]*PartitionError{ + "test-topic-2": { + { + Partition: 0, + Err: ErrNoError, + }, + }, + }, + }) + + produceResponse := new(ProduceResponse) + produceResponse.Version = 3 + produceResponse.AddTopicPartition("test-topic-2", 0, ErrNoError) + broker.Returns(produceResponse) + + broker.Returns(&AddPartitionsToTxnResponse{ + Errors: map[string][]*PartitionError{ + "test-topic": { + { + Partition: 0, + Err: ErrTopicAuthorizationFailed, + }, + }, + }, + }) + + // now broker is closed due to error. will now reopen it + broker.Returns(metadataLeader) + + endTxnResponse := &EndTxnResponse{ + Err: ErrNoError, + } + broker.Returns(endTxnResponse) + + require.Equal(t, ProducerTxnFlagReady, producer.txnmgr.status) + + err = producer.BeginTxn() + require.NoError(t, err) + require.Equal(t, ProducerTxnFlagInTransaction, producer.txnmgr.status) + + producer.Input() <- &ProducerMessage{Topic: "test-topic-2", Partition: 0, Key: nil, Value: StringEncoder(TestMessage)} + <-producer.Successes() + + producer.Input() <- &ProducerMessage{Topic: "test-topic", Partition: 0, Key: nil, Value: StringEncoder(TestMessage)} + + err = producer.CommitTxn() + require.Error(t, err) + require.NotEqual(t, producer.txnmgr.status&ProducerTxnFlagAbortableError, 0) + + err = producer.AbortTxn() + require.NoError(t, err) +} + // This example shows how to use the producer while simultaneously // reading the Errors channel to know about any failures. func ExampleAsyncProducer_select() { diff --git a/broker.go b/broker.go index d1ac71deb..5815fd567 100644 --- a/broker.go +++ b/broker.go @@ -274,6 +274,13 @@ func (b *Broker) Open(conf *Config) error { return nil } +func (b *Broker) ResponseSize() int { + b.lock.Lock() + defer b.lock.Unlock() + + return len(b.responses) +} + // Connected returns true if the broker is connected and false otherwise. If the broker is not // connected but it had tried to connect, the error from that connection attempt is also returned. func (b *Broker) Connected() (bool, error) { @@ -730,6 +737,7 @@ func (b *Broker) DeleteAcls(request *DeleteAclsRequest) (*DeleteAclsResponse, er // InitProducerID sends an init producer request and returns a response or error func (b *Broker) InitProducerID(request *InitProducerIDRequest) (*InitProducerIDResponse, error) { response := new(InitProducerIDResponse) + response.Version = request.version() err := b.sendAndReceive(request, response) if err != nil { diff --git a/client.go b/client.go index ee1760fe8..2b775a412 100644 --- a/client.go +++ b/client.go @@ -2,6 +2,7 @@ package sarama import ( "errors" + "math" "math/rand" "sort" "sync" @@ -87,9 +88,22 @@ type Client interface { // in local cache. This function only works on Kafka 0.8.2 and higher. RefreshCoordinator(consumerGroup string) error + // Coordinator returns the coordinating broker for a transaction id. It will + // return a locally cached value if it's available. You can call + // RefreshCoordinator to update the cached value. This function only works on + // Kafka 0.11.0.0 and higher. + TransactionCoordinator(transactionID string) (*Broker, error) + + // RefreshCoordinator retrieves the coordinator for a transaction id and stores it + // in local cache. This function only works on Kafka 0.11.0.0 and higher. + RefreshTransactionCoordinator(transactionID string) error + // InitProducerID retrieves information required for Idempotent Producer InitProducerID() (*InitProducerIDResponse, error) + // LeastLoadedBroker retrieves broker that has the least responses pending + LeastLoadedBroker() *Broker + // Close shuts down all broker connections managed by this client. It is required // to call this function before a client object passes out of scope, as it will // otherwise leak memory. You must close any Producers or Consumers using a client @@ -123,11 +137,12 @@ type client struct { seedBrokers []*Broker deadSeeds []*Broker - controllerID int32 // cluster controller broker id - brokers map[int32]*Broker // maps broker ids to brokers - metadata map[string]map[int32]*PartitionMetadata // maps topics to partition ids to metadata - metadataTopics map[string]none // topics that need to collect metadata - coordinators map[string]int32 // Maps consumer group names to coordinating broker IDs + controllerID int32 // cluster controller broker id + brokers map[int32]*Broker // maps broker ids to brokers + metadata map[string]map[int32]*PartitionMetadata // maps topics to partition ids to metadata + metadataTopics map[string]none // topics that need to collect metadata + coordinators map[string]int32 // Maps consumer group names to coordinating broker IDs + transactionCoordinators map[string]int32 // Maps transaction ids to coordinating broker IDs // If the number of partitions is large, we can get some churn calling cachedPartitions, // so the result is cached. It is important to update this value whenever metadata is changed @@ -165,6 +180,7 @@ func NewClient(addrs []string, conf *Config) (Client, error) { metadataTopics: make(map[string]none), cachedPartitionsResults: make(map[string][maxPartitionIndex][]int32), coordinators: make(map[string]int32), + transactionCoordinators: make(map[string]int32), } client.randomizeSeedBrokers(addrs) @@ -599,7 +615,7 @@ func (client *client) RefreshCoordinator(consumerGroup string) error { return ErrClosedClient } - response, err := client.getConsumerMetadata(consumerGroup, client.conf.Metadata.Retry.Max) + response, err := client.findCoordinator(consumerGroup, CoordinatorGroup, client.conf.Metadata.Retry.Max) if err != nil { return err } @@ -611,6 +627,45 @@ func (client *client) RefreshCoordinator(consumerGroup string) error { return nil } +func (client *client) TransactionCoordinator(transactionID string) (*Broker, error) { + if client.Closed() { + return nil, ErrClosedClient + } + + coordinator := client.cachedTransactionCoordinator(transactionID) + + if coordinator == nil { + if err := client.RefreshTransactionCoordinator(transactionID); err != nil { + return nil, err + } + coordinator = client.cachedTransactionCoordinator(transactionID) + } + + if coordinator == nil { + return nil, ErrConsumerCoordinatorNotAvailable + } + + _ = coordinator.Open(client.conf) + return coordinator, nil +} + +func (client *client) RefreshTransactionCoordinator(transactionID string) error { + if client.Closed() { + return ErrClosedClient + } + + response, err := client.findCoordinator(transactionID, CoordinatorTransaction, client.conf.Metadata.Retry.Max) + if err != nil { + return err + } + + client.lock.Lock() + defer client.lock.Unlock() + client.registerBroker(response.Coordinator) + client.transactionCoordinators[transactionID] = response.Coordinator.ID() + return nil +} + // private broker management helpers func (client *client) randomizeSeedBrokers(addrs []string) { @@ -709,6 +764,30 @@ func (client *client) anyBroker() *Broker { return nil } +func (client *client) LeastLoadedBroker() *Broker { + client.lock.RLock() + defer client.lock.RUnlock() + + if len(client.seedBrokers) > 0 { + _ = client.seedBrokers[0].Open(client.conf) + return client.seedBrokers[0] + } + + var leastLoadedBroker *Broker + pendingRequests := math.MaxInt + for _, broker := range client.brokers { + if pendingRequests > broker.ResponseSize() { + pendingRequests = broker.ResponseSize() + leastLoadedBroker = broker + } + } + + if leastLoadedBroker != nil { + _ = leastLoadedBroker.Open(client.conf) + } + return leastLoadedBroker +} + // private caching/lazy metadata helpers type partitionType int @@ -1045,6 +1124,15 @@ func (client *client) cachedCoordinator(consumerGroup string) *Broker { return nil } +func (client *client) cachedTransactionCoordinator(transactionID string) *Broker { + client.lock.RLock() + defer client.lock.RUnlock() + if coordinatorID, ok := client.transactionCoordinators[transactionID]; ok { + return client.brokers[coordinatorID] + } + return nil +} + func (client *client) cachedController() *Broker { client.lock.RLock() defer client.lock.RUnlock() @@ -1061,24 +1149,28 @@ func (client *client) computeBackoff(attemptsRemaining int) time.Duration { return client.conf.Metadata.Retry.Backoff } -func (client *client) getConsumerMetadata(consumerGroup string, attemptsRemaining int) (*FindCoordinatorResponse, error) { +func (client *client) findCoordinator(coordinatorKey string, coordinatorType CoordinatorType, attemptsRemaining int) (*FindCoordinatorResponse, error) { retry := func(err error) (*FindCoordinatorResponse, error) { if attemptsRemaining > 0 { backoff := client.computeBackoff(attemptsRemaining) Logger.Printf("client/coordinator retrying after %dms... (%d attempts remaining)\n", backoff/time.Millisecond, attemptsRemaining) time.Sleep(backoff) - return client.getConsumerMetadata(consumerGroup, attemptsRemaining-1) + return client.findCoordinator(coordinatorKey, coordinatorType, attemptsRemaining-1) } return nil, err } brokerErrors := make([]error, 0) for broker := client.anyBroker(); broker != nil; broker = client.anyBroker() { - DebugLogger.Printf("client/coordinator requesting coordinator for consumergroup %s from %s\n", consumerGroup, broker.Addr()) + DebugLogger.Printf("client/coordinator requesting coordinator for %s from %s\n", coordinatorKey, broker.Addr()) request := new(FindCoordinatorRequest) - request.CoordinatorKey = consumerGroup - request.CoordinatorType = CoordinatorGroup + request.CoordinatorKey = coordinatorKey + request.CoordinatorType = coordinatorType + + if client.conf.Version.IsAtLeast(V0_11_0_0) { + request.Version = 1 + } response, err := broker.FindCoordinator(request) if err != nil { @@ -1096,10 +1188,10 @@ func (client *client) getConsumerMetadata(consumerGroup string, attemptsRemainin } if errors.Is(response.Err, ErrNoError) { - DebugLogger.Printf("client/coordinator coordinator for consumergroup %s is #%d (%s)\n", consumerGroup, response.Coordinator.ID(), response.Coordinator.Addr()) + DebugLogger.Printf("client/coordinator coordinator for %s is #%d (%s)\n", coordinatorKey, response.Coordinator.ID(), response.Coordinator.Addr()) return response, nil } else if errors.Is(response.Err, ErrConsumerCoordinatorNotAvailable) { - Logger.Printf("client/coordinator coordinator for consumer group %s is not available\n", consumerGroup) + Logger.Printf("client/coordinator coordinator for %s is not available\n", coordinatorKey) // This is very ugly, but this scenario will only happen once per cluster. // The __consumer_offsets topic only has to be created one time. @@ -1108,10 +1200,16 @@ func (client *client) getConsumerMetadata(consumerGroup string, attemptsRemainin Logger.Printf("client/coordinator the __consumer_offsets topic is not initialized completely yet. Waiting 2 seconds...\n") time.Sleep(2 * time.Second) } + if coordinatorType == CoordinatorTransaction { + if _, err := client.Leader("__transaction_state", 0); err != nil { + Logger.Printf("client/coordinator the __transaction_state topic is not initialized completely yet. Waiting 2 seconds...\n") + time.Sleep(2 * time.Second) + } + } return retry(ErrConsumerCoordinatorNotAvailable) } else if errors.Is(response.Err, ErrGroupAuthorizationFailed) { - Logger.Printf("client was not authorized to access group %s while attempting to find coordinator", consumerGroup) + Logger.Printf("client was not authorized to access group %s while attempting to find coordinator", coordinatorKey) return retry(ErrGroupAuthorizationFailed) } else { return nil, response.Err diff --git a/config.go b/config.go index 85b9e8729..c3918ecfd 100644 --- a/config.go +++ b/config.go @@ -188,6 +188,28 @@ type Config struct { // If enabled, the producer will ensure that exactly one copy of each message is // written. Idempotent bool + // Transaction specify + Transaction struct { + // Used in transactions to identify an instance of a producer through restarts + ID string + // Amount of time a transaction can remain unresolved (neither committed nor aborted) + // default is 1 min + Timeout time.Duration + + Retry struct { + // The total number of times to retry sending a message (default 50). + // Similar to the `message.send.max.retries` setting of the JVM producer. + Max int + // How long to wait for the cluster to settle between retries + // (default 10ms). Similar to the `retry.backoff.ms` setting of the + // JVM producer. + Backoff time.Duration + // Called to compute backoff time dynamically. Useful for implementing + // more sophisticated backoff strategies. This takes precedence over + // `Backoff` if set. + BackoffFunc func(retries, maxRetries int) time.Duration + } + } // Return specifies what channels will be populated. If they are set to true, // you must read from the respective channels to prevent deadlock. If, @@ -500,6 +522,10 @@ func NewConfig() *Config { c.Producer.Return.Errors = true c.Producer.CompressionLevel = CompressionLevelDefault + c.Producer.Transaction.Timeout = 1 * time.Minute + c.Producer.Transaction.Retry.Max = 50 + c.Producer.Transaction.Retry.Backoff = 100 * time.Millisecond + c.Consumer.Fetch.Min = 1 c.Consumer.Fetch.Default = 1024 * 1024 c.Consumer.Retry.Backoff = 2 * time.Second @@ -726,6 +752,10 @@ func (c *Config) Validate() error { } } + if c.Producer.Transaction.ID != "" && !c.Producer.Idempotent { + return ConfigurationError("Transactional producer requires Idempotent to be true") + } + // validate the Consumer values switch { case c.Consumer.Fetch.Min <= 0: diff --git a/docker-compose.yml b/docker-compose.yml index 7a07e0a29..fb0a806c6 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -47,6 +47,7 @@ services: KAFKA_CFG_INTER_BROKER_LISTENER_NAME: 'LISTENER_INTERNAL' KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP: 'LISTENER_INTERNAL:PLAINTEXT,LISTENER_LOCAL:PLAINTEXT' KAFKA_CFG_DEFAULT_REPLICATION_FACTOR: '2' + KAFKA_CFG_TRANSACTION_STATE_LOG_REPLICATION_FACTOR: '2' KAFKA_CFG_BROKER_ID: '1' KAFKA_CFG_BROKER_RACK: '1' KAFKA_CFG_ZOOKEEPER_SESSION_TIMEOUT_MS: '6000' @@ -68,6 +69,7 @@ services: KAFKA_CFG_INTER_BROKER_LISTENER_NAME: 'LISTENER_INTERNAL' KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP: 'LISTENER_INTERNAL:PLAINTEXT,LISTENER_LOCAL:PLAINTEXT' KAFKA_CFG_DEFAULT_REPLICATION_FACTOR: '2' + KAFKA_CFG_TRANSACTION_STATE_LOG_REPLICATION_FACTOR: '2' KAFKA_CFG_BROKER_ID: '2' KAFKA_CFG_BROKER_RACK: '2' KAFKA_CFG_ZOOKEEPER_SESSION_TIMEOUT_MS: '6000' @@ -89,6 +91,7 @@ services: KAFKA_CFG_INTER_BROKER_LISTENER_NAME: 'LISTENER_INTERNAL' KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP: 'LISTENER_INTERNAL:PLAINTEXT,LISTENER_LOCAL:PLAINTEXT' KAFKA_CFG_DEFAULT_REPLICATION_FACTOR: '2' + KAFKA_CFG_TRANSACTION_STATE_LOG_REPLICATION_FACTOR: '2' KAFKA_CFG_BROKER_ID: '3' KAFKA_CFG_BROKER_RACK: '3' KAFKA_CFG_ZOOKEEPER_SESSION_TIMEOUT_MS: '6000' @@ -110,6 +113,7 @@ services: KAFKA_CFG_INTER_BROKER_LISTENER_NAME: 'LISTENER_INTERNAL' KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP: 'LISTENER_INTERNAL:PLAINTEXT,LISTENER_LOCAL:PLAINTEXT' KAFKA_CFG_DEFAULT_REPLICATION_FACTOR: '2' + KAFKA_CFG_TRANSACTION_STATE_LOG_REPLICATION_FACTOR: '2' KAFKA_CFG_BROKER_ID: '4' KAFKA_CFG_BROKER_RACK: '4' KAFKA_CFG_ZOOKEEPER_SESSION_TIMEOUT_MS: '6000' @@ -131,6 +135,7 @@ services: KAFKA_CFG_INTER_BROKER_LISTENER_NAME: 'LISTENER_INTERNAL' KAFKA_CFG_LISTENER_SECURITY_PROTOCOL_MAP: 'LISTENER_INTERNAL:PLAINTEXT,LISTENER_LOCAL:PLAINTEXT' KAFKA_CFG_DEFAULT_REPLICATION_FACTOR: '2' + KAFKA_CFG_TRANSACTION_STATE_LOG_REPLICATION_FACTOR: '2' KAFKA_CFG_BROKER_ID: '5' KAFKA_CFG_BROKER_RACK: '5' KAFKA_CFG_ZOOKEEPER_SESSION_TIMEOUT_MS: '6000' diff --git a/end_txn_response.go b/end_txn_response.go index 763976726..dd2a04504 100644 --- a/end_txn_response.go +++ b/end_txn_response.go @@ -32,7 +32,7 @@ func (e *EndTxnResponse) decode(pd packetDecoder, version int16) (err error) { } func (e *EndTxnResponse) key() int16 { - return 25 + return 26 } func (e *EndTxnResponse) version() int16 { diff --git a/errors.go b/errors.go index bb6409042..27977f166 100644 --- a/errors.go +++ b/errors.go @@ -67,6 +67,27 @@ var ErrDeleteRecords = errors.New("kafka server: failed to delete records") // ErrCreateACLs is the type of error returned when ACL creation failed var ErrCreateACLs = errors.New("kafka server: failed to create one or more ACL rules") +// ErrAddPartitionsToTxn is returned when AddPartitionsToTxn failed multiple times +var ErrAddPartitionsToTxn = errors.New("transaction manager: failed to send partitions to transaction") + +// ErrTxnOffsetCommit is returned when TxnOffsetCommit failed multiple times +var ErrTxnOffsetCommit = errors.New("transaction manager: failed to send offsets to transaction") + +// ErrTransactionNotReady when transaction status is invalid for the current action. +var ErrTransactionNotReady = errors.New("transaction manager: transaction is not ready") + +// ErrNonTransactedProducer when calling BeginTxn, CommitTxn or AbortTxn on a non transactional producer. +var ErrNonTransactedProducer = errors.New("transaction manager: you need to add TransactionalID to producer") + +// ErrTransitionNotAllowed when txnmgr state transiion is not valid. +var ErrTransitionNotAllowed = errors.New("transaction manager: invalid transition attempted") + +// ErrCannotTransitionNilError when transition is attempted with an nil error. +var ErrCannotTransitionNilError = errors.New("transaction manager: cannot transition with a nil error") + +// ErrTxnUnableToParseResponse when response is nil +var ErrTxnUnableToParseResponse = errors.New("transaction manager: unable to parse response") + // MultiErrorFormat specifies the formatter applied to format multierrors. The // default implementation is a consensed version of the hashicorp/go-multierror // default one @@ -242,6 +263,8 @@ const ( ErrGroupSubscribedToTopic KError = 86 ErrInvalidRecord KError = 87 ErrUnstableOffsetCommit KError = 88 + ErrThrottlingQuotaExceeded KError = 89 + ErrProducerFenced KError = 90 ) func (err KError) Error() string { diff --git a/examples/README.md b/examples/README.md index b523a6f0a..f50a325eb 100644 --- a/examples/README.md +++ b/examples/README.md @@ -11,3 +11,11 @@ In these examples, we use `github.com/Shopify/sarama` as import path. We do this #### Interceptors Basic example to use a producer interceptor that produces [OpenTelemetry](https://github.com/open-telemetry/opentelemetry-go/) spans and add some headers for each intercepted message. + +#### Transactional Producer + +[txn_producer](./txn_producer) Basic example to use a transactional producer that produce on some topic within a Kafka transaction. To ensure transactional-id uniqueness it implement some ***ProducerProvider*** that build a producer appending an integer that grow when producer is created. + +#### Exacly-once transactional paradigm + +[exactly_once](./exactly_once) Basic example to use a transactional producer that produce consumed message from some topics within a Kafka transaction. To ensure transactional-id uniqueness it implement some ***ProducerProvider*** that build a producer using current message topic-partition. diff --git a/examples/exactly_once/.gitignore b/examples/exactly_once/.gitignore new file mode 100644 index 000000000..9f2dfaffb --- /dev/null +++ b/examples/exactly_once/.gitignore @@ -0,0 +1 @@ +exactly_once diff --git a/examples/exactly_once/README.md b/examples/exactly_once/README.md new file mode 100644 index 000000000..0111bf65e --- /dev/null +++ b/examples/exactly_once/README.md @@ -0,0 +1,10 @@ +# Exactly-Once example + +This example shows you how to use the Sarama transactional producer to ensure exacly-once paradigm with Kafka transaction. The example simply starts consuming the given Kafka topics and produce the consumed message to another topic including current message offset in procuder transaction. + +```bash +$ go run main.go -brokers="127.0.0.1:9092" -topics="sarama" -destination-topic="destination-sarama" -group="example" +``` + +To ensure transactional-id uniqueness it implement some ***ProducerProvider*** that build a producer using current message topic-partition. +You can also toggle (pause/resume) the consumption by sending SIGUSR1. \ No newline at end of file diff --git a/examples/exactly_once/go.mod b/examples/exactly_once/go.mod new file mode 100644 index 000000000..da3acccca --- /dev/null +++ b/examples/exactly_once/go.mod @@ -0,0 +1,7 @@ +module github.com/Shopify/sarama/examples/exactly_once + +go 1.16 + +require github.com/Shopify/sarama v1.34.1 + +replace github.com/Shopify/sarama => ../../ diff --git a/examples/exactly_once/go.sum b/examples/exactly_once/go.sum new file mode 100644 index 000000000..ab9273a96 --- /dev/null +++ b/examples/exactly_once/go.sum @@ -0,0 +1,530 @@ +cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU= +cloud.google.com/go v0.44.1/go.mod h1:iSa0KzasP4Uvy3f1mN/7PiObzGgflwredwwASm/v6AU= +cloud.google.com/go v0.44.2/go.mod h1:60680Gw3Yr4ikxnPRS/oxxkBccT6SA1yMk63TGekxKY= +cloud.google.com/go v0.45.1/go.mod h1:RpBamKRgapWJb87xiFSdk4g1CME7QZg3uwTez+TSTjc= +cloud.google.com/go v0.46.3/go.mod h1:a6bKKbmY7er1mI7TEI4lsAkts/mkhTSZK8w33B4RAg0= +cloud.google.com/go v0.50.0/go.mod h1:r9sluTvynVuxRIOHXQEHMFffphuXHOMZMycpNR5e6To= +cloud.google.com/go v0.52.0/go.mod h1:pXajvRH/6o3+F9jDHZWQ5PbGhn+o8w9qiu/CffaVdO4= +cloud.google.com/go v0.53.0/go.mod h1:fp/UouUEsRkN6ryDKNW/Upv/JBKnv6WDthjR6+vze6M= +cloud.google.com/go v0.54.0/go.mod h1:1rq2OEkV3YMf6n/9ZvGWI3GWw0VoqH/1x2nd8Is/bPc= +cloud.google.com/go v0.56.0/go.mod h1:jr7tqZxxKOVYizybht9+26Z/gUq7tiRzu+ACVAMbKVk= +cloud.google.com/go v0.57.0/go.mod h1:oXiQ6Rzq3RAkkY7N6t3TcE6jE+CIBBbA36lwQ1JyzZs= +cloud.google.com/go v0.62.0/go.mod h1:jmCYTdRCQuc1PHIIJ/maLInMho30T/Y0M4hTdTShOYc= +cloud.google.com/go v0.65.0/go.mod h1:O5N8zS7uWy9vkA9vayVHs65eM1ubvY4h553ofrNHObY= +cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o= +cloud.google.com/go/bigquery v1.3.0/go.mod h1:PjpwJnslEMmckchkHFfq+HTD2DmtT67aNFKH1/VBDHE= +cloud.google.com/go/bigquery v1.4.0/go.mod h1:S8dzgnTigyfTmLBfrtrhyYhwRxG72rYxvftPBK2Dvzc= +cloud.google.com/go/bigquery v1.5.0/go.mod h1:snEHRnqQbz117VIFhE8bmtwIDY80NLUZUMb4Nv6dBIg= +cloud.google.com/go/bigquery v1.7.0/go.mod h1://okPTzCYNXSlb24MZs83e2Do+h+VXtc4gLoIoXIAPc= +cloud.google.com/go/bigquery v1.8.0/go.mod h1:J5hqkt3O0uAFnINi6JXValWIb1v0goeZM77hZzJN/fQ= +cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= +cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk= +cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= +cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+R3AArQw= +cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA= +cloud.google.com/go/pubsub v1.3.1/go.mod h1:i+ucay31+CNRpDW4Lu78I4xXG+O1r/MAHgjpRVR+TSU= +cloud.google.com/go/storage v1.0.0/go.mod h1:IhtSnM/ZTZV8YYJWCY8RULGVqBDmpoyjwiyrjsg+URw= +cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0ZeosJ0Rtdos= +cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohlUTyfDhBk= +cloud.google.com/go/storage v1.8.0/go.mod h1:Wv1Oy7z6Yz3DshWRJFhqM/UCfaWIRTdp0RXyy7KQOVs= +cloud.google.com/go/storage v1.10.0/go.mod h1:FLPqc6j+Ki4BU591ie1oL6qBQGu2Bl/tZ9ullr3+Kg0= +dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= +github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= +github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= +github.com/Shopify/toxiproxy/v2 v2.4.0 h1:O1e4Jfvr/hefNTNu+8VtdEG5lSeamJRo4aKhMOKNM64= +github.com/Shopify/toxiproxy/v2 v2.4.0/go.mod h1:3ilnjng821bkozDRxNoo64oI/DKqM+rOyJzb564+bvg= +github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= +github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= +github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= +github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= +github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho= +github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= +github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= +github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= +github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= +github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/cespare/xxhash/v2 v2.1.2/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= +github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= +github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= +github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= +github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= +github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= +github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= +github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= +github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/eapache/go-resiliency v1.3.0 h1:RRL0nge+cWGlxXbUzJ7yMcq6w2XBEr19dCN6HECGaT0= +github.com/eapache/go-resiliency v1.3.0/go.mod h1:5yPzW0MIvSe0JDsv0v+DvcjEv2FyD6iZYSs1ZI+iQho= +github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21 h1:YEetp8/yCZMuEPMUDHG0CW/brkkEp8mzqk2+ODEitlw= +github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21/go.mod h1:+020luEh2TKB4/GOp8oxxtq0Daoen/Cii55CzbTV6DU= +github.com/eapache/queue v1.1.0 h1:YOEu7KNc61ntiQlcEeUIoDTJ2o8mQznoNvUhiigpIqc= +github.com/eapache/queue v1.1.0/go.mod h1:6eCeP0CKFpHLu8blIFXhExK/dRa7WDZfr6jVFPTqq+I= +github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= +github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= +github.com/fortytw2/leaktest v1.3.0 h1:u8491cBMTQ8ft8aeV+adlcytMZylmA5nnwwkRZjI8vw= +github.com/fortytw2/leaktest v1.3.0/go.mod h1:jDsjWgpAGjm2CA7WthBh/CdZYEPF31XHquHwclZch5g= +github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= +github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= +github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= +github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= +github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= +github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= +github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= +github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= +github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= +github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= +github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= +github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= +github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= +github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= +github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y= +github.com/golang/mock v1.4.0/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= +github.com/golang/mock v1.4.1/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= +github.com/golang/mock v1.4.3/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= +github.com/golang/mock v1.4.4/go.mod h1:l3mdAwkq5BuhzHwde/uurv3sEJeZMXNpwsxVWU71h+4= +github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= +github.com/golang/protobuf v1.3.4/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= +github.com/golang/protobuf v1.3.5/go.mod h1:6O5/vntMXwX2lRkT1hjjk0nAC1IDOTvTlVgjlRvqsdk= +github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8= +github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA= +github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs= +github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w= +github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= +github.com/golang/protobuf v1.4.1/go.mod h1:U8fpvMrcmy5pZrNK1lt4xCsGvpyWQ/VVv6QDs8UjoX8= +github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= +github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= +github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= +github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= +github.com/golang/snappy v0.0.4 h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM= +github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= +github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= +github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= +github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= +github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.4.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= +github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= +github.com/google/martian/v3 v3.0.0/go.mod h1:y5Zk1BBys9G+gd6Jrk0W3cC1+ELVxBWuIGO+w/tUAp0= +github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= +github.com/google/pprof v0.0.0-20190515194954-54271f7e092f/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= +github.com/google/pprof v0.0.0-20191218002539-d4f498aebedc/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/pprof v0.0.0-20200212024743-f11f1df84d12/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/pprof v0.0.0-20200229191704-1ebb73c60ed3/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/pprof v0.0.0-20200430221834-fc25d7d30c6d/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/pprof v0.0.0-20200708004538-1a94d8640e99/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= +github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= +github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= +github.com/gorilla/mux v1.8.0/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= +github.com/gorilla/securecookie v1.1.1/go.mod h1:ra0sb63/xPlUeL+yeDciTfxMRAA+MP+HVt/4epWDjd4= +github.com/gorilla/sessions v1.2.1/go.mod h1:dk2InVEVJ0sfLlnXv9EAgkf6ecYs/i80K/zI+bUmuGM= +github.com/hashicorp/errwrap v1.0.0 h1:hLrqtEDnRye3+sgx6z4qVLNuviH3MR5aQ0ykNJa/UYA= +github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= +github.com/hashicorp/go-multierror v1.1.1 h1:H5DkEtf6CXdFp0N0Em5UCwQpXMWke8IA0+lD48awMYo= +github.com/hashicorp/go-multierror v1.1.1/go.mod h1:iw975J/qwKPdAO1clOe2L8331t/9/fmwbPZ6JB6eMoM= +github.com/hashicorp/go-uuid v1.0.2/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= +github.com/hashicorp/go-uuid v1.0.3 h1:2gKiV6YVmrJ1i2CKKa9obLvRieoRGviZFL26PcT/Co8= +github.com/hashicorp/go-uuid v1.0.3/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= +github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= +github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= +github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= +github.com/jcmturner/aescts/v2 v2.0.0 h1:9YKLH6ey7H4eDBXW8khjYslgyqG2xZikXP0EQFKrle8= +github.com/jcmturner/aescts/v2 v2.0.0/go.mod h1:AiaICIRyfYg35RUkr8yESTqvSy7csK90qZ5xfvvsoNs= +github.com/jcmturner/dnsutils/v2 v2.0.0 h1:lltnkeZGL0wILNvrNiVCR6Ro5PGU/SeBvVO/8c/iPbo= +github.com/jcmturner/dnsutils/v2 v2.0.0/go.mod h1:b0TnjGOvI/n42bZa+hmXL+kFJZsFT7G4t3HTlQ184QM= +github.com/jcmturner/gofork v1.7.6 h1:QH0l3hzAU1tfT3rZCnW5zXl+orbkNMMRGJfdJjHVETg= +github.com/jcmturner/gofork v1.7.6/go.mod h1:1622LH6i/EZqLloHfE7IeZ0uEJwMSUyQ/nDd82IeqRo= +github.com/jcmturner/goidentity/v6 v6.0.1 h1:VKnZd2oEIMorCTsFBnJWbExfNN7yZr3EhJAxwOkZg6o= +github.com/jcmturner/goidentity/v6 v6.0.1/go.mod h1:X1YW3bgtvwAXju7V3LCIMpY0Gbxyjn/mY9zx4tFonSg= +github.com/jcmturner/gokrb5/v8 v8.4.3 h1:iTonLeSJOn7MVUtyMT+arAn5AKAPrkilzhGw8wE/Tq8= +github.com/jcmturner/gokrb5/v8 v8.4.3/go.mod h1:dqRwJGXznQrzw6cWmyo6kH+E7jksEQG/CyVWsJEsJO0= +github.com/jcmturner/rpc/v2 v2.0.3 h1:7FXXj8Ti1IaVFpSAziCZWNzbNuZmnvw/i6CqLNdWfZY= +github.com/jcmturner/rpc/v2 v2.0.3/go.mod h1:VUJYCIDm3PVOEHw8sgt091/20OJjskO/YJki3ELg/Hc= +github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= +github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= +github.com/json-iterator/go v1.1.10/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= +github.com/json-iterator/go v1.1.11/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= +github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= +github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= +github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= +github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= +github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= +github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= +github.com/klauspost/compress v1.15.9 h1:wKRjX6JRtDdrE9qwa4b/Cip7ACOshUI4smpCQanqjSY= +github.com/klauspost/compress v1.15.9/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= +github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= +github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= +github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= +github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= +github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= +github.com/kr/pretty v0.3.0/go.mod h1:640gp4NfQd8pI5XOwp5fnNeVWj67G7CFk/SaSQn7NBk= +github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= +github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= +github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= +github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= +github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= +github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= +github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= +github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= +github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= +github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= +github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= +github.com/pierrec/lz4/v4 v4.1.15 h1:MO0/ucJhngq7299dKLwIMtgTfbkoSPF6AoMYDd8Q4q0= +github.com/pierrec/lz4/v4 v4.1.15/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= +github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= +github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= +github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= +github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= +github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= +github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= +github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP4j5+D6mVACh+pe2M= +github.com/prometheus/client_golang v1.11.0/go.mod h1:Z6t4BnS23TR94PD6BsDNk8yVqroYurpAkEiz0P2BEV0= +github.com/prometheus/client_golang v1.12.1/go.mod h1:3Z9XVyYiZYEO+YQWt3RD2R3jrbd179Rt297l4aS6nDY= +github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= +github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= +github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo= +github.com/prometheus/common v0.26.0/go.mod h1:M7rCNAaPfAosfx8veZJCuw84e35h3Cfd9VFqTh1DIvc= +github.com/prometheus/common v0.32.1/go.mod h1:vu+V0TpY+O6vW9J44gczi3Ap/oXXR10b+M/gUGO4Hls= +github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= +github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= +github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= +github.com/prometheus/procfs v0.6.0/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= +github.com/prometheus/procfs v0.7.3/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= +github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 h1:N/ElC8H3+5XpJzTSTfLsJV/mx9Q9g7kxmchpfZyxgzM= +github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= +github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= +github.com/rogpeppe/go-internal v1.6.1/go.mod h1:xXDCJY+GAPziupqXw64V24skbSoqbTEfhy4qGm1nDQc= +github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= +github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= +github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= +github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= +github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= +github.com/sirupsen/logrus v1.8.1/go.mod h1:yWOB1SBYBC5VeMP7gHvWumXLIWorT60ONWic61uBYv0= +github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= +github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= +github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= +github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= +github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= +github.com/stretchr/testify v1.8.0 h1:pSgiaMZlXftHpm5L7V1+rVB+AZJydKsMxsQBIJw4PKk= +github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= +github.com/urfave/cli/v2 v2.3.0/go.mod h1:LJmUH05zAU44vOAcrfzZQKsZbVcdbOG8rtL3/XcUArI= +github.com/xdg-go/pbkdf2 v1.0.0/go.mod h1:jrpuAogTd400dnrH08LKmI/xc1MbPOebTwRqcT5RDeI= +github.com/xdg-go/scram v1.1.1/go.mod h1:RaEWvsqvNKKvBPvcKeFjrG2cJqOkHTiyTpzz23ni57g= +github.com/xdg-go/stringprep v1.0.3/go.mod h1:W3f5j4i+9rC0kuIEJL0ky1VpHXQU3ocBgklLGvcBnW8= +github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= +go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= +go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= +go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= +go.opencensus.io v0.22.4/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= +golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= +golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20220722155217-630584e8d5aa h1:zuSxTR4o9y82ebqCUJYNGJbGPo6sKVl54f/TVDObg1c= +golang.org/x/crypto v0.0.0-20220722155217-630584e8d5aa/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= +golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= +golang.org/x/exp v0.0.0-20190829153037-c13cbed26979/go.mod h1:86+5VVa7VpoJ4kLfm080zCjGlMRFzhUhsZKEZO7MGek= +golang.org/x/exp v0.0.0-20191030013958-a1ab85dbe136/go.mod h1:JXzH8nQsPlswgeRAPE3MuO9GYsAcnJvJ4vnMwN/5qkY= +golang.org/x/exp v0.0.0-20191129062945-2f5052295587/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= +golang.org/x/exp v0.0.0-20191227195350-da58074b4299/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= +golang.org/x/exp v0.0.0-20200119233911-0405dc783f0a/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= +golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EHIKF9dgMWnmCNThgcyBT1FY9mM= +golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= +golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= +golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= +golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= +golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= +golang.org/x/lint v0.0.0-20190301231843-5614ed5bae6f/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= +golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20190409202823-959b441ac422/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20190909230951-414d861bb4ac/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20191125180803-fdd1cda4f05f/go.mod h1:5qLYkcX4OjUUV8bRuDixDT3tpyyb+LUpUlRWLxfhWrs= +golang.org/x/lint v0.0.0-20200130185559-910be7a94367/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= +golang.org/x/lint v0.0.0-20200302205851-738671d3881b/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= +golang.org/x/mobile v0.0.0-20190312151609-d3739f865fa6/go.mod h1:z+o9i4GpDbdi3rU15maQ/Ox0txvL9dWGYEHz965HBQE= +golang.org/x/mobile v0.0.0-20190719004257-d2bd2a29d028/go.mod h1:E/iHnbuqvinMTCcRqshq8CkpyQDoeVncDDYHnLhea+o= +golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= +golang.org/x/mod v0.1.0/go.mod h1:0QHyrYULN0/3qlju5TqG8bIK38QM8yzMo5ekMj3DlcY= +golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= +golang.org/x/mod v0.1.1-0.20191107180719-034126e5016b/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= +golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190501004415-9ce7a6920f09/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190603091049-60506f45cf65/go.mod h1:HSz+uSET+XFnRR8LxR5pz3Of3rY3CfYBVs4xY44aLks= +golang.org/x/net v0.0.0-20190613194153-d28f0bde5980/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20190628185345-da137c7871d7/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20190724013045-ca1201d0de80/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20191209160850-c0dbc17a3553/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200202094626-16171245cfb2/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200222125558-5a598a2470a0/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200301022130-244492dfa37a/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200324143707-d3edc9973b7e/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= +golang.org/x/net v0.0.0-20200501053045-e0ff5e5a1de5/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= +golang.org/x/net v0.0.0-20200506145744-7e3656a0809f/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= +golang.org/x/net v0.0.0-20200513185701-a91f0712d120/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= +golang.org/x/net v0.0.0-20200520182314-0ba52f642ac2/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= +golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= +golang.org/x/net v0.0.0-20200707034311-ab3426394381/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= +golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= +golang.org/x/net v0.0.0-20210525063256-abc453219eb5/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= +golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= +golang.org/x/net v0.0.0-20220725212005-46097bf591d3/go.mod h1:AaygXjzTFtRAg2ttMY5RMuhpJ3cNnI0XpyFJD1iQRSM= +golang.org/x/net v0.0.0-20220809184613-07c6da5e1ced h1:3dYNDff0VT5xj+mbj2XucFst9WKk6PdGOrb9n+SbIvw= +golang.org/x/net v0.0.0-20220809184613-07c6da5e1ced/go.mod h1:YDH+HFinaLZZlnHAfSS6ZXJJ9M9t4Dl22yv3iI2vPwk= +golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= +golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20191202225959-858c2ad4c8b6/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20210514164344-f6687ab2804c/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= +golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20200317015054-43a5402ce75a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4 h1:uVc8UZUe6tr40fFVnUP5Oj+veunVezqYl9z7DYw9xzw= +golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190502145724-3ef323f4f1fd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190507160741-ecd444e8653b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190606165138-5da285871e9c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190624142023-c5567b49c5d0/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190726091711-fc99dfbffb4e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191001151750-bb3f8db39f24/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191026070338-33540a1f6037/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191204072324-ce4227a45e2e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191228213918-04cbcbbfeed8/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200106162015-b016eb3dc98e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200113162924-86b910548bc1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200122134326-e047566fdf82/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200202164722-d101bd2416d5/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200212091648-12a6c2dcc1e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200223170610-d5e6a3e2c0ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200302150141-5c8b2ff67527/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200331124033-c3d80250170d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200501052902-10377860bb8e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200511232937-7e40ca221e25/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200515095857-1151b9dac4a9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200523222454-059865788121/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200615200032-f1bc736245b1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200625212154-ddb9806d33ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200803210538-64077c9b5642/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220114195835-da31bd327af9/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220728004956-3c1f35247d10/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= +golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= +golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= +golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= +golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= +golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190312151545-0bb0c0a6e846/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190312170243-e65039ee4138/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190425150028-36563e24a262/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20190506145303-2d16b83fe98c/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20190606124116-d0a3d012864b/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= +golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= +golang.org/x/tools v0.0.0-20190628153133-6cdbf07be9d0/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= +golang.org/x/tools v0.0.0-20190816200558-6889da9d5479/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20190911174233-4f2ddba30aff/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191012152004-8de300cfc20a/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191113191852-77e3bb0ad9e7/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191115202509-3a792d9c32b2/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191125144606-a911d9008d1f/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191130070609-6e064ea0cf2d/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191216173652-a0e659d51361/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20191227053925-7b8e75db28f4/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200117161641-43d50277825c/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200122220014-bf1340f18c4a/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200130002326-2f3ba24bd6e7/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200204074204-1cc6d1ef6c74/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200207183749-b753a1ba74fa/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200212150539-ea181f53ac56/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200224181240-023911ca70b2/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200227222343-706bc42d1f0d/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200304193943-95d2e580d8eb/go.mod h1:o4KQGtdN14AW+yjsvvwRTJJuXz8XRtIHtEnmAXLyFUw= +golang.org/x/tools v0.0.0-20200312045724-11d5b4c81c7d/go.mod h1:o4KQGtdN14AW+yjsvvwRTJJuXz8XRtIHtEnmAXLyFUw= +golang.org/x/tools v0.0.0-20200331025713-a30bf2db82d4/go.mod h1:Sl4aGygMT6LrqrWclx+PTx3U+LnKx/seiNR+3G19Ar8= +golang.org/x/tools v0.0.0-20200501065659-ab2804fb9c9d/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= +golang.org/x/tools v0.0.0-20200512131952-2bc93b1c0c88/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= +golang.org/x/tools v0.0.0-20200515010526-7d3b6ebf133d/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= +golang.org/x/tools v0.0.0-20200618134242-20370b0cb4b2/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= +golang.org/x/tools v0.0.0-20200729194436-6467de6f59a7/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= +golang.org/x/tools v0.0.0-20200804011535-6c149bb5ef0d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= +golang.org/x/tools v0.0.0-20200825202427-b303f430e36d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= +golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +google.golang.org/api v0.4.0/go.mod h1:8k5glujaEP+g9n7WNsDg8QP6cUVNI86fCNMcbazEtwE= +google.golang.org/api v0.7.0/go.mod h1:WtwebWUNSVBH/HAw79HIFXZNqEvBhG+Ra+ax0hx3E3M= +google.golang.org/api v0.8.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= +google.golang.org/api v0.9.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= +google.golang.org/api v0.13.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= +google.golang.org/api v0.14.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= +google.golang.org/api v0.15.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= +google.golang.org/api v0.17.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/api v0.18.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/api v0.19.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/api v0.20.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/api v0.22.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/api v0.24.0/go.mod h1:lIXQywCXRcnZPGlsd8NbLnOjtAoL6em04bJ9+z0MncE= +google.golang.org/api v0.28.0/go.mod h1:lIXQywCXRcnZPGlsd8NbLnOjtAoL6em04bJ9+z0MncE= +google.golang.org/api v0.29.0/go.mod h1:Lcubydp8VUV7KeIHD9z2Bys/sm/vGKnG1UHuDBSrHWM= +google.golang.org/api v0.30.0/go.mod h1:QGmEvQ87FHZNiUVJkT14jQNYJ4ZJjdRF23ZXz5138Fc= +google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= +google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/appengine v1.6.1/go.mod h1:i06prIuMbXzDqacNJfV5OdTW448YApPu5ww/cMBSeb0= +google.golang.org/appengine v1.6.5/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= +google.golang.org/appengine v1.6.6/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= +google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= +google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190425155659-357c62f0e4bb/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190502173448-54afdca5d873/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190801165951-fa694d86fc64/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +google.golang.org/genproto v0.0.0-20190911173649-1774047e7e51/go.mod h1:IbNlFCBrqXvoKpeg0TB2l7cyZUmoaFKYIwrEpbDKLA8= +google.golang.org/genproto v0.0.0-20191108220845-16a3f7862a1a/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20191115194625-c23dd37a84c9/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20191216164720-4f79533eabd1/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20191230161307-f3c370f40bfb/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20200115191322-ca5a22157cba/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20200122232147-0452cf42e150/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20200204135345-fa8e72b47b90/go.mod h1:GmwEX6Z4W5gMy59cAlVYjN9JhxgbQH6Gn+gFDQe2lzA= +google.golang.org/genproto v0.0.0-20200212174721-66ed5ce911ce/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200224152610-e50cd9704f63/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200228133532-8c2c7df3a383/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200305110556-506484158171/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200312145019-da6875a35672/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200331122359-1ee6d9798940/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200430143042-b979b6f78d84/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200511104702-f5ebc3bea380/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200515170657-fc4c6c6a6587/go.mod h1:YsZOwe1myG/8QRHRsmBRE1LrgQY60beZKjly0O1fX9U= +google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= +google.golang.org/genproto v0.0.0-20200618031413-b414f8b61790/go.mod h1:jDfRM7FcilCzHH/e9qn6dsT145K34l5v+OpcnNgKAAA= +google.golang.org/genproto v0.0.0-20200729003335-053ba62fc06f/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20200804131852-c06518451d9c/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20200825200019-8632dd797987/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= +google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= +google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= +google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= +google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= +google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.27.1/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.28.0/go.mod h1:rpkK4SK4GF4Ach/+MFLZUBavHOvF2JJB5uozKKal+60= +google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= +google.golang.org/grpc v1.30.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= +google.golang.org/grpc v1.31.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= +google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= +google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= +google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= +google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE= +google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= +google.golang.org/protobuf v1.22.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= +google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= +google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= +google.golang.org/protobuf v1.24.0/go.mod h1:r/3tXBNzIEhYS9I1OUVjXDlt8tc493IdKGjtUeSXeh4= +google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= +google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= +google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= +gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= +gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= +gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= +gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw= +gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.3/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= +gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= +gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= +honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= +honnef.co/go/tools v0.0.1-2020.1.4/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= +rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8= +rsc.io/quote/v3 v3.1.0/go.mod h1:yEA65RcK8LyAZtP9Kv3t0HmxON59tX3rD+tICJqUlj0= +rsc.io/sampler v1.3.0/go.mod h1:T1hPZKmBbMNahiBKFy5HrXp6adAjACjK9JXDnKaTXpA= diff --git a/examples/exactly_once/main.go b/examples/exactly_once/main.go new file mode 100644 index 000000000..61fbc04f2 --- /dev/null +++ b/examples/exactly_once/main.go @@ -0,0 +1,357 @@ +package main + +// SIGUSR1 toggle the pause/resume consumption +import ( + "context" + "flag" + "fmt" + "log" + "os" + "os/signal" + "strings" + "sync" + "syscall" + "time" + + "github.com/Shopify/sarama" +) + +// Sarama configuration options +var ( + brokers = "" + version = "" + group = "" + topics = "" + destinationTopic = "" + oldest = true + verbose = false +) + +func init() { + flag.StringVar(&brokers, "brokers", "", "Kafka bootstrap brokers to connect to, as a comma separated list") + flag.StringVar(&group, "group", "", "Kafka consumer group definition") + flag.StringVar(&version, "version", "2.6.0", "Kafka cluster version") + flag.StringVar(&topics, "topics", "", "Kafka topics to be consumed, as a comma separated list") + flag.StringVar(&destinationTopic, "destination-topic", "", "Kafka topic where records will be copied from topics.") + flag.BoolVar(&oldest, "oldest", true, "Kafka consumer consume initial offset from oldest") + flag.BoolVar(&verbose, "verbose", false, "Sarama logging") + flag.Parse() + + if len(brokers) == 0 { + panic("no Kafka bootstrap brokers defined, please set the -brokers flag") + } + + if len(topics) == 0 { + panic("no topics given to be consumed, please set the -topics flag") + } + + if len(destinationTopic) == 0 { + panic("no destination topics given to be consumed, please set the -destination-topics flag") + } + + if len(group) == 0 { + panic("no Kafka consumer group defined, please set the -group flag") + } +} + +func main() { + keepRunning := true + log.Println("Starting a new Sarama consumer") + + if verbose { + sarama.Logger = log.New(os.Stdout, "[sarama] ", log.LstdFlags) + } + + version, err := sarama.ParseKafkaVersion(version) + if err != nil { + log.Panicf("Error parsing Kafka version: %v", err) + } + + /** + * Construct a new Sarama configuration. + * The Kafka cluster version has to be defined before the consumer/producer is initialized. + */ + config := sarama.NewConfig() + config.Version = version + + config.Consumer.IsolationLevel = sarama.ReadCommitted + config.Consumer.Offsets.AutoCommit.Enable = false + config.Consumer.Group.Rebalance.Strategy = sarama.BalanceStrategyRoundRobin + + if oldest { + config.Consumer.Offsets.Initial = sarama.OffsetOldest + } + + producerProvider := newProducerProvider(strings.Split(brokers, ","), func() *sarama.Config { + producerConfig := sarama.NewConfig() + producerConfig.Version = version + + producerConfig.Net.MaxOpenRequests = 1 + producerConfig.Producer.RequiredAcks = sarama.WaitForAll + producerConfig.Producer.Idempotent = true + producerConfig.Producer.Transaction.ID = "sarama" + return producerConfig + }) + + /** + * Setup a new Sarama consumer group + */ + consumer := Consumer{ + groupId: group, + brokers: strings.Split(brokers, ","), + producerProvider: producerProvider, + ready: make(chan bool), + } + + ctx, cancel := context.WithCancel(context.Background()) + client, err := sarama.NewConsumerGroup(strings.Split(brokers, ","), group, config) + if err != nil { + log.Panicf("Error creating consumer group client: %v", err) + } + + consumptionIsPaused := false + wg := &sync.WaitGroup{} + wg.Add(1) + go func() { + defer wg.Done() + for { + // `Consume` should be called inside an infinite loop, when a + // server-side rebalance happens, the consumer session will need to be + // recreated to get the new claims + if err := client.Consume(ctx, strings.Split(topics, ","), &consumer); err != nil { + log.Panicf("Error from consumer: %v", err) + } + // check if context was cancelled, signaling that the consumer should stop + if ctx.Err() != nil { + return + } + consumer.ready = make(chan bool) + } + }() + + <-consumer.ready // Await till the consumer has been set up + log.Println("Sarama consumer up and running!...") + + sigusr1 := make(chan os.Signal, 1) + signal.Notify(sigusr1, syscall.SIGUSR1) + + sigterm := make(chan os.Signal, 1) + signal.Notify(sigterm, syscall.SIGINT, syscall.SIGTERM) + + for keepRunning { + select { + case <-ctx.Done(): + log.Println("terminating: context cancelled") + keepRunning = false + case <-sigterm: + log.Println("terminating: via signal") + keepRunning = false + case <-sigusr1: + toggleConsumptionFlow(client, &consumptionIsPaused) + } + } + cancel() + wg.Wait() + + producerProvider.clear() + + if err = client.Close(); err != nil { + log.Panicf("Error closing client: %v", err) + } +} + +func toggleConsumptionFlow(client sarama.ConsumerGroup, isPaused *bool) { + if *isPaused { + client.ResumeAll() + log.Println("Resuming consumption") + } else { + client.PauseAll() + log.Println("Pausing consumption") + } + + *isPaused = !*isPaused +} + +// Consumer represents a Sarama consumer group consumer +type Consumer struct { + ready chan bool + groupId string + brokers []string + producerProvider *producerProvider +} + +// Setup is run at the beginning of a new session, before ConsumeClaim +func (consumer *Consumer) Setup(session sarama.ConsumerGroupSession) error { + // Mark the consumer as ready + close(consumer.ready) + return nil +} + +// Cleanup is run at the end of a session, once all ConsumeClaim goroutines have exited +func (consumer *Consumer) Cleanup(sarama.ConsumerGroupSession) error { + return nil +} + +// ConsumeClaim must start a consumer loop of ConsumerGroupClaim's Messages(). +func (consumer *Consumer) ConsumeClaim(session sarama.ConsumerGroupSession, claim sarama.ConsumerGroupClaim) error { + // NOTE: + // Do not move the code below to a goroutine. + // The `ConsumeClaim` itself is called within a goroutine, see: + // https://github.com/Shopify/sarama/blob/main/consumer_group.go#L27-L2 + for { + select { + case message := <-claim.Messages(): + func() { + producer := consumer.producerProvider.borrow(message.Topic, message.Partition) + defer consumer.producerProvider.release(message.Topic, message.Partition, producer) + + startTime := time.Now() + + // BeginTxn must be called before any messages. + err := producer.BeginTxn() + if err != nil { + log.Printf("Message consumer: unable to start transaction: %+v", err) + return + } + // Produce current record in producer transaction. + producer.Input() <- &sarama.ProducerMessage{ + Topic: destinationTopic, + Key: sarama.ByteEncoder(message.Key), + Value: sarama.ByteEncoder(message.Value), + } + + // You can add current message to this transaction + err = producer.AddMessageToTxn(message, consumer.groupId, nil) + if err != nil { + log.Println("error on AddMessageToTxn") + consumer.handleTxnError(producer, message, session, err, func() error { + return producer.AddMessageToTxn(message, consumer.groupId, nil) + }) + return + } + + // Commit producer transaction. + err = producer.CommitTxn() + if err != nil { + log.Println("error on CommitTxn") + consumer.handleTxnError(producer, message, session, err, func() error { + return producer.CommitTxn() + }) + return + } + log.Printf("Message claimed [%s]: value = %s, timestamp = %v, topic = %s, partition = %d", time.Since(startTime), string(message.Value), message.Timestamp, message.Topic, message.Partition) + }() + // Should return when `session.Context()` is done. + // If not, will raise `ErrRebalanceInProgress` or `read tcp :: i/o timeout` when kafka rebalance. see: + // https://github.com/Shopify/sarama/issues/1192 + case <-session.Context().Done(): + return nil + } + } +} + +func (consumer *Consumer) handleTxnError(producer sarama.AsyncProducer, message *sarama.ConsumerMessage, session sarama.ConsumerGroupSession, err error, defaulthandler func() error) { + log.Printf("Message consumer: unable to process transaction: %+v", err) + for { + if producer.TxnStatus()&sarama.ProducerTxnFlagFatalError != 0 { + // fatal error. need to recreate producer. + log.Printf("Message consumer: producer is in a fatal state, need to recreate it") + // reset current consumer offset to retry consume this record. + session.ResetOffset(message.Topic, message.Partition, message.Offset, "") + return + } + if producer.TxnStatus()&sarama.ProducerTxnFlagAbortableError != 0 { + err = producer.AbortTxn() + if err != nil { + log.Printf("Message consumer: unable to abort transaction: %+v", err) + continue + } + // reset current consumer offset to retry consume this record. + session.ResetOffset(message.Topic, message.Partition, message.Offset, "") + return + } + // if not you can retry + err = defaulthandler() + if err == nil { + return + } + } +} + +type topicPartition struct { + topic string + partition int32 +} + +type producerProvider struct { + producersLock sync.Mutex + producers map[topicPartition][]sarama.AsyncProducer + + producerProvider func(topic string, partition int32) sarama.AsyncProducer +} + +func newProducerProvider(brokers []string, producerConfigurationProvider func() *sarama.Config) *producerProvider { + provider := &producerProvider{ + producers: make(map[topicPartition][]sarama.AsyncProducer), + } + provider.producerProvider = func(topic string, partition int32) sarama.AsyncProducer { + config := producerConfigurationProvider() + if config.Producer.Transaction.ID != "" { + config.Producer.Transaction.ID = config.Producer.Transaction.ID + "-" + topic + "-" + fmt.Sprint(partition) + } + producer, err := sarama.NewAsyncProducer(brokers, config) + if err != nil { + return nil + } + return producer + } + return provider +} + +func (p *producerProvider) borrow(topic string, partition int32) (producer sarama.AsyncProducer) { + p.producersLock.Lock() + defer p.producersLock.Unlock() + + tp := topicPartition{topic: topic, partition: partition} + + if producers, ok := p.producers[tp]; !ok || len(producers) == 0 { + for { + producer = p.producerProvider(topic, partition) + if producer != nil { + return + } + } + } + + index := len(p.producers[tp]) - 1 + producer = p.producers[tp][index] + p.producers[tp] = p.producers[tp][:index] + return +} + +func (p *producerProvider) release(topic string, partition int32, producer sarama.AsyncProducer) { + p.producersLock.Lock() + defer p.producersLock.Unlock() + + if producer.TxnStatus()&sarama.ProducerTxnFlagInError != 0 { + // Try to close it + _ = producer.Close() + return + } + tp := topicPartition{topic: topic, partition: partition} + p.producers[tp] = append(p.producers[tp], producer) +} + +func (p *producerProvider) clear() { + p.producersLock.Lock() + defer p.producersLock.Unlock() + + for _, producers := range p.producers { + for _, producer := range producers { + producer.Close() + } + } + for _, producers := range p.producers { + producers = producers[:0] + } +} diff --git a/examples/txn_producer/.gitignore b/examples/txn_producer/.gitignore new file mode 100644 index 000000000..4fcc75ab3 --- /dev/null +++ b/examples/txn_producer/.gitignore @@ -0,0 +1 @@ +txn_producer diff --git a/examples/txn_producer/README.md b/examples/txn_producer/README.md new file mode 100644 index 000000000..7f946e7b7 --- /dev/null +++ b/examples/txn_producer/README.md @@ -0,0 +1,10 @@ +# Transactional producer example + +This example shows you how to use the Sarama transactional producer. The example simply starts some goroutine that produce endlesslly on associated topic. + +```bash +$ go run main.go -brokers="127.0.0.1:9092" -topic="sarama" -producers=10 -record-numbers=10000 +``` + +To ensure transactional-id uniqueness it implement some ***ProducerProvider*** that build a producer appending an integer that grow when producer is created. +You can also see record-rate each 5s printing on stdout. \ No newline at end of file diff --git a/examples/txn_producer/go.mod b/examples/txn_producer/go.mod new file mode 100644 index 000000000..0b0027cb4 --- /dev/null +++ b/examples/txn_producer/go.mod @@ -0,0 +1,10 @@ +module github.com/Shopify/sarama/examples/txn_producer + +go 1.16 + +require ( + github.com/Shopify/sarama v1.34.1 + github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 // indirect +) + +replace github.com/Shopify/sarama => ../../ diff --git a/examples/txn_producer/go.sum b/examples/txn_producer/go.sum new file mode 100644 index 000000000..7e2a316c4 --- /dev/null +++ b/examples/txn_producer/go.sum @@ -0,0 +1,528 @@ +cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU= +cloud.google.com/go v0.44.1/go.mod h1:iSa0KzasP4Uvy3f1mN/7PiObzGgflwredwwASm/v6AU= +cloud.google.com/go v0.44.2/go.mod h1:60680Gw3Yr4ikxnPRS/oxxkBccT6SA1yMk63TGekxKY= +cloud.google.com/go v0.45.1/go.mod h1:RpBamKRgapWJb87xiFSdk4g1CME7QZg3uwTez+TSTjc= +cloud.google.com/go v0.46.3/go.mod h1:a6bKKbmY7er1mI7TEI4lsAkts/mkhTSZK8w33B4RAg0= +cloud.google.com/go v0.50.0/go.mod h1:r9sluTvynVuxRIOHXQEHMFffphuXHOMZMycpNR5e6To= +cloud.google.com/go v0.52.0/go.mod h1:pXajvRH/6o3+F9jDHZWQ5PbGhn+o8w9qiu/CffaVdO4= +cloud.google.com/go v0.53.0/go.mod h1:fp/UouUEsRkN6ryDKNW/Upv/JBKnv6WDthjR6+vze6M= +cloud.google.com/go v0.54.0/go.mod h1:1rq2OEkV3YMf6n/9ZvGWI3GWw0VoqH/1x2nd8Is/bPc= +cloud.google.com/go v0.56.0/go.mod h1:jr7tqZxxKOVYizybht9+26Z/gUq7tiRzu+ACVAMbKVk= +cloud.google.com/go v0.57.0/go.mod h1:oXiQ6Rzq3RAkkY7N6t3TcE6jE+CIBBbA36lwQ1JyzZs= +cloud.google.com/go v0.62.0/go.mod h1:jmCYTdRCQuc1PHIIJ/maLInMho30T/Y0M4hTdTShOYc= +cloud.google.com/go v0.65.0/go.mod h1:O5N8zS7uWy9vkA9vayVHs65eM1ubvY4h553ofrNHObY= +cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o= +cloud.google.com/go/bigquery v1.3.0/go.mod h1:PjpwJnslEMmckchkHFfq+HTD2DmtT67aNFKH1/VBDHE= +cloud.google.com/go/bigquery v1.4.0/go.mod h1:S8dzgnTigyfTmLBfrtrhyYhwRxG72rYxvftPBK2Dvzc= +cloud.google.com/go/bigquery v1.5.0/go.mod h1:snEHRnqQbz117VIFhE8bmtwIDY80NLUZUMb4Nv6dBIg= +cloud.google.com/go/bigquery v1.7.0/go.mod h1://okPTzCYNXSlb24MZs83e2Do+h+VXtc4gLoIoXIAPc= +cloud.google.com/go/bigquery v1.8.0/go.mod h1:J5hqkt3O0uAFnINi6JXValWIb1v0goeZM77hZzJN/fQ= +cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= +cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk= +cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= +cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+R3AArQw= +cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA= +cloud.google.com/go/pubsub v1.3.1/go.mod h1:i+ucay31+CNRpDW4Lu78I4xXG+O1r/MAHgjpRVR+TSU= +cloud.google.com/go/storage v1.0.0/go.mod h1:IhtSnM/ZTZV8YYJWCY8RULGVqBDmpoyjwiyrjsg+URw= +cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0ZeosJ0Rtdos= +cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohlUTyfDhBk= +cloud.google.com/go/storage v1.8.0/go.mod h1:Wv1Oy7z6Yz3DshWRJFhqM/UCfaWIRTdp0RXyy7KQOVs= +cloud.google.com/go/storage v1.10.0/go.mod h1:FLPqc6j+Ki4BU591ie1oL6qBQGu2Bl/tZ9ullr3+Kg0= +dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= +github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= +github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= +github.com/Shopify/toxiproxy/v2 v2.4.0 h1:O1e4Jfvr/hefNTNu+8VtdEG5lSeamJRo4aKhMOKNM64= +github.com/Shopify/toxiproxy/v2 v2.4.0/go.mod h1:3ilnjng821bkozDRxNoo64oI/DKqM+rOyJzb564+bvg= +github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= +github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= +github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= +github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= +github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho= +github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= +github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= +github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= +github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= +github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/cespare/xxhash/v2 v2.1.2/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= +github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= +github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= +github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= +github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= +github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= +github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= +github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= +github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/eapache/go-resiliency v1.3.0 h1:RRL0nge+cWGlxXbUzJ7yMcq6w2XBEr19dCN6HECGaT0= +github.com/eapache/go-resiliency v1.3.0/go.mod h1:5yPzW0MIvSe0JDsv0v+DvcjEv2FyD6iZYSs1ZI+iQho= +github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21 h1:YEetp8/yCZMuEPMUDHG0CW/brkkEp8mzqk2+ODEitlw= +github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21/go.mod h1:+020luEh2TKB4/GOp8oxxtq0Daoen/Cii55CzbTV6DU= +github.com/eapache/queue v1.1.0 h1:YOEu7KNc61ntiQlcEeUIoDTJ2o8mQznoNvUhiigpIqc= +github.com/eapache/queue v1.1.0/go.mod h1:6eCeP0CKFpHLu8blIFXhExK/dRa7WDZfr6jVFPTqq+I= +github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= +github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= +github.com/fortytw2/leaktest v1.3.0 h1:u8491cBMTQ8ft8aeV+adlcytMZylmA5nnwwkRZjI8vw= +github.com/fortytw2/leaktest v1.3.0/go.mod h1:jDsjWgpAGjm2CA7WthBh/CdZYEPF31XHquHwclZch5g= +github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= +github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= +github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= +github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= +github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= +github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= +github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= +github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= +github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= +github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= +github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= +github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= +github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= +github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= +github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y= +github.com/golang/mock v1.4.0/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= +github.com/golang/mock v1.4.1/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= +github.com/golang/mock v1.4.3/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= +github.com/golang/mock v1.4.4/go.mod h1:l3mdAwkq5BuhzHwde/uurv3sEJeZMXNpwsxVWU71h+4= +github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= +github.com/golang/protobuf v1.3.4/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= +github.com/golang/protobuf v1.3.5/go.mod h1:6O5/vntMXwX2lRkT1hjjk0nAC1IDOTvTlVgjlRvqsdk= +github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8= +github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA= +github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs= +github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w= +github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= +github.com/golang/protobuf v1.4.1/go.mod h1:U8fpvMrcmy5pZrNK1lt4xCsGvpyWQ/VVv6QDs8UjoX8= +github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= +github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= +github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= +github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= +github.com/golang/snappy v0.0.4 h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM= +github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= +github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= +github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= +github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= +github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.4.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= +github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= +github.com/google/martian/v3 v3.0.0/go.mod h1:y5Zk1BBys9G+gd6Jrk0W3cC1+ELVxBWuIGO+w/tUAp0= +github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= +github.com/google/pprof v0.0.0-20190515194954-54271f7e092f/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= +github.com/google/pprof v0.0.0-20191218002539-d4f498aebedc/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/pprof v0.0.0-20200212024743-f11f1df84d12/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/pprof v0.0.0-20200229191704-1ebb73c60ed3/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/pprof v0.0.0-20200430221834-fc25d7d30c6d/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/pprof v0.0.0-20200708004538-1a94d8640e99/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= +github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= +github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= +github.com/gorilla/mux v1.8.0/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= +github.com/gorilla/securecookie v1.1.1/go.mod h1:ra0sb63/xPlUeL+yeDciTfxMRAA+MP+HVt/4epWDjd4= +github.com/gorilla/sessions v1.2.1/go.mod h1:dk2InVEVJ0sfLlnXv9EAgkf6ecYs/i80K/zI+bUmuGM= +github.com/hashicorp/errwrap v1.0.0 h1:hLrqtEDnRye3+sgx6z4qVLNuviH3MR5aQ0ykNJa/UYA= +github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= +github.com/hashicorp/go-multierror v1.1.1 h1:H5DkEtf6CXdFp0N0Em5UCwQpXMWke8IA0+lD48awMYo= +github.com/hashicorp/go-multierror v1.1.1/go.mod h1:iw975J/qwKPdAO1clOe2L8331t/9/fmwbPZ6JB6eMoM= +github.com/hashicorp/go-uuid v1.0.2 h1:cfejS+Tpcp13yd5nYHWDI6qVCny6wyX2Mt5SGur2IGE= +github.com/hashicorp/go-uuid v1.0.2/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= +github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= +github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= +github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= +github.com/jcmturner/aescts/v2 v2.0.0 h1:9YKLH6ey7H4eDBXW8khjYslgyqG2xZikXP0EQFKrle8= +github.com/jcmturner/aescts/v2 v2.0.0/go.mod h1:AiaICIRyfYg35RUkr8yESTqvSy7csK90qZ5xfvvsoNs= +github.com/jcmturner/dnsutils/v2 v2.0.0 h1:lltnkeZGL0wILNvrNiVCR6Ro5PGU/SeBvVO/8c/iPbo= +github.com/jcmturner/dnsutils/v2 v2.0.0/go.mod h1:b0TnjGOvI/n42bZa+hmXL+kFJZsFT7G4t3HTlQ184QM= +github.com/jcmturner/gofork v1.0.0 h1:J7uCkflzTEhUZ64xqKnkDxq3kzc96ajM1Gli5ktUem8= +github.com/jcmturner/gofork v1.0.0/go.mod h1:MK8+TM0La+2rjBD4jE12Kj1pCCxK7d2LK/UM3ncEo0o= +github.com/jcmturner/goidentity/v6 v6.0.1 h1:VKnZd2oEIMorCTsFBnJWbExfNN7yZr3EhJAxwOkZg6o= +github.com/jcmturner/goidentity/v6 v6.0.1/go.mod h1:X1YW3bgtvwAXju7V3LCIMpY0Gbxyjn/mY9zx4tFonSg= +github.com/jcmturner/gokrb5/v8 v8.4.2 h1:6ZIM6b/JJN0X8UM43ZOM6Z4SJzla+a/u7scXFJzodkA= +github.com/jcmturner/gokrb5/v8 v8.4.2/go.mod h1:sb+Xq/fTY5yktf/VxLsE3wlfPqQjp0aWNYyvBVK62bc= +github.com/jcmturner/rpc/v2 v2.0.3 h1:7FXXj8Ti1IaVFpSAziCZWNzbNuZmnvw/i6CqLNdWfZY= +github.com/jcmturner/rpc/v2 v2.0.3/go.mod h1:VUJYCIDm3PVOEHw8sgt091/20OJjskO/YJki3ELg/Hc= +github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= +github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= +github.com/json-iterator/go v1.1.10/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= +github.com/json-iterator/go v1.1.11/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= +github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= +github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= +github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= +github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= +github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= +github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= +github.com/klauspost/compress v1.15.8 h1:JahtItbkWjf2jzm/T+qgMxkP9EMHsqEUA6vCMGmXvhA= +github.com/klauspost/compress v1.15.8/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= +github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= +github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= +github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= +github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= +github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= +github.com/kr/pretty v0.3.0/go.mod h1:640gp4NfQd8pI5XOwp5fnNeVWj67G7CFk/SaSQn7NBk= +github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= +github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= +github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= +github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= +github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= +github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= +github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= +github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= +github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= +github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= +github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= +github.com/pierrec/lz4/v4 v4.1.15 h1:MO0/ucJhngq7299dKLwIMtgTfbkoSPF6AoMYDd8Q4q0= +github.com/pierrec/lz4/v4 v4.1.15/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= +github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= +github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= +github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= +github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= +github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= +github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= +github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP4j5+D6mVACh+pe2M= +github.com/prometheus/client_golang v1.11.0/go.mod h1:Z6t4BnS23TR94PD6BsDNk8yVqroYurpAkEiz0P2BEV0= +github.com/prometheus/client_golang v1.12.1/go.mod h1:3Z9XVyYiZYEO+YQWt3RD2R3jrbd179Rt297l4aS6nDY= +github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= +github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= +github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo= +github.com/prometheus/common v0.26.0/go.mod h1:M7rCNAaPfAosfx8veZJCuw84e35h3Cfd9VFqTh1DIvc= +github.com/prometheus/common v0.32.1/go.mod h1:vu+V0TpY+O6vW9J44gczi3Ap/oXXR10b+M/gUGO4Hls= +github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= +github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= +github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= +github.com/prometheus/procfs v0.6.0/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= +github.com/prometheus/procfs v0.7.3/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= +github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 h1:N/ElC8H3+5XpJzTSTfLsJV/mx9Q9g7kxmchpfZyxgzM= +github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= +github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= +github.com/rogpeppe/go-internal v1.6.1/go.mod h1:xXDCJY+GAPziupqXw64V24skbSoqbTEfhy4qGm1nDQc= +github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= +github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= +github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= +github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= +github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= +github.com/sirupsen/logrus v1.8.1/go.mod h1:yWOB1SBYBC5VeMP7gHvWumXLIWorT60ONWic61uBYv0= +github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= +github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= +github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= +github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= +github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= +github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= +github.com/stretchr/testify v1.8.0 h1:pSgiaMZlXftHpm5L7V1+rVB+AZJydKsMxsQBIJw4PKk= +github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= +github.com/urfave/cli/v2 v2.3.0/go.mod h1:LJmUH05zAU44vOAcrfzZQKsZbVcdbOG8rtL3/XcUArI= +github.com/xdg-go/pbkdf2 v1.0.0/go.mod h1:jrpuAogTd400dnrH08LKmI/xc1MbPOebTwRqcT5RDeI= +github.com/xdg-go/scram v1.1.1/go.mod h1:RaEWvsqvNKKvBPvcKeFjrG2cJqOkHTiyTpzz23ni57g= +github.com/xdg-go/stringprep v1.0.3/go.mod h1:W3f5j4i+9rC0kuIEJL0ky1VpHXQU3ocBgklLGvcBnW8= +github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= +go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= +go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= +go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= +go.opencensus.io v0.22.4/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= +golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= +golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20201112155050-0c6587e931a9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20220214200702-86341886e292 h1:f+lwQ+GtmgoY+A2YaQxlSOnDjXcQ7ZRLWOHbC6HtRqE= +golang.org/x/crypto v0.0.0-20220214200702-86341886e292/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= +golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= +golang.org/x/exp v0.0.0-20190829153037-c13cbed26979/go.mod h1:86+5VVa7VpoJ4kLfm080zCjGlMRFzhUhsZKEZO7MGek= +golang.org/x/exp v0.0.0-20191030013958-a1ab85dbe136/go.mod h1:JXzH8nQsPlswgeRAPE3MuO9GYsAcnJvJ4vnMwN/5qkY= +golang.org/x/exp v0.0.0-20191129062945-2f5052295587/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= +golang.org/x/exp v0.0.0-20191227195350-da58074b4299/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= +golang.org/x/exp v0.0.0-20200119233911-0405dc783f0a/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= +golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EHIKF9dgMWnmCNThgcyBT1FY9mM= +golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= +golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= +golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= +golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= +golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= +golang.org/x/lint v0.0.0-20190301231843-5614ed5bae6f/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= +golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20190409202823-959b441ac422/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20190909230951-414d861bb4ac/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20191125180803-fdd1cda4f05f/go.mod h1:5qLYkcX4OjUUV8bRuDixDT3tpyyb+LUpUlRWLxfhWrs= +golang.org/x/lint v0.0.0-20200130185559-910be7a94367/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= +golang.org/x/lint v0.0.0-20200302205851-738671d3881b/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= +golang.org/x/mobile v0.0.0-20190312151609-d3739f865fa6/go.mod h1:z+o9i4GpDbdi3rU15maQ/Ox0txvL9dWGYEHz965HBQE= +golang.org/x/mobile v0.0.0-20190719004257-d2bd2a29d028/go.mod h1:E/iHnbuqvinMTCcRqshq8CkpyQDoeVncDDYHnLhea+o= +golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= +golang.org/x/mod v0.1.0/go.mod h1:0QHyrYULN0/3qlju5TqG8bIK38QM8yzMo5ekMj3DlcY= +golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= +golang.org/x/mod v0.1.1-0.20191107180719-034126e5016b/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= +golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190501004415-9ce7a6920f09/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190603091049-60506f45cf65/go.mod h1:HSz+uSET+XFnRR8LxR5pz3Of3rY3CfYBVs4xY44aLks= +golang.org/x/net v0.0.0-20190613194153-d28f0bde5980/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20190628185345-da137c7871d7/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20190724013045-ca1201d0de80/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20191209160850-c0dbc17a3553/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200202094626-16171245cfb2/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200222125558-5a598a2470a0/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200301022130-244492dfa37a/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200324143707-d3edc9973b7e/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= +golang.org/x/net v0.0.0-20200501053045-e0ff5e5a1de5/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= +golang.org/x/net v0.0.0-20200506145744-7e3656a0809f/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= +golang.org/x/net v0.0.0-20200513185701-a91f0712d120/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= +golang.org/x/net v0.0.0-20200520182314-0ba52f642ac2/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= +golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= +golang.org/x/net v0.0.0-20200707034311-ab3426394381/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= +golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= +golang.org/x/net v0.0.0-20210525063256-abc453219eb5/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= +golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= +golang.org/x/net v0.0.0-20220708220712-1185a9018129 h1:vucSRfWwTsoXro7P+3Cjlr6flUMtzCwzlvkxEQtHHB0= +golang.org/x/net v0.0.0-20220708220712-1185a9018129/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= +golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= +golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20191202225959-858c2ad4c8b6/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20210514164344-f6687ab2804c/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= +golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20200317015054-43a5402ce75a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20201207232520-09787c993a3a h1:DcqTD9SDLc+1P/r1EmRBwnVsrOwW+kk2vWf9n+1sGhs= +golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190502145724-3ef323f4f1fd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190507160741-ecd444e8653b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190606165138-5da285871e9c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190624142023-c5567b49c5d0/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190726091711-fc99dfbffb4e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191001151750-bb3f8db39f24/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191026070338-33540a1f6037/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191204072324-ce4227a45e2e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191228213918-04cbcbbfeed8/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200106162015-b016eb3dc98e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200113162924-86b910548bc1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200122134326-e047566fdf82/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200202164722-d101bd2416d5/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200212091648-12a6c2dcc1e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200223170610-d5e6a3e2c0ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200302150141-5c8b2ff67527/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200331124033-c3d80250170d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200501052902-10377860bb8e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200511232937-7e40ca221e25/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200515095857-1151b9dac4a9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200523222454-059865788121/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200615200032-f1bc736245b1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200625212154-ddb9806d33ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200803210538-64077c9b5642/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220114195835-da31bd327af9/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= +golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= +golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= +golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= +golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= +golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190312151545-0bb0c0a6e846/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190312170243-e65039ee4138/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190425150028-36563e24a262/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20190506145303-2d16b83fe98c/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20190606124116-d0a3d012864b/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= +golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= +golang.org/x/tools v0.0.0-20190628153133-6cdbf07be9d0/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= +golang.org/x/tools v0.0.0-20190816200558-6889da9d5479/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20190911174233-4f2ddba30aff/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191012152004-8de300cfc20a/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191113191852-77e3bb0ad9e7/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191115202509-3a792d9c32b2/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191125144606-a911d9008d1f/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191130070609-6e064ea0cf2d/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191216173652-a0e659d51361/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20191227053925-7b8e75db28f4/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200117161641-43d50277825c/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200122220014-bf1340f18c4a/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200130002326-2f3ba24bd6e7/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200204074204-1cc6d1ef6c74/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200207183749-b753a1ba74fa/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200212150539-ea181f53ac56/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200224181240-023911ca70b2/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200227222343-706bc42d1f0d/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200304193943-95d2e580d8eb/go.mod h1:o4KQGtdN14AW+yjsvvwRTJJuXz8XRtIHtEnmAXLyFUw= +golang.org/x/tools v0.0.0-20200312045724-11d5b4c81c7d/go.mod h1:o4KQGtdN14AW+yjsvvwRTJJuXz8XRtIHtEnmAXLyFUw= +golang.org/x/tools v0.0.0-20200331025713-a30bf2db82d4/go.mod h1:Sl4aGygMT6LrqrWclx+PTx3U+LnKx/seiNR+3G19Ar8= +golang.org/x/tools v0.0.0-20200501065659-ab2804fb9c9d/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= +golang.org/x/tools v0.0.0-20200512131952-2bc93b1c0c88/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= +golang.org/x/tools v0.0.0-20200515010526-7d3b6ebf133d/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= +golang.org/x/tools v0.0.0-20200618134242-20370b0cb4b2/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= +golang.org/x/tools v0.0.0-20200729194436-6467de6f59a7/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= +golang.org/x/tools v0.0.0-20200804011535-6c149bb5ef0d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= +golang.org/x/tools v0.0.0-20200825202427-b303f430e36d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= +golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +google.golang.org/api v0.4.0/go.mod h1:8k5glujaEP+g9n7WNsDg8QP6cUVNI86fCNMcbazEtwE= +google.golang.org/api v0.7.0/go.mod h1:WtwebWUNSVBH/HAw79HIFXZNqEvBhG+Ra+ax0hx3E3M= +google.golang.org/api v0.8.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= +google.golang.org/api v0.9.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= +google.golang.org/api v0.13.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= +google.golang.org/api v0.14.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= +google.golang.org/api v0.15.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= +google.golang.org/api v0.17.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/api v0.18.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/api v0.19.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/api v0.20.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/api v0.22.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/api v0.24.0/go.mod h1:lIXQywCXRcnZPGlsd8NbLnOjtAoL6em04bJ9+z0MncE= +google.golang.org/api v0.28.0/go.mod h1:lIXQywCXRcnZPGlsd8NbLnOjtAoL6em04bJ9+z0MncE= +google.golang.org/api v0.29.0/go.mod h1:Lcubydp8VUV7KeIHD9z2Bys/sm/vGKnG1UHuDBSrHWM= +google.golang.org/api v0.30.0/go.mod h1:QGmEvQ87FHZNiUVJkT14jQNYJ4ZJjdRF23ZXz5138Fc= +google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= +google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/appengine v1.6.1/go.mod h1:i06prIuMbXzDqacNJfV5OdTW448YApPu5ww/cMBSeb0= +google.golang.org/appengine v1.6.5/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= +google.golang.org/appengine v1.6.6/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= +google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= +google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190425155659-357c62f0e4bb/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190502173448-54afdca5d873/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190801165951-fa694d86fc64/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +google.golang.org/genproto v0.0.0-20190911173649-1774047e7e51/go.mod h1:IbNlFCBrqXvoKpeg0TB2l7cyZUmoaFKYIwrEpbDKLA8= +google.golang.org/genproto v0.0.0-20191108220845-16a3f7862a1a/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20191115194625-c23dd37a84c9/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20191216164720-4f79533eabd1/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20191230161307-f3c370f40bfb/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20200115191322-ca5a22157cba/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20200122232147-0452cf42e150/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20200204135345-fa8e72b47b90/go.mod h1:GmwEX6Z4W5gMy59cAlVYjN9JhxgbQH6Gn+gFDQe2lzA= +google.golang.org/genproto v0.0.0-20200212174721-66ed5ce911ce/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200224152610-e50cd9704f63/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200228133532-8c2c7df3a383/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200305110556-506484158171/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200312145019-da6875a35672/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200331122359-1ee6d9798940/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200430143042-b979b6f78d84/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200511104702-f5ebc3bea380/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200515170657-fc4c6c6a6587/go.mod h1:YsZOwe1myG/8QRHRsmBRE1LrgQY60beZKjly0O1fX9U= +google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= +google.golang.org/genproto v0.0.0-20200618031413-b414f8b61790/go.mod h1:jDfRM7FcilCzHH/e9qn6dsT145K34l5v+OpcnNgKAAA= +google.golang.org/genproto v0.0.0-20200729003335-053ba62fc06f/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20200804131852-c06518451d9c/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20200825200019-8632dd797987/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= +google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= +google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= +google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= +google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= +google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.27.1/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.28.0/go.mod h1:rpkK4SK4GF4Ach/+MFLZUBavHOvF2JJB5uozKKal+60= +google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= +google.golang.org/grpc v1.30.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= +google.golang.org/grpc v1.31.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= +google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= +google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= +google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= +google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE= +google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= +google.golang.org/protobuf v1.22.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= +google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= +google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= +google.golang.org/protobuf v1.24.0/go.mod h1:r/3tXBNzIEhYS9I1OUVjXDlt8tc493IdKGjtUeSXeh4= +google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= +google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= +google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= +gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= +gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= +gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= +gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw= +gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.3/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= +gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= +gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= +honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= +honnef.co/go/tools v0.0.1-2020.1.4/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= +rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8= +rsc.io/quote/v3 v3.1.0/go.mod h1:yEA65RcK8LyAZtP9Kv3t0HmxON59tX3rD+tICJqUlj0= +rsc.io/sampler v1.3.0/go.mod h1:T1hPZKmBbMNahiBKFy5HrXp6adAjACjK9JXDnKaTXpA= diff --git a/examples/txn_producer/main.go b/examples/txn_producer/main.go new file mode 100644 index 000000000..7342d6ac4 --- /dev/null +++ b/examples/txn_producer/main.go @@ -0,0 +1,232 @@ +package main + +// SIGUSR1 toggle the pause/resume consumption +import ( + "context" + "flag" + "fmt" + "log" + "os" + "os/signal" + "strings" + "sync" + "syscall" + "time" + + "github.com/Shopify/sarama" + "github.com/rcrowley/go-metrics" + + _ "net/http/pprof" +) + +// Sarama configuration options +var ( + brokers = "" + version = "" + topic = "" + producers = 1 + verbose = false + + recordsNumber int64 = 1 + + recordsRate = metrics.GetOrRegisterMeter("records.rate", nil) +) + +func init() { + flag.StringVar(&brokers, "brokers", "", "Kafka bootstrap brokers to connect to, as a comma separated list") + flag.StringVar(&version, "version", "0.11.0.0", "Kafka cluster version") + flag.StringVar(&topic, "topic", "", "Kafka topics where records will be copied from topics.") + flag.IntVar(&producers, "producers", 10, "Number of concurrent producers") + flag.Int64Var(&recordsNumber, "records-number", 10000, "Number of records sent per loop") + flag.BoolVar(&verbose, "verbose", false, "Sarama logging") + flag.Parse() + + if len(brokers) == 0 { + panic("no Kafka bootstrap brokers defined, please set the -brokers flag") + } + + if len(topic) == 0 { + panic("no topic given to be consumed, please set the -topic flag") + } +} + +func main() { + keepRunning := true + log.Println("Starting a new Sarama producer") + + if verbose { + sarama.Logger = log.New(os.Stdout, "[sarama] ", log.LstdFlags) + } + + version, err := sarama.ParseKafkaVersion(version) + if err != nil { + log.Panicf("Error parsing Kafka version: %v", err) + } + + producerProvider := newProducerProvider(strings.Split(brokers, ","), func() *sarama.Config { + config := sarama.NewConfig() + config.Version = version + config.Producer.Idempotent = true + config.Producer.Return.Errors = false + config.Producer.RequiredAcks = sarama.WaitForAll + config.Producer.Partitioner = sarama.NewRoundRobinPartitioner + config.Producer.Transaction.Retry.Backoff = 10 + config.Producer.Transaction.ID = "txn_producer" + config.Net.MaxOpenRequests = 1 + return config + }) + + go metrics.Log(metrics.DefaultRegistry, 5*time.Second, log.New(os.Stderr, "metrics: ", log.LstdFlags)) + + ctx, cancel := context.WithCancel(context.Background()) + + var wg sync.WaitGroup + for i := 0; i < producers; i++ { + wg.Add(1) + go func() { + defer wg.Done() + + for { + select { + case <-ctx.Done(): + return + default: + produceTestRecord(producerProvider) + } + } + }() + } + + sigterm := make(chan os.Signal, 1) + signal.Notify(sigterm, syscall.SIGINT, syscall.SIGTERM) + + for keepRunning { + <-sigterm + log.Println("terminating: via signal") + keepRunning = false + } + cancel() + wg.Wait() + + producerProvider.clear() +} + +func produceTestRecord(producerProvider *producerProvider) { + producer := producerProvider.borrow() + defer producerProvider.release(producer) + + // Start kafka transaction + err := producer.BeginTxn() + if err != nil { + log.Printf("unable to start txn %s\n", err) + return + } + + // Produce some records in transaction + var i int64 + for i = 0; i < recordsNumber; i++ { + producer.Input() <- &sarama.ProducerMessage{Topic: topic, Key: nil, Value: sarama.StringEncoder("test")} + } + + // commit transaction + err = producer.CommitTxn() + if err != nil { + log.Printf("Producer: unable to commit txn %s\n", err) + for { + if producer.TxnStatus()&sarama.ProducerTxnFlagFatalError != 0 { + // fatal error. need to recreate producer. + log.Printf("Producer: producer is in a fatal state, need to recreate it") + break + } + // If producer is in abortable state, try to abort current transaction. + if producer.TxnStatus()&sarama.ProducerTxnFlagAbortableError != 0 { + err = producer.AbortTxn() + if err != nil { + // If an error occured just retry it. + log.Printf("Producer: unable to abort transaction: %+v", err) + continue + } + break + } + // if not you can retry + err = producer.CommitTxn() + if err != nil { + log.Printf("Producer: unable to commit txn %s\n", err) + continue + } + } + return + } + recordsRate.Mark(recordsNumber) +} + +// pool of producers that ensure transactional-id is unique. +type producerProvider struct { + transactionIdGenerator int32 + + producersLock sync.Mutex + producers []sarama.AsyncProducer + + producerProvider func() sarama.AsyncProducer +} + +func newProducerProvider(brokers []string, producerConfigurationProvider func() *sarama.Config) *producerProvider { + provider := &producerProvider{} + provider.producerProvider = func() sarama.AsyncProducer { + config := producerConfigurationProvider() + suffix := provider.transactionIdGenerator + // Append transactionIdGenerator to current config.Producer.Transaction.ID to ensure transaction-id uniqueness. + if config.Producer.Transaction.ID != "" { + provider.transactionIdGenerator++ + config.Producer.Transaction.ID = config.Producer.Transaction.ID + "-" + fmt.Sprint(suffix) + } + producer, err := sarama.NewAsyncProducer(brokers, config) + if err != nil { + return nil + } + return producer + } + return provider +} + +func (p *producerProvider) borrow() (producer sarama.AsyncProducer) { + p.producersLock.Lock() + defer p.producersLock.Unlock() + + if len(p.producers) == 0 { + for { + producer = p.producerProvider() + if producer != nil { + return + } + } + } + + index := len(p.producers) - 1 + producer = p.producers[index] + p.producers = p.producers[:index] + return +} + +func (p *producerProvider) release(producer sarama.AsyncProducer) { + p.producersLock.Lock() + defer p.producersLock.Unlock() + + // If released producer is erroneous close it and don't return it to the producer pool. + if producer.TxnStatus()&sarama.ProducerTxnFlagInError != 0 { + // Try to close it + _ = producer.Close() + return + } + p.producers = append(p.producers, producer) +} + +func (p *producerProvider) clear() { + p.producersLock.Lock() + defer p.producersLock.Unlock() + + for _, producer := range p.producers { + producer.Close() + } + p.producers = p.producers[:0] +} diff --git a/functional_producer_test.go b/functional_producer_test.go index 1332a423c..6fda07637 100644 --- a/functional_producer_test.go +++ b/functional_producer_test.go @@ -4,6 +4,7 @@ package sarama import ( + "context" "errors" "fmt" "os" @@ -15,6 +16,7 @@ import ( toxiproxy "github.com/Shopify/toxiproxy/v2/client" "github.com/rcrowley/go-metrics" + "github.com/stretchr/testify/require" ) const TestBatchSize = 1000 @@ -89,6 +91,556 @@ func TestFuncMultiPartitionProduce(t *testing.T) { } } +func TestFuncTxnProduceNoBegin(t *testing.T) { + checkKafkaVersion(t, "0.11.0.0") + setupFunctionalTest(t) + defer teardownFunctionalTest(t) + + config := NewTestConfig() + config.ChannelBufferSize = 20 + config.Producer.Flush.Frequency = 50 * time.Millisecond + config.Producer.Flush.Messages = 200 + config.Producer.Idempotent = true + config.Producer.Transaction.ID = "TestFuncTxnProduceNoBegin" + config.Producer.RequiredAcks = WaitForAll + config.Producer.Retry.Max = 50 + config.Consumer.IsolationLevel = ReadCommitted + config.Producer.Return.Errors = true + config.Producer.Transaction.Retry.Max = 200 + config.Net.MaxOpenRequests = 1 + config.Version = V0_11_0_0 + producer, err := NewAsyncProducer(FunctionalTestEnv.KafkaBrokerAddrs, config) + require.NoError(t, err) + defer producer.Close() + + producer.Input() <- &ProducerMessage{Topic: "test.1", Key: nil, Value: StringEncoder("test")} + producerError := <-producer.Errors() + require.Error(t, producerError) +} + +func TestFuncTxnCommitNoMessages(t *testing.T) { + checkKafkaVersion(t, "0.11.0.0") + setupFunctionalTest(t) + defer teardownFunctionalTest(t) + + config := NewTestConfig() + config.ChannelBufferSize = 20 + config.Producer.Flush.Frequency = 50 * time.Millisecond + config.Producer.Flush.Messages = 200 + config.Producer.Idempotent = true + config.Producer.Transaction.ID = "TestFuncTxnCommitNoMessages" + config.Producer.RequiredAcks = WaitForAll + config.Producer.Retry.Max = 50 + config.Consumer.IsolationLevel = ReadCommitted + config.Producer.Return.Errors = true + config.Producer.Transaction.Retry.Max = 200 + config.Net.MaxOpenRequests = 1 + config.Version = V0_11_0_0 + producer, err := NewAsyncProducer(FunctionalTestEnv.KafkaBrokerAddrs, config) + require.NoError(t, err) + defer producer.Close() + + err = producer.BeginTxn() + require.NoError(t, err) + + err = producer.AbortTxn() + require.NoError(t, err) + + err = producer.BeginTxn() + require.NoError(t, err) + + err = producer.CommitTxn() + require.NoError(t, err) +} + +func TestFuncTxnProduce(t *testing.T) { + checkKafkaVersion(t, "0.11.0.0") + setupFunctionalTest(t) + defer teardownFunctionalTest(t) + + config := NewTestConfig() + config.ChannelBufferSize = 20 + config.Producer.Flush.Frequency = 50 * time.Millisecond + config.Producer.Flush.Messages = 200 + config.Producer.Idempotent = true + config.Producer.Transaction.ID = "TestFuncTxnProduce" + config.Producer.RequiredAcks = WaitForAll + config.Producer.Transaction.Retry.Max = 200 + config.Consumer.IsolationLevel = ReadCommitted + config.Net.MaxOpenRequests = 1 + config.Version = V0_11_0_0 + + consumer, err := NewConsumer(FunctionalTestEnv.KafkaBrokerAddrs, config) + require.NoError(t, err) + defer consumer.Close() + + pc, err := consumer.ConsumePartition("test.1", 0, OffsetNewest) + msgChannel := pc.Messages() + require.NoError(t, err) + defer pc.Close() + + nonTransactionalProducer, err := NewAsyncProducer(FunctionalTestEnv.KafkaBrokerAddrs, NewTestConfig()) + require.NoError(t, err) + defer nonTransactionalProducer.Close() + + // Ensure consumer is started + nonTransactionalProducer.Input() <- &ProducerMessage{Topic: "test.1", Key: nil, Value: StringEncoder("test")} + <-msgChannel + + producer, err := NewAsyncProducer(FunctionalTestEnv.KafkaBrokerAddrs, config) + require.NoError(t, err) + defer producer.Close() + + err = producer.BeginTxn() + require.NoError(t, err) + + for i := 0; i < 1; i++ { + producer.Input() <- &ProducerMessage{Topic: "test.1", Key: nil, Value: StringEncoder("test")} + } + + err = producer.CommitTxn() + require.NoError(t, err) + + for i := 0; i < 1; i++ { + msg := <-msgChannel + t.Logf("Received %s from %s-%d at offset %d", msg.Value, msg.Topic, msg.Partition, msg.Offset) + } +} + +func TestFuncTxnProduceWithBrokerFailure(t *testing.T) { + checkKafkaVersion(t, "0.11.0.0") + setupFunctionalTest(t) + defer teardownFunctionalTest(t) + + config := NewTestConfig() + config.ChannelBufferSize = 20 + config.Producer.Flush.Frequency = 50 * time.Millisecond + config.Producer.Flush.Messages = 200 + config.Producer.Idempotent = true + config.Producer.Transaction.ID = "TestFuncTxnProduceWithBrokerFailure" + config.Producer.RequiredAcks = WaitForAll + config.Producer.Transaction.Retry.Max = 200 + config.Consumer.IsolationLevel = ReadCommitted + config.Net.MaxOpenRequests = 1 + config.Version = V0_11_0_0 + + consumer, err := NewConsumer(FunctionalTestEnv.KafkaBrokerAddrs, config) + require.NoError(t, err) + defer consumer.Close() + + pc, err := consumer.ConsumePartition("test.1", 0, OffsetNewest) + msgChannel := pc.Messages() + require.NoError(t, err) + defer pc.Close() + + nonTransactionalProducer, err := NewAsyncProducer(FunctionalTestEnv.KafkaBrokerAddrs, NewTestConfig()) + require.NoError(t, err) + defer nonTransactionalProducer.Close() + + // Ensure consumer is started + nonTransactionalProducer.Input() <- &ProducerMessage{Topic: "test.1", Key: nil, Value: StringEncoder("test")} + <-msgChannel + + producer, err := NewAsyncProducer(FunctionalTestEnv.KafkaBrokerAddrs, config) + require.NoError(t, err) + defer producer.Close() + + txCoordinator, _ := producer.(*asyncProducer).client.TransactionCoordinator(config.Producer.Transaction.ID) + + err = producer.BeginTxn() + require.NoError(t, err) + + if err := stopDockerTestBroker(context.Background(), txCoordinator.id); err != nil { + t.Fatal(err) + } + + defer func() { + if err := startDockerTestBroker(context.Background(), txCoordinator.id); err != nil { + t.Fatal(err) + } + t.Logf("\n") + }() + + for i := 0; i < 1; i++ { + producer.Input() <- &ProducerMessage{Topic: "test.1", Key: nil, Value: StringEncoder("test")} + } + + err = producer.CommitTxn() + require.NoError(t, err) + + for i := 0; i < 1; i++ { + msg := <-msgChannel + t.Logf("Received %s from %s-%d at offset %d", msg.Value, msg.Topic, msg.Partition, msg.Offset) + } +} + +func TestFuncTxnProduceEpochBump(t *testing.T) { + checkKafkaVersion(t, "2.6.0") + setupFunctionalTest(t) + defer teardownFunctionalTest(t) + + config := NewTestConfig() + config.ChannelBufferSize = 20 + config.Producer.Flush.Frequency = 50 * time.Millisecond + config.Producer.Flush.Messages = 200 + config.Producer.Idempotent = true + config.Producer.Transaction.ID = "TestFuncTxnProduceEpochBump" + config.Producer.RequiredAcks = WaitForAll + config.Producer.Transaction.Retry.Max = 200 + config.Consumer.IsolationLevel = ReadCommitted + config.Net.MaxOpenRequests = 1 + config.Version = V2_6_0_0 + + consumer, err := NewConsumer(FunctionalTestEnv.KafkaBrokerAddrs, config) + require.NoError(t, err) + defer consumer.Close() + + pc, err := consumer.ConsumePartition("test.1", 0, OffsetNewest) + msgChannel := pc.Messages() + require.NoError(t, err) + defer pc.Close() + + nonTransactionalProducer, err := NewAsyncProducer(FunctionalTestEnv.KafkaBrokerAddrs, NewTestConfig()) + require.NoError(t, err) + defer nonTransactionalProducer.Close() + + // Ensure consumer is started + nonTransactionalProducer.Input() <- &ProducerMessage{Topic: "test.1", Key: nil, Value: StringEncoder("test")} + <-msgChannel + + producer, err := NewAsyncProducer(FunctionalTestEnv.KafkaBrokerAddrs, config) + require.NoError(t, err) + defer producer.Close() + + err = producer.BeginTxn() + require.NoError(t, err) + + for i := 0; i < 1; i++ { + producer.Input() <- &ProducerMessage{Topic: "test.1", Key: nil, Value: StringEncoder("test")} + } + + err = producer.CommitTxn() + require.NoError(t, err) + + for i := 0; i < 1; i++ { + msg := <-msgChannel + t.Logf("Received %s from %s-%d at offset %d", msg.Value, msg.Topic, msg.Partition, msg.Offset) + } + + err = producer.BeginTxn() + require.NoError(t, err) + + for i := 0; i < 1; i++ { + producer.Input() <- &ProducerMessage{Topic: "test.1", Key: nil, Value: StringEncoder("test")} + } + + err = producer.CommitTxn() + require.NoError(t, err) + + for i := 0; i < 1; i++ { + msg := <-msgChannel + t.Logf("Received %s from %s-%d at offset %d", msg.Value, msg.Topic, msg.Partition, msg.Offset) + } +} + +func TestFuncInitProducerId3(t *testing.T) { + checkKafkaVersion(t, "2.6.0") + setupFunctionalTest(t) + defer teardownFunctionalTest(t) + + config := NewTestConfig() + config.ChannelBufferSize = 20 + config.Producer.Flush.Frequency = 50 * time.Millisecond + config.Producer.Flush.Messages = 200 + config.Producer.Idempotent = true + config.Producer.Transaction.ID = "TestFuncInitProducerId3" + config.Producer.RequiredAcks = WaitForAll + config.Producer.Retry.Max = 50 + config.Consumer.IsolationLevel = ReadCommitted + config.Net.MaxOpenRequests = 1 + config.Version = V2_6_0_0 + + producer, err := NewAsyncProducer(FunctionalTestEnv.KafkaBrokerAddrs, config) + require.NoError(t, err) + defer producer.Close() + + require.Equal(t, true, producer.(*asyncProducer).txnmgr.coordinatorSupportsBumpingEpoch) +} + +type messageHandler struct { + *testing.T + h func(*ConsumerMessage) + started sync.WaitGroup +} + +func (h *messageHandler) Setup(s ConsumerGroupSession) error { return nil } +func (h *messageHandler) Cleanup(s ConsumerGroupSession) error { return nil } +func (h *messageHandler) ConsumeClaim(sess ConsumerGroupSession, claim ConsumerGroupClaim) error { + h.started.Done() + + for msg := range claim.Messages() { + h.Logf("consumed msg %v", msg) + h.h(msg) + } + return nil +} + +func TestFuncTxnProduceAndCommitOffset(t *testing.T) { + checkKafkaVersion(t, "0.11.0.0") + setupFunctionalTest(t) + defer teardownFunctionalTest(t) + + config := NewTestConfig() + config.ChannelBufferSize = 20 + config.Producer.Flush.Frequency = 50 * time.Millisecond + config.Producer.Flush.Messages = 200 + config.Producer.Idempotent = true + config.Producer.Transaction.ID = "TestFuncTxnProduceAndCommitOffset" + config.Producer.RequiredAcks = WaitForAll + config.Producer.Transaction.Retry.Max = 200 + config.Consumer.IsolationLevel = ReadCommitted + config.Consumer.Offsets.AutoCommit.Enable = false + config.Net.MaxOpenRequests = 1 + config.Version = V0_11_0_0 + + client, err := NewClient(FunctionalTestEnv.KafkaBrokerAddrs, config) + require.NoError(t, err) + defer client.Close() + + admin, err := NewClusterAdminFromClient(client) + require.NoError(t, err) + defer admin.Close() + + producer, err := NewAsyncProducerFromClient(client) + require.NoError(t, err) + defer producer.Close() + + cg, err := NewConsumerGroupFromClient("test-produce", client) + require.NoError(t, err) + defer cg.Close() + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + handler := &messageHandler{} + handler.T = t + handler.h = func(msg *ConsumerMessage) { + err := producer.BeginTxn() + require.NoError(t, err) + producer.Input() <- &ProducerMessage{Topic: "test.1", Value: StringEncoder("test-prod")} + err = producer.AddMessageToTxn(msg, "test-produce", nil) + require.NoError(t, err) + err = producer.CommitTxn() + require.NoError(t, err) + } + + handler.started.Add(4) + go func() { + err = cg.Consume(ctx, []string{"test.4"}, handler) + require.NoError(t, err) + }() + + handler.started.Wait() + + nonTransactionalProducer, err := NewAsyncProducer(FunctionalTestEnv.KafkaBrokerAddrs, NewTestConfig()) + require.NoError(t, err) + defer nonTransactionalProducer.Close() + + consumer, err := NewConsumerFromClient(client) + require.NoError(t, err) + defer consumer.Close() + + pc, err := consumer.ConsumePartition("test.1", 0, OffsetNewest) + msgChannel := pc.Messages() + require.NoError(t, err) + defer pc.Close() + + // Ensure consumer is started + nonTransactionalProducer.Input() <- &ProducerMessage{Topic: "test.1", Key: nil, Value: StringEncoder("test")} + <-msgChannel + + for i := 0; i < 1; i++ { + nonTransactionalProducer.Input() <- &ProducerMessage{Topic: "test.4", Key: nil, Value: StringEncoder("test")} + } + + for i := 0; i < 1; i++ { + msg := <-msgChannel + t.Logf("Received %s from %s-%d at offset %d", msg.Value, msg.Topic, msg.Partition, msg.Offset) + } + + topicPartitions := make(map[string][]int32) + topicPartitions["test.4"] = []int32{0, 1, 2, 3} + topicsDescription, err := admin.ListConsumerGroupOffsets("test-produce", topicPartitions) + require.NoError(t, err) + + for _, partition := range topicPartitions["test.4"] { + block := topicsDescription.GetBlock("test.4", partition) + _ = client.RefreshMetadata("test.4") + lastOffset, err := client.GetOffset("test.4", partition, OffsetNewest) + require.NoError(t, err) + if block.Offset > -1 { + require.Equal(t, lastOffset, block.Offset) + } + } +} + +func TestFuncTxnProduceMultiTxn(t *testing.T) { + checkKafkaVersion(t, "0.11.0.0") + setupFunctionalTest(t) + defer teardownFunctionalTest(t) + + config := NewTestConfig() + config.ChannelBufferSize = 20 + config.Producer.Flush.Frequency = 50 * time.Millisecond + config.Producer.Flush.Messages = 200 + config.Producer.Idempotent = true + config.Producer.Transaction.ID = "TestFuncTxnProduceMultiTxn" + config.Producer.RequiredAcks = WaitForAll + config.Producer.Transaction.Retry.Max = 200 + config.Consumer.IsolationLevel = ReadCommitted + config.Net.MaxOpenRequests = 1 + config.Version = V0_11_0_0 + + configSecond := NewTestConfig() + configSecond.ChannelBufferSize = 20 + configSecond.Producer.Flush.Frequency = 50 * time.Millisecond + configSecond.Producer.Flush.Messages = 200 + configSecond.Producer.Idempotent = true + configSecond.Producer.Transaction.ID = "TestFuncTxnProduceMultiTxn-second" + configSecond.Producer.RequiredAcks = WaitForAll + configSecond.Producer.Retry.Max = 50 + configSecond.Consumer.IsolationLevel = ReadCommitted + configSecond.Net.MaxOpenRequests = 1 + configSecond.Version = V0_11_0_0 + + consumer, err := NewConsumer(FunctionalTestEnv.KafkaBrokerAddrs, config) + require.NoError(t, err) + defer consumer.Close() + + pc, err := consumer.ConsumePartition("test.1", 0, OffsetNewest) + msgChannel := pc.Messages() + require.NoError(t, err) + defer pc.Close() + + nonTransactionalConfig := NewTestConfig() + nonTransactionalConfig.Producer.Return.Successes = true + nonTransactionalConfig.Producer.Return.Errors = true + + nonTransactionalProducer, err := NewAsyncProducer(FunctionalTestEnv.KafkaBrokerAddrs, nonTransactionalConfig) + require.NoError(t, err) + defer nonTransactionalProducer.Close() + + // Ensure consumer is started + nonTransactionalProducer.Input() <- &ProducerMessage{Topic: "test.1", Key: nil, Value: StringEncoder("test")} + <-msgChannel + + producer, err := NewAsyncProducer(FunctionalTestEnv.KafkaBrokerAddrs, config) + require.NoError(t, err) + defer producer.Close() + + producerSecond, err := NewAsyncProducer(FunctionalTestEnv.KafkaBrokerAddrs, configSecond) + require.NoError(t, err) + defer producerSecond.Close() + + err = producer.BeginTxn() + require.NoError(t, err) + + for i := 0; i < 2; i++ { + producer.Input() <- &ProducerMessage{Topic: "test.1", Key: nil, Value: StringEncoder("test-committed")} + } + + err = producerSecond.BeginTxn() + require.NoError(t, err) + + for i := 0; i < 2; i++ { + producerSecond.Input() <- &ProducerMessage{Topic: "test.1", Key: nil, Value: StringEncoder("test-aborted")} + } + + err = producer.CommitTxn() + require.NoError(t, err) + + err = producerSecond.AbortTxn() + require.NoError(t, err) + + for i := 0; i < 2; i++ { + msg := <-msgChannel + t.Logf("Received %s from %s-%d at offset %d", msg.Value, msg.Topic, msg.Partition, msg.Offset) + require.Equal(t, "test-committed", string(msg.Value)) + } +} + +func TestFuncTxnAbortedProduce(t *testing.T) { + checkKafkaVersion(t, "0.11.0.0") + setupFunctionalTest(t) + defer teardownFunctionalTest(t) + + config := NewTestConfig() + config.ChannelBufferSize = 20 + config.Producer.Flush.Frequency = 50 * time.Millisecond + config.Producer.Flush.Messages = 200 + config.Producer.Idempotent = true + config.Producer.Transaction.ID = "TestFuncTxnAbortedProduce" + config.Producer.RequiredAcks = WaitForAll + config.Producer.Return.Successes = true + config.Producer.Transaction.Retry.Max = 200 + config.Consumer.IsolationLevel = ReadCommitted + config.Net.MaxOpenRequests = 1 + config.Version = V0_11_0_0 + + client, err := NewClient(FunctionalTestEnv.KafkaBrokerAddrs, config) + require.NoError(t, err) + + consumer, err := NewConsumerFromClient(client) + require.NoError(t, err) + defer consumer.Close() + + pc, err := consumer.ConsumePartition("test.1", 0, OffsetNewest) + msgChannel := pc.Messages() + require.NoError(t, err) + defer pc.Close() + + nonTransactionalConfig := NewTestConfig() + nonTransactionalConfig.Producer.Return.Successes = true + nonTransactionalConfig.Producer.Return.Errors = true + + nonTransactionalProducer, err := NewAsyncProducer(FunctionalTestEnv.KafkaBrokerAddrs, nonTransactionalConfig) + require.NoError(t, err) + defer nonTransactionalProducer.Close() + + // Ensure consumer is started + nonTransactionalProducer.Input() <- &ProducerMessage{Topic: "test.1", Key: nil, Value: StringEncoder("test")} + <-msgChannel + + producer, err := NewAsyncProducerFromClient(client) + require.NoError(t, err) + defer producer.Close() + + err = producer.BeginTxn() + require.NoError(t, err) + + for i := 0; i < 2; i++ { + producer.Input() <- &ProducerMessage{Topic: "test.1", Key: nil, Value: StringEncoder("transactional")} + } + + for i := 0; i < 2; i++ { + <-producer.Successes() + } + + err = producer.AbortTxn() + require.NoError(t, err) + + for i := 0; i < 2; i++ { + nonTransactionalProducer.Input() <- &ProducerMessage{Topic: "test.1", Key: nil, Value: StringEncoder("non-transactional")} + <-nonTransactionalProducer.Successes() + } + + for i := 0; i < 2; i++ { + msg := <-msgChannel + t.Logf("Received %s from %s-%d at offset %d", msg.Value, msg.Topic, msg.Partition, msg.Offset) + require.Equal(t, "non-transactional", string(msg.Value)) + } +} + func TestFuncProducingToInvalidTopic(t *testing.T) { setupFunctionalTest(t) defer teardownFunctionalTest(t) diff --git a/init_producer_id_request.go b/init_producer_id_request.go index 689444397..33ce5fa41 100644 --- a/init_producer_id_request.go +++ b/init_producer_id_request.go @@ -3,22 +3,45 @@ package sarama import "time" type InitProducerIDRequest struct { + Version int16 TransactionalID *string TransactionTimeout time.Duration + ProducerID int64 + ProducerEpoch int16 } func (i *InitProducerIDRequest) encode(pe packetEncoder) error { - if err := pe.putNullableString(i.TransactionalID); err != nil { - return err + if i.Version < 2 { + if err := pe.putNullableString(i.TransactionalID); err != nil { + return err + } + } else { + if err := pe.putNullableCompactString(i.TransactionalID); err != nil { + return err + } } pe.putInt32(int32(i.TransactionTimeout / time.Millisecond)) + if i.Version >= 3 { + pe.putInt64(i.ProducerID) + pe.putInt16(i.ProducerEpoch) + } + if i.Version >= 2 { + pe.putEmptyTaggedFieldArray() + } return nil } func (i *InitProducerIDRequest) decode(pd packetDecoder, version int16) (err error) { - if i.TransactionalID, err = pd.getNullableString(); err != nil { - return err + i.Version = version + if i.Version < 2 { + if i.TransactionalID, err = pd.getNullableString(); err != nil { + return err + } + } else { + if i.TransactionalID, err = pd.getCompactNullableString(); err != nil { + return err + } } timeout, err := pd.getInt32() @@ -26,6 +49,21 @@ func (i *InitProducerIDRequest) decode(pd packetDecoder, version int16) (err err return err } i.TransactionTimeout = time.Duration(timeout) * time.Millisecond + if i.Version >= 3 { + if i.ProducerID, err = pd.getInt64(); err != nil { + return err + } + + if i.ProducerEpoch, err = pd.getInt16(); err != nil { + return err + } + } + + if i.Version >= 2 { + if _, err := pd.getEmptyTaggedFieldArray(); err != nil { + return err + } + } return nil } @@ -35,13 +73,30 @@ func (i *InitProducerIDRequest) key() int16 { } func (i *InitProducerIDRequest) version() int16 { - return 0 + return i.Version } func (i *InitProducerIDRequest) headerVersion() int16 { + if i.Version >= 2 { + return 2 + } + return 1 } func (i *InitProducerIDRequest) requiredVersion() KafkaVersion { - return V0_11_0_0 + switch i.Version { + case 2: + // Added tagged fields + return V2_4_0_0 + case 3: + // Added ProducerID/Epoch + return V2_5_0_0 + case 0: + fallthrough + case 1: + fallthrough + default: + return V0_11_0_0 + } } diff --git a/init_producer_id_request_test.go b/init_producer_id_request_test.go index 5c83d8514..4214342e1 100644 --- a/init_producer_id_request_test.go +++ b/init_producer_id_request_test.go @@ -15,6 +15,20 @@ var ( 0, 3, 't', 'x', 'n', 0, 0, 0, 100, } + + initProducerIDRequestTaggedFields = []byte{ + 4, 116, 120, 110, // TransactionID in compact string + 0, 0, 0, 100, // TransactionTimeout + 0, // empty TaggedFields + } + + initProducerIDRequestProducerId = []byte{ + 4, 116, 120, 110, // TransactionID in compact string + 0, 0, 0, 100, // TransactionTimeout + 0, 0, 0, 0, 0, 0, 0, 123, // ProducerID + 1, 65, // ProducerEpoch + 0, // empty TaggedFields + } ) func TestInitProducerIDRequest(t *testing.T) { @@ -28,4 +42,13 @@ func TestInitProducerIDRequest(t *testing.T) { req.TransactionalID = &transactionID testRequest(t, "transaction id", req, initProducerIDRequest) + + req.Version = 2 + testRequest(t, "tagged fields", req, initProducerIDRequestTaggedFields) + + req.Version = 3 + req.ProducerID = 123 + req.ProducerEpoch = 321 + + testRequest(t, "producer id", req, initProducerIDRequestProducerId) } diff --git a/init_producer_id_response.go b/init_producer_id_response.go index 3e1242bf6..006070189 100644 --- a/init_producer_id_response.go +++ b/init_producer_id_response.go @@ -5,6 +5,7 @@ import "time" type InitProducerIDResponse struct { ThrottleTime time.Duration Err KError + Version int16 ProducerID int64 ProducerEpoch int16 } @@ -15,10 +16,15 @@ func (i *InitProducerIDResponse) encode(pe packetEncoder) error { pe.putInt64(i.ProducerID) pe.putInt16(i.ProducerEpoch) + if i.Version >= 2 { + pe.putEmptyTaggedFieldArray() + } + return nil } func (i *InitProducerIDResponse) decode(pd packetDecoder, version int16) (err error) { + i.Version = version throttleTime, err := pd.getInt32() if err != nil { return err @@ -39,6 +45,12 @@ func (i *InitProducerIDResponse) decode(pd packetDecoder, version int16) (err er return err } + if i.Version >= 2 { + if _, err := pd.getEmptyTaggedFieldArray(); err != nil { + return err + } + } + return nil } @@ -47,13 +59,27 @@ func (i *InitProducerIDResponse) key() int16 { } func (i *InitProducerIDResponse) version() int16 { - return 0 + return i.Version } func (i *InitProducerIDResponse) headerVersion() int16 { + if i.Version >= 2 { + return 1 + } return 0 } func (i *InitProducerIDResponse) requiredVersion() KafkaVersion { - return V0_11_0_0 + switch i.Version { + case 2: + fallthrough + case 3: + return V2_4_0_0 + case 0: + fallthrough + case 1: + fallthrough + default: + return V0_11_0_0 + } } diff --git a/init_producer_id_response_test.go b/init_producer_id_response_test.go index b0649386a..ebc5d16ad 100644 --- a/init_producer_id_response_test.go +++ b/init_producer_id_response_test.go @@ -19,6 +19,14 @@ var ( 255, 255, 255, 255, 255, 255, 255, 255, 0, 0, } + + initProducerIdResponseWithTaggedFields = []byte{ + 0, 0, 0, 100, + 0, 51, + 255, 255, 255, 255, 255, 255, 255, 255, + 0, 0, + 0, + } ) func TestInitProducerIDResponse(t *testing.T) { @@ -34,4 +42,7 @@ func TestInitProducerIDResponse(t *testing.T) { resp.ProducerID = -1 testResponse(t, "with error", resp, initProducerIDRequestError) + + resp.Version = 2 + testResponse(t, "with tagged fields", resp, initProducerIdResponseWithTaggedFields) } diff --git a/mockresponses.go b/mockresponses.go index 7cfd8094b..15b4367f9 100644 --- a/mockresponses.go +++ b/mockresponses.go @@ -459,6 +459,7 @@ func (mr *MockFindCoordinatorResponse) SetError(coordinatorType CoordinatorType, func (mr *MockFindCoordinatorResponse) For(reqBody versionedDecoder) encoderWithHeader { req := reqBody.(*FindCoordinatorRequest) res := &FindCoordinatorResponse{} + res.Version = req.Version var v interface{} switch req.CoordinatorType { case CoordinatorGroup: diff --git a/mocks/async_producer.go b/mocks/async_producer.go index 0e2174b71..cdef5bfe7 100644 --- a/mocks/async_producer.go +++ b/mocks/async_producer.go @@ -14,14 +14,17 @@ import ( // function in each expectation so that the message is checked by this function and // an error is returned if the match fails. type AsyncProducer struct { - l sync.Mutex - t ErrorReporter - expectations []*producerExpectation - closed chan struct{} - input chan *sarama.ProducerMessage - successes chan *sarama.ProducerMessage - errors chan *sarama.ProducerError - lastOffset int64 + l sync.Mutex + t ErrorReporter + expectations []*producerExpectation + closed chan struct{} + input chan *sarama.ProducerMessage + successes chan *sarama.ProducerMessage + errors chan *sarama.ProducerError + isTransactional bool + txnLock sync.Mutex + txnStatus sarama.ProducerTxnStatusFlag + lastOffset int64 *TopicConfig } @@ -34,13 +37,15 @@ func NewAsyncProducer(t ErrorReporter, config *sarama.Config) *AsyncProducer { config = sarama.NewConfig() } mp := &AsyncProducer{ - t: t, - closed: make(chan struct{}), - expectations: make([]*producerExpectation, 0), - input: make(chan *sarama.ProducerMessage, config.ChannelBufferSize), - successes: make(chan *sarama.ProducerMessage, config.ChannelBufferSize), - errors: make(chan *sarama.ProducerError, config.ChannelBufferSize), - TopicConfig: NewTopicConfig(), + t: t, + closed: make(chan struct{}), + expectations: make([]*producerExpectation, 0), + input: make(chan *sarama.ProducerMessage, config.ChannelBufferSize), + successes: make(chan *sarama.ProducerMessage, config.ChannelBufferSize), + errors: make(chan *sarama.ProducerError, config.ChannelBufferSize), + isTransactional: config.Producer.Transaction.ID != "", + txnStatus: sarama.ProducerTxnFlagReady, + TopicConfig: NewTopicConfig(), } go func() { @@ -53,6 +58,13 @@ func NewAsyncProducer(t ErrorReporter, config *sarama.Config) *AsyncProducer { partitioners := make(map[string]sarama.Partitioner, 1) for msg := range mp.input { + mp.txnLock.Lock() + if mp.IsTransactional() && mp.txnStatus&sarama.ProducerTxnFlagInTransaction == 0 { + mp.t.Errorf("attempt to send message when transaction is not started or is in ending state.") + mp.errors <- &sarama.ProducerError{Err: errors.New("attempt to send message when transaction is not started or is in ending state"), Msg: msg} + continue + } + mp.txnLock.Unlock() partitioner := partitioners[msg.Topic] if partitioner == nil { partitioner = config.Producer.Partitioner(msg.Topic) @@ -142,6 +154,49 @@ func (mp *AsyncProducer) Errors() <-chan *sarama.ProducerError { return mp.errors } +func (mp *AsyncProducer) IsTransactional() bool { + return mp.isTransactional +} + +func (mp *AsyncProducer) BeginTxn() error { + mp.txnLock.Lock() + defer mp.txnLock.Unlock() + + mp.txnStatus = sarama.ProducerTxnFlagInTransaction + return nil +} + +func (mp *AsyncProducer) CommitTxn() error { + mp.txnLock.Lock() + defer mp.txnLock.Unlock() + + mp.txnStatus = sarama.ProducerTxnFlagReady + return nil +} + +func (mp *AsyncProducer) AbortTxn() error { + mp.txnLock.Lock() + defer mp.txnLock.Unlock() + + mp.txnStatus = sarama.ProducerTxnFlagReady + return nil +} + +func (mp *AsyncProducer) TxnStatus() sarama.ProducerTxnStatusFlag { + mp.txnLock.Lock() + defer mp.txnLock.Unlock() + + return mp.txnStatus +} + +func (mp *AsyncProducer) AddOffsetsToTxn(offsets map[string][]*sarama.PartitionOffsetMetadata, groupId string) error { + return nil +} + +func (mp *AsyncProducer) AddMessageToTxn(msg *sarama.ConsumerMessage, groupId string, metadata *string) error { + return nil +} + //////////////////////////////////////////////// // Setting expectations //////////////////////////////////////////////// diff --git a/mocks/async_producer_test.go b/mocks/async_producer_test.go index 63c5c87f2..69eedf2a1 100644 --- a/mocks/async_producer_test.go +++ b/mocks/async_producer_test.go @@ -108,6 +108,82 @@ func TestProducerWithTooManyExpectations(t *testing.T) { } } +func TestProducerFailTxn(t *testing.T) { + config := sarama.NewConfig() + config.Producer.Transaction.ID = "test" + config.Producer.RequiredAcks = sarama.WaitForAll + config.Producer.Retry.Backoff = 0 + config.Producer.Idempotent = true + config.Net.MaxOpenRequests = 1 + config.Version = sarama.V0_11_0_0 + + trm := newTestReporterMock() + mp := NewAsyncProducer(trm, config) + + mp.Input() <- &sarama.ProducerMessage{Topic: "test"} + + _ = mp.Close() + + if len(trm.errors) != 1 { + t.Error("must have fail with txn begin error") + } +} + +func TestProducerWithTxn(t *testing.T) { + config := sarama.NewConfig() + config.Producer.Transaction.ID = "test" + config.Producer.RequiredAcks = sarama.WaitForAll + config.Producer.Retry.Backoff = 0 + config.Producer.Idempotent = true + config.Net.MaxOpenRequests = 1 + config.Version = sarama.V0_11_0_0 + + trm := newTestReporterMock() + mp := NewAsyncProducer(trm, config). + ExpectInputAndSucceed() + + if !mp.IsTransactional() { + t.Error("producer must be transactional") + } + + if err := mp.BeginTxn(); err != nil { + t.Error(err) + } + + if mp.TxnStatus()&sarama.ProducerTxnFlagInTransaction == 0 { + t.Error("transaction must be started") + } + + mp.Input() <- &sarama.ProducerMessage{Topic: "test"} + + if err := mp.AddMessageToTxn(&sarama.ConsumerMessage{ + Topic: "original-topic", + Partition: 0, + Offset: 123, + }, "test-group", nil); err != nil { + t.Error(err) + } + + if err := mp.AddOffsetsToTxn(map[string][]*sarama.PartitionOffsetMetadata{ + "original-topic": { + { + Partition: 1, + Offset: 321, + }, + }, + }, "test-group"); err != nil { + t.Error(err) + } + + if err := mp.CommitTxn(); err != nil { + t.Error(err) + } + + if err := mp.Close(); err != nil { + t.Error(err) + } +} + func TestProducerWithCheckerFunction(t *testing.T) { trm := newTestReporterMock() mp := NewAsyncProducer(trm, nil). diff --git a/mocks/sync_producer.go b/mocks/sync_producer.go index ef7d67988..0acf10a6e 100644 --- a/mocks/sync_producer.go +++ b/mocks/sync_producer.go @@ -20,6 +20,10 @@ type SyncProducer struct { *TopicConfig newPartitioner sarama.PartitionerConstructor partitioners map[string]sarama.Partitioner + + isTransactional bool + txnLock sync.Mutex + txnStatus sarama.ProducerTxnStatusFlag } // NewSyncProducer instantiates a new SyncProducer mock. The t argument should @@ -30,11 +34,13 @@ func NewSyncProducer(t ErrorReporter, config *sarama.Config) *SyncProducer { config = sarama.NewConfig() } return &SyncProducer{ - t: t, - expectations: make([]*producerExpectation, 0), - TopicConfig: NewTopicConfig(), - newPartitioner: config.Producer.Partitioner, - partitioners: make(map[string]sarama.Partitioner, 1), + t: t, + expectations: make([]*producerExpectation, 0), + TopicConfig: NewTopicConfig(), + newPartitioner: config.Producer.Partitioner, + partitioners: make(map[string]sarama.Partitioner, 1), + isTransactional: config.Producer.Transaction.ID != "", + txnStatus: sarama.ProducerTxnFlagReady, } } @@ -52,6 +58,11 @@ func (sp *SyncProducer) SendMessage(msg *sarama.ProducerMessage) (partition int3 sp.l.Lock() defer sp.l.Unlock() + if sp.IsTransactional() && sp.txnStatus&sarama.ProducerTxnFlagInTransaction == 0 { + sp.t.Errorf("attempt to send message when transaction is not started or is in ending state.") + return -1, -1, errors.New("attempt to send message when transaction is not started or is in ending state") + } + if len(sp.expectations) > 0 { expectation := sp.expectations[0] sp.expectations = sp.expectations[1:] @@ -207,3 +218,43 @@ func (sp *SyncProducer) ExpectSendMessageAndFail(err error) *SyncProducer { return sp } + +func (sp *SyncProducer) IsTransactional() bool { + return sp.isTransactional +} + +func (sp *SyncProducer) BeginTxn() error { + sp.txnLock.Lock() + defer sp.txnLock.Unlock() + + sp.txnStatus = sarama.ProducerTxnFlagInTransaction + return nil +} + +func (sp *SyncProducer) CommitTxn() error { + sp.txnLock.Lock() + defer sp.txnLock.Unlock() + + sp.txnStatus = sarama.ProducerTxnFlagReady + return nil +} + +func (sp *SyncProducer) AbortTxn() error { + sp.txnLock.Lock() + defer sp.txnLock.Unlock() + + sp.txnStatus = sarama.ProducerTxnFlagReady + return nil +} + +func (sp *SyncProducer) TxnStatus() sarama.ProducerTxnStatusFlag { + return sp.txnStatus +} + +func (sp *SyncProducer) AddOffsetsToTxn(offsets map[string][]*sarama.PartitionOffsetMetadata, groupId string) error { + return nil +} + +func (sp *SyncProducer) AddMessageToTxn(msg *sarama.ConsumerMessage, groupId string, metadata *string) error { + return nil +} diff --git a/mocks/sync_producer_test.go b/mocks/sync_producer_test.go index fde215d97..3003711f9 100644 --- a/mocks/sync_producer_test.go +++ b/mocks/sync_producer_test.go @@ -55,6 +55,104 @@ func TestSyncProducerReturnsExpectationsToSendMessage(t *testing.T) { } } +func TestSyncProducerFailTxn(t *testing.T) { + config := sarama.NewConfig() + config.Producer.Transaction.ID = "test" + config.Producer.RequiredAcks = sarama.WaitForAll + config.Producer.Retry.Backoff = 0 + config.Producer.Idempotent = true + config.Net.MaxOpenRequests = 1 + config.Version = sarama.V0_11_0_0 + + tfm := newTestReporterMock() + + sp := NewSyncProducer(tfm, config) + defer func() { + if err := sp.Close(); err != nil { + t.Error(err) + } + }() + + msg := &sarama.ProducerMessage{Topic: "test", Value: sarama.StringEncoder("test")} + + _, _, err := sp.SendMessage(msg) + if err == nil { + t.Errorf("must have failed with txn begin error") + } + + if len(tfm.errors) != 1 { + t.Errorf("must have failed with txn begin error") + } +} + +func TestSyncProducerUseTxn(t *testing.T) { + config := sarama.NewConfig() + config.Producer.Transaction.ID = "test" + config.Producer.RequiredAcks = sarama.WaitForAll + config.Producer.Retry.Backoff = 0 + config.Producer.Idempotent = true + config.Net.MaxOpenRequests = 1 + config.Version = sarama.V0_11_0_0 + + sp := NewSyncProducer(t, config) + defer func() { + if err := sp.Close(); err != nil { + t.Error(err) + } + }() + + if !sp.IsTransactional() { + t.Error("producer must be transactional") + } + + sp.ExpectSendMessageAndSucceed() + + msg := &sarama.ProducerMessage{Topic: "test", Value: sarama.StringEncoder("test")} + + err := sp.BeginTxn() + if err != nil { + t.Errorf("txn can't be started, got %s", err) + } + if sp.TxnStatus()&sarama.ProducerTxnFlagInTransaction == 0 { + t.Error("transaction must be started") + } + _, offset, err := sp.SendMessage(msg) + if err != nil { + t.Errorf("The first message should have been produced successfully, but got %s", err) + } + if offset != 1 || offset != msg.Offset { + t.Errorf("The first message should have been assigned offset 1, but got %d", msg.Offset) + } + + if err := sp.AddMessageToTxn(&sarama.ConsumerMessage{ + Topic: "original-topic", + Partition: 0, + Offset: 123, + }, "test-group", nil); err != nil { + t.Error(err) + } + + if err := sp.AddOffsetsToTxn(map[string][]*sarama.PartitionOffsetMetadata{ + "original-topic": { + { + Partition: 1, + Offset: 321, + }, + }, + }, "test-group"); err != nil { + t.Error(err) + } + + err = sp.CommitTxn() + if err != nil { + t.Errorf("txn can't be committed, got %s", err) + } + + if err := sp.Close(); err != nil { + t.Error(err) + } +} + func TestSyncProducerWithTooManyExpectations(t *testing.T) { trm := newTestReporterMock() diff --git a/produce_set.go b/produce_set.go index b8cc5ceb7..8d6980479 100644 --- a/produce_set.go +++ b/produce_set.go @@ -137,6 +137,9 @@ func (ps *produceSet) buildRequest() *ProduceRequest { } if ps.parent.conf.Version.IsAtLeast(V0_11_0_0) { req.Version = 3 + if ps.parent.IsTransactional() { + req.TransactionalID = &ps.parent.conf.Producer.Transaction.ID + } } if ps.parent.conf.Producer.Compression == CompressionZSTD && ps.parent.conf.Version.IsAtLeast(V2_1_0_0) { @@ -160,6 +163,10 @@ func (ps *produceSet) buildRequest() *ProduceRequest { record.OffsetDelta = int64(i) } } + + // Set the batch as transactional when a transactionalID is set + rb.IsTransactional = ps.parent.IsTransactional() + req.AddBatch(topic, partition, rb) continue } diff --git a/request.go b/request.go index a2c92f8a9..5fec9776f 100644 --- a/request.go +++ b/request.go @@ -155,7 +155,7 @@ func allocateBody(key, version int16) protocolBody { case 21: return &DeleteRecordsRequest{} case 22: - return &InitProducerIDRequest{} + return &InitProducerIDRequest{Version: version} case 24: return &AddPartitionsToTxnRequest{} case 25: diff --git a/sync_producer.go b/sync_producer.go index eedece6b4..8765ac336 100644 --- a/sync_producer.go +++ b/sync_producer.go @@ -29,6 +29,27 @@ type SyncProducer interface { // object passes out of scope, as it may otherwise leak memory. // You must call this before calling Close on the underlying client. Close() error + + // TxnStatus return current producer transaction status. + TxnStatus() ProducerTxnStatusFlag + + // IsTransactional return true when current producer is is transactional. + IsTransactional() bool + + // BeginTxn mark current transaction as ready. + BeginTxn() error + + // CommitTxn commit current transaction. + CommitTxn() error + + // AbortTxn abort current transaction. + AbortTxn() error + + // AddOffsetsToTxn add associated offsets to current transaction. + AddOffsetsToTxn(offsets map[string][]*PartitionOffsetMetadata, groupId string) error + + // AddMessageToTxn add message offsets to current transaction. + AddMessageToTxn(msg *ConsumerMessage, groupId string, metadata *string) error } type syncProducer struct { @@ -146,3 +167,31 @@ func (sp *syncProducer) Close() error { sp.wg.Wait() return nil } + +func (sp *syncProducer) IsTransactional() bool { + return sp.producer.IsTransactional() +} + +func (sp *syncProducer) BeginTxn() error { + return sp.producer.BeginTxn() +} + +func (sp *syncProducer) CommitTxn() error { + return sp.producer.CommitTxn() +} + +func (sp *syncProducer) AbortTxn() error { + return sp.producer.AbortTxn() +} + +func (sp *syncProducer) AddOffsetsToTxn(offsets map[string][]*PartitionOffsetMetadata, groupId string) error { + return sp.producer.AddOffsetsToTxn(offsets, groupId) +} + +func (sp *syncProducer) AddMessageToTxn(msg *ConsumerMessage, groupId string, metadata *string) error { + return sp.producer.AddMessageToTxn(msg, groupId, metadata) +} + +func (p *syncProducer) TxnStatus() ProducerTxnStatusFlag { + return p.producer.TxnStatus() +} diff --git a/sync_producer_test.go b/sync_producer_test.go index d679e8280..3413b9b0f 100644 --- a/sync_producer_test.go +++ b/sync_producer_test.go @@ -57,6 +57,108 @@ func TestSyncProducer(t *testing.T) { seedBroker.Close() } +func TestSyncProducerTransactional(t *testing.T) { + seedBroker := NewMockBroker(t, 1) + defer seedBroker.Close() + leader := NewMockBroker(t, 2) + defer leader.Close() + + config := NewTestConfig() + config.Version = V0_11_0_0 + config.Producer.RequiredAcks = WaitForAll + config.Producer.Return.Successes = true + config.Producer.Transaction.ID = "test" + config.Producer.Idempotent = true + config.Producer.Retry.Max = 5 + config.Net.MaxOpenRequests = 1 + + metadataResponse := new(MetadataResponse) + metadataResponse.Version = 1 + metadataResponse.ControllerID = leader.BrokerID() + metadataResponse.AddBroker(leader.Addr(), leader.BrokerID()) + metadataResponse.AddTopic("my_topic", ErrNoError) + metadataResponse.AddTopicPartition("my_topic", 0, leader.BrokerID(), nil, nil, nil, ErrNoError) + seedBroker.Returns(metadataResponse) + + client, err := NewClient([]string{seedBroker.Addr()}, config) + if err != nil { + t.Fatal(err) + } + + findCoordinatorResponse := new(FindCoordinatorResponse) + findCoordinatorResponse.Coordinator = client.Brokers()[0] + findCoordinatorResponse.Version = 1 + seedBroker.Returns(findCoordinatorResponse) + + initProducerIdResponse := new(InitProducerIDResponse) + leader.Returns(initProducerIdResponse) + + addPartitionToTxn := new(AddPartitionsToTxnResponse) + addPartitionToTxn.Errors = map[string][]*PartitionError{ + "my_topic": { + { + Partition: 0, + }, + }, + } + leader.Returns(addPartitionToTxn) + + prodSuccess := new(ProduceResponse) + prodSuccess.Version = 3 + prodSuccess.AddTopicPartition("my_topic", 0, ErrNoError) + for i := 0; i < 10; i++ { + leader.Returns(prodSuccess) + } + + endTxnResponse := &EndTxnResponse{} + leader.Returns(endTxnResponse) + + producer, err := NewSyncProducerFromClient(client) + if err != nil { + t.Fatal(err) + } + if !producer.IsTransactional() { + t.Error("producer is not transactional") + } + + err = producer.BeginTxn() + if err != nil { + t.Fatal(err) + } + if producer.TxnStatus()&ProducerTxnFlagInTransaction == 0 { + t.Error("transaction must started") + } + + for i := 0; i < 10; i++ { + msg := &ProducerMessage{ + Topic: "my_topic", + Value: StringEncoder(TestMessage), + Metadata: "test", + } + + partition, offset, err := producer.SendMessage(msg) + + if partition != 0 || msg.Partition != partition { + t.Error("Unexpected partition") + } + if offset != 0 || msg.Offset != offset { + t.Error("Unexpected offset") + } + if str, ok := msg.Metadata.(string); !ok || str != "test" { + t.Error("Unexpected metadata") + } + if err != nil { + t.Error(err) + } + } + err = producer.CommitTxn() + if err != nil { + t.Fatal(err) + } + + safeClose(t, producer) +} + func TestSyncProducerBatch(t *testing.T) { seedBroker := NewMockBroker(t, 1) leader := NewMockBroker(t, 2) diff --git a/transaction_manager.go b/transaction_manager.go new file mode 100644 index 000000000..e18abecd3 --- /dev/null +++ b/transaction_manager.go @@ -0,0 +1,887 @@ +package sarama + +import ( + "errors" + "fmt" + "strings" + "sync" + "time" +) + +// ProducerTxnStatusFlag mark current transaction status. +type ProducerTxnStatusFlag int16 + +const ( + // ProducerTxnFlagUninitialized when txnmgr is created + ProducerTxnFlagUninitialized ProducerTxnStatusFlag = 1 << iota + // ProducerTxnFlagInitializing when txnmgr is initilizing + ProducerTxnFlagInitializing + // ProducerTxnFlagReady when is ready to receive transaction + ProducerTxnFlagReady + // ProducerTxnFlagInTransaction when transaction is started + ProducerTxnFlagInTransaction + // ProducerTxnFlagEndTransaction when transaction will be committed + ProducerTxnFlagEndTransaction + // ProducerTxnFlagInError whan having abortable or fatal error + ProducerTxnFlagInError + // ProducerTxnFlagCommittingTransaction when committing txn + ProducerTxnFlagCommittingTransaction + // ProducerTxnFlagAbortingTransaction when committing txn + ProducerTxnFlagAbortingTransaction + // ProducerTxnFlagAbortableError when producer encounter an abortable error + // Must call AbortTxn in this case. + ProducerTxnFlagAbortableError + // ProducerTxnFlagFatalError when producer encounter an fatal error + // Must Close an recreate it. + ProducerTxnFlagFatalError +) + +func (s ProducerTxnStatusFlag) String() string { + status := make([]string, 0) + if s&ProducerTxnFlagUninitialized != 0 { + status = append(status, "ProducerTxnStateUninitialized") + } + if s&ProducerTxnFlagInitializing != 0 { + status = append(status, "ProducerTxnStateInitializing") + } + if s&ProducerTxnFlagReady != 0 { + status = append(status, "ProducerTxnStateReady") + } + if s&ProducerTxnFlagInTransaction != 0 { + status = append(status, "ProducerTxnStateInTransaction") + } + if s&ProducerTxnFlagEndTransaction != 0 { + status = append(status, "ProducerTxnStateEndTransaction") + } + if s&ProducerTxnFlagInError != 0 { + status = append(status, "ProducerTxnStateInError") + } + if s&ProducerTxnFlagCommittingTransaction != 0 { + status = append(status, "ProducerTxnStateCommittingTransaction") + } + if s&ProducerTxnFlagAbortingTransaction != 0 { + status = append(status, "ProducerTxnStateAbortingTransaction") + } + if s&ProducerTxnFlagAbortableError != 0 { + status = append(status, "ProducerTxnStateAbortableError") + } + if s&ProducerTxnFlagFatalError != 0 { + status = append(status, "ProducerTxnStateFatalError") + } + return strings.Join(status, "|") +} + +// transactionManager keeps the state necessary to ensure idempotent production +type transactionManager struct { + producerID int64 + producerEpoch int16 + sequenceNumbers map[string]int32 + mutex sync.Mutex + transactionalID string + transactionTimeout time.Duration + client Client + + // when kafka cluster is at least 2.5.0. + // used to recover when producer failed. + coordinatorSupportsBumpingEpoch bool + + // When producer need to bump it's epoch. + epochBumpRequired bool + // Record last seen error. + lastError error + + // Ensure that status is never accessed with a race-condition. + statusLock sync.RWMutex + status ProducerTxnStatusFlag + + // Ensure that only one goroutine will update partitions in current transaction. + partitionInTxnLock sync.Mutex + pendingPartitionsInCurrentTxn topicPartitionSet + partitionsInCurrentTxn topicPartitionSet + + // Offsets to add to transaction. + offsetsInCurrentTxn map[string]topicPartitionOffsets +} + +const ( + noProducerID = -1 + noProducerEpoch = -1 + + // see publishTxnPartitions comment. + addPartitionsRetryBackoff = 20 * time.Millisecond +) + +// txnmngr allowed transitions. +var producerTxnTransitions = map[ProducerTxnStatusFlag][]ProducerTxnStatusFlag{ + ProducerTxnFlagUninitialized: { + ProducerTxnFlagReady, + ProducerTxnFlagInError, + }, + // When we need are initilizing + ProducerTxnFlagInitializing: { + ProducerTxnFlagInitializing, + ProducerTxnFlagReady, + ProducerTxnFlagInError, + }, + // When we have initilized transactional producer + ProducerTxnFlagReady: { + ProducerTxnFlagInTransaction, + }, + // When beginTxn has been called + ProducerTxnFlagInTransaction: { + // When calling commit or abort + ProducerTxnFlagEndTransaction, + // When got an error + ProducerTxnFlagInError, + }, + ProducerTxnFlagEndTransaction: { + // When epoch bump + ProducerTxnFlagInitializing, + // When commit is good + ProducerTxnFlagReady, + // When got an error + ProducerTxnFlagInError, + }, + // Need to abort transaction + ProducerTxnFlagAbortableError: { + // Call AbortTxn + ProducerTxnFlagAbortingTransaction, + // When got an error + ProducerTxnFlagInError, + }, + // Need to close producer + ProducerTxnFlagFatalError: { + ProducerTxnFlagFatalError, + }, +} + +type topicPartition struct { + topic string + partition int32 +} + +// to ensure that we don't do a full scan every time a partition or an offset is added. +type topicPartitionSet map[topicPartition]struct{} +type topicPartitionOffsets map[topicPartition]*PartitionOffsetMetadata + +func (s topicPartitionSet) mapToRequest() map[string][]int32 { + result := make(map[string][]int32, len(s)) + for tp := range s { + result[tp.topic] = append(result[tp.topic], tp.partition) + } + return result +} + +func (s topicPartitionOffsets) mapToRequest() map[string][]*PartitionOffsetMetadata { + result := make(map[string][]*PartitionOffsetMetadata, len(s)) + for tp, offset := range s { + result[tp.topic] = append(result[tp.topic], offset) + } + return result +} + +// Return true if current transition is allowed. +func (t *transactionManager) isTransitionValid(target ProducerTxnStatusFlag) bool { + for status, allowedTransitions := range producerTxnTransitions { + if status&t.status != 0 { + for _, allowedTransition := range allowedTransitions { + if allowedTransition&target != 0 { + return true + } + } + } + } + return false +} + +// Get current transaction status. +func (t *transactionManager) currentTxnStatus() ProducerTxnStatusFlag { + t.statusLock.RLock() + defer t.statusLock.RUnlock() + + return t.status +} + +// Try to transition to a valid status and return an error otherwise. +func (t *transactionManager) transitionTo(target ProducerTxnStatusFlag, err error) error { + t.statusLock.Lock() + defer t.statusLock.Unlock() + + if !t.isTransitionValid(target) { + return ErrTransitionNotAllowed + } + + if target&ProducerTxnFlagInError != 0 { + if err == nil { + return ErrCannotTransitionNilError + } + t.lastError = err + } else { + t.lastError = nil + } + + DebugLogger.Printf("txnmgr/transition [%s] transition from %s to %s\n", t.transactionalID, t.status, target) + + t.status = target + return err +} + +func (t *transactionManager) getAndIncrementSequenceNumber(topic string, partition int32) (int32, int16) { + key := fmt.Sprintf("%s-%d", topic, partition) + t.mutex.Lock() + defer t.mutex.Unlock() + sequence := t.sequenceNumbers[key] + t.sequenceNumbers[key] = sequence + 1 + return sequence, t.producerEpoch +} + +func (t *transactionManager) bumpEpoch() { + t.mutex.Lock() + defer t.mutex.Unlock() + t.producerEpoch++ + for k := range t.sequenceNumbers { + t.sequenceNumbers[k] = 0 + } +} + +func (t *transactionManager) getProducerID() (int64, int16) { + t.mutex.Lock() + defer t.mutex.Unlock() + return t.producerID, t.producerEpoch +} + +// Compute retry backoff considered current attempts. +func (t *transactionManager) computeBackoff(attemptsRemaining int) time.Duration { + if t.client.Config().Producer.Transaction.Retry.BackoffFunc != nil { + maxRetries := t.client.Config().Producer.Transaction.Retry.Max + retries := maxRetries - attemptsRemaining + return t.client.Config().Producer.Transaction.Retry.BackoffFunc(retries, maxRetries) + } + return t.client.Config().Producer.Transaction.Retry.Backoff +} + +// return true is txnmngr is transactinal. +func (t *transactionManager) isTransactional() bool { + return t.transactionalID != "" +} + +// add specified offsets to current transaction. +func (t *transactionManager) addOffsetsToTxn(offsetsToAdd map[string][]*PartitionOffsetMetadata, groupId string) error { + t.mutex.Lock() + defer t.mutex.Unlock() + + if t.currentTxnStatus()&ProducerTxnFlagInTransaction == 0 { + return ErrTransactionNotReady + } + + if t.currentTxnStatus()&ProducerTxnFlagFatalError != 0 { + return t.lastError + } + + if _, ok := t.offsetsInCurrentTxn[groupId]; !ok { + t.offsetsInCurrentTxn[groupId] = topicPartitionOffsets{} + } + + for topic, offsets := range offsetsToAdd { + for _, offset := range offsets { + tp := topicPartition{topic: topic, partition: offset.Partition} + t.offsetsInCurrentTxn[groupId][tp] = offset + } + } + return nil +} + +// send txnmgnr save offsets to transaction coordinator. +func (t *transactionManager) publishOffsetsToTxn(offsets topicPartitionOffsets, groupId string) (topicPartitionOffsets, error) { + // First AddOffsetsToTxn + attemptsRemaining := t.client.Config().Producer.Transaction.Retry.Max + exec := func(run func() (bool, error), err error) error { + for attemptsRemaining >= 0 { + var retry bool + retry, err = run() + if !retry { + return err + } + backoff := t.computeBackoff(attemptsRemaining) + Logger.Printf("txnmgr/add-offset-to-txn [%s] retrying after %dms... (%d attempts remaining) (%s)\n", + t.transactionalID, backoff/time.Millisecond, attemptsRemaining, err) + time.Sleep(backoff) + attemptsRemaining-- + } + return err + } + lastError := exec(func() (bool, error) { + coordinator, err := t.client.TransactionCoordinator(t.transactionalID) + if err != nil { + return true, err + } + response, err := coordinator.AddOffsetsToTxn(&AddOffsetsToTxnRequest{ + TransactionalID: t.transactionalID, + ProducerEpoch: t.producerEpoch, + ProducerID: t.producerID, + GroupID: groupId, + }) + if err != nil { + // If an error occurred try to refresh current transaction coordinator. + _ = coordinator.Close() + _ = t.client.RefreshTransactionCoordinator(t.transactionalID) + return true, err + } + if response == nil { + // If no response is returned just retry. + return true, ErrTxnUnableToParseResponse + } + if response.Err == ErrNoError { + DebugLogger.Printf("txnmgr/add-offset-to-txn [%s] successful add-offset-to-txn with group %s %+v\n", + t.transactionalID, groupId, response) + // If no error, just exit. + return false, nil + } + switch response.Err { + case ErrConsumerCoordinatorNotAvailable: + fallthrough + case ErrNotCoordinatorForConsumer: + _ = coordinator.Close() + _ = t.client.RefreshTransactionCoordinator(t.transactionalID) + fallthrough + case ErrOffsetsLoadInProgress: + fallthrough + case ErrConcurrentTransactions: + // Retry + case ErrUnknownProducerID: + fallthrough + case ErrInvalidProducerIDMapping: + return false, t.abortableErrorIfPossible(response.Err) + case ErrGroupAuthorizationFailed: + return false, t.transitionTo(ProducerTxnFlagInError|ProducerTxnFlagAbortableError, response.Err) + default: + // Others are fatal + return false, t.transitionTo(ProducerTxnFlagInError|ProducerTxnFlagFatalError, response.Err) + } + return true, response.Err + }, nil) + + if lastError != nil { + return offsets, lastError + } + + resultOffsets := offsets + // Then TxnOffsetCommit + // note the result is not completed until the TxnOffsetCommit returns + attemptsRemaining = t.client.Config().Producer.Transaction.Retry.Max + execTxnOffsetCommit := func(run func() (topicPartitionOffsets, bool, error), err error) (topicPartitionOffsets, error) { + var r topicPartitionOffsets + for attemptsRemaining >= 0 { + var retry bool + r, retry, err = run() + if !retry { + return r, err + } + backoff := t.computeBackoff(attemptsRemaining) + Logger.Printf("txnmgr/txn-offset-commit [%s] retrying after %dms... (%d attempts remaining) (%s)\n", + t.transactionalID, backoff/time.Millisecond, attemptsRemaining, err) + time.Sleep(backoff) + attemptsRemaining-- + } + return r, err + } + return execTxnOffsetCommit(func() (topicPartitionOffsets, bool, error) { + consumerGroupCoordinator, err := t.client.Coordinator(groupId) + if err != nil { + return resultOffsets, true, err + } + responses, err := consumerGroupCoordinator.TxnOffsetCommit(&TxnOffsetCommitRequest{ + TransactionalID: t.transactionalID, + ProducerEpoch: t.producerEpoch, + ProducerID: t.producerID, + GroupID: groupId, + Topics: offsets.mapToRequest(), + }) + if err != nil { + _ = consumerGroupCoordinator.Close() + _ = t.client.RefreshCoordinator(groupId) + return resultOffsets, true, err + } + + if responses == nil { + return resultOffsets, true, ErrTxnUnableToParseResponse + } + + var responseErrors []error + failedTxn := topicPartitionOffsets{} + for topic, partitionErrors := range responses.Topics { + for _, partitionError := range partitionErrors { + switch partitionError.Err { + case ErrNoError: + continue + // If the topic is unknown or the coordinator is loading, retry with the current coordinator + case ErrRequestTimedOut: + fallthrough + case ErrConsumerCoordinatorNotAvailable: + fallthrough + case ErrNotCoordinatorForConsumer: + _ = consumerGroupCoordinator.Close() + _ = t.client.RefreshCoordinator(groupId) + fallthrough + case ErrUnknownTopicOrPartition: + fallthrough + case ErrOffsetsLoadInProgress: + // Do nothing just retry + case ErrIllegalGeneration: + fallthrough + case ErrUnknownMemberId: + fallthrough + case ErrFencedInstancedId: + fallthrough + case ErrGroupAuthorizationFailed: + return resultOffsets, false, t.transitionTo(ProducerTxnFlagInError|ProducerTxnFlagAbortableError, partitionError.Err) + default: + // Others are fatal + return resultOffsets, false, t.transitionTo(ProducerTxnFlagInError|ProducerTxnFlagFatalError, partitionError.Err) + } + tp := topicPartition{topic: topic, partition: partitionError.Partition} + failedTxn[tp] = offsets[tp] + responseErrors = append(responseErrors, partitionError.Err) + } + } + + resultOffsets = failedTxn + + if len(resultOffsets) == 0 { + DebugLogger.Printf("txnmgr/txn-offset-commit [%s] successful txn-offset-commit with group %s %+v\n", + t.transactionalID, groupId) + return resultOffsets, false, nil + } + return resultOffsets, true, Wrap(ErrTxnOffsetCommit, responseErrors...) + }, nil) +} + +func (t *transactionManager) initProducerId() (int64, int16, error) { + isEpochBump := false + + req := &InitProducerIDRequest{} + if t.isTransactional() { + req.TransactionalID = &t.transactionalID + req.TransactionTimeout = t.transactionTimeout + } + + if t.client.Config().Version.IsAtLeast(V2_5_0_0) { + req.Version = 3 + isEpochBump = t.producerID != noProducerID && t.producerEpoch != noProducerEpoch + t.coordinatorSupportsBumpingEpoch = true + req.ProducerID = t.producerID + req.ProducerEpoch = t.producerEpoch + } else if t.client.Config().Version.IsAtLeast(V2_4_0_0) { + req.Version = 2 + } + + if isEpochBump { + err := t.transitionTo(ProducerTxnFlagInitializing, nil) + if err != nil { + return -1, -1, err + } + DebugLogger.Printf("txnmgr/init-producer-id [%s] invoking InitProducerId for the first time in order to acquire a producer ID\n", + t.transactionalID) + } else { + DebugLogger.Printf("txnmgr/init-producer-id [%s] invoking InitProducerId with current producer ID %d and epoch %d in order to bump the epoch\n", + t.transactionalID, t.producerID, t.producerEpoch) + } + + attemptsRemaining := t.client.Config().Producer.Transaction.Retry.Max + exec := func(run func() (int64, int16, bool, error), err error) (int64, int16, error) { + pid := int64(-1) + pepoch := int16(-1) + for attemptsRemaining >= 0 { + var retry bool + pid, pepoch, retry, err = run() + if !retry { + return pid, pepoch, err + } + backoff := t.computeBackoff(attemptsRemaining) + Logger.Printf("txnmgr/init-producer-id [%s] retrying after %dms... (%d attempts remaining) (%s)\n", + t.transactionalID, backoff/time.Millisecond, attemptsRemaining, err) + time.Sleep(backoff) + attemptsRemaining-- + } + return -1, -1, err + } + return exec(func() (int64, int16, bool, error) { + var err error + var coordinator *Broker + if t.isTransactional() { + coordinator, err = t.client.TransactionCoordinator(t.transactionalID) + } else { + coordinator = t.client.LeastLoadedBroker() + } + if err != nil { + return -1, -1, true, err + } + response, err := coordinator.InitProducerID(req) + if err != nil { + if t.isTransactional() { + _ = coordinator.Close() + _ = t.client.RefreshTransactionCoordinator(t.transactionalID) + } + return -1, -1, true, err + } + if response == nil { + return -1, -1, true, ErrTxnUnableToParseResponse + } + if response.Err == ErrNoError { + if isEpochBump { + t.sequenceNumbers = make(map[string]int32) + } + err := t.transitionTo(ProducerTxnFlagReady, nil) + if err != nil { + return -1, -1, true, err + } + DebugLogger.Printf("txnmgr/init-producer-id [%s] successful init producer id %+v\n", + t.transactionalID, response) + return response.ProducerID, response.ProducerEpoch, false, nil + } + switch response.Err { + case ErrConsumerCoordinatorNotAvailable: + fallthrough + case ErrNotCoordinatorForConsumer: + if t.isTransactional() { + _ = coordinator.Close() + _ = t.client.RefreshTransactionCoordinator(t.transactionalID) + } + // Fatal errors + default: + return -1, -1, false, t.transitionTo(ProducerTxnFlagInError|ProducerTxnFlagFatalError, response.Err) + } + return -1, -1, true, response.Err + }, nil) +} + +// if kafka cluster is at least 2.5.0 mark txnmngr to bump epoch else mark it as fatal. +func (t *transactionManager) abortableErrorIfPossible(err error) error { + if t.coordinatorSupportsBumpingEpoch { + t.epochBumpRequired = true + return t.transitionTo(ProducerTxnFlagInError|ProducerTxnFlagAbortableError, err) + } + return t.transitionTo(ProducerTxnFlagInError|ProducerTxnFlagFatalError, err) +} + +// End current transaction. +func (t *transactionManager) completeTransaction() error { + if t.epochBumpRequired { + err := t.transitionTo(ProducerTxnFlagInitializing, nil) + if err != nil { + return err + } + } else { + err := t.transitionTo(ProducerTxnFlagReady, nil) + if err != nil { + return err + } + } + + t.lastError = nil + t.epochBumpRequired = false + t.partitionsInCurrentTxn = topicPartitionSet{} + t.pendingPartitionsInCurrentTxn = topicPartitionSet{} + t.offsetsInCurrentTxn = map[string]topicPartitionOffsets{} + + return nil +} + +// send EndTxn request with commit flag. (true when committing false otherwise) +func (t *transactionManager) endTxn(commit bool) error { + attemptsRemaining := t.client.Config().Producer.Transaction.Retry.Max + exec := func(run func() (bool, error), err error) error { + for attemptsRemaining >= 0 { + var retry bool + retry, err = run() + if !retry { + return err + } + backoff := t.computeBackoff(attemptsRemaining) + Logger.Printf("txnmgr/endtxn [%s] retrying after %dms... (%d attempts remaining) (%s)\n", + t.transactionalID, backoff/time.Millisecond, attemptsRemaining, err) + time.Sleep(backoff) + attemptsRemaining-- + } + return err + } + return exec(func() (bool, error) { + coordinator, err := t.client.TransactionCoordinator(t.transactionalID) + if err != nil { + return true, err + } + response, err := coordinator.EndTxn(&EndTxnRequest{ + TransactionalID: t.transactionalID, + ProducerEpoch: t.producerEpoch, + ProducerID: t.producerID, + TransactionResult: commit, + }) + if err != nil { + // Always retry on network error + _ = coordinator.Close() + _ = t.client.RefreshTransactionCoordinator(t.transactionalID) + return true, err + } + if response == nil { + return true, ErrTxnUnableToParseResponse + } + if response.Err == ErrNoError { + DebugLogger.Printf("txnmgr/endtxn [%s] successful to end txn %+v\n", + t.transactionalID, response) + return false, t.completeTransaction() + } + switch response.Err { + // Need to refresh coordinator + case ErrConsumerCoordinatorNotAvailable: + fallthrough + case ErrNotCoordinatorForConsumer: + _ = coordinator.Close() + _ = t.client.RefreshTransactionCoordinator(t.transactionalID) + fallthrough + case ErrOffsetsLoadInProgress: + fallthrough + case ErrConcurrentTransactions: + // Just retry + case ErrUnknownProducerID: + fallthrough + case ErrInvalidProducerIDMapping: + return false, t.abortableErrorIfPossible(response.Err) + // Fatal errors + default: + return false, t.transitionTo(ProducerTxnFlagInError|ProducerTxnFlagFatalError, response.Err) + } + return true, response.Err + }, nil) +} + +// We will try to publish associated offsets for each groups +// then send endtxn request to mark transaction as finished. +func (t *transactionManager) finishTransaction(commit bool) error { + t.mutex.Lock() + defer t.mutex.Unlock() + + // Ensure no error when committing or abording + if commit && t.currentTxnStatus()&ProducerTxnFlagInError != 0 { + return t.lastError + } else if !commit && t.currentTxnStatus()&ProducerTxnFlagFatalError != 0 { + return t.lastError + } + + // if no records has been sent don't do anything. + if len(t.partitionsInCurrentTxn) == 0 { + return t.completeTransaction() + } + + epochBump := t.epochBumpRequired + // If we're aborting the transaction, so there should be no need to add offsets. + if commit && len(t.offsetsInCurrentTxn) > 0 { + for group, offsets := range t.offsetsInCurrentTxn { + newOffsets, err := t.publishOffsetsToTxn(offsets, group) + if err != nil { + t.offsetsInCurrentTxn[group] = newOffsets + return err + } + delete(t.offsetsInCurrentTxn, group) + } + } + + if t.currentTxnStatus()&ProducerTxnFlagFatalError != 0 { + return t.lastError + } + + if !errors.Is(t.lastError, ErrInvalidProducerIDMapping) { + err := t.endTxn(commit) + if err != nil { + return err + } + if !epochBump { + return nil + } + } + // reset pid and epoch if needed. + return t.initializeTransactions() +} + +// called before sending any transactional record +// won't do anything if current topic-partition is already added to transaction. +func (t *transactionManager) maybeAddPartitionToCurrentTxn(topic string, partition int32) { + if t.currentTxnStatus()&ProducerTxnFlagInError != 0 { + return + } + + tp := topicPartition{topic: topic, partition: partition} + + t.partitionInTxnLock.Lock() + defer t.partitionInTxnLock.Unlock() + if _, ok := t.partitionsInCurrentTxn[tp]; ok { + // partition is already added + return + } + + t.pendingPartitionsInCurrentTxn[tp] = struct{}{} +} + +// Makes a request to kafka to add a list of partitions ot the current transaction. +func (t *transactionManager) publishTxnPartitions() error { + t.partitionInTxnLock.Lock() + defer t.partitionInTxnLock.Unlock() + + if t.currentTxnStatus()&ProducerTxnFlagInError != 0 { + return t.lastError + } + + if len(t.pendingPartitionsInCurrentTxn) == 0 { + return nil + } + + // Remove the partitions from the pending set regardless of the result. We use the presence + // of partitions in the pending set to know when it is not safe to send batches. However, if + // the partitions failed to be added and we enter an error state, we expect the batches to be + // aborted anyway. In this case, we must be able to continue sending the batches which are in + // retry for partitions that were successfully added. + removeAllPartitionsOnFatalOrAbortedError := func() { + t.pendingPartitionsInCurrentTxn = topicPartitionSet{} + } + + // We only want to reduce the backoff when retrying the first AddPartition which errored out due to a + // CONCURRENT_TRANSACTIONS error since this means that the previous transaction is still completing and + // we don't want to wait too long before trying to start the new one. + // + // This is only a temporary fix, the long term solution is being tracked in + // https://issues.apache.org/jira/browse/KAFKA-5482 + retryBackoff := t.client.Config().Producer.Transaction.Retry.Backoff + computeBackoff := func(attemptsRemaining int) time.Duration { + if t.client.Config().Producer.Transaction.Retry.BackoffFunc != nil { + maxRetries := t.client.Config().Producer.Transaction.Retry.Max + retries := maxRetries - attemptsRemaining + return t.client.Config().Producer.Transaction.Retry.BackoffFunc(retries, maxRetries) + } + return retryBackoff + } + attemptsRemaining := t.client.Config().Producer.Transaction.Retry.Max + + exec := func(run func() (bool, error), err error) error { + for attemptsRemaining >= 0 { + var retry bool + retry, err = run() + if !retry { + return err + } + backoff := computeBackoff(attemptsRemaining) + Logger.Printf("txnmgr/add-partition-to-txn retrying after %dms... (%d attempts remaining) (%s)\n", backoff/time.Millisecond, attemptsRemaining, err) + time.Sleep(backoff) + attemptsRemaining-- + } + return err + } + return exec(func() (bool, error) { + coordinator, err := t.client.TransactionCoordinator(t.transactionalID) + if err != nil { + return true, err + } + addPartResponse, err := coordinator.AddPartitionsToTxn(&AddPartitionsToTxnRequest{ + TransactionalID: t.transactionalID, + ProducerID: t.producerID, + ProducerEpoch: t.producerEpoch, + TopicPartitions: t.pendingPartitionsInCurrentTxn.mapToRequest(), + }) + + if err != nil { + _ = coordinator.Close() + _ = t.client.RefreshTransactionCoordinator(t.transactionalID) + return true, err + } + + if addPartResponse == nil { + return true, ErrTxnUnableToParseResponse + } + + // remove from the list partitions that have been successfully updated + var responseErrors []error + for topic, results := range addPartResponse.Errors { + for _, response := range results { + tp := topicPartition{topic: topic, partition: response.Partition} + switch response.Err { + case ErrNoError: + // Mark partition as added to transaction + t.partitionsInCurrentTxn[tp] = struct{}{} + delete(t.pendingPartitionsInCurrentTxn, tp) + continue + case ErrConsumerCoordinatorNotAvailable: + fallthrough + case ErrNotCoordinatorForConsumer: + _ = coordinator.Close() + _ = t.client.RefreshTransactionCoordinator(t.transactionalID) + fallthrough + case ErrUnknownTopicOrPartition: + fallthrough + case ErrOffsetsLoadInProgress: + // Retry topicPartition + case ErrConcurrentTransactions: + if len(t.partitionsInCurrentTxn) == 0 && retryBackoff > addPartitionsRetryBackoff { + retryBackoff = addPartitionsRetryBackoff + } + case ErrOperationNotAttempted: + fallthrough + case ErrTopicAuthorizationFailed: + removeAllPartitionsOnFatalOrAbortedError() + return false, t.transitionTo(ProducerTxnFlagInError|ProducerTxnFlagAbortableError, response.Err) + case ErrUnknownProducerID: + fallthrough + case ErrInvalidProducerIDMapping: + removeAllPartitionsOnFatalOrAbortedError() + return false, t.abortableErrorIfPossible(response.Err) + // Fatal errors + default: + removeAllPartitionsOnFatalOrAbortedError() + return false, t.transitionTo(ProducerTxnFlagInError|ProducerTxnFlagFatalError, response.Err) + } + responseErrors = append(responseErrors, response.Err) + } + } + + // handle end + if len(t.pendingPartitionsInCurrentTxn) == 0 { + DebugLogger.Printf("txnmgr/add-partition-to-txn [%s] successful to add partitions txn %+v\n", + t.transactionalID, addPartResponse) + return false, nil + } + return true, Wrap(ErrAddPartitionsToTxn, responseErrors...) + }, nil) +} + +// Build a new transaction manager sharing producer client. +func newTransactionManager(conf *Config, client Client) (*transactionManager, error) { + txnmgr := &transactionManager{ + producerID: noProducerID, + producerEpoch: noProducerEpoch, + client: client, + pendingPartitionsInCurrentTxn: topicPartitionSet{}, + partitionsInCurrentTxn: topicPartitionSet{}, + offsetsInCurrentTxn: make(map[string]topicPartitionOffsets), + status: ProducerTxnFlagUninitialized, + } + + if conf.Producer.Idempotent { + txnmgr.transactionalID = conf.Producer.Transaction.ID + txnmgr.transactionTimeout = conf.Producer.Transaction.Timeout + txnmgr.sequenceNumbers = make(map[string]int32) + txnmgr.mutex = sync.Mutex{} + + var err error + txnmgr.producerID, txnmgr.producerEpoch, err = txnmgr.initProducerId() + if err != nil { + return nil, err + } + Logger.Printf("txnmgr/init-producer-id [%s] obtained a ProducerId: %d and ProducerEpoch: %d\n", + txnmgr.transactionalID, txnmgr.producerID, txnmgr.producerEpoch) + } + + return txnmgr, nil +} + +// re-init producer-id and producer-epoch if needed. +func (t *transactionManager) initializeTransactions() (err error) { + t.producerID, t.producerEpoch, err = t.initProducerId() + return +} diff --git a/transaction_manager_test.go b/transaction_manager_test.go new file mode 100644 index 000000000..9516f6dba --- /dev/null +++ b/transaction_manager_test.go @@ -0,0 +1,985 @@ +package sarama + +import ( + "errors" + "testing" + + "github.com/stretchr/testify/require" +) + +func TestTransitions(t *testing.T) { + testError := errors.New("test") + type testCase struct { + transitions []ProducerTxnStatusFlag + expectedError error + } + testCases := []testCase{ + { + transitions: []ProducerTxnStatusFlag{ + ProducerTxnFlagUninitialized, + ProducerTxnFlagReady, + ProducerTxnFlagInTransaction, + ProducerTxnFlagEndTransaction | ProducerTxnFlagCommittingTransaction, + ProducerTxnFlagReady, + }, + expectedError: nil, + }, + { + transitions: []ProducerTxnStatusFlag{ + ProducerTxnFlagUninitialized, + ProducerTxnFlagReady, + ProducerTxnFlagInTransaction, + ProducerTxnFlagEndTransaction | ProducerTxnFlagAbortingTransaction, + ProducerTxnFlagReady, + }, + expectedError: nil, + }, + { + transitions: []ProducerTxnStatusFlag{ + ProducerTxnFlagUninitialized, + ProducerTxnFlagReady, + ProducerTxnFlagInTransaction, + ProducerTxnFlagEndTransaction, + ProducerTxnFlagInError | ProducerTxnFlagAbortableError, + }, + expectedError: testError, + }, + { + transitions: []ProducerTxnStatusFlag{ + ProducerTxnFlagInError | ProducerTxnFlagAbortableError, + ProducerTxnFlagEndTransaction | ProducerTxnFlagAbortingTransaction, + ProducerTxnFlagReady, + }, + expectedError: nil, + }, + { + transitions: []ProducerTxnStatusFlag{ + ProducerTxnFlagInError | ProducerTxnFlagAbortableError, + ProducerTxnFlagEndTransaction | ProducerTxnFlagCommittingTransaction, + }, + expectedError: ErrTransitionNotAllowed, + }, + { + transitions: []ProducerTxnStatusFlag{ + ProducerTxnFlagInError | ProducerTxnFlagFatalError, + ProducerTxnFlagEndTransaction | ProducerTxnFlagAbortingTransaction, + }, + expectedError: ErrTransitionNotAllowed, + }, + } + for _, tc := range testCases { + txnmgr := transactionManager{} + txnmgr.status = tc.transitions[0] + var lastError error + for i := 1; i < len(tc.transitions); i++ { + var baseErr error + if tc.transitions[i]&ProducerTxnFlagInError != 0 { + baseErr = testError + } + lastError = txnmgr.transitionTo(tc.transitions[i], baseErr) + } + require.Equal(t, tc.expectedError, lastError, tc) + } +} + +func TestTxnmgrInitProducerIdTxn(t *testing.T) { + broker := NewMockBroker(t, 1) + defer broker.Close() + + metadataLeader := new(MetadataResponse) + metadataLeader.Version = 1 + metadataLeader.ControllerID = broker.brokerID + metadataLeader.AddBroker(broker.Addr(), broker.BrokerID()) + broker.Returns(metadataLeader) + + config := NewTestConfig() + config.Producer.Idempotent = true + config.Producer.Transaction.ID = "test" + config.Version = V0_11_0_0 + config.Producer.RequiredAcks = WaitForAll + config.Net.MaxOpenRequests = 1 + + client, err := NewClient([]string{broker.Addr()}, config) + require.NoError(t, err) + defer client.Close() + + findCoordinatorResponse := FindCoordinatorResponse{ + Coordinator: client.Brokers()[0], + Err: ErrNoError, + Version: 1, + } + broker.Returns(&findCoordinatorResponse) + + producerIdResponse := &InitProducerIDResponse{ + Err: ErrNoError, + ProducerID: 1, + ProducerEpoch: 0, + } + broker.Returns(producerIdResponse) + + txmng, err := newTransactionManager(config, client) + require.NoError(t, err) + + require.Equal(t, int64(1), txmng.producerID) + require.Equal(t, int16(0), txmng.producerEpoch) + require.Equal(t, ProducerTxnFlagReady, txmng.status) +} + +func TestMaybeAddPartitionToCurrentTxn(t *testing.T) { + type testCase struct { + initialFlags ProducerTxnStatusFlag + initialPartitionsInCurrentTxn topicPartitionSet + initialPendingPartitionsInCurrentTxn topicPartitionSet + + tpToAdd map[string][]int32 + + expectedPendingPartitions topicPartitionSet + expectedPartitionsInTxn topicPartitionSet + } + testCases := []testCase{ + { + initialFlags: ProducerTxnFlagInTransaction, + initialPartitionsInCurrentTxn: topicPartitionSet{ + {topic: "test-topic", partition: 0}: struct{}{}, + }, + initialPendingPartitionsInCurrentTxn: topicPartitionSet{}, + tpToAdd: map[string][]int32{ + "test-topic": { + 0, + }, + }, + expectedPendingPartitions: topicPartitionSet{}, + expectedPartitionsInTxn: topicPartitionSet{ + {topic: "test-topic", partition: 0}: struct{}{}, + }, + }, + { + initialFlags: ProducerTxnFlagInTransaction, + initialPartitionsInCurrentTxn: topicPartitionSet{}, + initialPendingPartitionsInCurrentTxn: topicPartitionSet{}, + tpToAdd: map[string][]int32{ + "test-topic": { + 0, + }, + }, + expectedPendingPartitions: topicPartitionSet{ + {topic: "test-topic", partition: 0}: struct{}{}, + }, + expectedPartitionsInTxn: topicPartitionSet{}, + }, + { + initialFlags: ProducerTxnFlagInTransaction, + initialPartitionsInCurrentTxn: topicPartitionSet{ + {topic: "test-topic", partition: 0}: struct{}{}, + }, + initialPendingPartitionsInCurrentTxn: topicPartitionSet{}, + tpToAdd: map[string][]int32{ + "test-topic": { + 0, + }, + }, + expectedPendingPartitions: topicPartitionSet{}, + expectedPartitionsInTxn: topicPartitionSet{ + {topic: "test-topic", partition: 0}: struct{}{}, + }, + }, + { + initialFlags: ProducerTxnFlagInTransaction, + initialPartitionsInCurrentTxn: topicPartitionSet{}, + initialPendingPartitionsInCurrentTxn: topicPartitionSet{ + {topic: "test-topic", partition: 0}: struct{}{}, + }, + tpToAdd: map[string][]int32{ + "test-topic": { + 0, + }, + }, + expectedPendingPartitions: topicPartitionSet{ + {topic: "test-topic", partition: 0}: struct{}{}, + }, + expectedPartitionsInTxn: topicPartitionSet{}, + }, + { + initialFlags: ProducerTxnFlagInError, + initialPartitionsInCurrentTxn: topicPartitionSet{}, + initialPendingPartitionsInCurrentTxn: topicPartitionSet{}, + tpToAdd: map[string][]int32{ + "test-topic": { + 0, + }, + }, + expectedPendingPartitions: topicPartitionSet{}, + expectedPartitionsInTxn: topicPartitionSet{}, + }, + } + + broker := NewMockBroker(t, 1) + defer broker.Close() + + metadataLeader := new(MetadataResponse) + metadataLeader.Version = 1 + metadataLeader.ControllerID = broker.brokerID + metadataLeader.AddBroker(broker.Addr(), broker.BrokerID()) + metadataLeader.AddTopic("test-topic", ErrNoError) + metadataLeader.AddTopicPartition("test-topic", 0, broker.BrokerID(), nil, nil, nil, ErrNoError) + + config := NewTestConfig() + config.Producer.Idempotent = true + config.Producer.Transaction.ID = "test" + config.Version = V0_11_0_0 + config.Producer.RequiredAcks = WaitForAll + config.Net.MaxOpenRequests = 1 + config.Producer.Transaction.Retry.Max = 0 + config.Producer.Transaction.Retry.Backoff = 0 + + for _, tc := range testCases { + func() { + broker.Returns(metadataLeader) + + client, err := NewClient([]string{broker.Addr()}, config) + require.NoError(t, err) + defer client.Close() + + findCoordinatorResponse := FindCoordinatorResponse{ + Coordinator: client.Brokers()[0], + Err: ErrNoError, + Version: 1, + } + broker.Returns(&findCoordinatorResponse) + + producerIdResponse := &InitProducerIDResponse{ + Err: ErrNoError, + ProducerID: 1, + ProducerEpoch: 0, + } + broker.Returns(producerIdResponse) + + txmng, err := newTransactionManager(config, client) + require.NoError(t, err) + + txmng.partitionsInCurrentTxn = tc.initialPartitionsInCurrentTxn + txmng.pendingPartitionsInCurrentTxn = tc.initialPendingPartitionsInCurrentTxn + txmng.status = tc.initialFlags + + for topic, partitions := range tc.tpToAdd { + for _, partition := range partitions { + txmng.maybeAddPartitionToCurrentTxn(topic, partition) + } + } + require.Equal(t, tc.expectedPartitionsInTxn, txmng.partitionsInCurrentTxn, tc) + require.Equal(t, tc.expectedPendingPartitions, txmng.pendingPartitionsInCurrentTxn, tc) + }() + } +} + +func TestAddOffsetsToTxn(t *testing.T) { + type testCase struct { + brokerErr KError + initialFlags ProducerTxnStatusFlag + expectedFlags ProducerTxnStatusFlag + expectedError error + newOffsets topicPartitionOffsets + } + + originalOffsets := topicPartitionOffsets{ + topicPartition{topic: "test-topic", partition: 0}: { + Partition: 0, + Offset: 0, + }, + } + + testCases := []testCase{ + { + brokerErr: ErrNoError, + initialFlags: ProducerTxnFlagInTransaction, + expectedFlags: ProducerTxnFlagInTransaction, + expectedError: nil, + newOffsets: topicPartitionOffsets{}, + }, + { + brokerErr: ErrConsumerCoordinatorNotAvailable, + initialFlags: ProducerTxnFlagInTransaction, + expectedFlags: ProducerTxnFlagInTransaction, + expectedError: ErrConsumerCoordinatorNotAvailable, + newOffsets: originalOffsets, + }, + { + brokerErr: ErrNotCoordinatorForConsumer, + initialFlags: ProducerTxnFlagInTransaction, + expectedFlags: ProducerTxnFlagInTransaction, + expectedError: ErrNotCoordinatorForConsumer, + newOffsets: originalOffsets, + }, + { + brokerErr: ErrOffsetsLoadInProgress, + initialFlags: ProducerTxnFlagInTransaction, + expectedFlags: ProducerTxnFlagInTransaction, + expectedError: ErrOffsetsLoadInProgress, + newOffsets: originalOffsets, + }, + { + brokerErr: ErrConcurrentTransactions, + initialFlags: ProducerTxnFlagInTransaction, + expectedFlags: ProducerTxnFlagInTransaction, + expectedError: ErrConcurrentTransactions, + newOffsets: originalOffsets, + }, + { + brokerErr: ErrUnknownProducerID, + initialFlags: ProducerTxnFlagInTransaction, + expectedFlags: ProducerTxnFlagFatalError, + expectedError: ErrUnknownProducerID, + newOffsets: originalOffsets, + }, + { + brokerErr: ErrInvalidProducerIDMapping, + initialFlags: ProducerTxnFlagInTransaction, + expectedFlags: ProducerTxnFlagFatalError, + expectedError: ErrInvalidProducerIDMapping, + newOffsets: originalOffsets, + }, + { + brokerErr: ErrGroupAuthorizationFailed, + initialFlags: ProducerTxnFlagInTransaction, + expectedFlags: ProducerTxnFlagAbortableError, + expectedError: ErrGroupAuthorizationFailed, + newOffsets: originalOffsets, + }, + } + + broker := NewMockBroker(t, 1) + defer broker.Close() + + metadataLeader := new(MetadataResponse) + metadataLeader.Version = 1 + metadataLeader.ControllerID = broker.brokerID + metadataLeader.AddBroker(broker.Addr(), broker.BrokerID()) + metadataLeader.AddTopic("test-topic", ErrNoError) + metadataLeader.AddTopicPartition("test-topic", 0, broker.BrokerID(), nil, nil, nil, ErrNoError) + + config := NewTestConfig() + config.Producer.Idempotent = true + config.Producer.Transaction.ID = "test" + config.Version = V0_11_0_0 + config.Producer.RequiredAcks = WaitForAll + config.Net.MaxOpenRequests = 1 + config.Producer.Transaction.Retry.Max = 0 + config.Producer.Transaction.Retry.Backoff = 0 + + offsets := topicPartitionOffsets{ + topicPartition{topic: "test-topic", partition: 0}: { + Partition: 0, + Offset: 0, + }, + } + + for _, tc := range testCases { + func() { + broker.Returns(metadataLeader) + + client, err := NewClient([]string{broker.Addr()}, config) + require.NoError(t, err) + + defer client.Close() + + findCoordinatorResponse := FindCoordinatorResponse{ + Coordinator: client.Brokers()[0], + Err: ErrNoError, + Version: 1, + } + broker.Returns(&findCoordinatorResponse) + + producerIdResponse := &InitProducerIDResponse{ + Err: ErrNoError, + ProducerID: 1, + ProducerEpoch: 0, + } + broker.Returns(producerIdResponse) + + txmng, err := newTransactionManager(config, client) + require.NoError(t, err) + + txmng.status = tc.initialFlags + + broker.Returns(&AddOffsetsToTxnResponse{ + Err: tc.brokerErr, + }) + + if errors.Is(tc.brokerErr, ErrRequestTimedOut) || + errors.Is(tc.brokerErr, ErrConsumerCoordinatorNotAvailable) || + errors.Is(tc.brokerErr, ErrNotCoordinatorForConsumer) { + broker.Returns(&FindCoordinatorResponse{ + Coordinator: client.Brokers()[0], + Err: ErrNoError, + Version: 1, + }) + } + + if tc.brokerErr == ErrNoError { + broker.Returns(&FindCoordinatorResponse{ + Coordinator: client.Brokers()[0], + Err: ErrNoError, + Version: 1, + }) + broker.Returns(&TxnOffsetCommitResponse{ + Topics: map[string][]*PartitionError{ + "test-topic": { + { + Partition: 0, + Err: ErrNoError, + }, + }, + }, + }) + } + + newOffsets, err := txmng.publishOffsetsToTxn(offsets, "test-group") + if tc.expectedError != nil { + require.Equal(t, tc.expectedError.Error(), err.Error()) + } else { + require.Equal(t, tc.expectedError, err) + } + require.Equal(t, tc.newOffsets, newOffsets) + require.True(t, tc.expectedFlags&txmng.status != 0) + }() + } +} + +func TestTxnOffsetsCommit(t *testing.T) { + type testCase struct { + brokerErr KError + initialFlags ProducerTxnStatusFlag + initialOffsets topicPartitionOffsets + expectedFlags ProducerTxnStatusFlag + expectedError error + expectedOffsets topicPartitionOffsets + } + + originalOffsets := topicPartitionOffsets{ + topicPartition{topic: "test-topic", partition: 0}: { + Partition: 0, + Offset: 0, + }, + } + + testCases := []testCase{ + { + brokerErr: ErrConsumerCoordinatorNotAvailable, + initialFlags: ProducerTxnFlagInTransaction, + initialOffsets: topicPartitionOffsets{ + topicPartition{topic: "test-topic", partition: 0}: { + Partition: 0, + Offset: 0, + }, + }, + expectedFlags: ProducerTxnFlagInTransaction, + expectedError: Wrap(ErrTxnOffsetCommit, ErrConsumerCoordinatorNotAvailable), + expectedOffsets: originalOffsets, + }, + { + brokerErr: ErrNotCoordinatorForConsumer, + initialFlags: ProducerTxnFlagInTransaction, + initialOffsets: topicPartitionOffsets{ + topicPartition{topic: "test-topic", partition: 0}: { + Partition: 0, + Offset: 0, + }, + }, + expectedFlags: ProducerTxnFlagInTransaction, + expectedError: Wrap(ErrTxnOffsetCommit, ErrNotCoordinatorForConsumer), + expectedOffsets: originalOffsets, + }, + { + brokerErr: ErrNoError, + initialFlags: ProducerTxnFlagInTransaction, + initialOffsets: topicPartitionOffsets{ + topicPartition{topic: "test-topic", partition: 0}: { + Partition: 0, + Offset: 0, + }, + }, + expectedFlags: ProducerTxnFlagInTransaction, + expectedError: nil, + expectedOffsets: topicPartitionOffsets{}, + }, + { + brokerErr: ErrUnknownTopicOrPartition, + initialFlags: ProducerTxnFlagInTransaction, + initialOffsets: topicPartitionOffsets{ + topicPartition{topic: "test-topic", partition: 0}: { + Partition: 0, + Offset: 0, + }, + }, + expectedFlags: ProducerTxnFlagInTransaction, + expectedError: Wrap(ErrTxnOffsetCommit, ErrUnknownTopicOrPartition), + expectedOffsets: originalOffsets, + }, + { + brokerErr: ErrOffsetsLoadInProgress, + initialFlags: ProducerTxnFlagInTransaction, + initialOffsets: topicPartitionOffsets{ + topicPartition{topic: "test-topic", partition: 0}: { + Partition: 0, + Offset: 0, + }, + }, + expectedFlags: ProducerTxnFlagInTransaction, + expectedError: Wrap(ErrTxnOffsetCommit, ErrOffsetsLoadInProgress), + expectedOffsets: originalOffsets, + }, + { + brokerErr: ErrIllegalGeneration, + initialFlags: ProducerTxnFlagInTransaction, + initialOffsets: topicPartitionOffsets{ + topicPartition{topic: "test-topic", partition: 0}: { + Partition: 0, + Offset: 0, + }, + }, + expectedFlags: ProducerTxnFlagAbortableError, + expectedError: ErrIllegalGeneration, + expectedOffsets: originalOffsets, + }, + { + brokerErr: ErrUnknownMemberId, + initialFlags: ProducerTxnFlagInTransaction, + initialOffsets: topicPartitionOffsets{ + topicPartition{topic: "test-topic", partition: 0}: { + Partition: 0, + Offset: 0, + }, + }, + expectedFlags: ProducerTxnFlagAbortableError, + expectedError: ErrUnknownMemberId, + expectedOffsets: originalOffsets, + }, + { + brokerErr: ErrFencedInstancedId, + initialFlags: ProducerTxnFlagInTransaction, + initialOffsets: topicPartitionOffsets{ + topicPartition{topic: "test-topic", partition: 0}: { + Partition: 0, + Offset: 0, + }, + }, + expectedFlags: ProducerTxnFlagAbortableError, + expectedError: ErrFencedInstancedId, + expectedOffsets: originalOffsets, + }, + { + brokerErr: ErrGroupAuthorizationFailed, + initialFlags: ProducerTxnFlagInTransaction, + initialOffsets: topicPartitionOffsets{ + topicPartition{topic: "test-topic", partition: 0}: { + Partition: 0, + Offset: 0, + }, + }, + expectedFlags: ProducerTxnFlagAbortableError, + expectedError: ErrGroupAuthorizationFailed, + expectedOffsets: originalOffsets, + }, + { + brokerErr: ErrKafkaStorageError, + initialFlags: ProducerTxnFlagInTransaction, + initialOffsets: topicPartitionOffsets{ + topicPartition{topic: "test-topic", partition: 0}: { + Partition: 0, + Offset: 0, + }, + }, + expectedFlags: ProducerTxnFlagFatalError, + expectedError: ErrKafkaStorageError, + expectedOffsets: originalOffsets, + }, + } + + broker := NewMockBroker(t, 1) + defer broker.Close() + + config := NewTestConfig() + config.Producer.Idempotent = true + config.Producer.Transaction.ID = "test" + config.Version = V0_11_0_0 + config.Producer.RequiredAcks = WaitForAll + config.Net.MaxOpenRequests = 1 + config.Producer.Transaction.Retry.Max = 0 + config.Producer.Transaction.Retry.Backoff = 0 + + metadataLeader := new(MetadataResponse) + metadataLeader.Version = 1 + metadataLeader.ControllerID = broker.brokerID + metadataLeader.AddBroker(broker.Addr(), broker.BrokerID()) + metadataLeader.AddTopic("test-topic", ErrNoError) + metadataLeader.AddTopicPartition("test-topic", 0, broker.BrokerID(), nil, nil, nil, ErrNoError) + + for _, tc := range testCases { + func() { + broker.Returns(metadataLeader) + + client, err := NewClient([]string{broker.Addr()}, config) + require.NoError(t, err) + defer client.Close() + + findCoordinatorResponse := FindCoordinatorResponse{ + Coordinator: client.Brokers()[0], + Err: ErrNoError, + Version: 1, + } + broker.Returns(&findCoordinatorResponse) + + producerIdResponse := &InitProducerIDResponse{ + Err: ErrNoError, + ProducerID: 1, + ProducerEpoch: 0, + } + broker.Returns(producerIdResponse) + + txmng, err := newTransactionManager(config, client) + require.NoError(t, err) + + txmng.status = tc.initialFlags + + broker.Returns(&AddOffsetsToTxnResponse{ + Err: ErrNoError, + }) + + broker.Returns(&FindCoordinatorResponse{ + Coordinator: client.Brokers()[0], + Err: ErrNoError, + Version: 1, + }) + broker.Returns(&TxnOffsetCommitResponse{ + Topics: map[string][]*PartitionError{ + "test-topic": { + { + Partition: 0, + Err: tc.brokerErr, + }, + }, + }, + }) + if errors.Is(tc.brokerErr, ErrRequestTimedOut) || + errors.Is(tc.brokerErr, ErrConsumerCoordinatorNotAvailable) || + errors.Is(tc.brokerErr, ErrNotCoordinatorForConsumer) { + broker.Returns(&FindCoordinatorResponse{ + Coordinator: client.Brokers()[0], + Err: ErrNoError, + Version: 1, + }) + } + + newOffsets, err := txmng.publishOffsetsToTxn(tc.initialOffsets, "test-group") + if tc.expectedError != nil { + require.Equal(t, tc.expectedError.Error(), err.Error()) + } else { + require.Equal(t, tc.expectedError, err) + } + require.Equal(t, tc.expectedOffsets, newOffsets) + require.True(t, tc.expectedFlags&txmng.status != 0) + }() + } +} + +func TestEndTxn(t *testing.T) { + type testCase struct { + brokerErr KError + commit bool + expectedFlags ProducerTxnStatusFlag + expectedError error + } + + testCases := []testCase{ + { + brokerErr: ErrNoError, + commit: true, + expectedFlags: ProducerTxnFlagReady, + expectedError: nil, + }, + { + brokerErr: ErrConsumerCoordinatorNotAvailable, + commit: true, + expectedFlags: ProducerTxnFlagEndTransaction, + expectedError: ErrConsumerCoordinatorNotAvailable, + }, + { + brokerErr: ErrNotCoordinatorForConsumer, + commit: true, + expectedFlags: ProducerTxnFlagEndTransaction, + expectedError: ErrNotCoordinatorForConsumer, + }, + { + brokerErr: ErrOffsetsLoadInProgress, + commit: true, + expectedFlags: ProducerTxnFlagEndTransaction, + expectedError: ErrOffsetsLoadInProgress, + }, + { + brokerErr: ErrConcurrentTransactions, + commit: true, + expectedFlags: ProducerTxnFlagEndTransaction, + expectedError: ErrConcurrentTransactions, + }, + { + brokerErr: ErrUnknownProducerID, + commit: true, + expectedFlags: ProducerTxnFlagFatalError, + expectedError: ErrUnknownProducerID, + }, + { + brokerErr: ErrInvalidProducerIDMapping, + commit: true, + expectedFlags: ProducerTxnFlagFatalError, + expectedError: ErrInvalidProducerIDMapping, + }, + } + + broker := NewMockBroker(t, 1) + defer broker.Close() + + metadataLeader := new(MetadataResponse) + metadataLeader.Version = 1 + metadataLeader.ControllerID = broker.brokerID + metadataLeader.AddBroker(broker.Addr(), broker.BrokerID()) + metadataLeader.AddTopic("test-topic", ErrNoError) + metadataLeader.AddTopicPartition("test-topic", 0, broker.BrokerID(), nil, nil, nil, ErrNoError) + + config := NewTestConfig() + config.Producer.Idempotent = true + config.Producer.Transaction.ID = "test" + config.Version = V0_11_0_0 + config.Producer.RequiredAcks = WaitForAll + config.Net.MaxOpenRequests = 1 + config.Producer.Transaction.Retry.Max = 0 + config.Producer.Transaction.Retry.Backoff = 0 + + for _, tc := range testCases { + func() { + broker.Returns(metadataLeader) + + client, err := NewClient([]string{broker.Addr()}, config) + require.NoError(t, err) + defer client.Close() + + findCoordinatorResponse := FindCoordinatorResponse{ + Coordinator: client.Brokers()[0], + Err: ErrNoError, + Version: 1, + } + broker.Returns(&findCoordinatorResponse) + + producerIdResponse := &InitProducerIDResponse{ + Err: ErrNoError, + ProducerID: 1, + ProducerEpoch: 0, + } + broker.Returns(producerIdResponse) + + txmng, err := newTransactionManager(config, client) + require.NoError(t, err) + + txmng.status = ProducerTxnFlagEndTransaction + + endTxnResponse := &EndTxnResponse{ + Err: tc.brokerErr, + ThrottleTime: 0, + } + broker.Returns(endTxnResponse) + + if errors.Is(tc.brokerErr, ErrRequestTimedOut) || + errors.Is(tc.brokerErr, ErrConsumerCoordinatorNotAvailable) || + errors.Is(tc.brokerErr, ErrNotCoordinatorForConsumer) { + broker.Returns(&FindCoordinatorResponse{ + Coordinator: client.Brokers()[0], + Err: ErrNoError, + Version: 1, + }) + } + + err = txmng.endTxn(tc.commit) + require.Equal(t, tc.expectedError, err) + require.True(t, txmng.currentTxnStatus()&tc.expectedFlags != 0) + }() + } +} + +func TestPublishPartitionToTxn(t *testing.T) { + type testCase struct { + brokerErr KError + expectedFlags ProducerTxnStatusFlag + expectedError error + expectedPendingPartitions topicPartitionSet + expectedPartitionsInTxn topicPartitionSet + } + + initialPendingTopicPartitionSet := topicPartitionSet{ + { + topic: "test-topic", + partition: 0, + }: struct{}{}, + } + + testCases := []testCase{ + { + brokerErr: ErrNoError, + expectedFlags: ProducerTxnFlagInTransaction, + expectedError: nil, + expectedPendingPartitions: topicPartitionSet{}, + expectedPartitionsInTxn: initialPendingTopicPartitionSet, + }, + { + brokerErr: ErrConsumerCoordinatorNotAvailable, + expectedFlags: ProducerTxnFlagInTransaction, + expectedError: Wrap(ErrAddPartitionsToTxn, ErrConsumerCoordinatorNotAvailable), + expectedPartitionsInTxn: topicPartitionSet{}, + expectedPendingPartitions: initialPendingTopicPartitionSet, + }, + { + brokerErr: ErrNotCoordinatorForConsumer, + expectedFlags: ProducerTxnFlagInTransaction, + expectedError: Wrap(ErrAddPartitionsToTxn, ErrNotCoordinatorForConsumer), + expectedPartitionsInTxn: topicPartitionSet{}, + expectedPendingPartitions: initialPendingTopicPartitionSet, + }, + { + brokerErr: ErrUnknownTopicOrPartition, + expectedFlags: ProducerTxnFlagInTransaction, + expectedError: Wrap(ErrAddPartitionsToTxn, ErrUnknownTopicOrPartition), + expectedPartitionsInTxn: topicPartitionSet{}, + expectedPendingPartitions: initialPendingTopicPartitionSet, + }, + { + brokerErr: ErrOffsetsLoadInProgress, + expectedFlags: ProducerTxnFlagInTransaction, + expectedError: Wrap(ErrAddPartitionsToTxn, ErrOffsetsLoadInProgress), + expectedPartitionsInTxn: topicPartitionSet{}, + expectedPendingPartitions: initialPendingTopicPartitionSet, + }, + { + brokerErr: ErrConcurrentTransactions, + expectedFlags: ProducerTxnFlagInTransaction, + expectedError: Wrap(ErrAddPartitionsToTxn, ErrConcurrentTransactions), + expectedPartitionsInTxn: topicPartitionSet{}, + expectedPendingPartitions: initialPendingTopicPartitionSet, + }, + { + brokerErr: ErrOperationNotAttempted, + expectedFlags: ProducerTxnFlagAbortableError, + expectedError: ErrOperationNotAttempted, + expectedPartitionsInTxn: topicPartitionSet{}, + expectedPendingPartitions: topicPartitionSet{}, + }, + { + brokerErr: ErrTopicAuthorizationFailed, + expectedFlags: ProducerTxnFlagAbortableError, + expectedError: ErrTopicAuthorizationFailed, + expectedPartitionsInTxn: topicPartitionSet{}, + expectedPendingPartitions: topicPartitionSet{}, + }, + { + brokerErr: ErrUnknownProducerID, + expectedFlags: ProducerTxnFlagFatalError, + expectedError: ErrUnknownProducerID, + expectedPartitionsInTxn: topicPartitionSet{}, + expectedPendingPartitions: topicPartitionSet{}, + }, + { + brokerErr: ErrInvalidProducerIDMapping, + expectedFlags: ProducerTxnFlagFatalError, + expectedError: ErrInvalidProducerIDMapping, + expectedPartitionsInTxn: topicPartitionSet{}, + expectedPendingPartitions: topicPartitionSet{}, + }, + { + brokerErr: ErrKafkaStorageError, + expectedFlags: ProducerTxnFlagFatalError, + expectedError: ErrKafkaStorageError, + expectedPartitionsInTxn: topicPartitionSet{}, + expectedPendingPartitions: topicPartitionSet{}, + }, + } + + broker := NewMockBroker(t, 1) + defer broker.Close() + + metadataLeader := new(MetadataResponse) + metadataLeader.Version = 1 + metadataLeader.ControllerID = broker.brokerID + metadataLeader.AddBroker(broker.Addr(), broker.BrokerID()) + metadataLeader.AddTopic("test-topic", ErrNoError) + metadataLeader.AddTopicPartition("test-topic", 0, broker.BrokerID(), nil, nil, nil, ErrNoError) + + config := NewTestConfig() + config.Producer.Idempotent = true + config.Producer.Transaction.ID = "test" + config.Version = V0_11_0_0 + config.Producer.RequiredAcks = WaitForAll + config.Net.MaxOpenRequests = 1 + config.Producer.Transaction.Retry.Max = 0 + config.Producer.Transaction.Retry.Backoff = 0 + + for _, tc := range testCases { + func() { + broker.Returns(metadataLeader) + + client, err := NewClient([]string{broker.Addr()}, config) + require.NoError(t, err) + defer client.Close() + + findCoordinatorResponse := FindCoordinatorResponse{ + Coordinator: client.Brokers()[0], + Err: ErrNoError, + Version: 1, + } + broker.Returns(&findCoordinatorResponse) + + producerIdResponse := &InitProducerIDResponse{ + Err: ErrNoError, + ProducerID: 1, + ProducerEpoch: 0, + } + broker.Returns(producerIdResponse) + + txmng, err := newTransactionManager(config, client) + require.NoError(t, err) + + txmng.status = ProducerTxnFlagInTransaction + txmng.pendingPartitionsInCurrentTxn = topicPartitionSet{ + { + topic: "test-topic", + partition: 0, + }: struct{}{}, + } + broker.Returns(&AddPartitionsToTxnResponse{ + Errors: map[string][]*PartitionError{ + "test-topic": { + { + Partition: 0, + Err: tc.brokerErr, + }, + }, + }, + }) + if errors.Is(tc.brokerErr, ErrRequestTimedOut) || + errors.Is(tc.brokerErr, ErrConsumerCoordinatorNotAvailable) || + errors.Is(tc.brokerErr, ErrNotCoordinatorForConsumer) { + broker.Returns(&FindCoordinatorResponse{ + Coordinator: client.Brokers()[0], + Err: ErrNoError, + Version: 1, + }) + } + err = txmng.publishTxnPartitions() + if tc.expectedError != nil { + require.Equal(t, tc.expectedError.Error(), err.Error(), tc) + } else { + require.Equal(t, tc.expectedError, err, tc) + } + + require.True(t, txmng.status&tc.expectedFlags != 0, tc) + require.Equal(t, tc.expectedPartitionsInTxn, txmng.partitionsInCurrentTxn, tc) + require.Equal(t, tc.expectedPendingPartitions, txmng.pendingPartitionsInCurrentTxn, tc) + }() + } +}