diff --git a/balance_strategy.go b/balance_strategy.go index 8635bdf7db..cc7a0a3697 100644 --- a/balance_strategy.go +++ b/balance_strategy.go @@ -19,6 +19,8 @@ const ( // StickyBalanceStrategyName identifies strategies that use the sticky-partition assignment strategy StickyBalanceStrategyName = "sticky" + CooperativeStickyBalanceStrategyName = "cooperative-sticky" + defaultGeneration = -1 ) @@ -40,6 +42,45 @@ func (p BalanceStrategyPlan) Add(memberID, topic string, partitions ...int32) { // -------------------------------------------------------------------- +type RebalanceProtocol int + +const ( + EAGER RebalanceProtocol = iota + COOPERATIVE +) + +func (p RebalanceProtocol) String() string { + switch p { + case EAGER: + return "EAGER" + case COOPERATIVE: + return "COOPERATIVE" + default: + return "UNKNOWN" + } +} + +type RebalanceProtocolSlice []RebalanceProtocol + +func (p RebalanceProtocolSlice) Len() int { return len(p) } +func (p RebalanceProtocolSlice) Less(i, j int) bool { return p[i] < p[j] } +func (p RebalanceProtocolSlice) Swap(i, j int) { p[i], p[j] = p[j], p[i] } + +func (p RebalanceProtocolSlice) retainAll(p2 RebalanceProtocolSlice) RebalanceProtocolSlice { + var result RebalanceProtocolSlice + set := make(map[RebalanceProtocol]bool, len(p2)) + for _, v := range p2 { + set[v] = true + } + + for _, v := range p { + if set[v] { + result = append(result, v) + } + } + return result +} + // BalanceStrategy is used to balance topics and partitions // across members of a consumer group type BalanceStrategy interface { @@ -53,6 +94,8 @@ type BalanceStrategy interface { // AssignmentData returns the serialized assignment data for the specified // memberID AssignmentData(memberID string, topics map[string][]int32, generationID int32) ([]byte, error) + + SupportedProtocols() RebalanceProtocolSlice } // -------------------------------------------------------------------- @@ -111,6 +154,12 @@ func NewBalanceStrategySticky() BalanceStrategy { // Deprecated: use NewBalanceStrategySticky to avoid data race issue var BalanceStrategySticky = NewBalanceStrategySticky() +func NewBalanceStrategyCooperativeSticky() BalanceStrategy { + return &cooperativeStickyBalanceStrategy{ + stickyBalanceStrategy: &stickyBalanceStrategy{}, + } +} + // -------------------------------------------------------------------- type balanceStrategy struct { @@ -161,6 +210,10 @@ func (s *balanceStrategy) AssignmentData(memberID string, topics map[string][]in return nil, nil } +func (s *balanceStrategy) SupportedProtocols() RebalanceProtocolSlice { + return []RebalanceProtocol{EAGER} +} + type stickyBalanceStrategy struct { movements partitionMovements } @@ -279,6 +332,10 @@ func (s *stickyBalanceStrategy) AssignmentData(memberID string, topics map[strin }, nil) } +func (s *stickyBalanceStrategy) SupportedProtocols() RebalanceProtocolSlice { + return []RebalanceProtocol{EAGER} +} + func strsContains(s []string, value string) bool { for _, entry := range s { if entry == value { @@ -343,6 +400,73 @@ func (s *stickyBalanceStrategy) balance(currentAssignment map[string][]topicPart } } +type cooperativeStickyBalanceStrategy struct { + stickyBalanceStrategy *stickyBalanceStrategy +} + +func (cs *cooperativeStickyBalanceStrategy) Plan(members map[string]ConsumerGroupMemberMetadata, topics map[string][]int32) (BalanceStrategyPlan, error) { + assignments, err := cs.stickyBalanceStrategy.Plan(members, topics) + if err != nil { + return nil, err + } + partitionsTransferredOwnership := computePartitionsTransferringOwnership(members, assignments) + return adjustAssignment(assignments, partitionsTransferredOwnership), nil +} + +// Following the cooperative rebalancing protocol requires removing partitions that must first be revoked from the assignment +func adjustAssignment(assignments BalanceStrategyPlan, partitionsTransferredOwnership map[topicPartitionAssignment]bool) BalanceStrategyPlan { + newAssignments := make(BalanceStrategyPlan) + for memberID, assignment := range assignments { + newAssignments[memberID] = make(map[string][]int32) + for topic, partitions := range assignment { + for _, partition := range partitions { + tp := topicPartitionAssignment{Topic: topic, Partition: partition} + if !partitionsTransferredOwnership[tp] { + newAssignments[memberID][topic] = append(newAssignments[memberID][topic], partition) + } + } + } + } + return newAssignments +} + +func computePartitionsTransferringOwnership(members map[string]ConsumerGroupMemberMetadata, assignments BalanceStrategyPlan) map[topicPartitionAssignment]bool { + partitionsTransferringOwnership := make(map[topicPartitionAssignment]bool) + previousAssignmentSet := make(map[topicPartitionAssignment]string) + for prevMember, metadata := range members { + for _, ownedTopicPartitions := range metadata.OwnedPartitions { + for _, partition := range ownedTopicPartitions.Partitions { + previousAssignmentSet[topicPartitionAssignment{Topic: ownedTopicPartitions.Topic, Partition: partition}] = prevMember + } + } + } + + for currMember, assignment := range assignments { + for topic, partitions := range assignment { + for _, partition := range partitions { + tp := topicPartitionAssignment{Topic: topic, Partition: partition} + prevMember, exist := previousAssignmentSet[tp] + if exist && prevMember != currMember { + partitionsTransferringOwnership[tp] = true + } + } + } + } + return partitionsTransferringOwnership +} + +func (cs *cooperativeStickyBalanceStrategy) AssignmentData(memberID string, topics map[string][]int32, generationID int32) ([]byte, error) { + return cs.stickyBalanceStrategy.AssignmentData(memberID, topics, generationID) +} + +func (cs *cooperativeStickyBalanceStrategy) SupportedProtocols() RebalanceProtocolSlice { + return []RebalanceProtocol{COOPERATIVE, EAGER} +} + +func (cs *cooperativeStickyBalanceStrategy) Name() string { + return CooperativeStickyBalanceStrategyName +} + // NewBalanceStrategyRoundRobin returns a round-robin balance strategy, // which assigns partitions to members in alternating order. // For example, there are two topics (t0, t1) and two consumer (m0, m1), and each topic has three partitions (p0, p1, p2): @@ -416,6 +540,10 @@ func (b *roundRobinBalancer) AssignmentData(memberID string, topics map[string][ return nil, nil // do nothing for now } +func (b *roundRobinBalancer) SupportedProtocols() RebalanceProtocolSlice { + return []RebalanceProtocol{EAGER} +} + type topicAndPartition struct { topic string partition int32 diff --git a/consumer_group.go b/consumer_group.go index a0a1e1a41b..0edebafc39 100644 --- a/consumer_group.go +++ b/consumer_group.go @@ -45,6 +45,37 @@ type ConsumerGroup interface { // recreated to get the new claims. Consume(ctx context.Context, topics []string, handler ConsumerGroupHandler) error + // ConsumeV2 joins a cluster of consumers for a given list of topics and ConsumerGroupHandlerV2. + // It should be wrapped in an infinite loop so that it would join the group again after a rebalance. + // + // Normally, you should always pass the same topic list and ConsumerGroupHandlerV2 instance in ConsumeV2, + // unless you want to change the subscribed topics or the handler implementation during the lifetime of the consumer group. + // + // Unlike the above Consumer interface, ConsumeV2 implements both COOPERATIVE and EAGER rebalance protocol. + // + // COOPERATIVE rebalance protocol works as follows: + // 1. The consumers join the group (as explained in https://kafka.apache.org/documentation/#intro_consumers) + // and is assigned their "fair share" of partitions, aka 'claims'. + // 2. Comparing to the previous assignments, the newly-added partitions and revoked partitions are calculated. + // 3. For revoked partitions, `ConsumeClaim` loops of these partitions should be exited as quickly as possible. + // Then `Cleanup` hook is called to allow the user to perform any final tasks. + // Finally, marked offsets are committed one last time before claims are released. + // 4. For newly-added partitions, `Setup` hook is called to notify the user + // of the claims and allow any necessary preparation or alteration of state. + // Then several `ConsumeClaim` functions are called in separate goroutines, which is required to be thread-safe. + // 5. For intersection of the previous and current assignments, nothing happens. + // 6. If there are revoked partitions, ConsumeV2 will return. + // ConsumeV2 will be called again to trigger a new rebalance so that leader can re-assigned the revoked partitions + // to other consumers. + // + // The difference between COOPERATIVE and EAGER rebalance protocol is that, + // when a rebalance happens, EAGER rebalance protocol will revoke all the partitions in the current generation, + // no matter whether they will be assigned to the same consumer later or not. + // + // Please note, once ctx is done, `ConsumeClaim` loops must exit as quickly as possible. + // Otherwise, it will be kicked out of the next generation and cause offset commit failures. + ConsumeV2(ctx context.Context, topics []string, handler ConsumerGroupHandlerV2) error + // Errors returns a read channel of errors that occurred during the consumer life-cycle. // By default, errors are logged and not returned over this channel. // If you want to implement any custom error handling, set your config's @@ -65,13 +96,13 @@ type ConsumerGroup interface { // New calls to the broker will return records from these partitions if there are any to be fetched. Resume(partitions map[string][]int32) - // Pause suspends fetching from all partitions. Future calls to the broker will not return any + // PauseAll suspends fetching from all partitions. Future calls to the broker will not return any // records from these partitions until they have been resumed using Resume()/ResumeAll(). // Note that this method does not affect partition subscription. // In particular, it does not cause a group rebalance when automatic assignment is used. PauseAll() - // Resume resumes all partitions which have been paused with Pause()/PauseAll(). + // ResumeAll resumes all partitions which have been paused with Pause()/PauseAll(). // New calls to the broker will return records from these partitions if there are any to be fetched. ResumeAll() } @@ -84,6 +115,7 @@ type consumerGroup struct { groupID string groupInstanceId *string memberID string + generationID int32 errors chan error lock sync.Mutex @@ -93,6 +125,19 @@ type consumerGroup struct { userData []byte + isLeader bool + protocol RebalanceProtocol + ownedPartitions map[string][]int32 + offsetManager *offsetManager + claims map[string]map[int32]*partitionClaim + claimsLock sync.RWMutex + handlerV2 ConsumerGroupHandlerV2 + strategy BalanceStrategy + allSubscribedTopicPartitions map[string][]int32 + allSubscribedTopics []string + wg sync.WaitGroup + subscribedTopics []string + metricRegistry metrics.Registry } @@ -107,6 +152,8 @@ func NewConsumerGroup(addrs []string, groupID string, config *Config) (ConsumerG if err != nil { _ = client.Close() } + + // start heartbeat loop return c, err } @@ -132,18 +179,48 @@ func newConsumerGroup(groupID string, client Client) (ConsumerGroup, error) { } cg := &consumerGroup{ - client: client, - consumer: consumer, - config: config, - groupID: groupID, - errors: make(chan error, config.ChannelBufferSize), - closed: make(chan none), - userData: config.Consumer.Group.Member.UserData, - metricRegistry: newCleanupRegistry(config.MetricRegistry), + client: client, + consumer: consumer, + config: config, + groupID: groupID, + errors: make(chan error, config.ChannelBufferSize), + closed: make(chan none), + userData: config.Consumer.Group.Member.UserData, + ownedPartitions: make(map[string][]int32), + metricRegistry: newCleanupRegistry(config.MetricRegistry), + claims: make(map[string]map[int32]*partitionClaim), } if config.Consumer.Group.InstanceId != "" && config.Version.IsAtLeast(V2_3_0_0) { cg.groupInstanceId = &config.Consumer.Group.InstanceId } + + // select the rebalance protocol such that: + // 1. only consider protocols that are supported by all BalanceStrategies. If there is no common protocols supported + // across all the BalanceStrategies, return an error. + // 2. if there are multiple protocols that are commonly supported, select the one with the highest value (i.e. the + // value indicates how advanced the protocol is) + var supportedProtocols RebalanceProtocolSlice + if config.Consumer.Group.Rebalance.Strategy != nil { + supportedProtocols = config.Consumer.Group.Rebalance.Strategy.SupportedProtocols() + } else { + supportedProtocols = config.Consumer.Group.Rebalance.GroupStrategies[0].SupportedProtocols() + for _, strategy := range config.Consumer.Group.Rebalance.GroupStrategies { + supportedProtocols = supportedProtocols.retainAll(strategy.SupportedProtocols()) + } + } + if len(supportedProtocols) == 0 { + return nil, ConfigurationError("no common rebalance protocol found") + } + sort.Sort(supportedProtocols) + cg.protocol = supportedProtocols[len(supportedProtocols)-1] + + Logger.Printf("select %s rebalance protocol", cg.protocol) + + cg.offsetManager, err = newOffsetManagerFromClient(cg.groupID, "", GroupGenerationUndefined, client, nil) + if err != nil { + return nil, err + } + return cg, nil } @@ -155,6 +232,18 @@ func (c *consumerGroup) Close() (err error) { c.closeOnce.Do(func() { close(c.closed) + // In cooperative rebalance protocol, we need to revoke all owned partitions before leaving the group. + c.lock.Lock() + c.revokedOwnedPartitions() + c.lock.Unlock() + + // wait for all ConsumeClaim goroutines to exit + c.wg.Wait() + + if c.offsetManager != nil { + err = c.offsetManager.Close() + } + // leave group if e := c.leave(); e != nil { err = e @@ -189,6 +278,9 @@ func (c *consumerGroup) Consume(ctx context.Context, topics []string, handler Co default: } + if c.protocol == COOPERATIVE { + return fmt.Errorf("use ConsumeV2 instead of Consume for cooperative rebalance protocol") + } c.lock.Lock() defer c.lock.Unlock() @@ -217,6 +309,267 @@ func (c *consumerGroup) Consume(ctx context.Context, topics []string, handler Co return sess.release(true) } +// todo: check pause & resume logic +func (c *consumerGroup) ConsumeV2(ctx context.Context, topics []string, handlerV2 ConsumerGroupHandlerV2) error { + // Quick exit when no topics are provided + if len(topics) == 0 { + return fmt.Errorf("no topics provided") + } + + if handlerV2 == nil { + return fmt.Errorf("nil handler provided") + } + + // Ensure group is not closed + select { + case <-c.closed: + return ErrClosedConsumerGroup + default: + } + + c.lock.Lock() + defer c.lock.Unlock() + + return c.start(ctx, topics, handlerV2) +} + +func (c *consumerGroup) start(ctx context.Context, topics []string, handlerV2 ConsumerGroupHandlerV2) error { + c.modifySubscribedTopicsAndListener(topics, handlerV2) + + ctx, cancelFunc := context.WithCancel(ctx) + defer cancelFunc() + + // todo: heartbeat should end here + syncGroupResponse, err := c.joinGroup(ctx, topics, c.config.Consumer.Group.Rebalance.Retry.Max) + if err != nil { + c.revokedOwnedPartitions() + return err + } + + // start heartbeat once the status is stable + hbDying, hbDone := c.startHeartbeatLoop(cancelFunc) + + // update the offset manager with new generation, memberID + c.offsetManager.Update(c.memberID, c.generationID) + + // Retrieve and sort claims + var claims map[string][]int32 + if len(syncGroupResponse.MemberAssignment) > 0 { + members, err := syncGroupResponse.GetMemberAssignment() + if err != nil { + return err + } + claims = members.Topics + + // in the case of stateful balance strategies, hold on to the returned + // assignment metadata, otherwise, reset the statically defined consumer + // group metadata + if members.UserData != nil { + c.userData = members.UserData + } else { + c.userData = c.config.Consumer.Group.Member.UserData + } + + for _, partitions := range claims { + sort.Sort(int32Slice(partitions)) + } + } + + newAssignedPartitions := diffAssignment(claims, c.ownedPartitions) + if c.protocol == COOPERATIVE { + revokedPartitions := diffAssignment(c.ownedPartitions, claims) + Logger.Printf("updating consumer(group:%s, member:%s, generation:%d)\n"+ + "All Assignments: %v\n"+ + "New Partitions: %v\n"+ + "Revoked Partitions: %v\n", + c.groupID, c.memberID, c.generationID, claims, newAssignedPartitions, revokedPartitions) + + if len(revokedPartitions) > 0 { + err = c.revokedPartitions(revokedPartitions) + if err != nil { + Logger.Printf("error when revoking partitions: %v", err) + } + cancelFunc() + } + } + c.ownedPartitions = claims + + err = c.startNewPartitions(newAssignedPartitions) + if err != nil { + c.revokedOwnedPartitions() + cancelFunc() + } + + // only the leader needs to check whether there are newly-added partitions in order to trigger a rebalance + if c.isLeader { + go c.loopCheckPartitionNumbers(ctx, cancelFunc, c.allSubscribedTopicPartitions, c.allSubscribedTopics) + } + + select { + case <-c.closed: + cancelFunc() + case <-ctx.Done(): + } + Logger.Printf("consumer(group:%s, member:%s, generation:%d) context is done", c.groupID, c.memberID, c.generationID) + + // if using EAGER rebalance protocol, we need to revoke all owned partitions before sending new JoinGroupRequest + if c.protocol == EAGER { + c.revokedOwnedPartitions() + } + + // make sure heartbeat loop is stopped + close(hbDying) + <-hbDone + return nil +} + +func (c *consumerGroup) joinGroup(ctx context.Context, topics []string, retries int) (*SyncGroupResponse, error) { + coordinator, err := c.joinPrepare(topics) + if err != nil { + return c.retryJoinGroup(ctx, topics, retries-1, err) + } + + // todo: add metrics + + // Join consumer group + join, err := c.joinGroupRequest(coordinator, topics) + if err != nil { + _ = coordinator.Close() + return nil, err + } + + switch join.Err { + case ErrNoError: + c.memberID = join.MemberId + case ErrUnknownMemberId, ErrIllegalGeneration: + // reset member ID and retry immediately + c.memberID = "" + return c.joinGroup(ctx, topics, retries) + case ErrNotCoordinatorForConsumer, ErrRebalanceInProgress, ErrOffsetsLoadInProgress: + // retry after backoff + return c.retryJoinGroup(ctx, topics, retries, join.Err) + case ErrMemberIdRequired: + // from JoinGroupRequest v4 onwards (due to KIP-394) if the client starts + // with an empty member id, it needs to get the assigned id from the + // response and send another join request with that id to actually join the + // group + c.memberID = join.MemberId + return c.retryJoinGroup(ctx, topics, retries+1 /*keep retry time*/, join.Err) + case ErrFencedInstancedId: + if c.groupInstanceId != nil { + Logger.Printf("JoinGroup failed: group instance id %s has been fenced\n", *c.groupInstanceId) + } + return nil, join.Err + default: + return nil, join.Err + } + + c.generationID = join.GenerationId + + var strategy BalanceStrategy + var ok bool + if strategy = c.config.Consumer.Group.Rebalance.Strategy; strategy == nil { + strategy, ok = c.findStrategy(join.GroupProtocol, c.config.Consumer.Group.Rebalance.GroupStrategies) + if !ok { + // this case shouldn't happen in practice, since the leader will choose the protocol + // that all the members support + return nil, fmt.Errorf("unable to find selected strategy: %s", join.GroupProtocol) + } + } + c.strategy = strategy + + // Prepare distribution plan if we joined as the leader + var plan BalanceStrategyPlan + var members map[string]ConsumerGroupMemberMetadata + + if join.LeaderId == join.MemberId { + members, err = join.GetMembers() + if err != nil { + return nil, err + } + + c.allSubscribedTopicPartitions, c.allSubscribedTopics, plan, err = c.balance(strategy, members) + if err != nil { + return nil, err + } + } + + // Sync consumer group + syncGroupResponse, err := c.syncGroupRequest(coordinator, members, plan, join.GenerationId, strategy) + if err != nil { + _ = coordinator.Close() + return nil, err + } + + switch syncGroupResponse.Err { + case ErrNoError: + c.memberID = join.MemberId + c.generationID = join.GenerationId + case ErrUnknownMemberId, ErrIllegalGeneration: + // reset member ID and retry immediately + c.memberID = "" + return c.joinGroup(ctx, topics, retries) + case ErrNotCoordinatorForConsumer, ErrRebalanceInProgress, ErrOffsetsLoadInProgress: + return c.retryJoinGroup(ctx, topics, retries, syncGroupResponse.Err) + case ErrFencedInstancedId: + if c.groupInstanceId != nil { + Logger.Printf("JoinGroup failed: group instance id %s has been fenced\n", *c.groupInstanceId) + } + return nil, syncGroupResponse.Err + default: + return nil, syncGroupResponse.Err + } + if join.LeaderId == join.MemberId { + c.isLeader = true + } + + if c.isLeader && c.protocol == COOPERATIVE && c.strategy.Name() != CooperativeStickyBalanceStrategyName { + err = validateCooperativeAssignment(members, plan) + if err != nil { + return nil, err + } + } + + return syncGroupResponse, nil +} + +func (c *consumerGroup) retryJoinGroup(ctx context.Context, topics []string, retries int, previousErr error) (*SyncGroupResponse, error) { + if retries <= 0 { + return nil, previousErr + } + + nextRetryTimer := time.NewTimer(c.config.Consumer.Group.Rebalance.Retry.Backoff) + defer nextRetryTimer.Stop() + + select { + case <-ctx.Done(): + return nil, previousErr + case <-nextRetryTimer.C: + return c.joinGroup(ctx, topics, retries) + case <-c.closed: + return nil, ErrClosedConsumerGroup + } +} + +// Used by COOPERATIVE rebalance protocol only. + +// Validate the assignments returned by the BalanceStrategy such that no owned partitions are going to +// be reassigned to a different consumer directly: if the BalanceStrategy wants to reassign an owned partition, +// it must first remove it from the new assignment of the current owner so that it is not assigned to any +// member, and then in the next rebalance it can finally reassign those partitions not owned by anyone to consumers. +func validateCooperativeAssignment(previousAssignment map[string]ConsumerGroupMemberMetadata, currentAssignment BalanceStrategyPlan) error { + set := computePartitionsTransferringOwnership(previousAssignment, currentAssignment) + if len(set) > 0 { + var topicPartitions []string + for k := range set { + topicPartitions = append(topicPartitions, fmt.Sprintf("%s/%d", k.Topic, k.Partition)) + } + return fmt.Errorf("in the customized cooperative rebalance strategy, "+ + "topic-partitions %v should be revoked before reassigning them to other consumers", topicPartitions) + } + return nil +} + // Pause implements ConsumerGroup. func (c *consumerGroup) Pause(partitions map[string][]int32) { c.consumer.Pause(partitions) @@ -302,6 +655,7 @@ func (c *consumerGroup) newSession(ctx context.Context, topics []string, handler switch join.Err { case ErrNoError: c.memberID = join.MemberId + c.generationID = join.GenerationId case ErrUnknownMemberId, ErrIllegalGeneration: // reset member ID and retry immediately c.memberID = "" @@ -417,15 +771,14 @@ func (c *consumerGroup) newSession(ctx context.Context, topics []string, handler sort.Sort(int32Slice(partitions)) } } - - session, err := newConsumerGroupSession(ctx, c, claims, join.MemberId, join.GenerationId, handler) + session, err := newConsumerGroupSession(ctx, c, claims, handler) if err != nil { return nil, err } // only the leader needs to check whether there are newly-added partitions in order to trigger a rebalance if join.LeaderId == join.MemberId { - go c.loopCheckPartitionNumbers(allSubscribedTopicPartitions, allSubscribedTopics, session) + go c.loopCheckPartitionNumbers(session.ctx, session.cancel, allSubscribedTopicPartitions, allSubscribedTopics) } return session, err @@ -464,9 +817,15 @@ func (c *consumerGroup) joinGroupRequest(coordinator *Broker, topics []string) ( } meta := &ConsumerGroupMemberMetadata{ + Version: 1, Topics: topics, UserData: c.userData, } + + for topic, partitions := range c.ownedPartitions { + meta.OwnedPartitions = append(meta.OwnedPartitions, &OwnedPartition{Topic: topic, Partitions: partitions}) + } + var strategy BalanceStrategy if strategy = c.config.Consumer.Group.Rebalance.Strategy; strategy != nil { if err := req.AddGroupProtocolMetadata(strategy.Name(), meta); err != nil { @@ -680,12 +1039,12 @@ func (c *consumerGroup) handleError(err error, topic string, partition int32) { } } -func (c *consumerGroup) loopCheckPartitionNumbers(allSubscribedTopicPartitions map[string][]int32, topics []string, session *consumerGroupSession) { +func (c *consumerGroup) loopCheckPartitionNumbers(ctx context.Context, cancelFunc context.CancelFunc, allSubscribedTopicPartitions map[string][]int32, topics []string) { if c.config.Metadata.RefreshFrequency == time.Duration(0) { return } - defer session.cancel() + defer cancelFunc() oldTopicToPartitionNum := make(map[string]int, len(allSubscribedTopicPartitions)) for topic, partitions := range allSubscribedTopicPartitions { @@ -703,17 +1062,16 @@ func (c *consumerGroup) loopCheckPartitionNumbers(allSubscribedTopicPartitions m Logger.Printf( "consumergroup/%s loop check partition number goroutine find partitions in topics %s changed from %d to %d\n", c.groupID, topics, num, newTopicToPartitionNum[topic]) - return // trigger the end of the session on exit + return // trigger defer cancelFunc() } } } select { case <-pause.C: - case <-session.ctx.Done(): + case <-ctx.Done(): Logger.Printf( "consumergroup/%s loop check partition number goroutine will exit, topics %s\n", c.groupID, topics) - // if session closed by other, should be exited return case <-c.closed: return @@ -736,6 +1094,122 @@ func (c *consumerGroup) topicToPartitionNumbers(topics []string) (map[string]int return topicToPartitionNum, nil } +func (c *consumerGroup) revokedPartitions(revokedPartitions map[string][]int32) error { + Logger.Printf("revoking partitions: %v", c.groupID, revokedPartitions) + + // close revoked partition consumers + c.removeClaims(revokedPartitions) + + c.handlerV2.Cleanup(c.offsetManager, revokedPartitions) + + // close partition offset managers for revoked partitions + if err := c.offsetManager.RemovePartitions(revokedPartitions); err != nil { + Logger.Printf("error when removing partition offset managers for %v, err: %v", revokedPartitions, err) + return err + } + return nil +} + +func (c *consumerGroup) revokedOwnedPartitions() { + if len(c.ownedPartitions) > 0 { + err := c.revokedPartitions(c.ownedPartitions) + if err != nil { + Logger.Printf("error revoking owned partitions: %v", err) + } + c.ownedPartitions = make(map[string][]int32) + } +} + +func (c *consumerGroup) startNewPartitions(newAssignedPartitions map[string][]int32) error { + // create partition offset managers for each new assigned partitions + for topic, partitions := range newAssignedPartitions { + for _, partition := range partitions { + pom, err := c.offsetManager.ManagePartition(topic, partition) + if err != nil { + Logger.Printf("unable to create partition offset manager for %s/%d, err: %v", topic, partition, err) + // todo maybe close all offset managers here + return err + } + + // handle POM errors + go func(topic string, partition int32) { + for err := range pom.Errors() { + c.handleError(err, topic, partition) + } + }(topic, partition) + } + } + + c.handlerV2.Setup(c.offsetManager, newAssignedPartitions) + + var errs ConsumerErrors + var errsLock sync.Mutex + + var wg sync.WaitGroup + // create partition consumers for each new assigned partitions + for topic, partitions := range newAssignedPartitions { + for _, partition := range partitions { + wg.Add(1) + // get next offset + go func(topic string, partition int32) { + defer wg.Done() + + offset := c.config.Consumer.Offsets.Initial + if pom := c.offsetManager.findPOM(topic, partition); pom != nil { + offset, _ = pom.NextOffset() + } + + claim, err := c.newConsumerGroupClaim(topic, partition, offset) + if err != nil { + Logger.Printf("unable to create consumer group claim for %s/%d, err: %v", topic, partition, err) + errsLock.Lock() + errs = append(errs, &ConsumerError{ + Topic: topic, + Partition: partition, + Err: err, + }) + errsLock.Unlock() + return + } + pc, err := c.addClaim(claim) + if err != nil { + Logger.Printf("unable to add consumer group claim for %s/%d, err: %v", topic, partition, err) + errsLock.Lock() + errs = append(errs, &ConsumerError{ + Topic: topic, + Partition: partition, + Err: err, + }) + errsLock.Unlock() + return + } + + // handle errors + go func(pc *partitionClaim) { + for err := range pc.claim.Errors() { + c.handleError(err, pc.claim.topic, pc.claim.partition) + } + }(pc) + + c.wg.Add(1) + go func(pc *partitionClaim) { + defer c.wg.Done() + + pc.wg.Add(1) + defer pc.wg.Done() + c.handlerV2.ConsumeClaim(pc.ctx, c.offsetManager, claim) + }(pc) + }(topic, partition) + } + } + wg.Wait() + + if len(errs) > 0 { + return errs + } + return nil +} + // -------------------------------------------------------------------- // ConsumerGroupSession represents a consumer group member session. @@ -799,12 +1273,12 @@ type consumerGroupSession struct { hbDying, hbDead chan none } -func newConsumerGroupSession(ctx context.Context, parent *consumerGroup, claims map[string][]int32, memberID string, generationID int32, handler ConsumerGroupHandler) (*consumerGroupSession, error) { +func newConsumerGroupSession(ctx context.Context, parent *consumerGroup, claims map[string][]int32, handler ConsumerGroupHandler) (*consumerGroupSession, error) { // init context ctx, cancel := context.WithCancel(ctx) // init offset manager - offsets, err := newOffsetManagerFromClient(parent.groupID, memberID, generationID, parent.client, cancel) + offsets, err := newOffsetManagerFromClient(parent.groupID, parent.memberID, parent.generationID, parent.client, cancel) if err != nil { return nil, err } @@ -812,8 +1286,8 @@ func newConsumerGroupSession(ctx context.Context, parent *consumerGroup, claims // init session sess := &consumerGroupSession{ parent: parent, - memberID: memberID, - generationID: generationID, + memberID: parent.memberID, + generationID: parent.generationID, handler: handler, offsets: offsets, claims: claims, @@ -915,7 +1389,7 @@ func (s *consumerGroupSession) consume(topic string, partition int32) { } // create new claim - claim, err := newConsumerGroupClaim(s, topic, partition, offset) + claim, err := s.parent.newConsumerGroupClaim(topic, partition, offset) if err != nil { s.parent.handleError(err, topic, partition) return @@ -1075,6 +1549,31 @@ type ConsumerGroupHandler interface { ConsumeClaim(ConsumerGroupSession, ConsumerGroupClaim) error } +// ConsumerGroupHandlerV2 instances are used to handle individual topic/partition claims. +// It also provides hooks triggered when adding new partitions or revoking existing ones. +// +// The difference with ConsumerGroupHandler interface is that ConsumerGroupHandlerV2 supports COOPERATIVE rebalancing protocol. +// You should always pass the same ConsumerGroupHandlerV2 instance in ConsumeV2, +// unless you want to change the handler implementation during the lifetime of the consumer group. +// +// PLEASE NOTE that ConsumeClaim is likely be called from several goroutines concurrently, +// ensure that all state is safely protected against race conditions. +type ConsumerGroupHandlerV2 interface { + // Setup runs at the beginning of setting up new assigned partitions, before ConsumeClaim. + // For EAGER rebalance strategy, this is to set up all assigned partitions. + // For COOPERATIVE rebalance strategy, this is only to set up new assigned partitions. + Setup(offsetManger OffsetManager, newAssignedPartitions map[string][]int32) + + // Cleanup runs after ConsumeClaim, but before the offsets are committed for the claim. + // For EAGER rebalance strategy, this is to clean up all assigned partitions. + // For COOPERATIVE rebalance strategy, this is only to clean up revoked partitions. + Cleanup(offsetManger OffsetManager, revokedPartitions map[string][]int32) + + // ConsumeClaim must start a consumer loop of ConsumerGroupClaim's Messages(). + // Once ctx is done, ConsumeClaim should return as soon as possible. + ConsumeClaim(ctx context.Context, offsetManger OffsetManager, claim ConsumerGroupClaim) +} + // ConsumerGroupClaim processes Kafka messages from a given topic and partition within a consumer group. type ConsumerGroupClaim interface { // Topic returns the consumed topic name. @@ -1106,12 +1605,12 @@ type consumerGroupClaim struct { PartitionConsumer } -func newConsumerGroupClaim(sess *consumerGroupSession, topic string, partition int32, offset int64) (*consumerGroupClaim, error) { - pcm, err := sess.parent.consumer.ConsumePartition(topic, partition, offset) +func (c *consumerGroup) newConsumerGroupClaim(topic string, partition int32, offset int64) (*consumerGroupClaim, error) { + pcm, err := c.consumer.ConsumePartition(topic, partition, offset) - if errors.Is(err, ErrOffsetOutOfRange) && sess.parent.config.Consumer.Group.ResetInvalidOffsets { - offset = sess.parent.config.Consumer.Offsets.Initial - pcm, err = sess.parent.consumer.ConsumePartition(topic, partition, offset) + if errors.Is(err, ErrOffsetOutOfRange) && c.config.Consumer.Group.ResetInvalidOffsets { + offset = c.config.Consumer.Offsets.Initial + pcm, err = c.consumer.ConsumePartition(topic, partition, offset) } if err != nil { return nil, err @@ -1119,7 +1618,7 @@ func newConsumerGroupClaim(sess *consumerGroupSession, topic string, partition i go func() { for err := range pcm.Errors() { - sess.parent.handleError(err, topic, partition) + c.handleError(err, topic, partition) } }() @@ -1131,6 +1630,229 @@ func newConsumerGroupClaim(sess *consumerGroupSession, topic string, partition i }, nil } +// todo: make it concurrent +func (c *consumerGroup) addClaim(claim *consumerGroupClaim) (*partitionClaim, error) { + ctx, cancelFunc := context.WithCancel(context.Background()) + // todo check duplication + pc := &partitionClaim{ + claim: claim, + ctx: ctx, + cancelFunc: cancelFunc, + } + + err := c.setPartitionClaim(pc) + if err != nil { + return nil, err + } + return pc, nil +} + +func (c *consumerGroup) removeClaims(revokedPartitions map[string][]int32) { + var wg sync.WaitGroup + + for topic, partitions := range revokedPartitions { + for _, partition := range partitions { + wg.Add(1) + go func(topic string, partition int32) { + defer wg.Done() + + pc := c.getPartitionClaim(topic, partition) + if pc == nil { + return + } + + pc.cancelFunc() + pc.claim.AsyncClose() + + // wait until ConsumerClaim goroutine returns + pc.wg.Wait() + // wait until claim is closed + for _, err := range pc.claim.waitClosed() { + c.handleError(err, topic, partition) + } + }(topic, partition) + } + } + wg.Wait() + + for topic, partitions := range revokedPartitions { + for _, partition := range partitions { + c.removePartitionClaim(topic, partition) + } + } +} + +func (c *consumerGroup) getPartitionClaim(topic string, partition int32) *partitionClaim { + c.claimsLock.RLock() + defer c.claimsLock.RUnlock() + + topicClaims, ok := c.claims[topic] + if !ok { + return nil + } + pc, ok := topicClaims[partition] + if !ok { + return nil + } + return pc +} + +func (c *consumerGroup) setPartitionClaim(pc *partitionClaim) error { + c.claimsLock.Lock() + defer c.claimsLock.Unlock() + + if _, ok := c.claims[pc.claim.topic]; !ok { + c.claims[pc.claim.topic] = make(map[int32]*partitionClaim) + } + if _, ok := c.claims[pc.claim.topic][pc.claim.partition]; ok { + // safeguard, should never happen + return fmt.Errorf("partition claim for %s/%d already exists", pc.claim.topic, pc.claim.partition) + } + c.claims[pc.claim.topic][pc.claim.partition] = pc + return nil +} + +func (c *consumerGroup) removePartitionClaim(topic string, partition int32) { + c.claimsLock.Lock() + defer c.claimsLock.Unlock() + + delete(c.claims[topic], partition) + if len(c.claims[topic]) == 0 { + delete(c.claims, topic) + } +} + +func (c *consumerGroup) joinPrepare(topics []string) (*Broker, error) { + // Refresh metadata for requested topics + if err := c.client.RefreshMetadata(topics...); err != nil { + return nil, err + } + + coordinator, err := c.client.Coordinator(c.groupID) + if err != nil { + return nil, err + } + + return coordinator, nil +} + +func (c *consumerGroup) startHeartbeatLoop(cancelFunc context.CancelFunc) (chan none, chan none) { + hbDone := make(chan none) + hbDying := make(chan none) + go func() { + defer close(hbDone) + + pause := time.NewTicker(c.config.Consumer.Group.Heartbeat.Interval) + defer pause.Stop() + + retryBackoff := time.NewTimer(c.config.Metadata.Retry.Backoff) + defer retryBackoff.Stop() + + retries := c.config.Metadata.Retry.Max + for { + coordinator, err := c.client.Coordinator(c.groupID) + if err != nil { + if retries <= 0 { + c.handleError(err, "", -1) + return + } + retryBackoff.Reset(c.config.Metadata.Retry.Backoff) + select { + case <-hbDying: + return + case <-retryBackoff.C: + retries-- + } + continue + } + + resp, err := c.heartbeatRequest(coordinator, c.memberID, c.generationID) + if err != nil { + _ = coordinator.Close() + + if retries <= 0 { + c.handleError(err, "", -1) + return + } + + retries-- + continue + } + + switch resp.Err { + case ErrNoError: + retries = c.config.Metadata.Retry.Max + case ErrRebalanceInProgress: + retries = c.config.Metadata.Retry.Max + cancelFunc() + case ErrUnknownMemberId, ErrIllegalGeneration: + retries = c.config.Metadata.Retry.Max + cancelFunc() + case ErrFencedInstancedId: + if c.groupInstanceId != nil { + Logger.Printf("heartbeat failed: group instance id %s has been fenced\n", *c.groupInstanceId) + } + c.handleError(resp.Err, "", -1) + retries = c.config.Metadata.Retry.Max + cancelFunc() + default: + c.handleError(resp.Err, "", -1) + Logger.Printf("heartbeat failed with unexpected error: %s\n", resp.Err) + retries = c.config.Metadata.Retry.Max + cancelFunc() + } + + select { + case <-pause.C: + case <-hbDying: + return + } + } + }() + return hbDying, hbDone +} + +func (c *consumerGroup) modifySubscribedTopicsAndListener(topics []string, handlerV2 ConsumerGroupHandlerV2) { + // for the first time + if c.handlerV2 == nil { + c.handlerV2 = handlerV2 + c.subscribedTopics = topics + return + } + + // if listener is changed, revoke all owned partitions + if c.handlerV2 != handlerV2 { + c.revokedOwnedPartitions() + c.handlerV2 = handlerV2 + c.subscribedTopics = topics + return + } + + // if topics are changed, only revoke removed topics + removedTopics := diffTopicSlice(c.subscribedTopics, topics) + + if len(removedTopics) > 0 { + removedTopicPartitions := make(map[string][]int32) + for topic, partitions := range c.ownedPartitions { + if _, ok := removedTopics[topic]; ok { + removedTopicPartitions[topic] = partitions + } + } + err := c.revokedPartitions(removedTopicPartitions) + if err != nil { + Logger.Printf("error revoking owned removed topic/partitions: %v", err) + } + } + c.subscribedTopics = topics +} + +type partitionClaim struct { + claim *consumerGroupClaim + ctx context.Context + cancelFunc context.CancelFunc + wg sync.WaitGroup +} + func (c *consumerGroupClaim) Topic() string { return c.topic } func (c *consumerGroupClaim) Partition() int32 { return c.partition } func (c *consumerGroupClaim) InitialOffset() int64 { return c.offset } diff --git a/consumer_group_members.go b/consumer_group_members.go index 3b8ca36f60..d222b97a5b 100644 --- a/consumer_group_members.go +++ b/consumer_group_members.go @@ -22,6 +22,16 @@ func (m *ConsumerGroupMemberMetadata) encode(pe packetEncoder) error { return err } + if m.Version >= 1 { + if err := pe.putArrayLength(len(m.OwnedPartitions)); err != nil { + return err + } + for _, op := range m.OwnedPartitions { + if err := op.encode(pe); err != nil { + return err + } + } + } return nil } @@ -68,6 +78,16 @@ type OwnedPartition struct { Partitions []int32 } +func (m *OwnedPartition) encode(pe packetEncoder) error { + if err := pe.putString(m.Topic); err != nil { + return err + } + if err := pe.putInt32Array(m.Partitions); err != nil { + return err + } + return nil +} + func (m *OwnedPartition) decode(pd packetDecoder) (err error) { if m.Topic, err = pd.getString(); err != nil { return err diff --git a/consumer_group_test.go b/consumer_group_test.go index 912b6aa4f3..ed32f7d5e4 100644 --- a/consumer_group_test.go +++ b/consumer_group_test.go @@ -224,3 +224,134 @@ func TestConsumerGroupSessionDoesNotRetryForever(t *testing.T) { wg.Wait() } + +func Test_validateCooperativeAssignment(t *testing.T) { + type args struct { + previousAssignment map[string]ConsumerGroupMemberMetadata + currentAssignment BalanceStrategyPlan + } + tests := []struct { + name string + args args + wantErr bool + }{ + { + name: "no previous assignment", + args: args{ + previousAssignment: nil, + currentAssignment: BalanceStrategyPlan{ + "member1": map[string][]int32{ + "topic1": {0, 1}, + }, + }, + }, + wantErr: false, + }, + { + name: "no current assignment", + args: args{ + previousAssignment: map[string]ConsumerGroupMemberMetadata{ + "member1": { + OwnedPartitions: []*OwnedPartition{ + { + Topic: "topic1", + Partitions: []int32{0, 1}, + }, + }, + }, + }, + currentAssignment: make(BalanceStrategyPlan), + }, + wantErr: false, + }, + { + name: "directly transfer one partition", + args: args{ + previousAssignment: map[string]ConsumerGroupMemberMetadata{ + "member1": { + OwnedPartitions: []*OwnedPartition{ + { + Topic: "topic1", + Partitions: []int32{0, 1}, + }, + }, + }, + "member2": { + OwnedPartitions: []*OwnedPartition{}, + }, + }, + currentAssignment: BalanceStrategyPlan{ + "member1": map[string][]int32{ + "topic1": {0}, + }, + "member2": map[string][]int32{ + "topic1": {1}, + }, + }, + }, + wantErr: true, + }, + { + name: "revoke one partition", + args: args{ + previousAssignment: map[string]ConsumerGroupMemberMetadata{ + "member1": { + OwnedPartitions: []*OwnedPartition{ + { + Topic: "topic1", + Partitions: []int32{0, 1}, + }, + }, + }, + "member2": { + OwnedPartitions: []*OwnedPartition{}, + }, + }, + currentAssignment: BalanceStrategyPlan{ + "member1": map[string][]int32{ + "topic1": {0}, + }, + "member2": map[string][]int32{}, + }, + }, + wantErr: false, + }, + { + name: "add one partition", + args: args{ + previousAssignment: map[string]ConsumerGroupMemberMetadata{ + "member1": { + OwnedPartitions: []*OwnedPartition{ + { + Topic: "topic1", + Partitions: []int32{0}, + }, + }, + }, + "member2": { + OwnedPartitions: []*OwnedPartition{}, + }, + }, + currentAssignment: BalanceStrategyPlan{ + "member1": map[string][]int32{ + "topic1": {0}, + }, + "member2": map[string][]int32{ + "topic1": {1}, + }, + }, + }, + wantErr: false, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + err := validateCooperativeAssignment(tt.args.previousAssignment, tt.args.currentAssignment) + if tt.wantErr { + assert.Error(t, err) + } else { + assert.NoError(t, err) + } + }) + } +} diff --git a/examples/consumergroup/go.sum b/examples/consumergroup/go.sum index f512224c0e..3ce0c297db 100644 --- a/examples/consumergroup/go.sum +++ b/examples/consumergroup/go.sum @@ -1,144 +1,17 @@ -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/toml v1.1.0/go.mod h1:CxXYINrC8qIiEnFrOxCa7Jy5BFHlXnUU2pbicEuybxQ= -github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= -github.com/Shopify/toxiproxy/v2 v2.5.0 h1:i4LPT+qrSlKNtQf5QliVjdP08GyAH8+BUIc9gT0eahc= -github.com/Shopify/toxiproxy/v2 v2.5.0/go.mod h1:yhM2epWtAmel9CB8r2+L+PCmhH6yH2pITaPAo7jxJl0= -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/coreos/go-systemd/v22 v22.3.3-0.20220203105225-a9a7ef127534/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= -github.com/cpuguy83/go-md2man/v2 v2.0.2/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= 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-20230111030713-bf00bc1b83b6 h1:8yY/I9ndfrgrXUbOGObLHKBR4Fl3nZXwM2c7OYTT8hM= -github.com/eapache/go-xerial-snappy v0.0.0-20230111030713-bf00bc1b83b6/go.mod h1:YvSRo5mw33fLEx1+DlK6L2VV43tJt5Eyel9n9XBcR+0= +github.com/eapache/go-resiliency v1.4.0 h1:3OK9bWpPk5q6pbFAaYSEwD9CLUSHG8bnZuqX2yMt3B0= +github.com/eapache/go-resiliency v1.4.0/go.mod h1:5yPzW0MIvSe0JDsv0v+DvcjEv2FyD6iZYSs1ZI+iQho= +github.com/eapache/go-xerial-snappy v0.0.0-20230731223053-c322873962e3 h1:Oy0F4ALJ04o5Qqpdz8XLIpNA3WM/iSIXqxtqo7UGVws= +github.com/eapache/go-xerial-snappy v0.0.0-20230731223053-c322873962e3/go.mod h1:YvSRo5mw33fLEx1+DlK6L2VV43tJt5Eyel9n9XBcR+0= 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-kit/log v0.2.0/go.mod h1:NwTd00d/i8cPZ3xOwwiv2PO5MOcx78fFErGNcVmBjv0= -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-logfmt/logfmt v0.5.1/go.mod h1:WYhtIu8zTZfxdn5+rREduYbwxfcBr/Vr6KEVveWlfTs= -github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= -github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= -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/go-cmp v0.5.8/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= -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= @@ -148,9 +21,6 @@ github.com/hashicorp/go-multierror v1.1.1/go.mod h1:iw975J/qwKPdAO1clOe2L8331t/9 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= @@ -159,420 +29,89 @@ github.com/jcmturner/gofork v1.7.6 h1:QH0l3hzAU1tfT3rZCnW5zXl+orbkNMMRGJfdJjHVET 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/gokrb5/v8 v8.4.4 h1:x1Sv4HaTpepFkXbt2IkL29DXRf8sOfZXo8eRKh687T8= +github.com/jcmturner/gokrb5/v8 v8.4.4/go.mod h1:1btQEpgT6k+unzCwX1KdWMEwPPkkgBtP+F6aCACiMrs= 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.16.6 h1:91SKEy4K37vkp255cJ8QesJhjyRO0hn9i9G0GoUwLsk= -github.com/klauspost/compress v1.16.6/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= -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/klauspost/compress v1.16.7 h1:2mk3MPGNzKyxErAw8YaohYh69+pa4sIQSC0fPGCFR9I= +github.com/klauspost/compress v1.16.7/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= 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/mattn/go-colorable v0.1.12/go.mod h1:u5H1YNBxpqRaxsYJYSkiCWKzEfiAb1Gb520KVy5xxl4= -github.com/mattn/go-colorable v0.1.13/go.mod h1:7S9/ev0klgBDR4GtXTXX8a3vIGJpMovkB8vQcUbaXHg= -github.com/mattn/go-isatty v0.0.14/go.mod h1:7GGIvUiUoEMVVmxf/4nioHXj79iQHKdU27kJ6hsGG94= -github.com/mattn/go-isatty v0.0.16/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/yFXSvRLM= -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.17 h1:kV4Ip+/hUBC+8T6+2EgburRtkE9ef4nbY3f4dFhGjMc= -github.com/pierrec/lz4/v4 v4.1.17/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/pierrec/lz4/v4 v4.1.18 h1:xaKrnTkyoqfh1YItXl56+6KJNVYWlEEPuAQW9xsplYQ= +github.com/pierrec/lz4/v4 v4.1.18/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= 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_golang v1.13.0/go.mod h1:vTeo+zgvILHsnnj/39Ou/1fPN5nJFOEMgftOUOmlvYQ= -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/common v0.37.0/go.mod h1:phzohg0JFMnBEFGxTDbfu3QyL5GI8gTQJFhYO5B3mfA= -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/prometheus/procfs v0.8.0/go.mod h1:z7EfXMXOkbkqb9IINtpCn86r/to3BnA0uaxHdg830/4= 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/rs/xid v1.4.0/go.mod h1:trrq9SKmegXys3aeAKXMUTdJsYXVwGY3RLcfgqegfbg= -github.com/rs/zerolog v1.28.0/go.mod h1:NILgTygv/Uej1ra5XxGf82ZFSLk58MFGAUS2o6usyD0= -github.com/russross/blackfriday/v2 v2.1.0/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= -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/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/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo= -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/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= +github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= -github.com/urfave/cli/v2 v2.11.0/go.mod h1:f8iq5LtQ/bLxafbdBSLPPNsgaW0l/2fYYEHhAyPlwvo= github.com/xdg-go/pbkdf2 v1.0.0/go.mod h1:jrpuAogTd400dnrH08LKmI/xc1MbPOebTwRqcT5RDeI= github.com/xdg-go/scram v1.1.2/go.mod h1:RT/sEzTbU5y00aCK8UOx6R7YryM0iF1N2MOmC3kKLN4= github.com/xdg-go/stringprep v1.0.4/go.mod h1:mPGuuIYwz7CmR2bT9j4GbQqutWS1zV24gijq1dTyGkM= -github.com/xrash/smetrics v0.0.0-20201216005158-039620a65673/go.mod h1:N3UwUGtsrSj3ccvlPHLoLsHnpR27oXr4ZE984MbSER8= -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= github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= -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-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= -golang.org/x/crypto v0.0.0-20220722155217-630584e8d5aa/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= -golang.org/x/crypto v0.11.0 h1:6Ewdq3tDic1mg5xRO4milcWCfMVQhI4NkqWWvqejpuA= -golang.org/x/crypto v0.11.0/go.mod h1:xgJhtzW8F9jGdVFWZESrid1U1bjeNy4zgy5cRr/CIio= -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/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= +golang.org/x/crypto v0.12.0 h1:tFM/ta59kqch6LlvYnPa0yx5a83cL2nHflFhYKvv9Yk= +golang.org/x/crypto v0.12.0/go.mod h1:NF0Gs7EO5K4qLn+Ylc+fih8BSTeIjAP05siRnAh98yw= golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= golang.org/x/mod v0.8.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= -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-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= -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-20220127200216-cd36cc0744dd/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= -golang.org/x/net v0.0.0-20220225172249-27dd8689420f/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= -golang.org/x/net v0.0.0-20220725212005-46097bf591d3/go.mod h1:AaygXjzTFtRAg2ttMY5RMuhpJ3cNnI0XpyFJD1iQRSM= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= +golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.10.0/go.mod h1:0qNGK6F8kojg2nk9dLZ2mShWaEBan6FAoqfSigmmuDg= -golang.org/x/net v0.13.0 h1:Nvo8UFsZ8X3BhAC9699Z1j7XQ3rsZnUUm7jfBEk1ueY= -golang.org/x/net v0.13.0/go.mod h1:zEVYFnQC7m/vmpQFELhcD1EWkZlX69l4oqgmer6hfKA= -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/oauth2 v0.0.0-20220223155221-ee480838109b/go.mod h1:DAh4E804XQdzx2j+YRIaUnCqCV2RuMz24cGBJ5QYIrc= -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/net v0.14.0 h1:BONx9s002vGdD9umnlX1Po8vOZmrgH34qlHcD1MfK14= +golang.org/x/net v0.14.0/go.mod h1:PpSgVXXLK0OxS0F31C1/tv6XNguvCrnXIDrFMspZIUI= 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-20220601150217-0de741cfad7f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.3.0 h1:ftCYgMx6zT/asHUrPw8BLLscYtGznsLAnjq5RH9P66E= golang.org/x/sync v0.3.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= -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-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-20210630005230-0f9fa26af87c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20210927094055-39ccf1dd6fa6/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e/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/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20220909162455-aba9fc2a8ff2/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.10.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.11.0/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/term v0.0.0-20220526004731-065cf7ba2467/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= golang.org/x/term v0.8.0/go.mod h1:xPskH00ivmX89bAKVGSKKtLOWNx2+17Eiy94tnKShWo= -golang.org/x/term v0.10.0/go.mod h1:lpqdcUyK/oCiQxvxVrppt5ggO2KCZ5QblwqPnfZ6d5o= -golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/term v0.11.0/go.mod h1:zC9APTIj3jG3FdV/Ons+XE1riIZXG4aZ4GTHiPZJPIU= 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/text v0.3.8/go.mod h1:E6s5w1FMmriuDzIBO73fBruAKo1PCIq6d2Q6DHfQ8WQ= golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.9.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= -golang.org/x/text v0.11.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= -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/text v0.12.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= 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/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= golang.org/x/tools v0.6.0/go.mod h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU= 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= -google.golang.org/protobuf v1.28.1/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= -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.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/consumergroup/main.go b/examples/consumergroup/main.go index 1af67792d8..83c437734c 100644 --- a/examples/consumergroup/main.go +++ b/examples/consumergroup/main.go @@ -83,9 +83,6 @@ func main() { config.Consumer.Offsets.Initial = sarama.OffsetOldest } - /** - * Setup a new Sarama consumer group - */ consumer := Consumer{ ready: make(chan bool), } diff --git a/examples/consumergroup_cooperative/README.md b/examples/consumergroup_cooperative/README.md new file mode 100644 index 0000000000..65ccee6eb4 --- /dev/null +++ b/examples/consumergroup_cooperative/README.md @@ -0,0 +1,9 @@ +# Consumergroup example + +This example shows you how to use the Sarama consumer group consumer. The example simply starts consuming the given Kafka topics and logs the consumed messages. + +```bash +$ go run main.go -brokers="127.0.0.1:9092" -topics="sarama" -group="example" +``` + +You can also toggle (pause/resume) the consumption by sending SIGUSR1 diff --git a/examples/consumergroup_cooperative/go.mod b/examples/consumergroup_cooperative/go.mod new file mode 100644 index 0000000000..62862a216f --- /dev/null +++ b/examples/consumergroup_cooperative/go.mod @@ -0,0 +1,7 @@ +module github.com/IBM/sarama/examples/consumergroup_cooperative + +go 1.16 + +require github.com/IBM/sarama v1.34.1 + +replace github.com/IBM/sarama => ../../ diff --git a/examples/consumergroup_cooperative/go.sum b/examples/consumergroup_cooperative/go.sum new file mode 100644 index 0000000000..3ce0c297db --- /dev/null +++ b/examples/consumergroup_cooperative/go.sum @@ -0,0 +1,117 @@ +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.4.0 h1:3OK9bWpPk5q6pbFAaYSEwD9CLUSHG8bnZuqX2yMt3B0= +github.com/eapache/go-resiliency v1.4.0/go.mod h1:5yPzW0MIvSe0JDsv0v+DvcjEv2FyD6iZYSs1ZI+iQho= +github.com/eapache/go-xerial-snappy v0.0.0-20230731223053-c322873962e3 h1:Oy0F4ALJ04o5Qqpdz8XLIpNA3WM/iSIXqxtqo7UGVws= +github.com/eapache/go-xerial-snappy v0.0.0-20230731223053-c322873962e3/go.mod h1:YvSRo5mw33fLEx1+DlK6L2VV43tJt5Eyel9n9XBcR+0= +github.com/eapache/queue v1.1.0 h1:YOEu7KNc61ntiQlcEeUIoDTJ2o8mQznoNvUhiigpIqc= +github.com/eapache/queue v1.1.0/go.mod h1:6eCeP0CKFpHLu8blIFXhExK/dRa7WDZfr6jVFPTqq+I= +github.com/fortytw2/leaktest v1.3.0 h1:u8491cBMTQ8ft8aeV+adlcytMZylmA5nnwwkRZjI8vw= +github.com/fortytw2/leaktest v1.3.0/go.mod h1:jDsjWgpAGjm2CA7WthBh/CdZYEPF31XHquHwclZch5g= +github.com/golang/snappy v0.0.4 h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM= +github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= +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/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.4 h1:x1Sv4HaTpepFkXbt2IkL29DXRf8sOfZXo8eRKh687T8= +github.com/jcmturner/gokrb5/v8 v8.4.4/go.mod h1:1btQEpgT6k+unzCwX1KdWMEwPPkkgBtP+F6aCACiMrs= +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/klauspost/compress v1.16.7 h1:2mk3MPGNzKyxErAw8YaohYh69+pa4sIQSC0fPGCFR9I= +github.com/klauspost/compress v1.16.7/go.mod h1:ntbaceVETuRiXiv4DpjP66DpAtAGkEQskQzEyD//IeE= +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/pierrec/lz4/v4 v4.1.18 h1:xaKrnTkyoqfh1YItXl56+6KJNVYWlEEPuAQW9xsplYQ= +github.com/pierrec/lz4/v4 v4.1.18/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= +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/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.6.1/go.mod h1:xXDCJY+GAPziupqXw64V24skbSoqbTEfhy4qGm1nDQc= +github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= +github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo= +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/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= +github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= +github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= +github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= +github.com/xdg-go/pbkdf2 v1.0.0/go.mod h1:jrpuAogTd400dnrH08LKmI/xc1MbPOebTwRqcT5RDeI= +github.com/xdg-go/scram v1.1.2/go.mod h1:RT/sEzTbU5y00aCK8UOx6R7YryM0iF1N2MOmC3kKLN4= +github.com/xdg-go/stringprep v1.0.4/go.mod h1:mPGuuIYwz7CmR2bT9j4GbQqutWS1zV24gijq1dTyGkM= +github.com/yuin/goldmark v1.4.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= +golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= +golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= +golang.org/x/crypto v0.12.0 h1:tFM/ta59kqch6LlvYnPa0yx5a83cL2nHflFhYKvv9Yk= +golang.org/x/crypto v0.12.0/go.mod h1:NF0Gs7EO5K4qLn+Ylc+fih8BSTeIjAP05siRnAh98yw= +golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= +golang.org/x/mod v0.8.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= +golang.org/x/net v0.0.0-20190620200207-3b0461eec859/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-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= +golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= +golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= +golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= +golang.org/x/net v0.10.0/go.mod h1:0qNGK6F8kojg2nk9dLZ2mShWaEBan6FAoqfSigmmuDg= +golang.org/x/net v0.14.0 h1:BONx9s002vGdD9umnlX1Po8vOZmrgH34qlHcD1MfK14= +golang.org/x/net v0.14.0/go.mod h1:PpSgVXXLK0OxS0F31C1/tv6XNguvCrnXIDrFMspZIUI= +golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.3.0 h1:ftCYgMx6zT/asHUrPw8BLLscYtGznsLAnjq5RH9P66E= +golang.org/x/sync v0.3.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= +golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.11.0/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/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= +golang.org/x/term v0.8.0/go.mod h1:xPskH00ivmX89bAKVGSKKtLOWNx2+17Eiy94tnKShWo= +golang.org/x/term v0.11.0/go.mod h1:zC9APTIj3jG3FdV/Ons+XE1riIZXG4aZ4GTHiPZJPIU= +golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= +golang.org/x/text v0.3.8/go.mod h1:E6s5w1FMmriuDzIBO73fBruAKo1PCIq6d2Q6DHfQ8WQ= +golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= +golang.org/x/text v0.9.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= +golang.org/x/text v0.12.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= +golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= +golang.org/x/tools v0.6.0/go.mod h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU= +golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +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-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= +gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= +gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +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= diff --git a/examples/consumergroup_cooperative/main.go b/examples/consumergroup_cooperative/main.go new file mode 100644 index 0000000000..add8fdd14d --- /dev/null +++ b/examples/consumergroup_cooperative/main.go @@ -0,0 +1,215 @@ +package main + +// SIGUSR1 toggle the pause/resume consumption +import ( + "context" + "errors" + "flag" + "log" + "os" + "os/signal" + "strings" + "sync" + "syscall" + + "github.com/IBM/sarama" +) + +// Sarama configuration options +var ( + brokers = "" + version = "" + group = "" + topics = "" + assignors = "" + 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.1.1", "Kafka cluster version") + flag.StringVar(&topics, "topics", "", "Kafka topics to be consumed, as a comma separated list") + flag.StringVar(&assignors, "assignors", "cooperative-sticky", "Consumer group partition assignment strategies (range, roundrobin, sticky, cooperative-sticky)") + flag.BoolVar(&oldest, "oldest", true, "Kafka consumer consume initial offset from oldest") + flag.BoolVar(&verbose, "verbose", false, "Sarama logging") + // flag string slice + // https://stackoverflow.com/questions/1752414/how-to-receive-variable-number-of-arguments-as-a-command-line-flag + + 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(group) == 0 { + panic("no Kafka consumer group defined, please set the -group flag") + } +} + +func main() { + sarama.Logger = log.New(os.Stdout, "[sarama] ", log.LstdFlags|log.Lshortfile|log.Lmsgprefix) + 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 + + var strategies []sarama.BalanceStrategy + for _, assignor := range strings.Split(assignors, " ") { + switch assignor { + case "sticky": + strategies = append(strategies, sarama.NewBalanceStrategySticky()) + case "roundrobin": + strategies = append(strategies, sarama.NewBalanceStrategyRoundRobin()) + case "range": + strategies = append(strategies, sarama.NewBalanceStrategyRange()) + case "cooperative-sticky": + strategies = append(strategies, sarama.NewBalanceStrategyCooperativeSticky()) + default: + log.Panicf("Unrecognized consumer group partition assignor: %s", assignor) + } + } + config.Consumer.Group.Rebalance.GroupStrategies = strategies + + if oldest { + config.Consumer.Offsets.Initial = sarama.OffsetOldest + } + + /** + * Setup a new Sarama consumer group + */ + consumer := Consumer{} + + 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() + + var err error + 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.ConsumeV2(ctx, strings.Split(topics, ","), &consumer); err != nil { + log.Panicf("Error from consumer: %v", err) + } + // return if consumer is closed + if errors.Is(err, sarama.ErrClosedConsumerGroup) { + return + } + // check if context was cancelled, signaling that the consumer should stop + if ctx.Err() != nil { + return + } + } + }() + + 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) + + select { + case <-ctx.Done(): + log.Println("terminating: context cancelled") + case <-sigterm: + log.Println("terminating: via signal") + case <-sigusr1: + toggleConsumptionFlow(client, &consumptionIsPaused) + } + cancel() + if err = client.Close(); err != nil { + log.Panicf("Error closing client: %v", err) + } + wg.Wait() +} + +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{} + +// Setup runs at the beginning of setting up new assigned partitions, before ConsumeClaim. +// For EAGER rebalance strategy, this is to set up all assigned partitions. +// For COOPERATIVE rebalance strategy, this is only to set up new assigned partitions. +func (consumer *Consumer) Setup(offsetManger sarama.OffsetManager, newAssignedPartitions map[string][]int32) { + log.Printf("newAssignedPartitions: %v", newAssignedPartitions) +} + +// Cleanup runs after ConsumeClaim, but before the offsets are committed for the claim. +// For EAGER rebalance strategy, this is to clean up all assigned partitions. +// For COOPERATIVE rebalance strategy, this is only to clean up revoked partitions. +func (consumer *Consumer) Cleanup(offsetManger sarama.OffsetManager, revokedPartitions map[string][]int32) { + log.Printf("revokedPartitions: %v", revokedPartitions) +} + +// ConsumeClaim must start a consumer loop of ConsumerGroupClaim's Messages(). +// Once ctx is done, ConsumeClaim should return as soon as possible. +func (consumer *Consumer) ConsumeClaim(ctx context.Context, om sarama.OffsetManager, claim sarama.ConsumerGroupClaim) { + // NOTE: + // Do not move the code below to a goroutine. + // The `ConsumeClaim` itself is called within a goroutine, see: + // https://github.com/IBM/sarama/blob/main/consumer_group.go#L27-L29 + for { + // `<-ctx.Done()` has a higher priority than `<-claim.Messages()` + select { + case <-ctx.Done(): + return + default: + } + + select { + // Should return immediately when `ctx.Context()` is done. + // If not, will raise `ErrRebalanceInProgress` or `read tcp :: i/o timeout` when kafka rebalance. see: + // https://github.com/IBM/sarama/issues/1192 + case <-ctx.Done(): + return + + case message, ok := <-claim.Messages(): + if !ok { + log.Printf("message channel was closed") + return + } + log.Printf("received message topic:%s, partition:%d, offset:%d, value:%s", message.Topic, message.Partition, message.Offset, message.Value) + om.MarkMessage(message, "") + } + } +} diff --git a/functional_offset_manager_test.go b/functional_offset_manager_test.go index 7f324b22bd..1df562518f 100644 --- a/functional_offset_manager_test.go +++ b/functional_offset_manager_test.go @@ -17,12 +17,12 @@ func TestFuncOffsetManager(t *testing.T) { t.Fatal(err) } - offsetManager, err := NewOffsetManagerFromClient("sarama.TestFuncOffsetManager", client) + om, err := NewOffsetManagerFromClient("sarama.TestFuncOffsetManager", client) if err != nil { t.Fatal(err) } - pom1, err := offsetManager.ManagePartition("test.1", 0) + pom1, err := om.ManagePartition("test.1", 0) if err != nil { t.Fatal(err) } @@ -30,7 +30,10 @@ func TestFuncOffsetManager(t *testing.T) { pom1.MarkOffset(10, "test metadata") safeClose(t, pom1) - pom2, err := offsetManager.ManagePartition("test.1", 0) + // Avoid flaky test: let om cleanup removed poms + om.(*offsetManager).releasePOMs(true) + + pom2, err := om.ManagePartition("test.1", 0) if err != nil { t.Fatal(err) } @@ -45,6 +48,6 @@ func TestFuncOffsetManager(t *testing.T) { } safeClose(t, pom2) - safeClose(t, offsetManager) + safeClose(t, om) safeClose(t, client) } diff --git a/metrics.go b/metrics.go index 7b7705f2e3..de8ad95c74 100644 --- a/metrics.go +++ b/metrics.go @@ -32,7 +32,7 @@ func getMetricNameForBroker(name string, broker *Broker) string { func getMetricNameForTopic(name string, topic string) string { // Convert dot to _ since reporters like Graphite typically use dot to represent hierarchy // cf. KAFKA-1902 and KAFKA-2337 - return fmt.Sprintf(name+"-for-topic-%s", strings.Replace(topic, ".", "_", -1)) + return fmt.Sprintf(name+"-for-topic-%s", strings.ReplaceAll(topic, ".", "_")) } func getOrRegisterTopicMeter(name string, topic string, r metrics.Registry) metrics.Meter { diff --git a/metrics_test.go b/metrics_test.go index c9144df38c..cfa25e7a4e 100644 --- a/metrics_test.go +++ b/metrics_test.go @@ -57,6 +57,7 @@ func (m *metricValidators) registerForBroker(broker *Broker, validator *metricVa m.register(&metricValidator{getMetricNameForBroker(validator.name, broker), validator.validator}) } +// nolint:unused func (m *metricValidators) registerForGlobalAndTopic(topic string, validator *metricValidator) { m.register(&metricValidator{validator.name, validator.validator}) m.register(&metricValidator{getMetricNameForTopic(validator.name, topic), validator.validator}) @@ -103,6 +104,7 @@ func countMeterValidator(name string, expectedCount int) *metricValidator { }) } +// nolint:unused func minCountMeterValidator(name string, minCount int) *metricValidator { return meterValidator(name, func(t *testing.T, meter metrics.Meter) { t.Helper() @@ -137,6 +139,7 @@ func countHistogramValidator(name string, expectedCount int) *metricValidator { }) } +// nolint:unused func minCountHistogramValidator(name string, minCount int) *metricValidator { return histogramValidator(name, func(t *testing.T, histogram metrics.Histogram) { t.Helper() @@ -161,6 +164,7 @@ func minMaxHistogramValidator(name string, expectedMin int, expectedMax int) *me }) } +// nolint:unused func minValHistogramValidator(name string, minMin int) *metricValidator { return histogramValidator(name, func(t *testing.T, histogram metrics.Histogram) { t.Helper() @@ -171,6 +175,7 @@ func minValHistogramValidator(name string, minMin int) *metricValidator { }) } +// nolint:unused func maxValHistogramValidator(name string, maxMax int) *metricValidator { return histogramValidator(name, func(t *testing.T, histogram metrics.Histogram) { t.Helper() diff --git a/offset_manager.go b/offset_manager.go index 332679fd77..ec50c83a53 100644 --- a/offset_manager.go +++ b/offset_manager.go @@ -1,6 +1,8 @@ package sarama import ( + "errors" + "fmt" "sync" "time" ) @@ -14,6 +16,35 @@ type OffsetManager interface { // topic/partition. ManagePartition(topic string, partition int32) (PartitionOffsetManager, error) + RemovePartitions(partitions map[string][]int32) error + + Update(memberID string, generation int32) + + // MarkOffset marks the provided offset, alongside a metadata string + // that represents the state of the partition consumer at that point in time. The + // metadata string can be used by another consumer to restore that state, so it + // can resume consumption. + // + // To follow upstream conventions, you are expected to mark the offset of the + // next message to read, not the last message read. Thus, when calling `MarkOffset` + // you should typically add one to the offset of the last consumed message. + // + // Note: calling MarkOffset does not necessarily commit the offset to the backend + // store immediately for efficiency reasons, and it may never be committed if + // your application crashes. This means that you may end up processing the same + // message twice, and your processing should ideally be idempotent. + MarkOffset(topic string, partition int32, offset int64, metadata string) + + // ResetOffset resets to the provided offset, alongside a metadata string that + // represents the state of the partition consumer at that point in time. Reset + // acts as a counterpart to MarkOffset, the difference being that it allows to + // reset an offset to an earlier or smaller value, where MarkOffset only + // allows incrementing the offset. cf MarkOffset for more details. + ResetOffset(topic string, partition int32, offset int64, metadata string) + + // MarkMessage marks a message as consumed. + MarkMessage(msg *ConsumerMessage, metadata string) + // Close stops the OffsetManager from managing offsets. It is required to call // this function before an OffsetManager object passes out of scope, as it // will otherwise leak memory. You must call this after all the @@ -35,6 +66,7 @@ type offsetManager struct { memberID string groupInstanceId *string generation int32 + updateLock sync.RWMutex broker *Broker brokerLock sync.RWMutex @@ -100,13 +132,81 @@ func (om *offsetManager) ManagePartition(topic string, partition int32) (Partiti } if topicManagers[partition] != nil { - return nil, ConfigurationError("That topic/partition is already being managed") + return nil, ConfigurationError(fmt.Sprintf("topic:%s/partition:%d is already being managed", topic, partition)) } topicManagers[partition] = pom return pom, nil } +func (om *offsetManager) RemovePartitions(topicPartitions map[string][]int32) error { + var errs ConsumerErrors + var errsLock sync.Mutex + + var wg sync.WaitGroup + for topic, partitions := range topicPartitions { + for _, partition := range partitions { + wg.Add(1) + go func(topic string, partition int32) { + defer wg.Done() + + om.pomsLock.RLock() + pom := om.poms[topic][partition] + om.pomsLock.RUnlock() + err := pom.Close() + if err != nil { + errsLock.Lock() + var consumerErrs ConsumerErrors + if errors.As(err, &consumerErrs) { + errs = append(errs, consumerErrs...) + } + errsLock.Unlock() + } + }(topic, partition) + } + } + wg.Wait() + + // flush one last time + if om.conf.Consumer.Offsets.AutoCommit.Enable { + for attempt := 0; attempt <= om.conf.Consumer.Offsets.Retry.Max; attempt++ { + om.flushToBroker() + } + } + + om.pomsLock.Lock() + for topic, partitions := range topicPartitions { + for _, partition := range partitions { + delete(om.poms[topic], partition) + if len(om.poms[topic]) == 0 { + delete(om.poms, topic) + } + } + } + + om.pomsLock.Unlock() + if len(errs) > 0 { + return errs + } + return nil +} + +func (om *offsetManager) MarkOffset(topic string, partition int32, offset int64, metadata string) { + if pom := om.findPOM(topic, partition); pom != nil { + pom.MarkOffset(offset, metadata) + } +} + +func (om *offsetManager) ResetOffset(topic string, partition int32, offset int64, metadata string) { + if pom := om.findPOM(topic, partition); pom != nil { + pom.ResetOffset(offset, metadata) + } +} + +func (om *offsetManager) MarkMessage(msg *ConsumerMessage, metadata string) { + om.MarkOffset(msg.Topic, msg.Partition, msg.Offset+1, metadata) +} + func (om *offsetManager) Close() error { om.closeOnce.Do(func() { // exit the mainLoop @@ -248,6 +348,14 @@ func (om *offsetManager) mainLoop() { } } +func (om *offsetManager) Update(memberID string, generation int32) { + om.updateLock.Lock() + defer om.updateLock.Unlock() + + om.memberID = memberID + om.generation = generation +} + func (om *offsetManager) Commit() { om.flushToBroker() om.releasePOMs(false) @@ -277,12 +385,15 @@ func (om *offsetManager) flushToBroker() { } func (om *offsetManager) constructRequest() *OffsetCommitRequest { + om.updateLock.RLock() r := &OffsetCommitRequest{ Version: 1, ConsumerGroup: om.group, ConsumerID: om.memberID, ConsumerGroupGeneration: om.generation, } + om.updateLock.RUnlock() + // Version 1 adds timestamp and group membership information, as well as the commit timestamp. // // Version 2 adds retention time. It removes the commit timestamp added in version 1. @@ -369,6 +480,14 @@ func (om *offsetManager) handleResponse(broker *Broker, req *OffsetCommitRequest case ErrNoError: block := req.blocks[pom.topic][pom.partition] pom.updateCommitted(block.offset, block.metadata) + case ErrRebalanceInProgress: + // do nothing here + case ErrIllegalGeneration: + // - For EAGER protocol: commit request with a stale generation is rejected by the coordinator. + // - For COOPERATIVE protocol: the partition is not assigned to this consumer in the new generation. + // Normally users don't need to do anything(include retry), Kafka has at-lease-once semantics + // and these messages will be redelivered to the consumer in the new generation. + pom.handleError(err) case ErrNotLeaderForPartition, ErrLeaderNotAvailable, ErrConsumerCoordinatorNotAvailable, ErrNotCoordinatorForConsumer: // not a critical error, we just need to redispatch @@ -605,6 +724,7 @@ func (pom *partitionOffsetManager) AsyncClose() { func (pom *partitionOffsetManager) Close() error { pom.AsyncClose() + pom.release() var errors ConsumerErrors for err := range pom.errors { errors = append(errors, err) diff --git a/offset_manager_test.go b/offset_manager_test.go index af95bc9e4c..6e168db8c5 100644 --- a/offset_manager_test.go +++ b/offset_manager_test.go @@ -534,6 +534,9 @@ func TestPartitionOffsetManagerCommitErr(t *testing.T) { pom.MarkOffset(100, "modified_meta") + // flaky test: wait for sending commit requests + time.Sleep(10 * time.Millisecond) + err := pom.Close() if err != nil { t.Error(err) diff --git a/utils.go b/utils.go index 748d664126..82d2f1d3d7 100644 --- a/utils.go +++ b/utils.go @@ -264,6 +264,7 @@ var ( DefaultVersion = V2_1_0_0 // reduced set of protocol versions to matrix test + // nolint:unused fvtRangeVersions = []KafkaVersion{ V0_8_2_2, V0_10_2_2, @@ -313,3 +314,40 @@ func (v KafkaVersion) String() string { return fmt.Sprintf("%d.%d.%d", v.version[0], v.version[1], v.version[2]) } + +func diffAssignment(map1 map[string][]int32, map2 map[string][]int32) map[string][]int32 { + set := make(map[string]map[int32]bool) + for topic, partitions := range map2 { + if _, exist := set[topic]; !exist { + set[topic] = make(map[int32]bool) + } + for _, partition := range partitions { + set[topic][partition] = true + } + } + + diff := make(map[string][]int32) + for topic, partitions := range map1 { + for _, partition := range partitions { + if _, exist := set[topic][partition]; !exist { + diff[topic] = append(diff[topic], partition) + } + } + } + return diff +} + +func diffTopicSlice(s1 []string, s2 []string) map[string]bool { + set := make(map[string]bool) + for _, s := range s2 { + set[s] = true + } + + diff := make(map[string]bool) + for _, s := range s1 { + if _, exist := set[s]; !exist { + diff[s] = true + } + } + return diff +}