From 91849b426142019203b249fb0e2f324545978ee7 Mon Sep 17 00:00:00 2001 From: Vytenis Darulis Date: Mon, 2 May 2022 12:38:15 -0400 Subject: [PATCH 1/8] [aggregator] Fix an unintended allocation during element flushing (#4108) --- src/aggregator/aggregator/counter_elem_gen.go | 38 ++++++++++--------- src/aggregator/aggregator/elem_base.go | 10 ++++- src/aggregator/aggregator/elem_base_test.go | 23 +++++++++++ src/aggregator/aggregator/gauge_elem_gen.go | 38 ++++++++++--------- src/aggregator/aggregator/generic_elem.go | 36 +++++++++--------- src/aggregator/aggregator/timer_elem_gen.go | 38 ++++++++++--------- 6 files changed, 111 insertions(+), 72 deletions(-) diff --git a/src/aggregator/aggregator/counter_elem_gen.go b/src/aggregator/aggregator/counter_elem_gen.go index 3c187f0541..7cdcf087e6 100644 --- a/src/aggregator/aggregator/counter_elem_gen.go +++ b/src/aggregator/aggregator/counter_elem_gen.go @@ -1,4 +1,4 @@ -// Copyright (c) 2021 Uber Technologies, Inc. +// Copyright (c) 2022 Uber Technologies, Inc. // // Permission is hereby granted, free of charge, to any person obtaining a copy // of this software and associated documentation files (the "Software"), to deal @@ -43,14 +43,14 @@ import ( ) type lockedCounterAggregation struct { - aggregation counterAggregation - sourcesSeen map[uint32]*bitset.BitSet - mtx sync.Mutex - dirty bool + aggregation counterAggregation + sourcesSeen map[uint32]*bitset.BitSet + mtx sync.Mutex + lastUpdatedAt xtime.UnixNano + dirty bool // resendEnabled is allowed to change while an aggregation is open, so it must be behind the lock. resendEnabled bool closed bool - lastUpdatedAt xtime.UnixNano } type timedCounter struct { @@ -469,7 +469,7 @@ func (e *CounterElem) dirtyToConsumeWithLock(targetNanos int64, } } -func (e *CounterElem) isFlushed(c consumeState) bool { +func (e *CounterElem) isFlushed(c *consumeState) bool { return e.flushState[c.startAt].flushed } @@ -478,14 +478,18 @@ func (e *CounterElem) isFlushed(c consumeState) bool { func (e *CounterElem) appendConsumeStateWithLock( agg timedCounter, toConsume []consumeState, - includeFilter func(consumeState) bool) ([]consumeState, bool) { - // eagerly append a new element so we can try reusing memory already allocated in the slice. - toConsume = append(toConsume, consumeState{}) - cState := toConsume[len(toConsume)-1] - if cState.values == nil { - cState.values = make([]float64, len(e.aggTypes)) - } - cState.values = cState.values[:0] + includeFilter func(*consumeState) bool, +) ([]consumeState, bool) { + // try reusing memory already allocated in the slice. + if cap(toConsume) >= len(toConsume)+1 { + toConsume = toConsume[:len(toConsume)+1] + } else { + toConsume = append(toConsume, consumeState{ + values: make([]float64, 0, len(e.aggTypes)), + }) + } + cState := &toConsume[len(toConsume)-1] + cState.Reset() // copy the lockedAgg data while holding the lock. agg.lockedAgg.mtx.Lock() cState.dirty = agg.lockedAgg.dirty @@ -494,8 +498,7 @@ func (e *CounterElem) appendConsumeStateWithLock( for _, aggType := range e.aggTypes { cState.values = append(cState.values, agg.lockedAgg.aggregation.ValueOf(aggType)) } - cState.annotation = raggregation.MaybeReplaceAnnotation( - cState.annotation, agg.lockedAgg.aggregation.Annotation()) + cState.annotation = raggregation.MaybeReplaceAnnotation(cState.annotation, agg.lockedAgg.aggregation.Annotation()) agg.lockedAgg.dirty = false agg.lockedAgg.mtx.Unlock() @@ -507,7 +510,6 @@ func (e *CounterElem) appendConsumeStateWithLock( cState.prevStartTime = 0 } cState.startAt = agg.startAt - toConsume[len(toConsume)-1] = cState // update the flush state with the latestResendEnabled since expireValuesWithLock needs it before actual processing. fState := e.flushState[cState.startAt] fState.latestResendEnabled = cState.resendEnabled diff --git a/src/aggregator/aggregator/elem_base.go b/src/aggregator/aggregator/elem_base.go index bdb767fd86..1dd943320f 100644 --- a/src/aggregator/aggregator/elem_base.go +++ b/src/aggregator/aggregator/elem_base.go @@ -216,6 +216,14 @@ type consumeState struct { resendEnabled bool } +// Reset resets the consume state for reuse. +func (c *consumeState) Reset() { + *c = consumeState{ + annotation: c.annotation[:0], + values: c.values[:0], + } +} + // mutable state for a timedAggregation that is local to the flusher. does not need to be synchronized. // this state is kept around for the lifetime of the timedAggregation. type flushState struct { @@ -233,7 +241,7 @@ type flushState struct { latestResendEnabled bool } -var isDirty = func(state consumeState) bool { +var isDirty = func(state *consumeState) bool { return state.dirty } diff --git a/src/aggregator/aggregator/elem_base_test.go b/src/aggregator/aggregator/elem_base_test.go index 63bd786dbc..e9f16f29b6 100644 --- a/src/aggregator/aggregator/elem_base_test.go +++ b/src/aggregator/aggregator/elem_base_test.go @@ -593,3 +593,26 @@ func TestParsePipelineTransformationDerivativeOrderTooHigh(t *testing.T) { require.Error(t, err) require.True(t, strings.Contains(err.Error(), "transformation derivative order is 2 higher than supported 1")) } + +func TestConsumeStateReset(t *testing.T) { + s := &consumeState{} + s.Reset() + require.EqualValues(t, consumeState{}, *s) + + s = &consumeState{ + annotation: []byte("foo1"), + values: []float64{1.0, 2.0, 3.0}, + startAt: 123, + prevStartTime: 42, + lastUpdatedAt: 100, + dirty: true, + resendEnabled: true, + } + s.Reset() + require.EqualValues(t, consumeState{ + annotation: []byte{}, + values: []float64{}, + }, *s) + require.Equal(t, 4, cap(s.annotation)) + require.Equal(t, 3, cap(s.values)) +} diff --git a/src/aggregator/aggregator/gauge_elem_gen.go b/src/aggregator/aggregator/gauge_elem_gen.go index 550b95ea22..d16c246c88 100644 --- a/src/aggregator/aggregator/gauge_elem_gen.go +++ b/src/aggregator/aggregator/gauge_elem_gen.go @@ -1,4 +1,4 @@ -// Copyright (c) 2021 Uber Technologies, Inc. +// Copyright (c) 2022 Uber Technologies, Inc. // // Permission is hereby granted, free of charge, to any person obtaining a copy // of this software and associated documentation files (the "Software"), to deal @@ -43,14 +43,14 @@ import ( ) type lockedGaugeAggregation struct { - aggregation gaugeAggregation - sourcesSeen map[uint32]*bitset.BitSet - mtx sync.Mutex - dirty bool + aggregation gaugeAggregation + sourcesSeen map[uint32]*bitset.BitSet + mtx sync.Mutex + lastUpdatedAt xtime.UnixNano + dirty bool // resendEnabled is allowed to change while an aggregation is open, so it must be behind the lock. resendEnabled bool closed bool - lastUpdatedAt xtime.UnixNano } type timedGauge struct { @@ -469,7 +469,7 @@ func (e *GaugeElem) dirtyToConsumeWithLock(targetNanos int64, } } -func (e *GaugeElem) isFlushed(c consumeState) bool { +func (e *GaugeElem) isFlushed(c *consumeState) bool { return e.flushState[c.startAt].flushed } @@ -478,14 +478,18 @@ func (e *GaugeElem) isFlushed(c consumeState) bool { func (e *GaugeElem) appendConsumeStateWithLock( agg timedGauge, toConsume []consumeState, - includeFilter func(consumeState) bool) ([]consumeState, bool) { - // eagerly append a new element so we can try reusing memory already allocated in the slice. - toConsume = append(toConsume, consumeState{}) - cState := toConsume[len(toConsume)-1] - if cState.values == nil { - cState.values = make([]float64, len(e.aggTypes)) - } - cState.values = cState.values[:0] + includeFilter func(*consumeState) bool, +) ([]consumeState, bool) { + // try reusing memory already allocated in the slice. + if cap(toConsume) >= len(toConsume)+1 { + toConsume = toConsume[:len(toConsume)+1] + } else { + toConsume = append(toConsume, consumeState{ + values: make([]float64, 0, len(e.aggTypes)), + }) + } + cState := &toConsume[len(toConsume)-1] + cState.Reset() // copy the lockedAgg data while holding the lock. agg.lockedAgg.mtx.Lock() cState.dirty = agg.lockedAgg.dirty @@ -494,8 +498,7 @@ func (e *GaugeElem) appendConsumeStateWithLock( for _, aggType := range e.aggTypes { cState.values = append(cState.values, agg.lockedAgg.aggregation.ValueOf(aggType)) } - cState.annotation = raggregation.MaybeReplaceAnnotation( - cState.annotation, agg.lockedAgg.aggregation.Annotation()) + cState.annotation = raggregation.MaybeReplaceAnnotation(cState.annotation, agg.lockedAgg.aggregation.Annotation()) agg.lockedAgg.dirty = false agg.lockedAgg.mtx.Unlock() @@ -507,7 +510,6 @@ func (e *GaugeElem) appendConsumeStateWithLock( cState.prevStartTime = 0 } cState.startAt = agg.startAt - toConsume[len(toConsume)-1] = cState // update the flush state with the latestResendEnabled since expireValuesWithLock needs it before actual processing. fState := e.flushState[cState.startAt] fState.latestResendEnabled = cState.resendEnabled diff --git a/src/aggregator/aggregator/generic_elem.go b/src/aggregator/aggregator/generic_elem.go index 42c4cf4ae5..e14ec1aa8b 100644 --- a/src/aggregator/aggregator/generic_elem.go +++ b/src/aggregator/aggregator/generic_elem.go @@ -106,14 +106,14 @@ type typeSpecificElemBase interface { } type lockedAggregation struct { - aggregation typeSpecificAggregation - sourcesSeen map[uint32]*bitset.BitSet - mtx sync.Mutex - dirty bool + aggregation typeSpecificAggregation + sourcesSeen map[uint32]*bitset.BitSet + mtx sync.Mutex + lastUpdatedAt xtime.UnixNano + dirty bool // resendEnabled is allowed to change while an aggregation is open, so it must be behind the lock. resendEnabled bool closed bool - lastUpdatedAt xtime.UnixNano } type timedAggregation struct { @@ -532,7 +532,7 @@ func (e *GenericElem) dirtyToConsumeWithLock(targetNanos int64, } } -func (e *GenericElem) isFlushed(c consumeState) bool { +func (e *GenericElem) isFlushed(c *consumeState) bool { return e.flushState[c.startAt].flushed } @@ -541,14 +541,18 @@ func (e *GenericElem) isFlushed(c consumeState) bool { func (e *GenericElem) appendConsumeStateWithLock( agg timedAggregation, toConsume []consumeState, - includeFilter func(consumeState) bool) ([]consumeState, bool) { - // eagerly append a new element so we can try reusing memory already allocated in the slice. - toConsume = append(toConsume, consumeState{}) - cState := toConsume[len(toConsume)-1] - if cState.values == nil { - cState.values = make([]float64, len(e.aggTypes)) - } - cState.values = cState.values[:0] + includeFilter func(*consumeState) bool, +) ([]consumeState, bool) { + // try reusing memory already allocated in the slice. + if cap(toConsume) >= len(toConsume)+1 { + toConsume = toConsume[:len(toConsume)+1] + } else { + toConsume = append(toConsume, consumeState{ + values: make([]float64, 0, len(e.aggTypes)), + }) + } + cState := &toConsume[len(toConsume)-1] + cState.Reset() // copy the lockedAgg data while holding the lock. agg.lockedAgg.mtx.Lock() cState.dirty = agg.lockedAgg.dirty @@ -557,8 +561,7 @@ func (e *GenericElem) appendConsumeStateWithLock( for _, aggType := range e.aggTypes { cState.values = append(cState.values, agg.lockedAgg.aggregation.ValueOf(aggType)) } - cState.annotation = raggregation.MaybeReplaceAnnotation( - cState.annotation, agg.lockedAgg.aggregation.Annotation()) + cState.annotation = raggregation.MaybeReplaceAnnotation(cState.annotation, agg.lockedAgg.aggregation.Annotation()) agg.lockedAgg.dirty = false agg.lockedAgg.mtx.Unlock() @@ -570,7 +573,6 @@ func (e *GenericElem) appendConsumeStateWithLock( cState.prevStartTime = 0 } cState.startAt = agg.startAt - toConsume[len(toConsume)-1] = cState // update the flush state with the latestResendEnabled since expireValuesWithLock needs it before actual processing. fState := e.flushState[cState.startAt] fState.latestResendEnabled = cState.resendEnabled diff --git a/src/aggregator/aggregator/timer_elem_gen.go b/src/aggregator/aggregator/timer_elem_gen.go index f4624b555e..80786cc640 100644 --- a/src/aggregator/aggregator/timer_elem_gen.go +++ b/src/aggregator/aggregator/timer_elem_gen.go @@ -1,4 +1,4 @@ -// Copyright (c) 2021 Uber Technologies, Inc. +// Copyright (c) 2022 Uber Technologies, Inc. // // Permission is hereby granted, free of charge, to any person obtaining a copy // of this software and associated documentation files (the "Software"), to deal @@ -43,14 +43,14 @@ import ( ) type lockedTimerAggregation struct { - aggregation timerAggregation - sourcesSeen map[uint32]*bitset.BitSet - mtx sync.Mutex - dirty bool + aggregation timerAggregation + sourcesSeen map[uint32]*bitset.BitSet + mtx sync.Mutex + lastUpdatedAt xtime.UnixNano + dirty bool // resendEnabled is allowed to change while an aggregation is open, so it must be behind the lock. resendEnabled bool closed bool - lastUpdatedAt xtime.UnixNano } type timedTimer struct { @@ -469,7 +469,7 @@ func (e *TimerElem) dirtyToConsumeWithLock(targetNanos int64, } } -func (e *TimerElem) isFlushed(c consumeState) bool { +func (e *TimerElem) isFlushed(c *consumeState) bool { return e.flushState[c.startAt].flushed } @@ -478,14 +478,18 @@ func (e *TimerElem) isFlushed(c consumeState) bool { func (e *TimerElem) appendConsumeStateWithLock( agg timedTimer, toConsume []consumeState, - includeFilter func(consumeState) bool) ([]consumeState, bool) { - // eagerly append a new element so we can try reusing memory already allocated in the slice. - toConsume = append(toConsume, consumeState{}) - cState := toConsume[len(toConsume)-1] - if cState.values == nil { - cState.values = make([]float64, len(e.aggTypes)) - } - cState.values = cState.values[:0] + includeFilter func(*consumeState) bool, +) ([]consumeState, bool) { + // try reusing memory already allocated in the slice. + if cap(toConsume) >= len(toConsume)+1 { + toConsume = toConsume[:len(toConsume)+1] + } else { + toConsume = append(toConsume, consumeState{ + values: make([]float64, 0, len(e.aggTypes)), + }) + } + cState := &toConsume[len(toConsume)-1] + cState.Reset() // copy the lockedAgg data while holding the lock. agg.lockedAgg.mtx.Lock() cState.dirty = agg.lockedAgg.dirty @@ -494,8 +498,7 @@ func (e *TimerElem) appendConsumeStateWithLock( for _, aggType := range e.aggTypes { cState.values = append(cState.values, agg.lockedAgg.aggregation.ValueOf(aggType)) } - cState.annotation = raggregation.MaybeReplaceAnnotation( - cState.annotation, agg.lockedAgg.aggregation.Annotation()) + cState.annotation = raggregation.MaybeReplaceAnnotation(cState.annotation, agg.lockedAgg.aggregation.Annotation()) agg.lockedAgg.dirty = false agg.lockedAgg.mtx.Unlock() @@ -507,7 +510,6 @@ func (e *TimerElem) appendConsumeStateWithLock( cState.prevStartTime = 0 } cState.startAt = agg.startAt - toConsume[len(toConsume)-1] = cState // update the flush state with the latestResendEnabled since expireValuesWithLock needs it before actual processing. fState := e.flushState[cState.startAt] fState.latestResendEnabled = cState.resendEnabled From f01d7e22c5b569698d60ca3c9ab8d656e05cd3a6 Mon Sep 17 00:00:00 2001 From: Vytenis Darulis Date: Mon, 2 May 2022 17:30:51 -0400 Subject: [PATCH 2/8] [aggregator] Refactor elem/flush metrics (#4110) --- src/aggregator/aggregator/counter_elem_gen.go | 17 ++- src/aggregator/aggregator/elem_base.go | 124 +++++++++--------- src/aggregator/aggregator/entry_test.go | 2 +- src/aggregator/aggregator/gauge_elem_gen.go | 17 ++- src/aggregator/aggregator/generic_elem.go | 17 ++- src/aggregator/aggregator/list.go | 2 + src/aggregator/aggregator/timer_elem_gen.go | 17 ++- 7 files changed, 110 insertions(+), 86 deletions(-) diff --git a/src/aggregator/aggregator/counter_elem_gen.go b/src/aggregator/aggregator/counter_elem_gen.go index 7cdcf087e6..f98438dd9f 100644 --- a/src/aggregator/aggregator/counter_elem_gen.go +++ b/src/aggregator/aggregator/counter_elem_gen.go @@ -228,7 +228,9 @@ func (e *CounterElem) AddUnique( func (e *CounterElem) expireValuesWithLock( targetNanos int64, isEarlierThanFn isEarlierThanFn, - flushMetrics flushMetrics) { + flushMetrics *flushMetrics, +) { + var expiredCount int64 e.flushStateToExpire = e.flushStateToExpire[:0] if len(e.values) == 0 { return @@ -272,7 +274,7 @@ func (e *CounterElem) expireValuesWithLock( e.flushStateToExpire = append(e.flushStateToExpire, e.minStartTime) delete(e.values, e.minStartTime) e.minStartTime = currAgg.startAt - flushMetrics.valuesExpired.Inc(1) + expiredCount++ // it's safe to access this outside the agg lock since it was closed in a previous iteration. // This is to make sure there aren't too many cached source sets taking up @@ -288,6 +290,7 @@ func (e *CounterElem) expireValuesWithLock( break } } + flushMetrics.valuesExpired.Inc(expiredCount) } func (e *CounterElem) expireFlushState() { @@ -737,7 +740,8 @@ func (e *CounterElem) processValue( resolution time.Duration, latenessAllowed time.Duration, jitter time.Duration, - flushMetrics flushMetrics) { + flushMetrics *flushMetrics, +) { var ( transformations = e.parsedPipeline.Transformations discardNaNValues = e.opts.DiscardNaNAggregatedValues() @@ -753,7 +757,7 @@ func (e *CounterElem) processValue( l.Error("reflushing aggregation without resendEnabled", zap.Any("consumeState", cState)) }) } - flushMetrics.valuesProcessed.Inc(1) + for aggTypeIdx, aggType := range e.aggTypes { var extraDp transformation.Datapoint value := cState.values[aggTypeIdx] @@ -867,10 +871,11 @@ func (e *CounterElem) processValue( // forward lag = current time - (agg timestamp + lateness allowed + jitter) // use expectedProcessingTime instead of the aggregation timestamp since the aggregation timestamp could be // in the past for updated aggregations (resendEnabled). + lag := xtime.Since(expectedProcessingTime.Add(latenessAllowed)) flushMetrics.forwardLag(forwardKey{fwdType: fwdType, jitter: false}). - RecordDuration(xtime.Since(expectedProcessingTime.Add(latenessAllowed + jitter))) + RecordDuration(lag) flushMetrics.forwardLag(forwardKey{fwdType: fwdType, jitter: true}). - RecordDuration(xtime.Since(expectedProcessingTime.Add(latenessAllowed))) + RecordDuration(lag + jitter) } fState.flushed = true e.flushState[cState.startAt] = fState diff --git a/src/aggregator/aggregator/elem_base.go b/src/aggregator/aggregator/elem_base.go index 1dd943320f..650ee49613 100644 --- a/src/aggregator/aggregator/elem_base.go +++ b/src/aggregator/aggregator/elem_base.go @@ -101,8 +101,10 @@ const ( type forwardType int const ( + // NB(vytenis): keep this zero-indexed forwardTypeLocal forwardType = iota forwardTypeRemote + forwardTypeInvalid // must be the last value in the list - used as a sentinel value for metric scope generation ) // metricElem is the common interface for metric elements. @@ -189,7 +191,7 @@ type elemBase struct { // Mutable states. cachedSourceSets []map[uint32]*bitset.BitSet // nolint: structcheck // a cache of the flush metrics that don't require grabbing a lock to access. - flushMetricsCache map[flushKey]flushMetrics + flushMetricsCache map[flushKey]*flushMetrics writeMetrics writeMetrics tombstoned bool closed bool @@ -251,11 +253,16 @@ func (f *flushState) close() { f.emittedValues = f.emittedValues[:0] } -type elemMetrics struct { - scope tally.Scope - write map[metricListType]writeMetrics - flush map[flushKey]flushMetrics - mtx sync.RWMutex +type writeMetrics struct { + writes tally.Counter + updatedValues tally.Counter +} + +func newWriteMetrics(scope tally.Scope) writeMetrics { + return writeMetrics{ + updatedValues: scope.Counter("updated-values"), + writes: scope.Counter("writes"), + } } // flushMetrics are the metrics produced by a flush task processing the metric element. @@ -267,15 +274,11 @@ type flushMetrics struct { // count of values expired. valuesExpired tally.Counter // the difference between actual and expected processing for a value. - forwardLags map[forwardKey]tally.Histogram + jitteredForwardLags [int(forwardTypeInvalid)]tally.Histogram + nonJitteredForwardLags [int(forwardTypeInvalid)]tally.Histogram } -type writeMetrics struct { - writes tally.Counter - updatedValues tally.Counter -} - -func newFlushMetrics(scope tally.Scope) flushMetrics { +func newFlushMetrics(scope tally.Scope) *flushMetrics { forwardLagBuckets := tally.DurationBuckets{ 10 * time.Millisecond, 500 * time.Millisecond, @@ -294,32 +297,31 @@ func newFlushMetrics(scope tally.Scope) flushMetrics { 90 * time.Second, 120 * time.Second, } - jitterVals := []bool{true, false} - typeVals := []forwardType{forwardTypeRemote, forwardTypeLocal} + m := flushMetrics{ elemsScanned: scope.Counter("elements-scanned"), valuesProcessed: scope.Counter("values-processed"), valuesExpired: scope.Counter("values-expired"), - forwardLags: make(map[forwardKey]tally.Histogram), } - for _, jv := range jitterVals { - for _, tv := range typeVals { - key := forwardKey{jitter: jv, fwdType: tv} - m.forwardLags[key] = scope.Tagged(key.toTags()).Histogram("forward-lag", forwardLagBuckets) - } + // forwardTypeInvalid is a sentinel value, marking the maximum index for forwardMetricType consts + for i := 0; i < int(forwardTypeInvalid); i++ { + tv := forwardType(i) + m.jitteredForwardLags[i] = scope. + Tagged(forwardKey{fwdType: tv, jitter: true}.toTags()). + Histogram("forward-lag", forwardLagBuckets) + m.nonJitteredForwardLags[i] = scope. + Tagged(forwardKey{fwdType: tv, jitter: false}.toTags()). + Histogram("forward-lag", forwardLagBuckets) } - return m + return &m } -func newWriteMetrics(scope tally.Scope) writeMetrics { - return writeMetrics{ - updatedValues: scope.Counter("updated-values"), - writes: scope.Counter("writes"), +func (f flushMetrics) forwardLag(key forwardKey) tally.Histogram { + if key.jitter { + return f.jitteredForwardLags[key.fwdType] } -} -func (f flushMetrics) forwardLag(key forwardKey) tally.Histogram { - return f.forwardLags[key] + return f.nonJitteredForwardLags[key.fwdType] } // flushKey identifies a flush task. @@ -358,46 +360,50 @@ func (f flushKey) toTags() map[string]string { } } -func (e *elemMetrics) flushMetrics(key flushKey) flushMetrics { - e.mtx.RLock() - m, ok := e.flush[key] - if ok { - e.mtx.RUnlock() - return m +type elemMetrics struct { + scope tally.Scope + write [int(invalidMetricListType)]writeMetrics + flush map[flushKey]*flushMetrics + mtx sync.RWMutex +} + +func newElemMetrics(scope tally.Scope) *elemMetrics { + m := elemMetrics{ + scope: scope, + flush: make(map[flushKey]*flushMetrics), } - e.mtx.RUnlock() - e.mtx.Lock() - m, ok = e.flush[key] - if ok { - e.mtx.Unlock() - return m + // invalidMetricListType is a sentinel value, marking the maximum index for metricListType consts + for i := 0; i < int(invalidMetricListType); i++ { + m.write[i] = newWriteMetrics(scope.Tagged(map[string]string{listTypeLabel: (metricListType)(i).String()})) } - m = newFlushMetrics(e.scope.Tagged(key.toTags())) - e.flush[key] = m - e.mtx.Unlock() - return m + return &m } -func (e *elemMetrics) writeMetrics(key metricListType) writeMetrics { +func (e *elemMetrics) flushMetrics(key flushKey) *flushMetrics { e.mtx.RLock() - m, ok := e.write[key] - if ok { + if m, ok := e.flush[key]; ok { e.mtx.RUnlock() return m } e.mtx.RUnlock() + e.mtx.Lock() - m, ok = e.write[key] - if ok { - e.mtx.Unlock() + defer e.mtx.Unlock() + + if m, ok := e.flush[key]; ok { return m } - m = newWriteMetrics(e.scope.Tagged(map[string]string{listTypeLabel: key.String()})) - e.write[key] = m - e.mtx.Unlock() + + m := newFlushMetrics(e.scope.Tagged(key.toTags())) + e.flush[key] = m + return m } +func (e *elemMetrics) writeMetrics(key metricListType) writeMetrics { + return e.write[key] +} + // ElemOptions are the parameters for constructing a new elemBase. type ElemOptions struct { aggregatorOpts Options @@ -411,11 +417,7 @@ func NewElemOptions(aggregatorOpts Options) ElemOptions { return ElemOptions{ aggregatorOpts: aggregatorOpts, aggregationOpts: raggregation.NewOptions(aggregatorOpts.InstrumentOptions()), - elemMetrics: &elemMetrics{ - scope: scope, - write: make(map[metricListType]writeMetrics), - flush: make(map[flushKey]flushMetrics), - }, + elemMetrics: newElemMetrics(scope), } } @@ -426,11 +428,11 @@ func newElemBase(opts ElemOptions) elemBase { aggOpts: opts.aggregationOpts, metrics: opts.elemMetrics, bufferForPastTimedMetricFn: opts.aggregatorOpts.BufferForPastTimedMetricFn(), - flushMetricsCache: make(map[flushKey]flushMetrics), + flushMetricsCache: make(map[flushKey]*flushMetrics), } } -func (e *elemBase) flushMetrics(resolution time.Duration, flushType flushType) flushMetrics { +func (e *elemBase) flushMetrics(resolution time.Duration, flushType flushType) *flushMetrics { key := flushKey{ resolution: resolution, flushType: flushType, diff --git a/src/aggregator/aggregator/entry_test.go b/src/aggregator/aggregator/entry_test.go index f8b0f4f2ad..be98f28e69 100644 --- a/src/aggregator/aggregator/entry_test.go +++ b/src/aggregator/aggregator/entry_test.go @@ -1164,7 +1164,7 @@ func TestAddUntimed_ResendEnabledMigrationRaceWithFlusher(t *testing.T) { require.NoError(t, e.addUntimed(mu, metadatas)) // continue consuming the aggregation - elem.expireValuesWithLock(int64(t2), isStandardMetricEarlierThan, flushMetrics{}) + elem.expireValuesWithLock(int64(t2), isStandardMetricEarlierThan, newFlushMetrics(tally.NoopScope)) // target the aggregation being flushed again...it should still be open since it migrated to resendEnabled before // closing. diff --git a/src/aggregator/aggregator/gauge_elem_gen.go b/src/aggregator/aggregator/gauge_elem_gen.go index d16c246c88..a910f0a8ef 100644 --- a/src/aggregator/aggregator/gauge_elem_gen.go +++ b/src/aggregator/aggregator/gauge_elem_gen.go @@ -228,7 +228,9 @@ func (e *GaugeElem) AddUnique( func (e *GaugeElem) expireValuesWithLock( targetNanos int64, isEarlierThanFn isEarlierThanFn, - flushMetrics flushMetrics) { + flushMetrics *flushMetrics, +) { + var expiredCount int64 e.flushStateToExpire = e.flushStateToExpire[:0] if len(e.values) == 0 { return @@ -272,7 +274,7 @@ func (e *GaugeElem) expireValuesWithLock( e.flushStateToExpire = append(e.flushStateToExpire, e.minStartTime) delete(e.values, e.minStartTime) e.minStartTime = currAgg.startAt - flushMetrics.valuesExpired.Inc(1) + expiredCount++ // it's safe to access this outside the agg lock since it was closed in a previous iteration. // This is to make sure there aren't too many cached source sets taking up @@ -288,6 +290,7 @@ func (e *GaugeElem) expireValuesWithLock( break } } + flushMetrics.valuesExpired.Inc(expiredCount) } func (e *GaugeElem) expireFlushState() { @@ -737,7 +740,8 @@ func (e *GaugeElem) processValue( resolution time.Duration, latenessAllowed time.Duration, jitter time.Duration, - flushMetrics flushMetrics) { + flushMetrics *flushMetrics, +) { var ( transformations = e.parsedPipeline.Transformations discardNaNValues = e.opts.DiscardNaNAggregatedValues() @@ -753,7 +757,7 @@ func (e *GaugeElem) processValue( l.Error("reflushing aggregation without resendEnabled", zap.Any("consumeState", cState)) }) } - flushMetrics.valuesProcessed.Inc(1) + for aggTypeIdx, aggType := range e.aggTypes { var extraDp transformation.Datapoint value := cState.values[aggTypeIdx] @@ -867,10 +871,11 @@ func (e *GaugeElem) processValue( // forward lag = current time - (agg timestamp + lateness allowed + jitter) // use expectedProcessingTime instead of the aggregation timestamp since the aggregation timestamp could be // in the past for updated aggregations (resendEnabled). + lag := xtime.Since(expectedProcessingTime.Add(latenessAllowed)) flushMetrics.forwardLag(forwardKey{fwdType: fwdType, jitter: false}). - RecordDuration(xtime.Since(expectedProcessingTime.Add(latenessAllowed + jitter))) + RecordDuration(lag) flushMetrics.forwardLag(forwardKey{fwdType: fwdType, jitter: true}). - RecordDuration(xtime.Since(expectedProcessingTime.Add(latenessAllowed))) + RecordDuration(lag + jitter) } fState.flushed = true e.flushState[cState.startAt] = fState diff --git a/src/aggregator/aggregator/generic_elem.go b/src/aggregator/aggregator/generic_elem.go index e14ec1aa8b..a233255452 100644 --- a/src/aggregator/aggregator/generic_elem.go +++ b/src/aggregator/aggregator/generic_elem.go @@ -291,7 +291,9 @@ func (e *GenericElem) AddUnique( func (e *GenericElem) expireValuesWithLock( targetNanos int64, isEarlierThanFn isEarlierThanFn, - flushMetrics flushMetrics) { + flushMetrics *flushMetrics, +) { + var expiredCount int64 e.flushStateToExpire = e.flushStateToExpire[:0] if len(e.values) == 0 { return @@ -335,7 +337,7 @@ func (e *GenericElem) expireValuesWithLock( e.flushStateToExpire = append(e.flushStateToExpire, e.minStartTime) delete(e.values, e.minStartTime) e.minStartTime = currAgg.startAt - flushMetrics.valuesExpired.Inc(1) + expiredCount++ // it's safe to access this outside the agg lock since it was closed in a previous iteration. // This is to make sure there aren't too many cached source sets taking up @@ -351,6 +353,7 @@ func (e *GenericElem) expireValuesWithLock( break } } + flushMetrics.valuesExpired.Inc(expiredCount) } func (e *GenericElem) expireFlushState() { @@ -800,7 +803,8 @@ func (e *GenericElem) processValue( resolution time.Duration, latenessAllowed time.Duration, jitter time.Duration, - flushMetrics flushMetrics) { + flushMetrics *flushMetrics, +) { var ( transformations = e.parsedPipeline.Transformations discardNaNValues = e.opts.DiscardNaNAggregatedValues() @@ -816,7 +820,7 @@ func (e *GenericElem) processValue( l.Error("reflushing aggregation without resendEnabled", zap.Any("consumeState", cState)) }) } - flushMetrics.valuesProcessed.Inc(1) + for aggTypeIdx, aggType := range e.aggTypes { var extraDp transformation.Datapoint value := cState.values[aggTypeIdx] @@ -930,10 +934,11 @@ func (e *GenericElem) processValue( // forward lag = current time - (agg timestamp + lateness allowed + jitter) // use expectedProcessingTime instead of the aggregation timestamp since the aggregation timestamp could be // in the past for updated aggregations (resendEnabled). + lag := xtime.Since(expectedProcessingTime.Add(latenessAllowed)) flushMetrics.forwardLag(forwardKey{fwdType: fwdType, jitter: false}). - RecordDuration(xtime.Since(expectedProcessingTime.Add(latenessAllowed + jitter))) + RecordDuration(lag) flushMetrics.forwardLag(forwardKey{fwdType: fwdType, jitter: true}). - RecordDuration(xtime.Since(expectedProcessingTime.Add(latenessAllowed))) + RecordDuration(lag + jitter) } fState.flushed = true e.flushState[cState.startAt] = fState diff --git a/src/aggregator/aggregator/list.go b/src/aggregator/aggregator/list.go index b2e50cab6b..83871469a0 100644 --- a/src/aggregator/aggregator/list.go +++ b/src/aggregator/aggregator/list.go @@ -781,9 +781,11 @@ func (l *timedMetricList) Close() { type metricListType int const ( + // NB(vytenis): keep this zero-indexed standardMetricListType metricListType = iota forwardedMetricListType timedMetricListType + invalidMetricListType // must be the last value in the list - used as a sentinel value for metric scope generation ) func (t metricListType) String() string { diff --git a/src/aggregator/aggregator/timer_elem_gen.go b/src/aggregator/aggregator/timer_elem_gen.go index 80786cc640..8d19d9022d 100644 --- a/src/aggregator/aggregator/timer_elem_gen.go +++ b/src/aggregator/aggregator/timer_elem_gen.go @@ -228,7 +228,9 @@ func (e *TimerElem) AddUnique( func (e *TimerElem) expireValuesWithLock( targetNanos int64, isEarlierThanFn isEarlierThanFn, - flushMetrics flushMetrics) { + flushMetrics *flushMetrics, +) { + var expiredCount int64 e.flushStateToExpire = e.flushStateToExpire[:0] if len(e.values) == 0 { return @@ -272,7 +274,7 @@ func (e *TimerElem) expireValuesWithLock( e.flushStateToExpire = append(e.flushStateToExpire, e.minStartTime) delete(e.values, e.minStartTime) e.minStartTime = currAgg.startAt - flushMetrics.valuesExpired.Inc(1) + expiredCount++ // it's safe to access this outside the agg lock since it was closed in a previous iteration. // This is to make sure there aren't too many cached source sets taking up @@ -288,6 +290,7 @@ func (e *TimerElem) expireValuesWithLock( break } } + flushMetrics.valuesExpired.Inc(expiredCount) } func (e *TimerElem) expireFlushState() { @@ -737,7 +740,8 @@ func (e *TimerElem) processValue( resolution time.Duration, latenessAllowed time.Duration, jitter time.Duration, - flushMetrics flushMetrics) { + flushMetrics *flushMetrics, +) { var ( transformations = e.parsedPipeline.Transformations discardNaNValues = e.opts.DiscardNaNAggregatedValues() @@ -753,7 +757,7 @@ func (e *TimerElem) processValue( l.Error("reflushing aggregation without resendEnabled", zap.Any("consumeState", cState)) }) } - flushMetrics.valuesProcessed.Inc(1) + for aggTypeIdx, aggType := range e.aggTypes { var extraDp transformation.Datapoint value := cState.values[aggTypeIdx] @@ -867,10 +871,11 @@ func (e *TimerElem) processValue( // forward lag = current time - (agg timestamp + lateness allowed + jitter) // use expectedProcessingTime instead of the aggregation timestamp since the aggregation timestamp could be // in the past for updated aggregations (resendEnabled). + lag := xtime.Since(expectedProcessingTime.Add(latenessAllowed)) flushMetrics.forwardLag(forwardKey{fwdType: fwdType, jitter: false}). - RecordDuration(xtime.Since(expectedProcessingTime.Add(latenessAllowed + jitter))) + RecordDuration(lag) flushMetrics.forwardLag(forwardKey{fwdType: fwdType, jitter: true}). - RecordDuration(xtime.Since(expectedProcessingTime.Add(latenessAllowed))) + RecordDuration(lag + jitter) } fState.flushed = true e.flushState[cState.startAt] = fState From 736243d0fcc38d8989d4258a462073497474c24e Mon Sep 17 00:00:00 2001 From: Vytenis Darulis Date: Mon, 2 May 2022 17:31:22 -0400 Subject: [PATCH 3/8] [aggregator] Pool locked aggregations (#4109) --- src/aggregator/aggregation/timer.go | 4 +- src/aggregator/aggregator/counter_elem_gen.go | 15 ++-- src/aggregator/aggregator/elem_base.go | 73 +++++++++++++++++++ src/aggregator/aggregator/gauge_elem_gen.go | 15 ++-- src/aggregator/aggregator/generic_elem.go | 15 ++-- src/aggregator/aggregator/timer_elem_gen.go | 15 ++-- 6 files changed, 116 insertions(+), 21 deletions(-) diff --git a/src/aggregator/aggregation/timer.go b/src/aggregator/aggregation/timer.go index 34ccd5b8b5..edbb5860e2 100644 --- a/src/aggregator/aggregation/timer.go +++ b/src/aggregator/aggregation/timer.go @@ -158,4 +158,6 @@ func (t *Timer) Annotation() []byte { } // Close closes the timer. -func (t *Timer) Close() { t.stream.Close() } +func (t *Timer) Close() { + t.stream.Close() +} diff --git a/src/aggregator/aggregator/counter_elem_gen.go b/src/aggregator/aggregator/counter_elem_gen.go index f98438dd9f..abccbbb657 100644 --- a/src/aggregator/aggregator/counter_elem_gen.go +++ b/src/aggregator/aggregator/counter_elem_gen.go @@ -63,7 +63,7 @@ type timedCounter struct { // close is called when the aggregation has been expired or the element is being closed. func (ta *timedCounter) close() { - ta.lockedAgg.aggregation.Close() + ta.lockedAgg.close() ta.lockedAgg = nil } @@ -684,12 +684,17 @@ func (e *CounterElem) findOrCreate( sourcesSeen = make(map[uint32]*bitset.BitSet) } } + // NB(vytenis): lockedCounterAggregation will be returned to pool on timedCounter close. + // this is a bit different from regular pattern of using a pool object due to codegen with Genny limitations, + // so we can avoid writing more boilerplate. + // timedCounter itself is always pass-by-value, but lockedCounterAggregation incurs an expensive allocation on heap + // in the critical path (30%+, depending on workload as of 2020-05-01): see https://github.com/m3db/m3/pull/4109 timedAgg = timedCounter{ startAt: alignedStart, - lockedAgg: &lockedCounterAggregation{ - sourcesSeen: sourcesSeen, - aggregation: e.NewAggregation(e.opts, e.aggOpts), - }, + lockedAgg: lockedCounterAggregationFromPool( + e.NewAggregation(e.opts, e.aggOpts), + sourcesSeen, + ), inDirtySet: true, } diff --git a/src/aggregator/aggregator/elem_base.go b/src/aggregator/aggregator/elem_base.go index 650ee49613..ea06c93b04 100644 --- a/src/aggregator/aggregator/elem_base.go +++ b/src/aggregator/aggregator/elem_base.go @@ -741,3 +741,76 @@ func newParsedPipeline(pipeline applied.Pipeline) (parsedPipeline, error) { Rollup: rollup, }, nil } + +// Placeholder to make compiler happy about generic elem base. +// NB: lockedAggregationFromPool and not newLockedAggregation to avoid yet another rename hack in makefile +func lockedAggregationFromPool( + aggregation typeSpecificAggregation, + sourcesSeen map[uint32]*bitset.BitSet, +) *lockedAggregation { + return &lockedAggregation{ + aggregation: aggregation, + sourcesSeen: sourcesSeen, + } +} + +func (l *lockedAggregation) close() { + l.aggregation.Close() +} + +var lockedCounterAggregationPool = sync.Pool{New: func() interface{} { return &lockedCounterAggregation{} }} + +func lockedCounterAggregationFromPool( + aggregation counterAggregation, + sourcesSeen map[uint32]*bitset.BitSet, +) *lockedCounterAggregation { + l := lockedCounterAggregationPool.Get().(*lockedCounterAggregation) + l.aggregation = aggregation + l.sourcesSeen = sourcesSeen + + return l +} + +func (l *lockedCounterAggregation) close() { + l.aggregation.Close() + *l = lockedCounterAggregation{} + lockedCounterAggregationPool.Put(l) +} + +var lockedGaugeAggregationPool = sync.Pool{New: func() interface{} { return &lockedGaugeAggregation{} }} + +func lockedGaugeAggregationFromPool( + aggregation gaugeAggregation, + sourcesSeen map[uint32]*bitset.BitSet, +) *lockedGaugeAggregation { + l := lockedGaugeAggregationPool.Get().(*lockedGaugeAggregation) + l.aggregation = aggregation + l.sourcesSeen = sourcesSeen + + return l +} + +func (l *lockedGaugeAggregation) close() { + l.aggregation.Close() + *l = lockedGaugeAggregation{} + lockedGaugeAggregationPool.Put(l) +} + +var lockedTimerAggregationPool = sync.Pool{New: func() interface{} { return &lockedTimerAggregation{} }} + +func lockedTimerAggregationFromPool( + aggregation timerAggregation, + sourcesSeen map[uint32]*bitset.BitSet, +) *lockedTimerAggregation { + l := lockedTimerAggregationPool.Get().(*lockedTimerAggregation) + l.aggregation = aggregation + l.sourcesSeen = sourcesSeen + + return l +} + +func (l *lockedTimerAggregation) close() { + l.aggregation.Close() + *l = lockedTimerAggregation{} + lockedTimerAggregationPool.Put(l) +} diff --git a/src/aggregator/aggregator/gauge_elem_gen.go b/src/aggregator/aggregator/gauge_elem_gen.go index a910f0a8ef..2586ef42dd 100644 --- a/src/aggregator/aggregator/gauge_elem_gen.go +++ b/src/aggregator/aggregator/gauge_elem_gen.go @@ -63,7 +63,7 @@ type timedGauge struct { // close is called when the aggregation has been expired or the element is being closed. func (ta *timedGauge) close() { - ta.lockedAgg.aggregation.Close() + ta.lockedAgg.close() ta.lockedAgg = nil } @@ -684,12 +684,17 @@ func (e *GaugeElem) findOrCreate( sourcesSeen = make(map[uint32]*bitset.BitSet) } } + // NB(vytenis): lockedGaugeAggregation will be returned to pool on timedGauge close. + // this is a bit different from regular pattern of using a pool object due to codegen with Genny limitations, + // so we can avoid writing more boilerplate. + // timedGauge itself is always pass-by-value, but lockedGaugeAggregation incurs an expensive allocation on heap + // in the critical path (30%+, depending on workload as of 2020-05-01): see https://github.com/m3db/m3/pull/4109 timedAgg = timedGauge{ startAt: alignedStart, - lockedAgg: &lockedGaugeAggregation{ - sourcesSeen: sourcesSeen, - aggregation: e.NewAggregation(e.opts, e.aggOpts), - }, + lockedAgg: lockedGaugeAggregationFromPool( + e.NewAggregation(e.opts, e.aggOpts), + sourcesSeen, + ), inDirtySet: true, } diff --git a/src/aggregator/aggregator/generic_elem.go b/src/aggregator/aggregator/generic_elem.go index a233255452..0d53fec568 100644 --- a/src/aggregator/aggregator/generic_elem.go +++ b/src/aggregator/aggregator/generic_elem.go @@ -126,7 +126,7 @@ type timedAggregation struct { // close is called when the aggregation has been expired or the element is being closed. func (ta *timedAggregation) close() { - ta.lockedAgg.aggregation.Close() + ta.lockedAgg.close() ta.lockedAgg = nil } @@ -747,12 +747,17 @@ func (e *GenericElem) findOrCreate( sourcesSeen = make(map[uint32]*bitset.BitSet) } } + // NB(vytenis): lockedAggregation will be returned to pool on timedAggregation close. + // this is a bit different from regular pattern of using a pool object due to codegen with Genny limitations, + // so we can avoid writing more boilerplate. + // timedAggregation itself is always pass-by-value, but lockedAggregation incurs an expensive allocation on heap + // in the critical path (30%+, depending on workload as of 2020-05-01): see https://github.com/m3db/m3/pull/4109 timedAgg = timedAggregation{ startAt: alignedStart, - lockedAgg: &lockedAggregation{ - sourcesSeen: sourcesSeen, - aggregation: e.NewAggregation(e.opts, e.aggOpts), - }, + lockedAgg: lockedAggregationFromPool( + e.NewAggregation(e.opts, e.aggOpts), + sourcesSeen, + ), inDirtySet: true, } diff --git a/src/aggregator/aggregator/timer_elem_gen.go b/src/aggregator/aggregator/timer_elem_gen.go index 8d19d9022d..8205b2a3a2 100644 --- a/src/aggregator/aggregator/timer_elem_gen.go +++ b/src/aggregator/aggregator/timer_elem_gen.go @@ -63,7 +63,7 @@ type timedTimer struct { // close is called when the aggregation has been expired or the element is being closed. func (ta *timedTimer) close() { - ta.lockedAgg.aggregation.Close() + ta.lockedAgg.close() ta.lockedAgg = nil } @@ -684,12 +684,17 @@ func (e *TimerElem) findOrCreate( sourcesSeen = make(map[uint32]*bitset.BitSet) } } + // NB(vytenis): lockedTimerAggregation will be returned to pool on timedTimer close. + // this is a bit different from regular pattern of using a pool object due to codegen with Genny limitations, + // so we can avoid writing more boilerplate. + // timedTimer itself is always pass-by-value, but lockedTimerAggregation incurs an expensive allocation on heap + // in the critical path (30%+, depending on workload as of 2020-05-01): see https://github.com/m3db/m3/pull/4109 timedAgg = timedTimer{ startAt: alignedStart, - lockedAgg: &lockedTimerAggregation{ - sourcesSeen: sourcesSeen, - aggregation: e.NewAggregation(e.opts, e.aggOpts), - }, + lockedAgg: lockedTimerAggregationFromPool( + e.NewAggregation(e.opts, e.aggOpts), + sourcesSeen, + ), inDirtySet: true, } From 665fd30acaa803f9c9c18aee1804e94df43d3751 Mon Sep 17 00:00:00 2001 From: Vytenis Darulis Date: Tue, 3 May 2022 10:31:20 -0400 Subject: [PATCH 4/8] [m3msg] Remove messageWriter interface as it only has a single impl (#4111) --- .../writer/consumer_service_writer_test.go | 4 +- src/msg/producer/writer/message_writer.go | 161 +++++++----------- .../writer/message_writer_benchmark_test.go | 2 +- .../producer/writer/message_writer_test.go | 46 ++--- src/msg/producer/writer/router.go | 12 +- src/msg/producer/writer/router_mock.go | 2 +- src/msg/producer/writer/shard_writer.go | 21 ++- src/msg/producer/writer/shard_writer_test.go | 20 +-- 8 files changed, 115 insertions(+), 153 deletions(-) diff --git a/src/msg/producer/writer/consumer_service_writer_test.go b/src/msg/producer/writer/consumer_service_writer_test.go index f88804e3e7..93172a80fa 100644 --- a/src/msg/producer/writer/consumer_service_writer_test.go +++ b/src/msg/producer/writer/consumer_service_writer_test.go @@ -135,7 +135,7 @@ func TestConsumerServiceWriterWithSharedConsumerWithNonShardedPlacement(t *testi } for _, sw := range w.(*consumerServiceWriterImpl).shardWriters { - require.Equal(t, 3, len(sw.(*sharedShardWriter).mw.(*messageWriterImpl).consumerWriters)) + require.Equal(t, 3, len(sw.(*sharedShardWriter).mw.consumerWriters)) } p2 := placement.NewPlacement(). @@ -162,7 +162,7 @@ func TestConsumerServiceWriterWithSharedConsumerWithNonShardedPlacement(t *testi } for _, sw := range w.(*consumerServiceWriterImpl).shardWriters { - require.Equal(t, 2, len(sw.(*sharedShardWriter).mw.(*messageWriterImpl).consumerWriters)) + require.Equal(t, 2, len(sw.(*sharedShardWriter).mw.consumerWriters)) } csw.Close() diff --git a/src/msg/producer/writer/message_writer.go b/src/msg/producer/writer/message_writer.go index ceac0bfdd6..f96e651d3e 100644 --- a/src/msg/producer/writer/message_writer.go +++ b/src/msg/producer/writer/message_writer.go @@ -48,60 +48,6 @@ var ( const _recordMessageDelayEvery = 4 // keep it a power of two value to keep modulo fast -type messageWriter interface { - // Write writes a message, messages not acknowledged in time will be retried. - // New messages will be written in order, but retries could be out of order. - Write(rm *producer.RefCountedMessage) - - // Ack acknowledges the metadata. - Ack(meta metadata) bool - - // Init initialize the message writer. - Init() - - // Close closes the writer. - // It should block until all buffered messages have been acknowledged. - Close() - - // AddConsumerWriter adds a consumer writer. - AddConsumerWriter(cw consumerWriter) - - // RemoveConsumerWriter removes the consumer writer for the given address. - RemoveConsumerWriter(addr string) - - // Metrics returns the metrics - Metrics() messageWriterMetrics - - // SetMetrics sets the metrics - // - // This allows changing the labels of the metrics when the downstream consumer instance changes. - SetMetrics(m messageWriterMetrics) - - // ReplicatedShardID returns the replicated shard id. - ReplicatedShardID() uint64 - - // CutoverNanos returns the cutover nanoseconds. - CutoverNanos() int64 - - // SetCutoverNanos sets the cutover nanoseconds. - SetCutoverNanos(nanos int64) - - // CutoffNanos returns the cutoff nanoseconds. - CutoffNanos() int64 - - // SetCutoffNanos sets the cutoff nanoseconds. - SetCutoffNanos(nanos int64) - - // MessageTTLNanos returns the message ttl nanoseconds. - MessageTTLNanos() int64 - - // SetMessageTTLNanos sets the message ttl nanoseconds. - SetMessageTTLNanos(nanos int64) - - // QueueSize returns the number of messages queued in the writer. - QueueSize() int -} - type messageWriterMetrics struct { withoutConsumerScope bool scope tally.Scope @@ -131,7 +77,7 @@ type messageWriterMetrics struct { processedDrop tally.Counter } -func (m messageWriterMetrics) withConsumer(consumer string) messageWriterMetrics { +func (m *messageWriterMetrics) withConsumer(consumer string) *messageWriterMetrics { if m.withoutConsumerScope { return m } @@ -142,7 +88,7 @@ func newMessageWriterMetrics( scope tally.Scope, opts instrument.TimerOptions, withoutConsumerScope bool, -) messageWriterMetrics { +) *messageWriterMetrics { return newMessageWriterMetricsWithConsumer(scope, opts, "unknown", withoutConsumerScope) } @@ -150,12 +96,13 @@ func newMessageWriterMetricsWithConsumer( scope tally.Scope, opts instrument.TimerOptions, consumer string, - withoutConsumerScope bool) messageWriterMetrics { + withoutConsumerScope bool, +) *messageWriterMetrics { consumerScope := scope if !withoutConsumerScope { consumerScope = scope.Tagged(map[string]string{"consumer": consumer}) } - return messageWriterMetrics{ + return &messageWriterMetrics{ withoutConsumerScope: withoutConsumerScope, scope: scope, opts: opts, @@ -209,7 +156,7 @@ func newMessageWriterMetricsWithConsumer( } } -type messageWriterImpl struct { +type messageWriter struct { sync.RWMutex replicatedShardID uint64 @@ -243,13 +190,13 @@ func newMessageWriter( replicatedShardID uint64, mPool *messagePool, opts Options, - m messageWriterMetrics, -) messageWriter { + m *messageWriterMetrics, +) *messageWriter { if opts == nil { opts = NewOptions() } nowFn := time.Now - return &messageWriterImpl{ + return &messageWriter{ replicatedShardID: replicatedShardID, mPool: mPool, opts: opts, @@ -264,12 +211,14 @@ func newMessageWriter( msgsToWrite: make([]*message, 0, opts.MessageQueueScanBatchSize()), isClosed: false, doneCh: make(chan struct{}), - m: &m, + m: m, nowFn: nowFn, } } -func (w *messageWriterImpl) Write(rm *producer.RefCountedMessage) { +// Write writes a message, messages not acknowledged in time will be retried. +// New messages will be written in order, but retries could be out of order. +func (w *messageWriter) Write(rm *producer.RefCountedMessage) { var ( nowNanos = w.nowFn().UnixNano() msg = w.newMessage() @@ -300,7 +249,7 @@ func (w *messageWriterImpl) Write(rm *producer.RefCountedMessage) { w.Unlock() } -func (w *messageWriterImpl) isValidWriteWithLock(nowNanos int64) bool { +func (w *messageWriter) isValidWriteWithLock(nowNanos int64) bool { if w.opts.IgnoreCutoffCutover() { return true } @@ -317,7 +266,7 @@ func (w *messageWriterImpl) isValidWriteWithLock(nowNanos int64) bool { return true } -func (w *messageWriterImpl) write( +func (w *messageWriter) write( iterationIndexes []int, consumerWriters []consumerWriter, metrics *messageWriterMetrics, @@ -367,15 +316,8 @@ func (w *messageWriterImpl) write( return errFailAllConsumers } -func randIndex(iterationIndexes []int, i int) int { - j := int(unsafe.Fastrandn(uint32(i + 1))) - // NB: we should only mutate the order in the iteration indexes and - // keep the order of consumer writers unchanged to prevent data race. - iterationIndexes[i], iterationIndexes[j] = iterationIndexes[j], iterationIndexes[i] - return iterationIndexes[i] -} - -func (w *messageWriterImpl) Ack(meta metadata) bool { +// Ack acknowledges the metadata. +func (w *messageWriter) Ack(meta metadata) bool { acked, expectedProcessNanos := w.acks.ack(meta) if acked { w.RLock() @@ -387,7 +329,8 @@ func (w *messageWriterImpl) Ack(meta metadata) bool { return false } -func (w *messageWriterImpl) Init() { +// Init initialize the message writer. +func (w *messageWriter) Init() { w.wg.Add(1) go func() { w.scanMessageQueueUntilClose() @@ -395,7 +338,7 @@ func (w *messageWriterImpl) Init() { }() } -func (w *messageWriterImpl) scanMessageQueueUntilClose() { +func (w *messageWriter) scanMessageQueueUntilClose() { var ( interval = w.opts.MessageQueueNewWritesScanInterval() jitter = time.Duration( @@ -419,7 +362,7 @@ func (w *messageWriterImpl) scanMessageQueueUntilClose() { } } -func (w *messageWriterImpl) scanMessageQueue() { +func (w *messageWriter) scanMessageQueue() { w.RLock() e := w.queue.Front() w.lastNewWrite = nil @@ -471,7 +414,7 @@ func (w *messageWriterImpl) scanMessageQueue() { } } -func (w *messageWriterImpl) writeBatch( +func (w *messageWriter) writeBatch( iterationIndexes []int, consumerWriters []consumerWriter, metrics *messageWriterMetrics, @@ -498,7 +441,7 @@ func (w *messageWriterImpl) writeBatch( // scanBatchWithLock iterates the message queue with a lock. It returns after // visited enough elements. So it holds the lock for less time and allows new // writes to be unblocked. -func (w *messageWriterImpl) scanBatchWithLock( +func (w *messageWriter) scanBatchWithLock( start *list.Element, nowNanos int64, batchSize int, @@ -582,7 +525,9 @@ func (w *messageWriterImpl) scanBatchWithLock( return next, w.msgsToWrite } -func (w *messageWriterImpl) Close() { +// Close closes the writer. +// It should block until all buffered messages have been acknowledged. +func (w *messageWriter) Close() { w.Lock() if w.isClosed { w.Unlock() @@ -596,7 +541,7 @@ func (w *messageWriterImpl) Close() { w.wg.Wait() } -func (w *messageWriterImpl) waitUntilAllMessageRemoved() { +func (w *messageWriter) waitUntilAllMessageRemoved() { // The message writers are being closed sequentially, checking isEmpty() // before always waiting for the first tick can speed up Close() a lot. if w.isEmpty() { @@ -612,57 +557,59 @@ func (w *messageWriterImpl) waitUntilAllMessageRemoved() { } } -func (w *messageWriterImpl) isEmpty() bool { +func (w *messageWriter) isEmpty() bool { w.RLock() l := w.queue.Len() w.RUnlock() return l == 0 } -func (w *messageWriterImpl) ReplicatedShardID() uint64 { +// ReplicatedShardID returns the replicated shard id. +func (w *messageWriter) ReplicatedShardID() uint64 { return w.replicatedShardID } -func (w *messageWriterImpl) CutoffNanos() int64 { +func (w *messageWriter) CutoffNanos() int64 { w.RLock() res := w.cutOffNanos w.RUnlock() return res } -func (w *messageWriterImpl) SetCutoffNanos(nanos int64) { +func (w *messageWriter) SetCutoffNanos(nanos int64) { w.Lock() w.cutOffNanos = nanos w.Unlock() } -func (w *messageWriterImpl) CutoverNanos() int64 { +func (w *messageWriter) CutoverNanos() int64 { w.RLock() res := w.cutOverNanos w.RUnlock() return res } -func (w *messageWriterImpl) SetCutoverNanos(nanos int64) { +func (w *messageWriter) SetCutoverNanos(nanos int64) { w.Lock() w.cutOverNanos = nanos w.Unlock() } -func (w *messageWriterImpl) MessageTTLNanos() int64 { +func (w *messageWriter) MessageTTLNanos() int64 { w.RLock() res := w.messageTTLNanos w.RUnlock() return res } -func (w *messageWriterImpl) SetMessageTTLNanos(nanos int64) { +func (w *messageWriter) SetMessageTTLNanos(nanos int64) { w.Lock() w.messageTTLNanos = nanos w.Unlock() } -func (w *messageWriterImpl) AddConsumerWriter(cw consumerWriter) { +// AddConsumerWriter adds a consumer writer. +func (w *messageWriter) AddConsumerWriter(cw consumerWriter) { w.Lock() newConsumerWriters := make([]consumerWriter, 0, len(w.consumerWriters)+1) newConsumerWriters = append(newConsumerWriters, w.consumerWriters...) @@ -676,7 +623,8 @@ func (w *messageWriterImpl) AddConsumerWriter(cw consumerWriter) { w.Unlock() } -func (w *messageWriterImpl) RemoveConsumerWriter(addr string) { +// RemoveConsumerWriter removes the consumer writer for the given address. +func (w *messageWriter) RemoveConsumerWriter(addr string) { w.Lock() newConsumerWriters := make([]consumerWriter, 0, len(w.consumerWriters)-1) for _, cw := range w.consumerWriters { @@ -694,33 +642,38 @@ func (w *messageWriterImpl) RemoveConsumerWriter(addr string) { w.Unlock() } -func (w *messageWriterImpl) Metrics() messageWriterMetrics { +// Metrics returns the metrics. These are dynamic and change if downstream consumer instance changes. +func (w *messageWriter) Metrics() *messageWriterMetrics { w.RLock() defer w.RUnlock() - return *w.m + return w.m } -func (w *messageWriterImpl) SetMetrics(m messageWriterMetrics) { +// SetMetrics sets the metrics +// +// This allows changing the labels of the metrics when the downstream consumer instance changes. +func (w *messageWriter) SetMetrics(m *messageWriterMetrics) { w.Lock() - w.m = &m + w.m = m w.Unlock() } -func (w *messageWriterImpl) QueueSize() int { +// QueueSize returns the number of messages queued in the writer. +func (w *messageWriter) QueueSize() int { return w.acks.size() } -func (w *messageWriterImpl) newMessage() *message { +func (w *messageWriter) newMessage() *message { return w.mPool.Get() } -func (w *messageWriterImpl) removeFromQueueWithLock(e *list.Element, m *message) { +func (w *messageWriter) removeFromQueueWithLock(e *list.Element, m *message) { w.queue.Remove(e) w.m.dequeuedMessages.Inc(1) w.close(m) } -func (w *messageWriterImpl) close(m *message) { +func (w *messageWriter) close(m *message) { m.Close() w.mPool.Put(m) } @@ -865,3 +818,11 @@ func StaticRetryNanosFn(backoffDurations []time.Duration) (MessageRetryNanosFn, return backoffInt64s[l-1] }, nil } + +func randIndex(iterationIndexes []int, i int) int { + j := int(unsafe.Fastrandn(uint32(i + 1))) + // NB: we should only mutate the order in the iteration indexes and + // keep the order of consumer writers unchanged to prevent data race. + iterationIndexes[i], iterationIndexes[j] = iterationIndexes[j], iterationIndexes[i] + return iterationIndexes[i] +} diff --git a/src/msg/producer/writer/message_writer_benchmark_test.go b/src/msg/producer/writer/message_writer_benchmark_test.go index d8433db0c5..ffe8e3c729 100644 --- a/src/msg/producer/writer/message_writer_benchmark_test.go +++ b/src/msg/producer/writer/message_writer_benchmark_test.go @@ -45,7 +45,7 @@ func BenchmarkScanMessageQueue(b *testing.B) { newMessagePool(), opts, testMessageWriterMetrics(), - ).(*messageWriterImpl) + ) w.consumerWriters = append(w.consumerWriters, noopWriter{}) diff --git a/src/msg/producer/writer/message_writer_test.go b/src/msg/producer/writer/message_writer_test.go index 34c0d5f1a5..087bb761e6 100644 --- a/src/msg/producer/writer/message_writer_test.go +++ b/src/msg/producer/writer/message_writer_test.go @@ -111,7 +111,7 @@ func TestMessageWriter(t *testing.T) { wg.Done() }() - w := newMessageWriter(200, newMessagePool(), opts, testMessageWriterMetrics()).(*messageWriterImpl) + w := newMessageWriter(200, newMessagePool(), opts, testMessageWriterMetrics()) require.Equal(t, 200, int(w.ReplicatedShardID())) w.Init() @@ -184,7 +184,7 @@ func TestMessageWriterRetry(t *testing.T) { addr := lis.Addr().String() opts := testOptions() - w := newMessageWriter(200, newMessagePool(), opts, testMessageWriterMetrics()).(*messageWriterImpl) + w := newMessageWriter(200, newMessagePool(), opts, testMessageWriterMetrics()) w.Init() defer w.Close() @@ -254,30 +254,30 @@ func TestMessageWriterCleanupDroppedMessage(t *testing.T) { w.Write(rm) // A get will allocate a new message because the old one has not been returned to pool yet. - m := w.(*messageWriterImpl).mPool.Get() + m := w.mPool.Get() require.Nil(t, m.RefCountedMessage) - require.Equal(t, 1, w.(*messageWriterImpl).queue.Len()) + require.Equal(t, 1, w.queue.Len()) w.Init() defer w.Close() for { - w.(*messageWriterImpl).Lock() - l := w.(*messageWriterImpl).queue.Len() - w.(*messageWriterImpl).Unlock() + w.Lock() + l := w.queue.Len() + w.Unlock() if l != 1 { break } time.Sleep(100 * time.Millisecond) } - require.True(t, isEmptyWithLock(w.(*messageWriterImpl).acks)) + require.True(t, isEmptyWithLock(w.acks)) } func TestMessageWriterCleanupAckedMessage(t *testing.T) { defer leaktest.Check(t)() opts := testOptions() - w := newMessageWriter(200, newMessagePool(), opts, testMessageWriterMetrics()).(*messageWriterImpl) + w := newMessageWriter(200, newMessagePool(), opts, testMessageWriterMetrics()) w.Init() defer w.Close() @@ -330,7 +330,7 @@ func TestMessageWriterCutoverCutoff(t *testing.T) { ctrl := xtest.NewController(t) defer ctrl.Finish() - w := newMessageWriter(200, newMessagePool(), nil, testMessageWriterMetrics()).(*messageWriterImpl) + w := newMessageWriter(200, newMessagePool(), nil, testMessageWriterMetrics()) now := time.Now() w.nowFn = func() time.Time { return now } require.True(t, w.isValidWriteWithLock(now.UnixNano())) @@ -357,7 +357,7 @@ func TestMessageWriterIgnoreCutoverCutoff(t *testing.T) { opts := NewOptions().SetIgnoreCutoffCutover(true) - w := newMessageWriter(200, newMessagePool(), opts, testMessageWriterMetrics()).(*messageWriterImpl) + w := newMessageWriter(200, newMessagePool(), opts, testMessageWriterMetrics()) now := time.Now() w.nowFn = func() time.Time { return now } @@ -382,7 +382,7 @@ func TestMessageWriterKeepNewWritesInOrderInFrontOfTheQueue(t *testing.T) { opts := testOptions().SetMessageRetryNanosFn( NextRetryNanosFn(retry.NewOptions().SetInitialBackoff(2 * time.Nanosecond).SetMaxBackoff(5 * time.Nanosecond)), ) - w := newMessageWriter(200, newMessagePool(), opts, testMessageWriterMetrics()).(*messageWriterImpl) + w := newMessageWriter(200, newMessagePool(), opts, testMessageWriterMetrics()) now := time.Now() w.nowFn = func() time.Time { return now } @@ -426,7 +426,7 @@ func TestMessageWriterRetryIterateBatchFullScan(t *testing.T) { opts := testOptions().SetMessageQueueScanBatchSize(retryBatchSize).SetMessageRetryNanosFn( NextRetryNanosFn(retry.NewOptions().SetInitialBackoff(2 * time.Nanosecond).SetMaxBackoff(5 * time.Nanosecond)), ) - w := newMessageWriter(200, newMessagePool(), opts, testMessageWriterMetrics()).(*messageWriterImpl) + w := newMessageWriter(200, newMessagePool(), opts, testMessageWriterMetrics()) now := time.Now() w.nowFn = func() time.Time { return now } @@ -491,7 +491,7 @@ func TestMessageWriterRetryIterateBatchFullScanWithMessageTTL(t *testing.T) { opts := testOptions().SetMessageQueueScanBatchSize(retryBatchSize).SetMessageRetryNanosFn( NextRetryNanosFn(retry.NewOptions().SetInitialBackoff(2 * time.Nanosecond).SetMaxBackoff(5 * time.Nanosecond)), ) - w := newMessageWriter(200, newMessagePool(), opts, testMessageWriterMetrics()).(*messageWriterImpl) + w := newMessageWriter(200, newMessagePool(), opts, testMessageWriterMetrics()) now := time.Now() w.nowFn = func() time.Time { return now } @@ -553,7 +553,7 @@ func TestMessageWriterRetryIterateBatchNotFullScan(t *testing.T) { opts := testOptions().SetMessageQueueScanBatchSize(retryBatchSize).SetMessageRetryNanosFn( NextRetryNanosFn(retry.NewOptions().SetInitialBackoff(2 * time.Nanosecond).SetMaxBackoff(5 * time.Nanosecond)), ) - w := newMessageWriter(200, newMessagePool(), opts, testMessageWriterMetrics()).(*messageWriterImpl) + w := newMessageWriter(200, newMessagePool(), opts, testMessageWriterMetrics()) now := time.Now() w.nowFn = func() time.Time { return now } @@ -629,7 +629,7 @@ func TestNextRetryAfterNanos(t *testing.T) { SetJitter(true), ), ) - w := newMessageWriter(200, nil, opts, testMessageWriterMetrics()).(*messageWriterImpl) + w := newMessageWriter(200, nil, opts, testMessageWriterMetrics()) nowNanos := time.Now().UnixNano() m := newMessage() @@ -653,7 +653,7 @@ func TestStaticRetryAfterNanos(t *testing.T) { require.NoError(t, err) opts := testOptions().SetMessageRetryNanosFn(fn) - w := newMessageWriter(200, nil, opts, testMessageWriterMetrics()).(*messageWriterImpl) + w := newMessageWriter(200, nil, opts, testMessageWriterMetrics()) m := newMessage() m.IncWriteTimes() @@ -686,7 +686,7 @@ func TestMessageWriterCloseCleanupAllMessages(t *testing.T) { defer leaktest.Check(t)() opts := testOptions() - w := newMessageWriter(200, newMessagePool(), opts, testMessageWriterMetrics()).(*messageWriterImpl) + w := newMessageWriter(200, newMessagePool(), opts, testMessageWriterMetrics()) ctrl := xtest.NewController(t) defer ctrl.Finish() @@ -713,7 +713,7 @@ func TestMessageWriterQueueFullScanOnWriteErrors(t *testing.T) { opts := testOptions().SetMessageQueueScanBatchSize(1) scope := tally.NewTestScope("", nil) metrics := testMessageWriterMetricsWithScope(scope).withConsumer("c1") - w := newMessageWriter(200, newMessagePool(), opts, metrics).(*messageWriterImpl) + w := newMessageWriter(200, newMessagePool(), opts, metrics) w.AddConsumerWriter(newConsumerWriter("bad", nil, opts, testConsumerWriterMetrics())) mm1 := producer.NewMockMessage(ctrl) @@ -748,7 +748,7 @@ func TestMessageWriter_WithoutConsumerScope(t *testing.T) { opts := testOptions().SetMessageQueueScanBatchSize(1) scope := tally.NewTestScope("", nil) metrics := newMessageWriterMetrics(scope, instrument.TimerOptions{}, true) - w := newMessageWriter(200, nil, opts, metrics).(*messageWriterImpl) + w := newMessageWriter(200, nil, opts, metrics) w.AddConsumerWriter(newConsumerWriter("bad", nil, opts, testConsumerWriterMetrics())) snapshot := scope.Snapshot() @@ -763,15 +763,15 @@ func isEmptyWithLock(h *acks) bool { return len(h.ackMap) == 0 } -func testMessageWriterMetrics() messageWriterMetrics { +func testMessageWriterMetrics() *messageWriterMetrics { return newMessageWriterMetrics(tally.NoopScope, instrument.TimerOptions{}, false) } -func testMessageWriterMetricsWithScope(scope tally.TestScope) messageWriterMetrics { +func testMessageWriterMetricsWithScope(scope tally.TestScope) *messageWriterMetrics { return newMessageWriterMetrics(scope, instrument.TimerOptions{}, false) } -func validateMessages(t *testing.T, msgs []*producer.RefCountedMessage, w *messageWriterImpl) { +func validateMessages(t *testing.T, msgs []*producer.RefCountedMessage, w *messageWriter) { w.RLock() idx := 0 for e := w.queue.Front(); e != nil; e = e.Next() { diff --git a/src/msg/producer/writer/router.go b/src/msg/producer/writer/router.go index 4218720c07..f5b0fe177d 100644 --- a/src/msg/producer/writer/router.go +++ b/src/msg/producer/writer/router.go @@ -30,21 +30,23 @@ type ackRouter interface { Ack(ack metadata) error // Register registers a message writer. - Register(replicatedShardID uint64, mw messageWriter) + Register(replicatedShardID uint64, mw *messageWriter) // Unregister removes a message writer. Unregister(replicatedShardID uint64) } +var _ ackRouter = (*router)(nil) + type router struct { sync.RWMutex - messageWriters map[uint64]messageWriter + messageWriters map[uint64]*messageWriter } -func newAckRouter(size int) ackRouter { +func newAckRouter(size int) *router { return &router{ - messageWriters: make(map[uint64]messageWriter, size), + messageWriters: make(map[uint64]*messageWriter, size), } } @@ -60,7 +62,7 @@ func (r *router) Ack(meta metadata) error { return nil } -func (r *router) Register(replicatedShardID uint64, mw messageWriter) { +func (r *router) Register(replicatedShardID uint64, mw *messageWriter) { r.Lock() r.messageWriters[replicatedShardID] = mw r.Unlock() diff --git a/src/msg/producer/writer/router_mock.go b/src/msg/producer/writer/router_mock.go index 35828cb49c..c96beadf45 100644 --- a/src/msg/producer/writer/router_mock.go +++ b/src/msg/producer/writer/router_mock.go @@ -68,7 +68,7 @@ func (mr *MockackRouterMockRecorder) Ack(ack interface{}) *gomock.Call { } // Register mocks base method. -func (m *MockackRouter) Register(replicatedShardID uint64, mw messageWriter) { +func (m *MockackRouter) Register(replicatedShardID uint64, mw *messageWriter) { m.ctrl.T.Helper() m.ctrl.Call(m, "Register", replicatedShardID, mw) } diff --git a/src/msg/producer/writer/shard_writer.go b/src/msg/producer/writer/shard_writer.go index 4216a54d8b..feb8995841 100644 --- a/src/msg/producer/writer/shard_writer.go +++ b/src/msg/producer/writer/shard_writer.go @@ -52,8 +52,8 @@ type shardWriter interface { type sharedShardWriter struct { instances map[string]struct{} - mw messageWriter - isClosed *atomic.Bool + mw *messageWriter + isClosed atomic.Bool } func newSharedShardWriter( @@ -61,7 +61,7 @@ func newSharedShardWriter( router ackRouter, mPool *messagePool, opts Options, - m messageWriterMetrics, + m *messageWriterMetrics, ) shardWriter { replicatedShardID := uint64(shard) mw := newMessageWriter(replicatedShardID, mPool, opts, m) @@ -70,7 +70,6 @@ func newSharedShardWriter( return &sharedShardWriter{ instances: make(map[string]struct{}), mw: mw, - isClosed: atomic.NewBool(false), } } @@ -129,9 +128,9 @@ type replicatedShardWriter struct { ackRouter ackRouter opts Options logger *zap.Logger - m messageWriterMetrics + m *messageWriterMetrics - messageWriters map[string]messageWriter + messageWriters map[string]*messageWriter messageTTLNanos int64 replicaID uint32 isClosed bool @@ -142,7 +141,7 @@ func newReplicatedShardWriter( router ackRouter, mPool *messagePool, opts Options, - m messageWriterMetrics, + m *messageWriterMetrics, ) shardWriter { return &replicatedShardWriter{ shard: shard, @@ -152,7 +151,7 @@ func newReplicatedShardWriter( logger: opts.InstrumentOptions().Logger(), ackRouter: router, replicaID: 0, - messageWriters: make(map[string]messageWriter), + messageWriters: make(map[string]*messageWriter), isClosed: false, m: m, } @@ -181,8 +180,8 @@ func (w *replicatedShardWriter) UpdateInstances( // are already cutoff. Otherwise it will wait until next placement change // to clean up. var ( - newMessageWriters = make(map[string]messageWriter, len(instances)) - toBeClosed []messageWriter + newMessageWriters = make(map[string]*messageWriter, len(instances)) + toBeClosed []*messageWriter toBeAdded = make(map[placement.Instance]consumerWriter, len(instances)) oldMessageWriters = w.messageWriters ) @@ -253,7 +252,7 @@ func (w *replicatedShardWriter) UpdateInstances( } func (w *replicatedShardWriter) updateCutoverCutoffNanos( - mw messageWriter, + mw *messageWriter, instance placement.Instance, ) { s, ok := instance.Shards().Shard(w.shard) diff --git a/src/msg/producer/writer/shard_writer_test.go b/src/msg/producer/writer/shard_writer_test.go index 9a877fbe55..ece56d1148 100644 --- a/src/msg/producer/writer/shard_writer_test.go +++ b/src/msg/producer/writer/shard_writer_test.go @@ -89,7 +89,7 @@ func TestSharedShardWriter(t *testing.T) { sw.Write(producer.NewRefCountedMessage(mm, nil)) - mw := sw.(*sharedShardWriter).mw.(*messageWriterImpl) + mw := sw.(*sharedShardWriter).mw mw.RLock() require.Equal(t, 1, len(mw.consumerWriters)) require.Equal(t, 1, mw.queue.Len()) @@ -172,9 +172,9 @@ func TestReplicatedShardWriter(t *testing.T) { sw.Write(producer.NewRefCountedMessage(mm, nil)) - mw1 := sw.messageWriters[i1.Endpoint()].(*messageWriterImpl) + mw1 := sw.messageWriters[i1.Endpoint()] require.Equal(t, 1, mw1.queue.Len()) - mw3 := sw.messageWriters[i3.Endpoint()].(*messageWriterImpl) + mw3 := sw.messageWriters[i3.Endpoint()] require.Equal(t, 1, mw3.queue.Len()) var wg sync.WaitGroup @@ -219,7 +219,7 @@ func TestReplicatedShardWriter(t *testing.T) { time.Sleep(100 * time.Millisecond) } - mw2 := sw.messageWriters[i2.Endpoint()].(*messageWriterImpl) + mw2 := sw.messageWriters[i2.Endpoint()] require.Equal(t, mw3, mw2) _, ok := sw.messageWriters[i3.Endpoint()] require.False(t, ok) @@ -228,7 +228,7 @@ func TestReplicatedShardWriter(t *testing.T) { func TestReplicatedShardWriterRemoveMessageWriter(t *testing.T) { defer leaktest.Check(t)() - router := newAckRouter(2).(*router) + router := newAckRouter(2) opts := testOptions() sw := newReplicatedShardWriter( 1, 200, router, newMessagePool(), opts, testMessageWriterMetrics(), @@ -269,8 +269,8 @@ func TestReplicatedShardWriterRemoveMessageWriter(t *testing.T) { require.Equal(t, 2, len(sw.messageWriters)) - mw1 := sw.messageWriters[i1.Endpoint()].(*messageWriterImpl) - mw2 := sw.messageWriters[i2.Endpoint()].(*messageWriterImpl) + mw1 := sw.messageWriters[i1.Endpoint()] + mw2 := sw.messageWriters[i2.Endpoint()] require.Equal(t, 0, mw1.queue.Len()) require.Equal(t, 0, mw2.queue.Len()) @@ -410,9 +410,9 @@ func TestReplicatedShardWriterUpdate(t *testing.T) { require.NotNil(t, sw.messageWriters[i3.Endpoint()]) require.Equal(t, 500, int(mw3.MessageTTLNanos())) for { - mw2.(*messageWriterImpl).RLock() - isClosed := mw2.(*messageWriterImpl).isClosed - mw2.(*messageWriterImpl).RUnlock() + mw2.RLock() + isClosed := mw2.isClosed + mw2.RUnlock() if isClosed { break } From e4c8e418a71e20a07b9970c6bbeff971212526d6 Mon Sep 17 00:00:00 2001 From: Vytenis Darulis Date: Wed, 4 May 2022 18:15:00 -0400 Subject: [PATCH 5/8] [aggregator] Clean up aggregated metric writer and encoder (#4112) --- .../aggregator/handler/writer/options.go | 27 +++----- .../aggregator/handler/writer/protobuf.go | 63 +++++++------------ .../handler/writer/protobuf_test.go | 29 +-------- .../server/m3msg/protobuf_handler_test.go | 23 +++---- .../encoding/protobuf/aggregated_decoder.go | 2 +- .../aggregated_decoder_benchmark_test.go | 2 +- .../encoding/protobuf/aggregated_encoder.go | 24 +++---- .../protobuf/aggregated_roundtrip_test.go | 16 ++--- 8 files changed, 57 insertions(+), 129 deletions(-) diff --git a/src/aggregator/aggregator/handler/writer/options.go b/src/aggregator/aggregator/handler/writer/options.go index edbea6ad72..eddb5187e0 100644 --- a/src/aggregator/aggregator/handler/writer/options.go +++ b/src/aggregator/aggregator/handler/writer/options.go @@ -26,16 +26,12 @@ import ( "github.com/m3db/m3/src/x/pool" ) -const ( - defaultEncodingTimeSamplingRate = 0 -) - // Options provide a set of options for the writer. type Options interface { - // SetClockOptions sets the clock options. + // SetClockOptions is deprecated SetClockOptions(value clock.Options) Options - // ClockOptions returns the clock options. + // ClockOptions is deprecated ClockOptions() clock.Options // SetInstrumentOptions sets the instrument options. @@ -50,14 +46,10 @@ type Options interface { // BytesPool returns the bytes pool. BytesPool() pool.BytesPool - // SetEncodingTimeSamplingRate sets the sampling rate at which the encoding time is - // included in the encoded data. A value of 0 means the encoding time is never included, - // and a value of 1 means the encoding time is always included. + // SetEncodingTimeSamplingRate is deprecated SetEncodingTimeSamplingRate(value float64) Options - // EncodingTimeSamplingRate returns the sampling rate at which the encoding time is - // included in the encoded data. A value of 0 means the encoding time is never included, - // and a value of 1 means the encoding time is always included. + // EncodingTimeSamplingRate is deprecated EncodingTimeSamplingRate() float64 } @@ -71,9 +63,8 @@ type options struct { // NewOptions provide a set of writer options. func NewOptions() Options { return &options{ - clockOpts: clock.NewOptions(), - instrumentOpts: instrument.NewOptions(), - encodingTimeSamplingRate: defaultEncodingTimeSamplingRate, + clockOpts: clock.NewOptions(), + instrumentOpts: instrument.NewOptions(), } } @@ -108,11 +99,9 @@ func (o *options) BytesPool() pool.BytesPool { } func (o *options) SetEncodingTimeSamplingRate(value float64) Options { - opts := *o - opts.encodingTimeSamplingRate = value - return &opts + return o } func (o *options) EncodingTimeSamplingRate() float64 { - return o.encodingTimeSamplingRate + return 0 } diff --git a/src/aggregator/aggregator/handler/writer/protobuf.go b/src/aggregator/aggregator/handler/writer/protobuf.go index e8ff002d3b..2ab738c10a 100644 --- a/src/aggregator/aggregator/handler/writer/protobuf.go +++ b/src/aggregator/aggregator/handler/writer/protobuf.go @@ -22,14 +22,12 @@ package writer import ( "errors" - "math/rand" "github.com/m3db/m3/src/aggregator/sharding" "github.com/m3db/m3/src/metrics/encoding/protobuf" "github.com/m3db/m3/src/metrics/metric/aggregated" "github.com/m3db/m3/src/metrics/policy" "github.com/m3db/m3/src/msg/producer" - "github.com/m3db/m3/src/x/clock" "github.com/uber-go/tally" ) @@ -38,44 +36,34 @@ var ( errWriterClosed = errors.New("writer is closed") ) -type randFn func() float64 - type protobufWriterMetrics struct { - writerClosed tally.Counter - encodeSuccess tally.Counter - encodeErrors tally.Counter - routeSuccess tally.Counter - routeErrors tally.Counter + writerClosed tally.Counter + encodeErrors tally.Counter + routeErrors tally.Counter } func newProtobufWriterMetrics(scope tally.Scope) protobufWriterMetrics { encodeScope := scope.SubScope("encode") routeScope := scope.SubScope("route") return protobufWriterMetrics{ - writerClosed: scope.Counter("writer-closed"), - encodeSuccess: encodeScope.Counter("success"), - encodeErrors: encodeScope.Counter("errors"), - routeSuccess: routeScope.Counter("success"), - routeErrors: routeScope.Counter("errors"), + writerClosed: scope.Counter("writer-closed"), + encodeErrors: encodeScope.Counter("errors"), + routeErrors: routeScope.Counter("errors"), } } // protobufWriter encodes data and routes them to the backend. // protobufWriter is not thread safe. type protobufWriter struct { - encodingTimeSamplingRate float64 - encoder protobuf.AggregatedEncoder - p producer.Producer - numShards uint32 + encoder *protobuf.AggregatedEncoder + p producer.Producer + numShards uint32 - closed bool m aggregated.MetricWithStoragePolicy - rand *rand.Rand metrics protobufWriterMetrics - nowFn clock.NowFn - randFn randFn shardFn sharding.ShardFn + closed bool } // NewProtobufWriter creates a writer that encodes metric in protobuf. @@ -84,20 +72,15 @@ func NewProtobufWriter( shardFn sharding.ShardFn, opts Options, ) Writer { - nowFn := opts.ClockOptions().NowFn() instrumentOpts := opts.InstrumentOptions() w := &protobufWriter{ - encodingTimeSamplingRate: opts.EncodingTimeSamplingRate(), - encoder: protobuf.NewAggregatedEncoder(opts.BytesPool()), - p: producer, - numShards: producer.NumShards(), - closed: false, - rand: rand.New(rand.NewSource(nowFn().UnixNano())), - metrics: newProtobufWriterMetrics(instrumentOpts.MetricsScope()), - nowFn: nowFn, - shardFn: shardFn, + encoder: protobuf.NewAggregatedEncoder(opts.BytesPool()), + p: producer, + numShards: producer.NumShards(), + closed: false, + metrics: newProtobufWriterMetrics(instrumentOpts.MetricsScope()), + shardFn: shardFn, } - w.randFn = w.rand.Float64 return w } @@ -106,22 +89,18 @@ func (w *protobufWriter) Write(mp aggregated.ChunkedMetricWithStoragePolicy) err w.metrics.writerClosed.Inc(1) return errWriterClosed } - var encodeNanos int64 - if w.encodingTimeSamplingRate > 0 && w.randFn() < w.encodingTimeSamplingRate { - encodeNanos = w.nowFn().UnixNano() - } + m, shard := w.prepare(mp) - if err := w.encoder.Encode(m, encodeNanos); err != nil { + if err := w.encoder.Encode(m); err != nil { w.metrics.encodeErrors.Inc(1) return err } - w.metrics.encodeSuccess.Inc(1) if err := w.p.Produce(newMessage(shard, mp.StoragePolicy, w.encoder.Buffer())); err != nil { w.metrics.routeErrors.Inc(1) return err } - w.metrics.routeSuccess.Inc(1) + return nil } @@ -198,8 +177,8 @@ func (f storagePolicyFilter) Filter(m producer.Message) bool { if !ok { return true } - for _, accepted := range f.acceptedStoragePolicies { - if accepted == msg.sp { + for i := 0; i < len(f.acceptedStoragePolicies); i++ { + if f.acceptedStoragePolicies[i].Equivalent(msg.sp) { return true } } diff --git a/src/aggregator/aggregator/handler/writer/protobuf_test.go b/src/aggregator/aggregator/handler/writer/protobuf_test.go index 8fe686d8d2..2e67f2e813 100644 --- a/src/aggregator/aggregator/handler/writer/protobuf_test.go +++ b/src/aggregator/aggregator/handler/writer/protobuf_test.go @@ -31,7 +31,6 @@ import ( "github.com/m3db/m3/src/metrics/metric/id" "github.com/m3db/m3/src/metrics/policy" "github.com/m3db/m3/src/msg/producer" - "github.com/m3db/m3/src/x/clock" xtime "github.com/m3db/m3/src/x/time" "github.com/golang/mock/gomock" @@ -108,12 +107,7 @@ func TestProtobufWriterWriteClosed(t *testing.T) { } func TestProtobufWriterWrite(t *testing.T) { - now := time.Now() - nowFn := func() time.Time { return now } - opts := NewOptions(). - SetClockOptions(clock.NewOptions().SetNowFn(nowFn)). - SetEncodingTimeSamplingRate(0.5) - + opts := NewOptions() ctrl := gomock.NewController(t) defer ctrl.Finish() writer := testProtobufWriter(t, ctrl, opts) @@ -133,7 +127,6 @@ func TestProtobufWriterWrite(t *testing.T) { }, StoragePolicy: sp, }, - encodedAtNanos: d.EncodeNanos(), }) return nil }).AnyTimes() @@ -147,14 +140,6 @@ func TestProtobufWriterWrite(t *testing.T) { require.Fail(t, "unexpected chunked id %v", id) return 0 } - var iter int - writer.randFn = func() float64 { - iter++ - if iter%2 == 0 { - return 0.1 - } - return 0.9 - } inputs := []aggregated.ChunkedMetricWithStoragePolicy{ testChunkedMetricWithStoragePolicy, @@ -165,27 +150,21 @@ func TestProtobufWriterWrite(t *testing.T) { for _, input := range inputs { require.NoError(t, writer.Write(input)) } - - encodedAtNanos := now.UnixNano() expectedData := map[uint32][]decodeData{ - 1: []decodeData{ + 1: { { MetricWithStoragePolicy: testMetricWithStoragePolicy, - encodedAtNanos: 0, }, { MetricWithStoragePolicy: testMetricWithStoragePolicy, - encodedAtNanos: encodedAtNanos, }, }, - 2: []decodeData{ + 2: { { MetricWithStoragePolicy: testMetricWithStoragePolicy2, - encodedAtNanos: encodedAtNanos, }, { MetricWithStoragePolicy: testMetricWithStoragePolicy2, - encodedAtNanos: 0, }, }, } @@ -205,6 +184,4 @@ func testProtobufWriter(t *testing.T, ctrl *gomock.Controller, opts Options) *pr type decodeData struct { aggregated.MetricWithStoragePolicy - - encodedAtNanos int64 } diff --git a/src/cmd/services/m3coordinator/server/m3msg/protobuf_handler_test.go b/src/cmd/services/m3coordinator/server/m3msg/protobuf_handler_test.go index b15cf8af9b..1a11d16696 100644 --- a/src/cmd/services/m3coordinator/server/m3msg/protobuf_handler_test.go +++ b/src/cmd/services/m3coordinator/server/m3msg/protobuf_handler_test.go @@ -22,7 +22,6 @@ package m3msg import ( "context" - "fmt" "net" "sync" "testing" @@ -85,7 +84,7 @@ func TestM3MsgServerWithProtobufHandler(t *testing.T) { } encoder := protobuf.NewAggregatedEncoder(nil) - require.NoError(t, encoder.Encode(m1, 2000)) + require.NoError(t, encoder.Encode(m1)) enc := proto.NewEncoder(opts.EncoderOptions()) require.NoError(t, enc.Encode(&msgpb.Message{ Value: encoder.Buffer().Bytes(), @@ -107,7 +106,7 @@ func TestM3MsgServerWithProtobufHandler(t *testing.T) { }, StoragePolicy: precisionStoragePolicy, } - require.NoError(t, encoder.Encode(m2, 3000)) + require.NoError(t, encoder.Encode(m2)) enc = proto.NewEncoder(opts.EncoderOptions()) require.NoError(t, enc.Encode(&msgpb.Message{ Value: encoder.Buffer().Bytes(), @@ -117,19 +116,17 @@ func TestM3MsgServerWithProtobufHandler(t *testing.T) { require.NoError(t, dec.Decode(&a)) require.Equal(t, 2, w.ingested()) - payload, ok := w.m[key(string(m1.ID), 2000)] + payload, ok := w.m[string(m1.ID)] //nolint:govet require.True(t, ok) require.Equal(t, string(m1.ID), payload.id) require.Equal(t, m1.TimeNanos, payload.metricNanos) - require.Equal(t, 2000, int(payload.encodeNanos)) require.Equal(t, m1.Value, payload.value) require.Equal(t, m1.StoragePolicy, payload.sp) - payload, ok = w.m[key(string(m2.ID), 3000)] + payload, ok = w.m[string(m2.ID)] require.True(t, ok) require.Equal(t, string(m2.ID), payload.id) require.Equal(t, m2.TimeNanos, payload.metricNanos) - require.Equal(t, 3000, int(payload.encodeNanos)) require.Equal(t, m2.Value, payload.value) require.Equal(t, m2.StoragePolicy, payload.sp) } @@ -168,7 +165,7 @@ func TestM3MsgServerWithProtobufHandler_Blackhole(t *testing.T) { } encoder := protobuf.NewAggregatedEncoder(nil) - require.NoError(t, encoder.Encode(m1, 2000)) + require.NoError(t, encoder.Encode(m1)) enc := proto.NewEncoder(opts.EncoderOptions()) require.NoError(t, enc.Encode(&msgpb.Message{ Value: encoder.Buffer().Bytes(), @@ -191,7 +188,7 @@ func TestM3MsgServerWithProtobufHandler_Blackhole(t *testing.T) { }, StoragePolicy: policy.MustParseStoragePolicy("5m:180d"), } - require.NoError(t, encoder.Encode(m2, 3000)) + require.NoError(t, encoder.Encode(m2)) enc = proto.NewEncoder(opts.EncoderOptions()) require.NoError(t, enc.Encode(&msgpb.Message{ Value: encoder.Buffer().Bytes(), @@ -212,7 +209,7 @@ func TestM3MsgServerWithProtobufHandler_Blackhole(t *testing.T) { }, StoragePolicy: baseStoragePolicy, } - require.NoError(t, encoder.Encode(m3, 3000)) + require.NoError(t, encoder.Encode(m3)) enc = proto.NewEncoder(opts.EncoderOptions()) require.NoError(t, enc.Encode(&msgpb.Message{ Value: encoder.Buffer().Bytes(), @@ -248,7 +245,7 @@ func (m *mockWriter) write( value: value, sp: sp, } - m.m[key(payload.id, encodeNanos)] = payload + m.m[payload.id] = payload m.Unlock() callbackable.Callback(OnSuccess) } @@ -260,10 +257,6 @@ func (m *mockWriter) ingested() int { return m.n } -func key(id string, encodeTime int64) string { - return fmt.Sprintf("%s%d", id, encodeTime) -} - type payload struct { id string metricNanos int64 diff --git a/src/metrics/encoding/protobuf/aggregated_decoder.go b/src/metrics/encoding/protobuf/aggregated_decoder.go index c89ac2c524..c37610cd69 100644 --- a/src/metrics/encoding/protobuf/aggregated_decoder.go +++ b/src/metrics/encoding/protobuf/aggregated_decoder.go @@ -72,7 +72,7 @@ func (d AggregatedDecoder) StoragePolicy() policy.StoragePolicy { return d.sp } -// EncodeNanos returns the decoded encodeNanos. +// EncodeNanos is deprecated. func (d AggregatedDecoder) EncodeNanos() int64 { return d.pb.EncodeNanos } diff --git a/src/metrics/encoding/protobuf/aggregated_decoder_benchmark_test.go b/src/metrics/encoding/protobuf/aggregated_decoder_benchmark_test.go index 56e83b4891..997ec2c84d 100644 --- a/src/metrics/encoding/protobuf/aggregated_decoder_benchmark_test.go +++ b/src/metrics/encoding/protobuf/aggregated_decoder_benchmark_test.go @@ -33,7 +33,7 @@ func BenchmarkDecodeStoragePolicy(b *testing.B) { dec = NewAggregatedDecoder(nil) sp policy.StoragePolicy ) - if err := enc.Encode(testAggregatedMetric1, 2000); err != nil { + if err := enc.Encode(testAggregatedMetric1); err != nil { b.Fatal(err) } diff --git a/src/metrics/encoding/protobuf/aggregated_encoder.go b/src/metrics/encoding/protobuf/aggregated_encoder.go index 1fe7108799..7a8911930c 100644 --- a/src/metrics/encoding/protobuf/aggregated_encoder.go +++ b/src/metrics/encoding/protobuf/aggregated_encoder.go @@ -27,38 +27,27 @@ import ( ) // AggregatedEncoder is an encoder for encoding aggregated metrics. -type AggregatedEncoder interface { - // Encode encodes a metric with an applicable storage policy, - // alongside the time at which encoding happens. - Encode(m aggregated.MetricWithStoragePolicy, encodedAtNanos int64) error - - // Buffer returns the encoded buffer. - Buffer() Buffer -} - -type aggregatedEncoder struct { +type AggregatedEncoder struct { pool pool.BytesPool buf []byte pb metricpb.AggregatedMetric } // NewAggregatedEncoder creates a new aggregated encoder. -func NewAggregatedEncoder(p pool.BytesPool) AggregatedEncoder { - e := &aggregatedEncoder{ +func NewAggregatedEncoder(p pool.BytesPool) *AggregatedEncoder { + return &AggregatedEncoder{ pool: p, } - return e } -func (enc *aggregatedEncoder) Encode( +// Encode encodes a metric with an applicable storage policy. +func (enc *AggregatedEncoder) Encode( m aggregated.MetricWithStoragePolicy, - encodedAtNanos int64, ) error { ReuseAggregatedMetricProto(&enc.pb) if err := m.ToProto(&enc.pb.Metric); err != nil { return err } - enc.pb.EncodeNanos = encodedAtNanos // Always allocate a new byte slice to avoid modifying the existing one which may still being used. enc.buf = allocate(enc.pool, enc.pb.Size()) n, err := enc.pb.MarshalTo(enc.buf) @@ -66,7 +55,8 @@ func (enc *aggregatedEncoder) Encode( return err } -func (enc *aggregatedEncoder) Buffer() Buffer { +// Buffer returns the encoded buffer +func (enc *AggregatedEncoder) Buffer() Buffer { var fn PoolReleaseFn if enc.pool != nil { fn = enc.pool.Put diff --git a/src/metrics/encoding/protobuf/aggregated_roundtrip_test.go b/src/metrics/encoding/protobuf/aggregated_roundtrip_test.go index 516d41f6e6..02a8fe437e 100644 --- a/src/metrics/encoding/protobuf/aggregated_roundtrip_test.go +++ b/src/metrics/encoding/protobuf/aggregated_roundtrip_test.go @@ -54,9 +54,9 @@ var ( func TestAggregatedEncoderDecoder_RoundTrip(t *testing.T) { enc := NewAggregatedEncoder(nil) dec := NewAggregatedDecoder(nil) - require.NoError(t, enc.Encode(testAggregatedMetric1, 2000)) + require.NoError(t, enc.Encode(testAggregatedMetric1)) require.NoError(t, dec.Decode(enc.Buffer().Bytes())) - require.Equal(t, int64(2000), dec.EncodeNanos()) + require.Equal(t, int64(0), dec.EncodeNanos()) sp := dec.StoragePolicy() require.Equal(t, testAggregatedMetric1.StoragePolicy, sp) require.Equal(t, string(testAggregatedMetric1.ID), string(dec.ID())) @@ -73,9 +73,9 @@ func TestAggregatedEncoderDecoder_WithBytesPool(t *testing.T) { p.Init() enc := NewAggregatedEncoder(p) dec := NewAggregatedDecoder(nil) - require.NoError(t, enc.Encode(testAggregatedMetric1, 2000)) + require.NoError(t, enc.Encode(testAggregatedMetric1)) require.NoError(t, dec.Decode(enc.Buffer().Bytes())) - require.Equal(t, int64(2000), dec.EncodeNanos()) + require.Equal(t, int64(0), dec.EncodeNanos()) sp := dec.StoragePolicy() require.Equal(t, testAggregatedMetric1.StoragePolicy, sp) require.Equal(t, string(testAggregatedMetric1.ID), string(dec.ID())) @@ -86,9 +86,9 @@ func TestAggregatedEncoderDecoder_WithBytesPool(t *testing.T) { func TestAggregatedEncoderDecoder_ResetProtobuf(t *testing.T) { enc := NewAggregatedEncoder(nil) dec := NewAggregatedDecoder(nil) - require.NoError(t, enc.Encode(testAggregatedMetric1, 2000)) + require.NoError(t, enc.Encode(testAggregatedMetric1)) require.NoError(t, dec.Decode(enc.Buffer().Bytes())) - require.Equal(t, int64(2000), dec.EncodeNanos()) + require.Equal(t, int64(0), dec.EncodeNanos()) sp := dec.StoragePolicy() require.Equal(t, testAggregatedMetric1.StoragePolicy, sp) require.Equal(t, string(testAggregatedMetric1.ID), string(dec.ID())) @@ -99,9 +99,9 @@ func TestAggregatedEncoderDecoder_ResetProtobuf(t *testing.T) { // to avoid previously decoded value being applied in the next decoding. dec.Close() - require.NoError(t, enc.Encode(testAggregatedMetric2, 3000)) + require.NoError(t, enc.Encode(testAggregatedMetric2)) require.NoError(t, dec.Decode(enc.Buffer().Bytes())) - require.Equal(t, int64(3000), dec.EncodeNanos()) + require.Equal(t, int64(0), dec.EncodeNanos()) sp = dec.StoragePolicy() require.Equal(t, testAggregatedMetric2.StoragePolicy, sp) require.Equal(t, string(testAggregatedMetric2.ID), string(dec.ID())) From 898bcf844a65dd8b7314d18e082720f67d050cf8 Mon Sep 17 00:00:00 2001 From: Vytenis Darulis Date: Thu, 5 May 2022 14:01:31 -0400 Subject: [PATCH 6/8] [m3msg] Specialize messageWriter acker map (#4113) --- .golangci.yml | 2 + src/msg/producer/writer/message_writer.go | 92 ++++++++++--------- .../writer/message_writer_benchmark_test.go | 9 ++ .../producer/writer/message_writer_test.go | 31 ++++--- 4 files changed, 75 insertions(+), 59 deletions(-) diff --git a/.golangci.yml b/.golangci.yml index 0785343a08..19c9240798 100644 --- a/.golangci.yml +++ b/.golangci.yml @@ -298,6 +298,8 @@ issues: - "G104: Errors unhandled" # Random numbers here are not used in a security-sensitive context. - "G404: Use of weak random number generator (math/rand instead of crypto/rand)" + # Pointless, every time we'll just end up adding a //nolint comment on this. + - "G103: Use of unsafe calls should be audited" # Excluding configuration per-path, per-linter, per-text and per-source exclude-rules: diff --git a/src/msg/producer/writer/message_writer.go b/src/msg/producer/writer/message_writer.go index f96e651d3e..1ca18bd57a 100644 --- a/src/msg/producer/writer/message_writer.go +++ b/src/msg/producer/writer/message_writer.go @@ -26,6 +26,7 @@ import ( "math" "sync" "time" + stdunsafe "unsafe" "github.com/m3db/m3/src/msg/producer" "github.com/m3db/m3/src/msg/protocol/proto" @@ -35,6 +36,7 @@ import ( "github.com/m3db/m3/src/x/unsafe" "github.com/uber-go/tally" + "go.uber.org/atomic" ) // MessageRetryNanosFn returns the message backoff time for retry in nanoseconds. @@ -179,7 +181,7 @@ type messageWriter struct { doneCh chan struct{} wg sync.WaitGroup // metrics can be updated when a consumer instance changes, so must be guarded with RLock - m *messageWriterMetrics + metrics atomic.UnsafePointer // *messageWriterMetrics nextFullScan time.Time lastNewWrite *list.Element @@ -196,7 +198,7 @@ func newMessageWriter( opts = NewOptions() } nowFn := time.Now - return &messageWriter{ + mw := &messageWriter{ replicatedShardID: replicatedShardID, mPool: mPool, opts: opts, @@ -211,9 +213,10 @@ func newMessageWriter( msgsToWrite: make([]*message, 0, opts.MessageQueueScanBatchSize()), isClosed: false, doneCh: make(chan struct{}), - m: m, nowFn: nowFn, } + mw.metrics.Store(stdunsafe.Pointer(m)) + return mw } // Write writes a message, messages not acknowledged in time will be retried. @@ -222,9 +225,10 @@ func (w *messageWriter) Write(rm *producer.RefCountedMessage) { var ( nowNanos = w.nowFn().UnixNano() msg = w.newMessage() + metrics = w.Metrics() ) w.Lock() - if !w.isValidWriteWithLock(nowNanos) { + if !w.isValidWriteWithLock(nowNanos, metrics) { w.Unlock() w.close(msg) return @@ -240,7 +244,7 @@ func (w *messageWriter) Write(rm *producer.RefCountedMessage) { msg.Set(meta, rm, nowNanos) w.acks.add(meta, msg) // Make sure all the new writes are ordered in queue. - w.m.enqueuedMessages.Inc(1) + metrics.enqueuedMessages.Inc(1) if w.lastNewWrite != nil { w.lastNewWrite = w.queue.InsertAfter(msg, w.lastNewWrite) } else { @@ -249,17 +253,17 @@ func (w *messageWriter) Write(rm *producer.RefCountedMessage) { w.Unlock() } -func (w *messageWriter) isValidWriteWithLock(nowNanos int64) bool { +func (w *messageWriter) isValidWriteWithLock(nowNanos int64, metrics *messageWriterMetrics) bool { if w.opts.IgnoreCutoffCutover() { return true } if w.cutOffNanos > 0 && nowNanos >= w.cutOffNanos { - w.m.writeAfterCutoff.Inc(1) + metrics.writeAfterCutoff.Inc(1) return false } if w.cutOverNanos > 0 && nowNanos < w.cutOverNanos { - w.m.writeBeforeCutover.Inc(1) + metrics.writeBeforeCutover.Inc(1) return false } @@ -318,12 +322,10 @@ func (w *messageWriter) write( // Ack acknowledges the metadata. func (w *messageWriter) Ack(meta metadata) bool { - acked, expectedProcessNanos := w.acks.ack(meta) - if acked { - w.RLock() - defer w.RUnlock() - w.m.messageConsumeLatency.Record(time.Duration(w.nowFn().UnixNano() - expectedProcessNanos)) - w.m.messageAcked.Inc(1) + if acked, expectedProcessNanos := w.acks.ack(meta); acked { + m := w.Metrics() + m.messageConsumeLatency.Record(time.Duration(w.nowFn().UnixNano() - expectedProcessNanos)) + m.messageAcked.Inc(1) return true } return false @@ -367,8 +369,8 @@ func (w *messageWriter) scanMessageQueue() { e := w.queue.Front() w.lastNewWrite = nil isClosed := w.isClosed - m := w.m w.RUnlock() + var ( nowFn = w.nowFn msgsToWrite []*message @@ -378,6 +380,7 @@ func (w *messageWriter) scanMessageQueue() { consumerWriters []consumerWriter iterationIndexes []int fullScan = isClosed || beforeScan.After(w.nextFullScan) + m = w.Metrics() scanMetrics scanBatchMetrics skipWrites bool ) @@ -452,6 +455,7 @@ func (w *messageWriter) scanBatchWithLock( iterated int next *list.Element ) + metrics := w.Metrics() w.msgsToWrite = w.msgsToWrite[:0] for e := start; e != nil; e = next { iterated++ @@ -469,7 +473,7 @@ func (w *messageWriter) scanBatchWithLock( // do not stay in memory forever. // NB: The message must be added to the ack map to be acked here. w.acks.ack(m.Metadata()) - w.removeFromQueueWithLock(e, m) + w.removeFromQueueWithLock(e, m, metrics) scanMetrics[_messageClosed]++ continue } @@ -491,12 +495,12 @@ func (w *messageWriter) scanBatchWithLock( if acked, _ := w.acks.ack(m.Metadata()); acked { scanMetrics[_messageDroppedTTLExpire]++ } - w.removeFromQueueWithLock(e, m) + w.removeFromQueueWithLock(e, m, metrics) continue } if m.IsAcked() { scanMetrics[_processedAck]++ - w.removeFromQueueWithLock(e, m) + w.removeFromQueueWithLock(e, m, metrics) continue } if m.IsDroppedOrConsumed() { @@ -509,7 +513,7 @@ func (w *messageWriter) scanBatchWithLock( continue } w.acks.remove(m.Metadata()) - w.removeFromQueueWithLock(e, m) + w.removeFromQueueWithLock(e, m, metrics) scanMetrics[_messageDroppedBufferFull]++ continue } @@ -644,18 +648,14 @@ func (w *messageWriter) RemoveConsumerWriter(addr string) { // Metrics returns the metrics. These are dynamic and change if downstream consumer instance changes. func (w *messageWriter) Metrics() *messageWriterMetrics { - w.RLock() - defer w.RUnlock() - return w.m + return (*messageWriterMetrics)(w.metrics.Load()) } // SetMetrics sets the metrics // // This allows changing the labels of the metrics when the downstream consumer instance changes. func (w *messageWriter) SetMetrics(m *messageWriterMetrics) { - w.Lock() - w.m = m - w.Unlock() + w.metrics.Store(stdunsafe.Pointer(m)) } // QueueSize returns the number of messages queued in the writer. @@ -667,9 +667,9 @@ func (w *messageWriter) newMessage() *message { return w.mPool.Get() } -func (w *messageWriter) removeFromQueueWithLock(e *list.Element, m *message) { +func (w *messageWriter) removeFromQueueWithLock(e *list.Element, m *message, metrics *messageWriterMetrics) { w.queue.Remove(e) - w.m.dequeuedMessages.Inc(1) + metrics.dequeuedMessages.Inc(1) w.close(m) } @@ -679,51 +679,53 @@ func (w *messageWriter) close(m *message) { } type acks struct { - sync.Mutex - - ackMap map[metadataKey]*message + mtx sync.Mutex + acks map[uint64]*message } // nolint: unparam func newAckHelper(size int) *acks { return &acks{ - ackMap: make(map[metadataKey]*message, size), + acks: make(map[uint64]*message, size), } } func (a *acks) add(meta metadata, m *message) { - a.Lock() - a.ackMap[meta.metadataKey] = m - a.Unlock() + a.mtx.Lock() + a.acks[meta.metadataKey.id] = m + a.mtx.Unlock() } func (a *acks) remove(meta metadata) { - a.Lock() - delete(a.ackMap, meta.metadataKey) - a.Unlock() + a.mtx.Lock() + delete(a.acks, meta.metadataKey.id) + a.mtx.Unlock() } // ack processes the ack. returns true if the message was not already acked. additionally returns the expected // processing time for lag calculations. func (a *acks) ack(meta metadata) (bool, int64) { - a.Lock() - m, ok := a.ackMap[meta.metadataKey] + a.mtx.Lock() + m, ok := a.acks[meta.metadataKey.id] if !ok { - a.Unlock() + a.mtx.Unlock() // Acking a message that is already acked, which is ok. return false, 0 } - delete(a.ackMap, meta.metadataKey) - a.Unlock() + + delete(a.acks, meta.metadataKey.id) + a.mtx.Unlock() + expectedProcessAtNanos := m.ExpectedProcessAtNanos() m.Ack() + return true, expectedProcessAtNanos } func (a *acks) size() int { - a.Lock() - l := len(a.ackMap) - a.Unlock() + a.mtx.Lock() + l := len(a.acks) + a.mtx.Unlock() return l } diff --git a/src/msg/producer/writer/message_writer_benchmark_test.go b/src/msg/producer/writer/message_writer_benchmark_test.go index ffe8e3c729..73e9dc3b91 100644 --- a/src/msg/producer/writer/message_writer_benchmark_test.go +++ b/src/msg/producer/writer/message_writer_benchmark_test.go @@ -69,3 +69,12 @@ func (noopWriter) Address() string { return "" } func (noopWriter) Write(int, []byte) error { return nil } func (noopWriter) Init() {} func (noopWriter) Close() {} + +func BenchmarkAck(b *testing.B) { + ack := newAckHelper(64) + for i := 0; i < b.N; i++ { + meta := metadata{metadataKey: metadataKey{200, uint64(b.N)}} + ack.add(meta, nil) + ack.remove(meta) + } +} diff --git a/src/msg/producer/writer/message_writer_test.go b/src/msg/producer/writer/message_writer_test.go index 087bb761e6..9b38f59e04 100644 --- a/src/msg/producer/writer/message_writer_test.go +++ b/src/msg/producer/writer/message_writer_test.go @@ -212,8 +212,11 @@ func TestMessageWriterRetry(t *testing.T) { time.Sleep(100 * time.Millisecond) } - _, ok := w.acks.ackMap[metadataKey{shard: 200, id: 1}] + require.Equal(t, 1, w.acks.size()) + w.acks.mtx.Lock() + _, ok := w.acks.acks[uint64(1)] require.True(t, ok) + w.acks.mtx.Unlock() cw := newConsumerWriter(addr, a, opts, testConsumerWriterMetrics()) cw.Init() @@ -333,16 +336,17 @@ func TestMessageWriterCutoverCutoff(t *testing.T) { w := newMessageWriter(200, newMessagePool(), nil, testMessageWriterMetrics()) now := time.Now() w.nowFn = func() time.Time { return now } - require.True(t, w.isValidWriteWithLock(now.UnixNano())) - require.True(t, w.isValidWriteWithLock(now.UnixNano()+150)) - require.True(t, w.isValidWriteWithLock(now.UnixNano()+250)) - require.True(t, w.isValidWriteWithLock(now.UnixNano()+50)) + met := w.Metrics() + require.True(t, w.isValidWriteWithLock(now.UnixNano(), met)) + require.True(t, w.isValidWriteWithLock(now.UnixNano()+150, met)) + require.True(t, w.isValidWriteWithLock(now.UnixNano()+250, met)) + require.True(t, w.isValidWriteWithLock(now.UnixNano()+50, met)) w.SetCutoffNanos(now.UnixNano() + 200) w.SetCutoverNanos(now.UnixNano() + 100) - require.True(t, w.isValidWriteWithLock(now.UnixNano()+150)) - require.False(t, w.isValidWriteWithLock(now.UnixNano()+250)) - require.False(t, w.isValidWriteWithLock(now.UnixNano()+50)) + require.True(t, w.isValidWriteWithLock(now.UnixNano()+150, met)) + require.False(t, w.isValidWriteWithLock(now.UnixNano()+250, met)) + require.False(t, w.isValidWriteWithLock(now.UnixNano()+50, met)) require.Equal(t, 0, w.queue.Len()) mm := producer.NewMockMessage(ctrl) @@ -363,9 +367,10 @@ func TestMessageWriterIgnoreCutoverCutoff(t *testing.T) { w.SetCutoffNanos(now.UnixNano() + 200) w.SetCutoverNanos(now.UnixNano() + 100) - require.True(t, w.isValidWriteWithLock(now.UnixNano()+150)) - require.True(t, w.isValidWriteWithLock(now.UnixNano()+250)) - require.True(t, w.isValidWriteWithLock(now.UnixNano()+50)) + met := w.Metrics() + require.True(t, w.isValidWriteWithLock(now.UnixNano()+150, met)) + require.True(t, w.isValidWriteWithLock(now.UnixNano()+250, met)) + require.True(t, w.isValidWriteWithLock(now.UnixNano()+50, met)) require.Equal(t, 0, w.queue.Len()) mm := producer.NewMockMessage(ctrl) @@ -758,9 +763,7 @@ func TestMessageWriter_WithoutConsumerScope(t *testing.T) { } func isEmptyWithLock(h *acks) bool { - h.Lock() - defer h.Unlock() - return len(h.ackMap) == 0 + return h.size() == 0 } func testMessageWriterMetrics() *messageWriterMetrics { From ddcfc1f80fdddcdd1dc30b873919df461990f3b8 Mon Sep 17 00:00:00 2001 From: Vytenis Darulis Date: Mon, 6 Jun 2022 22:01:29 -0400 Subject: [PATCH 7/8] Upgrade Tally and Prometheus dependencies (#4118) --- go.mod | 50 +- go.sum | 607 ++++++++++++++---- .../tools/query_index_segments/main/main.go | 2 +- src/query/api/v1/handler/prom/mocks.go | 6 +- src/query/api/v1/handler/prom/read_test.go | 2 +- .../api/v1/handler/prometheus/remote/read.go | 2 +- src/query/parser/promql/matcher_test.go | 2 +- src/query/parser/promql/matchers.go | 2 +- src/query/parser/promql/options.go | 6 +- .../storage/prometheus/prometheus_storage.go | 6 +- .../prometheus/prometheus_storage_test.go | 3 +- src/query/test/compatibility/test.go | 2 +- 12 files changed, 540 insertions(+), 150 deletions(-) diff --git a/go.mod b/go.mod index 6bb85e152e..b551cb7ba9 100644 --- a/go.mod +++ b/go.mod @@ -21,7 +21,7 @@ require ( github.com/google/uuid v1.3.0 github.com/gorilla/mux v1.8.0 github.com/hydrogen18/stalecucumber v0.0.0-20151102144322-9b38526d4bdf - github.com/influxdata/influxdb v1.9.2 + github.com/influxdata/influxdb v1.9.5 github.com/jhump/protoreflect v1.6.1 github.com/jonboulle/clockwork v0.2.2 github.com/json-iterator/go v1.1.12 @@ -30,9 +30,9 @@ require ( github.com/lightstep/lightstep-tracer-go v0.18.1 github.com/m3db/bitset v2.0.0+incompatible github.com/m3db/bloom/v4 v4.0.0-20200901140942-52efb8544fe9 - github.com/m3db/prometheus_client_golang v0.8.1 - github.com/m3db/prometheus_client_model v0.0.0-20180517145114-8b2299a4bf7d - github.com/m3db/prometheus_common v0.0.0-20180517030744-25aaa3dff79b + github.com/m3db/prometheus_client_golang v1.12.8 + github.com/m3db/prometheus_client_model v0.2.1 + github.com/m3db/prometheus_common v0.34.7 github.com/m3db/prometheus_procfs v0.8.1 github.com/m3db/stackadler32 v0.0.0-20180104200216-bfebcd73ef6f github.com/m3db/stackmurmur3/v2 v2.0.2 @@ -47,19 +47,19 @@ require ( github.com/pkg/errors v0.9.1 github.com/pkg/profile v1.2.1 github.com/prometheus/client_golang v1.12.1 - github.com/prometheus/common v0.32.1 - github.com/prometheus/prometheus v1.8.2-0.20210621150501-ff58416a0b02 + github.com/prometheus/common v0.34.0 + github.com/prometheus/prometheus v0.0.0-20211110084043-4ef8c7c1d8e4 github.com/rakyll/statik v0.1.6 github.com/sergi/go-diff v1.1.0 github.com/spf13/cobra v1.3.0 github.com/stretchr/testify v1.7.0 github.com/twotwotwo/sorts v0.0.0-20160814051341-bf5c1f2b8553 - github.com/uber-go/tally v3.4.3+incompatible + github.com/uber-go/tally v3.5.0+incompatible github.com/uber/jaeger-client-go v2.29.1+incompatible github.com/uber/jaeger-lib v2.4.1+incompatible github.com/uber/tchannel-go v1.20.1 github.com/valyala/tcplisten v0.0.0-20161114210144-ceec8f93295a - github.com/willf/bitset v1.1.10 + github.com/willf/bitset v1.1.11 // etcd is currently on an alpha version to accomodate a GRPC version upgrade. See // https://github.com/m3db/m3/issues/4090 for the followup task to move back to a stable version. // Gory details (why we're doing this): @@ -82,7 +82,7 @@ require ( go.uber.org/config v1.4.0 go.uber.org/goleak v1.1.12 go.uber.org/zap v1.21.0 - golang.org/x/net v0.0.0-20220105145211-5b0dc2dfae98 + golang.org/x/net v0.0.0-20220225172249-27dd8689420f golang.org/x/sync v0.0.0-20210220032951-036812b2e83c golang.org/x/sys v0.0.0-20220422013727-9388b58f7150 google.golang.org/grpc v1.44.0 @@ -94,30 +94,30 @@ require ( ) require ( - github.com/Azure/go-ansiterm v0.0.0-20170929234023-d6e3b3328b78 // indirect + github.com/Azure/go-ansiterm v0.0.0-20210617225240-d185dfc1b5a1 // indirect github.com/CAFxX/gcnotifier v0.0.0-20190112062741-224a280d589d // indirect github.com/DataDog/datadog-go v3.7.1+incompatible // indirect - github.com/Microsoft/go-winio v0.4.16 // indirect + github.com/Microsoft/go-winio v0.4.17 // indirect github.com/Nvveen/Gotty v0.0.0-20120604004816-cd527374f1e5 // indirect github.com/StackExchange/wmi v0.0.0-20210224194228-fe8f1750fd46 // indirect - github.com/alecthomas/units v0.0.0-20210208195552-ff826a37aa15 // indirect - github.com/aws/aws-sdk-go v1.38.68 // indirect + github.com/alecthomas/units v0.0.0-20210927113745-59d0afb8317a // indirect + github.com/aws/aws-sdk-go v1.41.7 // indirect github.com/benbjohnson/clock v1.3.0 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/bmizerany/perks v0.0.0-20141205001514-d9a9656a3a4b // indirect github.com/cenkalti/backoff/v4 v4.1.2 // indirect - github.com/containerd/continuity v0.0.0-20200413184840-d3ef23f19fbb // indirect + github.com/containerd/continuity v0.1.0 // indirect github.com/coreos/go-semver v0.3.0 // indirect github.com/coreos/go-systemd/v22 v22.3.2 // indirect + github.com/dennwc/varint v1.0.0 // indirect github.com/docker/go-connections v0.4.0 // indirect github.com/docker/go-units v0.4.0 // indirect github.com/dustin/go-humanize v1.0.0 // indirect github.com/edsrzf/mmap-go v1.0.0 // indirect github.com/felixge/httpsnoop v1.0.2 // indirect - github.com/form3tech-oss/jwt-go v3.2.3+incompatible // indirect github.com/glycerine/go-unsnap-stream v0.0.0-20181221182339-f9677308dec2 // indirect - github.com/go-kit/log v0.1.0 // indirect - github.com/go-logfmt/logfmt v0.5.0 // indirect + github.com/go-kit/log v0.2.0 // indirect + github.com/go-logfmt/logfmt v0.5.1 // indirect github.com/go-logr/logr v1.2.2 // indirect github.com/go-logr/stdr v1.2.2 // indirect github.com/go-playground/locales v0.13.0 // indirect @@ -138,11 +138,11 @@ require ( github.com/leodido/go-urn v1.2.1 // indirect github.com/lib/pq v1.9.0 // indirect github.com/lightstep/lightstep-tracer-common/golang/gogo v0.0.0-20190605223551-bc2310a04743 // indirect - github.com/matttproud/golang_protobuf_extensions v1.0.1 // indirect + github.com/matttproud/golang_protobuf_extensions v1.0.2-0.20181231171920-c182affec369 // indirect github.com/mitchellh/copystructure v1.2.0 // indirect github.com/mitchellh/mapstructure v1.4.3 // indirect github.com/mitchellh/reflectwalk v1.0.2 // indirect - github.com/moby/term v0.0.0-20201216013528-df9cb8a40635 // indirect + github.com/moby/term v0.0.0-20210619224110-3f7ff695adc6 // indirect github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect github.com/modern-go/reflect2 v1.0.2 // indirect github.com/mostynb/go-grpc-compression v1.1.16 // indirect @@ -153,11 +153,12 @@ require ( github.com/onsi/gomega v1.10.4 // indirect github.com/opencontainers/go-digest v1.0.0 // indirect github.com/opencontainers/image-spec v1.0.1 // indirect - github.com/opencontainers/runc v1.0.0-rc9 // indirect + github.com/opencontainers/runc v1.0.2 // indirect github.com/philhofer/fwd v1.0.0 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/prashantv/protectmem v0.0.0-20171002184600-e20412882b3a // indirect github.com/prometheus/client_model v0.2.0 // indirect + github.com/prometheus/common/sigv4 v0.1.0 // indirect github.com/prometheus/procfs v0.7.3 // indirect github.com/rs/cors v1.8.2 // indirect github.com/shirou/gopsutil v3.21.6+incompatible // indirect @@ -187,15 +188,14 @@ require ( go.uber.org/multierr v1.7.0 // indirect golang.org/x/crypto v0.0.0-20210817164053-32db794688a5 // indirect golang.org/x/lint v0.0.0-20210508222113-6edffad5e616 // indirect - golang.org/x/oauth2 v0.0.0-20211104180415-d3ed0bb246c8 // indirect + golang.org/x/oauth2 v0.0.0-20220223155221-ee480838109b // indirect golang.org/x/text v0.3.7 // indirect - golang.org/x/time v0.0.0-20210611083556-38a9dc6acbc6 // indirect - golang.org/x/tools v0.1.5 // indirect + golang.org/x/time v0.0.0-20210723032227-1f47c861a9ac // indirect + golang.org/x/tools v0.1.7 // indirect golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1 // indirect google.golang.org/appengine v1.6.7 // indirect google.golang.org/genproto v0.0.0-20211208223120-3a66f561d7aa // indirect gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c // indirect - gopkg.in/go-playground/assert.v1 v1.2.1 // indirect gopkg.in/natefinch/lumberjack.v2 v2.0.0 // indirect sigs.k8s.io/yaml v1.2.0 // indirect ) @@ -220,8 +220,6 @@ replace github.com/google/flatbuffers => github.com/google/flatbuffers v1.12.1 // Fix legacy import path - https://github.com/uber-go/atomic/pull/60 replace github.com/uber-go/atomic => github.com/uber-go/atomic v1.4.0 -replace github.com/envoyproxy/go-control-plane => github.com/envoyproxy/go-control-plane v0.9.7 - replace google.golang.org/grpc => google.golang.org/grpc v1.40.1 // TODO: this can be removed once M3 is on Go 1.17.x diff --git a/go.sum b/go.sum index ff650e59d5..b962300c4b 100644 --- a/go.sum +++ b/go.sum @@ -54,31 +54,34 @@ cloud.google.com/go/storage v1.10.0/go.mod h1:FLPqc6j+Ki4BU591ie1oL6qBQGu2Bl/tZ9 code.cloudfoundry.org/bytefmt v0.0.0-20190710193110-1eb035ffe2b6/go.mod h1:wN/zk7mhREp/oviagqUXY3EwuHhWyOvAdsn5Y4CzOrc= collectd.org v0.3.0/go.mod h1:A/8DzQBkF6abtvrT2j/AU/4tiBgJWYyh0y/oB/4MlWE= dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= +github.com/Azure/azure-sdk-for-go v16.2.1+incompatible/go.mod h1:9XXNKU+eRnpl9moKnB4QOLf1HestfXbmab5FXxiDBjc= github.com/Azure/azure-sdk-for-go v41.3.0+incompatible/go.mod h1:9XXNKU+eRnpl9moKnB4QOLf1HestfXbmab5FXxiDBjc= -github.com/Azure/azure-sdk-for-go v55.2.0+incompatible h1:TL2/vJWJEPOrmv97nHcbvjXES0Ntlb9P95hqGA1J2dU= -github.com/Azure/azure-sdk-for-go v55.2.0+incompatible/go.mod h1:9XXNKU+eRnpl9moKnB4QOLf1HestfXbmab5FXxiDBjc= -github.com/Azure/go-ansiterm v0.0.0-20170929234023-d6e3b3328b78 h1:w+iIsaOQNcT7OZ575w+acHgRric5iCyQh+xv+KJ4HB8= +github.com/Azure/azure-sdk-for-go v58.2.0+incompatible h1:iCb2tuoEm3N7ZpUDOvu1Yxl1B3iOVDmaD6weaRuIPzs= +github.com/Azure/azure-sdk-for-go v58.2.0+incompatible/go.mod h1:9XXNKU+eRnpl9moKnB4QOLf1HestfXbmab5FXxiDBjc= github.com/Azure/go-ansiterm v0.0.0-20170929234023-d6e3b3328b78/go.mod h1:LmzpDX56iTiv29bbRTIsUNlaFfuhWRQBWjQdVyAevI8= +github.com/Azure/go-ansiterm v0.0.0-20210617225240-d185dfc1b5a1 h1:UQHMgLO+TxOElx5B5HZ4hJQsoJ/PvUvKRhJHDQXO8P8= +github.com/Azure/go-ansiterm v0.0.0-20210617225240-d185dfc1b5a1/go.mod h1:xomTg63KZ2rFqZQzSB4Vz2SUXa1BpHTVz9L5PTmPC4E= +github.com/Azure/go-autorest v10.8.1+incompatible/go.mod h1:r+4oMnoxhatjLLJ6zxSWATqVooLgysK6ZNox3g/xq24= github.com/Azure/go-autorest v14.2.0+incompatible h1:V5VMDjClD3GiElqLWO7mz2MxNAK/vTfRHdAubSIPRgs= github.com/Azure/go-autorest v14.2.0+incompatible/go.mod h1:r+4oMnoxhatjLLJ6zxSWATqVooLgysK6ZNox3g/xq24= github.com/Azure/go-autorest/autorest v0.9.0/go.mod h1:xyHB1BMZT0cuDHU7I0+g046+BFDTQ8rEZB0s4Yfa6bI= -github.com/Azure/go-autorest/autorest v0.9.3/go.mod h1:GsRuLYvwzLjjjRoWEIyMUaYq8GNUx2nRB378IPt/1p0= github.com/Azure/go-autorest/autorest v0.10.0/go.mod h1:/FALq9T/kS7b5J5qsQ+RSTUdAmGFqi0vUdVNNx8q630= -github.com/Azure/go-autorest/autorest v0.10.1/go.mod h1:/FALq9T/kS7b5J5qsQ+RSTUdAmGFqi0vUdVNNx8q630= -github.com/Azure/go-autorest/autorest v0.11.12/go.mod h1:eipySxLmqSyC5s5k1CLupqet0PSENBEDP93LQ9a8QYw= -github.com/Azure/go-autorest/autorest v0.11.19 h1:7/IqD2fEYVha1EPeaiytVKhzmPV223pfkRIQUGOK2IE= -github.com/Azure/go-autorest/autorest v0.11.19/go.mod h1:dSiJPy22c3u0OtOKDNttNgqpNFY/GeWa7GH/Pz56QRA= +github.com/Azure/go-autorest/autorest v0.11.1/go.mod h1:JFgpikqFJ/MleTTxwepExTKnFUKKszPS8UavbQYUMuw= +github.com/Azure/go-autorest/autorest v0.11.9/go.mod h1:eipySxLmqSyC5s5k1CLupqet0PSENBEDP93LQ9a8QYw= +github.com/Azure/go-autorest/autorest v0.11.18/go.mod h1:dSiJPy22c3u0OtOKDNttNgqpNFY/GeWa7GH/Pz56QRA= +github.com/Azure/go-autorest/autorest v0.11.21 h1:w77zY/9RnUAWcIQyDC0Fc89mCvwftR8F+zsR/OH6enk= +github.com/Azure/go-autorest/autorest v0.11.21/go.mod h1:Do/yuMSW/13ayUkcVREpsMHGG+MvV81uzSCFgYPj4tM= github.com/Azure/go-autorest/autorest/adal v0.5.0/go.mod h1:8Z9fGy2MpX0PvDjB1pEgQTmVqjGhiHBW7RJJEciWzS0= -github.com/Azure/go-autorest/autorest/adal v0.8.0/go.mod h1:Z6vX6WXXuyieHAXwMj0S6HY6e6wcHn37qQMBQlvY3lc= -github.com/Azure/go-autorest/autorest/adal v0.8.1/go.mod h1:ZjhuQClTqx435SRJ2iMlOxPYt3d2C/T/7TiQCVZSn3Q= github.com/Azure/go-autorest/autorest/adal v0.8.2/go.mod h1:ZjhuQClTqx435SRJ2iMlOxPYt3d2C/T/7TiQCVZSn3Q= github.com/Azure/go-autorest/autorest/adal v0.8.3/go.mod h1:ZjhuQClTqx435SRJ2iMlOxPYt3d2C/T/7TiQCVZSn3Q= +github.com/Azure/go-autorest/autorest/adal v0.9.0/go.mod h1:/c022QCutn2P7uY+/oQWWNcK9YU+MH96NgK+jErpbcg= github.com/Azure/go-autorest/autorest/adal v0.9.5/go.mod h1:B7KF7jKIeC9Mct5spmyCB/A8CG/sEz1vwIRGv/bbw7A= github.com/Azure/go-autorest/autorest/adal v0.9.13/go.mod h1:W/MM4U6nLxnIskrw4UwWzlHfGjwUS50aOsc/I3yuU8M= -github.com/Azure/go-autorest/autorest/adal v0.9.14 h1:G8hexQdV5D4khOXrWG2YuLCFKhWYmWD8bHYaXN5ophk= github.com/Azure/go-autorest/autorest/adal v0.9.14/go.mod h1:W/MM4U6nLxnIskrw4UwWzlHfGjwUS50aOsc/I3yuU8M= -github.com/Azure/go-autorest/autorest/azure/auth v0.4.2/go.mod h1:90gmfKdlmKgfjUpnCEpOJzsUEjrWDSLwHIG73tSXddM= -github.com/Azure/go-autorest/autorest/azure/cli v0.3.1/go.mod h1:ZG5p860J94/0kI9mNJVoIoLgXcirM2gF5i2kWloofxw= +github.com/Azure/go-autorest/autorest/adal v0.9.16 h1:P8An8Z9rH1ldbOLdFpxYorgOt2sywL9V24dAwWHPuGc= +github.com/Azure/go-autorest/autorest/adal v0.9.16/go.mod h1:tGMin8I49Yij6AQ+rvV+Xa/zwxYQB5hmsd6DkfAx2+A= +github.com/Azure/go-autorest/autorest/azure/auth v0.5.3/go.mod h1:4bJZhUhcq8LB20TruwHbAQsmUs2Xh+QR7utuJpLXX3A= +github.com/Azure/go-autorest/autorest/azure/cli v0.4.2/go.mod h1:7qkJkT+j6b+hIpzMOwPChJhTqS8VbsqqgULzMNRugoM= github.com/Azure/go-autorest/autorest/date v0.1.0/go.mod h1:plvfp3oPSKwf2DNjlBjWF/7vwR+cUD/ELuzDCXwHUVA= github.com/Azure/go-autorest/autorest/date v0.2.0/go.mod h1:vcORJHLJEh643/Ioh9+vPmf1Ij9AEBM5FuBIXLmIy0g= github.com/Azure/go-autorest/autorest/date v0.3.0 h1:7gUk1U5M/CQbp9WoqinNzJar+8KY+LPI6wiWrP/myHw= @@ -86,6 +89,7 @@ github.com/Azure/go-autorest/autorest/date v0.3.0/go.mod h1:BI0uouVdmngYNUzGWeSY github.com/Azure/go-autorest/autorest/mocks v0.1.0/go.mod h1:OTyCOPRA2IgIlWxVYxBee2F5Gr4kF2zd2J5cFRaIDN0= github.com/Azure/go-autorest/autorest/mocks v0.2.0/go.mod h1:OTyCOPRA2IgIlWxVYxBee2F5Gr4kF2zd2J5cFRaIDN0= github.com/Azure/go-autorest/autorest/mocks v0.3.0/go.mod h1:a8FDP3DYzQ4RYfVAxAN3SVSiiO77gL2j2ronKKP0syM= +github.com/Azure/go-autorest/autorest/mocks v0.4.0/go.mod h1:LTp+uSrOhSkaKrUy935gNZuuIPPVsHlr9DSOxSayd+k= github.com/Azure/go-autorest/autorest/mocks v0.4.1/go.mod h1:LTp+uSrOhSkaKrUy935gNZuuIPPVsHlr9DSOxSayd+k= github.com/Azure/go-autorest/autorest/to v0.3.0/go.mod h1:MgwOyqaIuKdG4TL/2ywSsIWKAfJfgHDo8ObuUk3t5sA= github.com/Azure/go-autorest/autorest/to v0.4.0 h1:oXVqrxakqqV1UZdSazDOPOLvOIz+XA683u8EctwboHk= @@ -111,16 +115,32 @@ github.com/DATA-DOG/go-sqlmock v1.4.1/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q github.com/DataDog/datadog-go v3.2.0+incompatible/go.mod h1:LButxg5PwREeZtORoXG3tL4fMGNddJ+vMq1mwgfaqoQ= github.com/DataDog/datadog-go v3.7.1+incompatible h1:HmA9qHVrHIAqpSvoCYJ+c6qst0lgqEhNW6/KwfkHbS8= github.com/DataDog/datadog-go v3.7.1+incompatible/go.mod h1:LButxg5PwREeZtORoXG3tL4fMGNddJ+vMq1mwgfaqoQ= -github.com/HdrHistogram/hdrhistogram-go v1.0.1 h1:GX8GAYDuhlFQnI2fRDHQhTlkHMz8bEn0jTI6LJU0mpw= -github.com/HdrHistogram/hdrhistogram-go v1.0.1/go.mod h1:BWJ+nMSHY3L41Zj7CA3uXnloDp7xxV0YvstAE7nKTaM= +github.com/HdrHistogram/hdrhistogram-go v1.1.0 h1:6dpdDPTRoo78HxAJ6T1HfMiKSnqhgRRqzCuPshRkQ7I= +github.com/HdrHistogram/hdrhistogram-go v1.1.0/go.mod h1:yDgFjdqOqDEKOvasDdhWNXYg9BVp4O+o5f6V/ehm6Oo= github.com/Knetic/govaluate v3.0.1-0.20171022003610-9aa49832a739+incompatible/go.mod h1:r7JcOSlj0wfOMncg0iLm8Leh48TZaKVeNIfJntJ2wa0= github.com/Masterminds/semver v1.4.2/go.mod h1:MB6lktGJrhw8PrUyiEoblNEGEQ+RzHPF078ddwwvV3Y= github.com/Masterminds/sprig v2.16.0+incompatible/go.mod h1:y6hNFY5UBTIWBxnzTeuNhlNS5hqE0NB0E6fgfo2Br3o= github.com/MichaelTJones/pcg v0.0.0-20180122055547-df440c6ed7ed h1:hQC4FSwvsLH6rOLJTndsHnANARF9RwW4PbrDTjks/0A= github.com/MichaelTJones/pcg v0.0.0-20180122055547-df440c6ed7ed/go.mod h1:NQ4UMHqyfXyYVmZopcfwPRWJa0rw2aH16eDIltReVUo= +github.com/Microsoft/go-winio v0.4.11/go.mod h1:VhR8bwka0BXejwEJY73c50VrPtXAaKcyvVC4A4RozmA= github.com/Microsoft/go-winio v0.4.14/go.mod h1:qXqCSQ3Xa7+6tgxaGTIe4Kpcdsi+P8jBhyzoq1bpyYA= -github.com/Microsoft/go-winio v0.4.16 h1:FtSW/jqD+l4ba5iPBj9CODVtgfYAD8w2wS923g/cFDk= +github.com/Microsoft/go-winio v0.4.15-0.20190919025122-fc70bd9a86b5/go.mod h1:tTuCMEN+UleMWgg9dVx4Hu52b1bJo+59jBh3ajtinzw= +github.com/Microsoft/go-winio v0.4.16-0.20201130162521-d1ffc52c7331/go.mod h1:XB6nPKklQyQ7GC9LdcBEcBl8PF76WugXOPRXwdLnMv0= github.com/Microsoft/go-winio v0.4.16/go.mod h1:XB6nPKklQyQ7GC9LdcBEcBl8PF76WugXOPRXwdLnMv0= +github.com/Microsoft/go-winio v0.4.17-0.20210211115548-6eac466e5fa3/go.mod h1:JPGBdM1cNvN/6ISo+n8V5iA4v8pBzdOpzfwIujj1a84= +github.com/Microsoft/go-winio v0.4.17-0.20210324224401-5516f17a5958/go.mod h1:JPGBdM1cNvN/6ISo+n8V5iA4v8pBzdOpzfwIujj1a84= +github.com/Microsoft/go-winio v0.4.17 h1:iT12IBVClFevaf8PuVyi3UmZOVh4OqnaLxDTW2O6j3w= +github.com/Microsoft/go-winio v0.4.17/go.mod h1:JPGBdM1cNvN/6ISo+n8V5iA4v8pBzdOpzfwIujj1a84= +github.com/Microsoft/hcsshim v0.8.6/go.mod h1:Op3hHsoHPAvb6lceZHDtd9OkTew38wNoXnJs8iY7rUg= +github.com/Microsoft/hcsshim v0.8.7-0.20190325164909-8abdbb8205e4/go.mod h1:Op3hHsoHPAvb6lceZHDtd9OkTew38wNoXnJs8iY7rUg= +github.com/Microsoft/hcsshim v0.8.7/go.mod h1:OHd7sQqRFrYd3RmSgbgji+ctCwkbq2wbEYNSzOYtcBQ= +github.com/Microsoft/hcsshim v0.8.9/go.mod h1:5692vkUqntj1idxauYlpoINNKeqCiG6Sg38RRsjT5y8= +github.com/Microsoft/hcsshim v0.8.14/go.mod h1:NtVKoYxQuTLx6gEq0L96c9Ju4JbRJ4nY2ow3VK6a9Lg= +github.com/Microsoft/hcsshim v0.8.15/go.mod h1:x38A4YbHbdxJtc0sF6oIz+RG0npwSCAvn69iY6URG00= +github.com/Microsoft/hcsshim v0.8.16/go.mod h1:o5/SZqmR7x9JNKsW3pu+nqHm0MF8vbA+VxGOoXdC600= +github.com/Microsoft/hcsshim v0.8.21/go.mod h1:+w2gRZ5ReXQhFOrvSQeNfhrYB/dg3oDwTOcER2fw4I4= +github.com/Microsoft/hcsshim/test v0.0.0-20201218223536-d3e5debf77da/go.mod h1:5hlzMzRKMLyo42nCZ9oml8AdTlq/0cvIaBv6tK1RehU= +github.com/Microsoft/hcsshim/test v0.0.0-20210227013316-43a75bb4edd3/go.mod h1:mw7qgWloBUl75W/gVH3cQszUg1+gUITj7D6NY7ywVnY= github.com/NYTimes/gziphandler v0.0.0-20170623195520-56545f4a5d46/go.mod h1:3wb06e3pkSAbeQ52E9H9iFoQsEEwGN64994WTCIhntQ= github.com/Nvveen/Gotty v0.0.0-20120604004816-cd527374f1e5 h1:TngWCqHvy9oXAN6lEVMRuU21PR1EtLVZJmdB18Gu3Rw= github.com/Nvveen/Gotty v0.0.0-20120604004816-cd527374f1e5/go.mod h1:lmUJ/7eu/Q8D7ML55dXQrVaamCz2vxCfdQBasLZfHKk= @@ -134,6 +154,7 @@ github.com/PuerkitoBio/urlesc v0.0.0-20170810143723-de5bf2ad4578/go.mod h1:uGdko github.com/RoaringBitmap/roaring v0.4.21 h1:WJ/zIlNX4wQZ9x8Ey33O1UaD9TCTakYsdLFSBcTwH+8= github.com/RoaringBitmap/roaring v0.4.21/go.mod h1:D0gp8kJQgE1A4LQ5wFLggQEyvDi06Mq5mKs52e1TwOo= github.com/SAP/go-hdb v0.14.1/go.mod h1:7fdQLVC2lER3urZLjZCm0AuMQfApof92n3aylBPEkMo= +github.com/Shopify/logrus-bugsnag v0.0.0-20171204204709-577dee27f20d/go.mod h1:HI8ITrYtUY+O+ZhtlqUnD8+KwNPOyugEhfP9fdUIaEQ= github.com/Shopify/sarama v1.19.0/go.mod h1:FVkBWblsNy7DGZRfXLU0O9RCGt5g3g3yEuWXgklEdEo= github.com/Shopify/toxiproxy v2.1.4+incompatible/go.mod h1:OXgGpZ6Cli1/URJOF1DMxUHB2q5Ap20/P/eIdh4G0pI= github.com/StackExchange/wmi v0.0.0-20180116203802-5d049714c4a6/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg= @@ -148,13 +169,16 @@ github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuy 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/alecthomas/units v0.0.0-20210208195552-ff826a37aa15 h1:AUNCr9CiJuwrRYS3XieqF+Z9B9gNxo/eANAJCF2eiN4= github.com/alecthomas/units v0.0.0-20210208195552-ff826a37aa15/go.mod h1:OMCwj8VM1Kc9e19TLln2VL61YJF0x1XFtfdL4JdbSyE= +github.com/alecthomas/units v0.0.0-20210927113745-59d0afb8317a h1:E/8AP5dFtMhl5KPJz66Kt9G0n+7Sn41Fy1wv9/jHOrc= +github.com/alecthomas/units v0.0.0-20210927113745-59d0afb8317a/go.mod h1:OMCwj8VM1Kc9e19TLln2VL61YJF0x1XFtfdL4JdbSyE= +github.com/alexflint/go-filemutex v0.0.0-20171022225611-72bdc8eae2ae/go.mod h1:CgnQgUtFrFz9mxFNtED3jI5tLDjKlOM+oUF/sTk6ps0= github.com/andreyvit/diff v0.0.0-20170406064948-c7f18ee00883/go.mod h1:rCTlJbsFo29Kk6CurOXKm700vrz8f0KW0JNfpkRJY/8= github.com/antihax/optional v0.0.0-20180407024304-ca021399b1a6/go.mod h1:V8iCPQYkqmusNa815XgQio277wI47sdRh1dUOLdyC6Q= github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= github.com/aokoli/goutils v1.0.1/go.mod h1:SijmP0QR8LtwsmDs8Yii5Z/S4trXFGFC2oO5g9DP+DQ= github.com/apache/arrow/go/arrow v0.0.0-20191024131854-af6fa24be0db/go.mod h1:VTxUBvSJ3s3eHAg65PNgrsn5BtqCRPdmyXh6rAfdxN0= +github.com/apache/arrow/go/arrow v0.0.0-20200601151325-b2287a20f230/go.mod h1:QNYViu/X0HXDHw7m3KXzWSVXIbfUvJqBFe6Gj8/pYA0= github.com/apache/arrow/go/arrow v0.0.0-20200923215132-ac86123a3f01/go.mod h1:QNYViu/X0HXDHw7m3KXzWSVXIbfUvJqBFe6Gj8/pYA0= github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e/go.mod h1:3U/XgcO3hCbHZ8TKRvWD2dDTCfh9M9ya+I9JpbB7O8o= github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8= @@ -172,13 +196,15 @@ github.com/asaskevich/govalidator v0.0.0-20200108200545-475eaeb16496/go.mod h1:o github.com/asaskevich/govalidator v0.0.0-20200428143746-21a406dcc535/go.mod h1:oGkLhpf+kjZl6xBf758TQhh5XrAeiJv/7FRz/2spLIg= github.com/asaskevich/govalidator v0.0.0-20200907205600-7a23bdc65eef/go.mod h1:WaHUgvxTVq04UNunO+XhnAqY/wQc+bxr74GqbsZ/Jqw= github.com/aws/aws-lambda-go v1.13.3/go.mod h1:4UKl9IzQMoD+QF79YdCuzCwp8VbmG4VAQwij/eHl5CU= +github.com/aws/aws-sdk-go v1.15.11/go.mod h1:mFuSZ37Z9YOHbQEwBWztmVzqXrEkub65tZoCYDt7FT0= github.com/aws/aws-sdk-go v1.27.0/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpiN924inxo= github.com/aws/aws-sdk-go v1.29.16/go.mod h1:1KvfttTE3SPKMpo8g2c6jL3ZKfXtFvKscTgahTma5Xg= github.com/aws/aws-sdk-go v1.30.12/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZveU8YkpAk0= github.com/aws/aws-sdk-go v1.34.28/go.mod h1:H7NKnBqNVzoTJpGfLrQkkD+ytBA93eiDYi/+8rV9s48= -github.com/aws/aws-sdk-go v1.38.60/go.mod h1:hcU610XS61/+aQV88ixoOzUoG7v3b31pl2zKMmprdro= -github.com/aws/aws-sdk-go v1.38.68 h1:aOG8geU4SohNp659eKBHRBgbqSrZ6jNZlfimIuJAwL8= -github.com/aws/aws-sdk-go v1.38.68/go.mod h1:hcU610XS61/+aQV88ixoOzUoG7v3b31pl2zKMmprdro= +github.com/aws/aws-sdk-go v1.38.35/go.mod h1:hcU610XS61/+aQV88ixoOzUoG7v3b31pl2zKMmprdro= +github.com/aws/aws-sdk-go v1.40.11/go.mod h1:585smgzpB/KqRA+K3y/NL/oYRqQvpNJYvLm+LY1U59Q= +github.com/aws/aws-sdk-go v1.41.7 h1:vlpR8Cky3ZxUVNINgeRZS6N0p6zmFvu/ZqRRwrTI25U= +github.com/aws/aws-sdk-go v1.41.7/go.mod h1:585smgzpB/KqRA+K3y/NL/oYRqQvpNJYvLm+LY1U59Q= github.com/aws/aws-sdk-go-v2 v0.18.0/go.mod h1:JWVYvqSMppoMJC0x5wdwiImzgXTI9FuZwxzkQq9wy+g= github.com/aws/aws-sdk-go-v2 v1.9.2/go.mod h1:cK/D0BBs0b/oWPIcX/Z/obahJK1TT7IPVjy53i/mX/4= github.com/aws/aws-sdk-go-v2/config v1.8.3/go.mod h1:4AEiLtAb8kLs7vgw2ZV3p2VZ1+hBavOc84hqxVNpCyw= @@ -195,17 +221,28 @@ github.com/benbjohnson/clock v1.3.0 h1:ip6w0uFQkncKQ979AypyG0ER7mqUSBdKLOgAle/AT github.com/benbjohnson/clock v1.3.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/benbjohnson/immutable v0.2.1/go.mod h1:uc6OHo6PN2++n98KHLxW8ef4W42ylHiQSENghE1ezxI= github.com/benbjohnson/tmpl v1.0.0/go.mod h1:igT620JFIi44B6awvU9IsDhR77IXWtFigTLil/RPdps= +github.com/beorn7/perks v0.0.0-20160804104726-4c0e84591b9a/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= 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 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs= +github.com/bitly/go-simplejson v0.5.0/go.mod h1:cXHtHw4XUPsvGaxgjIAn8PhEWG9NfngEKAMDJEczWVA= +github.com/bits-and-blooms/bitset v1.2.0/go.mod h1:gIdJ4wp64HaoK2YrL1Q5/N7Y16edYb8uY+O0FJTyyDA= github.com/bketelsen/crypt v0.0.3-0.20200106085610-5cbc8cc4026c/go.mod h1:MKsuJmJgSg28kpZDP6UIiPt0e0Oz0kqKNGyRaWEPv84= +github.com/blang/semver v3.1.0+incompatible/go.mod h1:kRBLl5iJ+tD4TcOOxsy/0fnwebNt5EWlYSAyrTnjyyk= +github.com/blang/semver v3.5.1+incompatible/go.mod h1:kRBLl5iJ+tD4TcOOxsy/0fnwebNt5EWlYSAyrTnjyyk= +github.com/bmizerany/assert v0.0.0-20160611221934-b7ed37b82869/go.mod h1:Ekp36dRnpXw/yCqJaO+ZrUyxD+3VXMFFr56k5XYrpB4= github.com/bmizerany/pat v0.0.0-20170815010413-6226ea591a40/go.mod h1:8rLXio+WjiTceGBHIoTvn60HIbs7Hm7bcHjyrSqYB9c= github.com/bmizerany/perks v0.0.0-20141205001514-d9a9656a3a4b h1:AP/Y7sqYicnjGDfD5VcY4CIfh1hRXBUavxrvELjTiOE= github.com/bmizerany/perks v0.0.0-20141205001514-d9a9656a3a4b/go.mod h1:ac9efd0D1fsDb3EJvhqgXRbFx7bs2wqZ10HQPeU8U/Q= github.com/boltdb/bolt v1.3.1/go.mod h1:clJnj/oiGkjum5o1McbSZDSLxVThjynRyGBgiAx27Ps= github.com/bonitoo-io/go-sql-bigquery v0.3.4-1.4.0/go.mod h1:J4Y6YJm0qTWB9aFziB7cPeSyc6dOZFyJdteSeybVpXQ= +github.com/bshuster-repo/logrus-logstash-hook v0.4.1/go.mod h1:zsTqEiSzDgAa/8GZR7E1qaXrhYNDKBYy5/dWPTIflbk= +github.com/buger/jsonparser v0.0.0-20180808090653-f4dd9f5a6b44/go.mod h1:bbYlZJ7hK1yFx9hf58LP0zeX7UjIGs20ufpu3evjr+s= +github.com/bugsnag/bugsnag-go v0.0.0-20141110184014-b1d153021fcd/go.mod h1:2oa8nejYd4cQ/b0hMIopN0lCRxU0bueqREvZLWFrtK8= +github.com/bugsnag/osext v0.0.0-20130617224835-0dd3f918b21b/go.mod h1:obH5gd0BsqsP2LwDJ9aOkm/6J86V6lyAXCoQWGw3K50= +github.com/bugsnag/panicwrap v0.0.0-20151223152923-e2c28503fcd0/go.mod h1:D/8v3kj0zr8ZAKg1AQ6crr+5VwKN5eIywRkfhyM/+dE= github.com/c-bata/go-prompt v0.2.2/go.mod h1:VzqtzE2ksDBcdln8G7mk2RX9QyGjH+OVqOCSiVIqS34= github.com/c2h5oh/datasize v0.0.0-20171227191756-4eba002a5eae h1:2Zmk+8cNvAGuY8AyvZuWpUdpQUAXwfom4ReVMe/CTIo= github.com/c2h5oh/datasize v0.0.0-20171227191756-4eba002a5eae/go.mod h1:S/7n9copUssQ56c7aAgHqftWO4LTf4xY6CGWt8Bc+3M= @@ -215,7 +252,7 @@ github.com/cenkalti/backoff v0.0.0-20181003080854-62661b46c409/go.mod h1:90ReRw6 github.com/cenkalti/backoff v2.2.1+incompatible/go.mod h1:90ReRw6GdpyfrHakVjL/QHaoyV4aDUVVkXQJJJ3NXXM= github.com/cenkalti/backoff/v3 v3.0.0 h1:ske+9nBpD9qZsTBoF41nW5L+AIuFBKMeze18XQ3eG1c= github.com/cenkalti/backoff/v3 v3.0.0/go.mod h1:cIeZDE3IrqwwJl6VUwCN6trj1oXrTS4rc0ij+ULvLYs= -github.com/cenkalti/backoff/v4 v4.1.0/go.mod h1:scbssz8iZGpm3xbr14ovlUdkxfGXNInqkPWOWmG2CLw= +github.com/cenkalti/backoff/v4 v4.1.1/go.mod h1:scbssz8iZGpm3xbr14ovlUdkxfGXNInqkPWOWmG2CLw= github.com/cenkalti/backoff/v4 v4.1.2 h1:6Yo7N8UP2K6LWZnW94DLVSSrbobcWdVzAYOisuDPIFo= github.com/cenkalti/backoff/v4 v4.1.2/go.mod h1:scbssz8iZGpm3xbr14ovlUdkxfGXNInqkPWOWmG2CLw= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= @@ -227,71 +264,184 @@ github.com/cespare/xxhash/v2 v2.1.0/go.mod h1:dgIUBU3pDso/gPgZ1osOZ0iQf77oPR28Tj github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= github.com/cespare/xxhash/v2 v2.1.2 h1:YRXhKfTDauu4ajMg1TPgFO5jnlC2HCbmLXMcTG5cbYE= github.com/cespare/xxhash/v2 v2.1.2/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/checkpoint-restore/go-criu/v4 v4.1.0/go.mod h1:xUQBLp4RLc5zJtWY++yjOoMoB5lihDt7fai+75m+rGw= +github.com/checkpoint-restore/go-criu/v5 v5.0.0/go.mod h1:cfwC0EG7HMUenopBsUf9d89JlCLQIfgVcNsNN0t6T2M= 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/cilium/ebpf v0.0.0-20200110133405-4032b1d8aae3/go.mod h1:MA5e5Lr8slmEg9bt0VpxxWqJlO4iwu3FBdHUzV7wQVg= +github.com/cilium/ebpf v0.0.0-20200702112145-1c8d4c9ef775/go.mod h1:7cR51M8ViRLIdUjrmSXlK9pkrsDlLHbO8jiB8X8JnOc= +github.com/cilium/ebpf v0.2.0/go.mod h1:To2CFviqOWL/M0gIMsvSMlqe7em/l1ALkX1PyjrX2Qs= +github.com/cilium/ebpf v0.4.0/go.mod h1:4tRaxcgiL706VnOzHOdBlY8IEAIdxINsQBcU4xJJXRs= +github.com/cilium/ebpf v0.6.2/go.mod h1:4tRaxcgiL706VnOzHOdBlY8IEAIdxINsQBcU4xJJXRs= github.com/circonus-labs/circonus-gometrics v2.3.1+incompatible/go.mod h1:nmEj6Dob7S7YxXgwXpfOuvO54S+tGdZdw9fuRZt25Ag= github.com/circonus-labs/circonusllhist v0.1.3/go.mod h1:kMXHVDlOchFAehlya5ePtbp5jckzBHf4XRpQvBOLI+I= github.com/clbanning/x2j v0.0.0-20191024224557-825249438eec/go.mod h1:jMjuTZXRI4dUb/I5gc9Hdhagfvm9+RyrPryS/auMzxE= -github.com/cncf/udpa/go v0.0.0-20200629203442-efcf912fb354/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk= github.com/cncf/udpa/go v0.0.0-20201120205902-5459f2c99403/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk= github.com/cncf/udpa/go v0.0.0-20210930031921-04548b0d99d4/go.mod h1:6pvJx4me5XPnfI9Z40ddWsdw2W/uZgQLFXToKeRcDiI= +github.com/cncf/xds/go v0.0.0-20210312221358-fbca930ec8ed/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/cncf/xds/go v0.0.0-20210922020428-25de7278fc84/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= +github.com/cncf/xds/go v0.0.0-20211001041855-01bcc9b48dfe/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= +github.com/cncf/xds/go v0.0.0-20211130200136-a8f946100490 h1:KwaoQzs/WeUxxJqiJsZ4euOly1Az/IgZXXSxlD/UBNk= github.com/cncf/xds/go v0.0.0-20211130200136-a8f946100490/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa/go.mod h1:zn76sxSg3SzpJ0PPJaLDCu+Bu0Lg3sKTORVIj19EIF8= github.com/cockroachdb/datadriven v0.0.0-20200714090401-bf6692d28da5 h1:xD/lrqdvwsc+O2bjSSi3YqY73Ke3LAiSCx49aCesA0E= github.com/cockroachdb/errors v1.2.4 h1:Lap807SXTH5tri2TivECb/4abUkMZC9zRoLarvcKDqs= github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f h1:o/kfcElHqOiXqcou5a3rIlMc7oJbMQkeLk0VQJ7zgqY= github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd/go.mod h1:sE/e/2PUdi/liOCUjSTXgM1o87ZssimdTWN964YiIeI= -github.com/containerd/containerd v1.4.3 h1:ijQT13JedHSHrQGWFcGEwzcNKrAGIiZ+jSD5QQG07SY= +github.com/containerd/aufs v0.0.0-20200908144142-dab0cbea06f4/go.mod h1:nukgQABAEopAHvB6j7cnP5zJ+/3aVcE7hCYqvIwAHyE= +github.com/containerd/aufs v0.0.0-20201003224125-76a6863f2989/go.mod h1:AkGGQs9NM2vtYHaUen+NljV0/baGCAPELGm2q9ZXpWU= +github.com/containerd/aufs v0.0.0-20210316121734-20793ff83c97/go.mod h1:kL5kd6KM5TzQjR79jljyi4olc1Vrx6XBlcyj3gNv2PU= +github.com/containerd/aufs v1.0.0/go.mod h1:kL5kd6KM5TzQjR79jljyi4olc1Vrx6XBlcyj3gNv2PU= +github.com/containerd/btrfs v0.0.0-20201111183144-404b9149801e/go.mod h1:jg2QkJcsabfHugurUvvPhS3E08Oxiuh5W/g1ybB4e0E= +github.com/containerd/btrfs v0.0.0-20210316141732-918d888fb676/go.mod h1:zMcX3qkXTAi9GI50+0HOeuV8LU2ryCE/V2vG/ZBiTss= +github.com/containerd/btrfs v1.0.0/go.mod h1:zMcX3qkXTAi9GI50+0HOeuV8LU2ryCE/V2vG/ZBiTss= +github.com/containerd/cgroups v0.0.0-20190717030353-c4b9ac5c7601/go.mod h1:X9rLEHIqSf/wfK8NsPqxJmeZgW4pcfzdXITDrUSJ6uI= +github.com/containerd/cgroups v0.0.0-20190919134610-bf292b21730f/go.mod h1:OApqhQ4XNSNC13gXIwDjhOQxjWa/NxkwZXJ1EvqT0ko= +github.com/containerd/cgroups v0.0.0-20200531161412-0dbf7f05ba59/go.mod h1:pA0z1pT8KYB3TCXK/ocprsh7MAkoW8bZVzPdih9snmM= +github.com/containerd/cgroups v0.0.0-20200710171044-318312a37340/go.mod h1:s5q4SojHctfxANBDvMeIaIovkq29IP48TKAxnhYRxvo= +github.com/containerd/cgroups v0.0.0-20200824123100-0b889c03f102/go.mod h1:s5q4SojHctfxANBDvMeIaIovkq29IP48TKAxnhYRxvo= +github.com/containerd/cgroups v0.0.0-20210114181951-8a68de567b68/go.mod h1:ZJeTFisyysqgcCdecO57Dj79RfL0LNeGiFUqLYQRYLE= +github.com/containerd/cgroups v1.0.1/go.mod h1:0SJrPIenamHDcZhEcJMNBB85rHcUsw4f25ZfBiPYRkU= +github.com/containerd/console v0.0.0-20180822173158-c12b1e7919c1/go.mod h1:Tj/on1eG8kiEhd0+fhSDzsPAFESxzBBvdyEgyryXffw= +github.com/containerd/console v0.0.0-20181022165439-0650fd9eeb50/go.mod h1:Tj/on1eG8kiEhd0+fhSDzsPAFESxzBBvdyEgyryXffw= +github.com/containerd/console v0.0.0-20191206165004-02ecf6a7291e/go.mod h1:8Pf4gM6VEbTNRIT26AyyU7hxdQU3MvAvxVI0sc00XBE= +github.com/containerd/console v1.0.1/go.mod h1:XUsP6YE/mKtz6bxc+I8UiKKTP04qjQL4qcS3XoQ5xkw= +github.com/containerd/console v1.0.2/go.mod h1:ytZPjGgY2oeTkAONYafi2kSj0aYggsf8acV1PGKCbzQ= +github.com/containerd/containerd v1.2.10/go.mod h1:bC6axHOhabU15QhwfG7w5PipXdVtMXFTttgp+kVtyUA= +github.com/containerd/containerd v1.3.0-beta.2.0.20190828155532-0293cbd26c69/go.mod h1:bC6axHOhabU15QhwfG7w5PipXdVtMXFTttgp+kVtyUA= +github.com/containerd/containerd v1.3.0/go.mod h1:bC6axHOhabU15QhwfG7w5PipXdVtMXFTttgp+kVtyUA= +github.com/containerd/containerd v1.3.1-0.20191213020239-082f7e3aed57/go.mod h1:bC6axHOhabU15QhwfG7w5PipXdVtMXFTttgp+kVtyUA= +github.com/containerd/containerd v1.3.2/go.mod h1:bC6axHOhabU15QhwfG7w5PipXdVtMXFTttgp+kVtyUA= +github.com/containerd/containerd v1.4.0-beta.2.0.20200729163537-40b22ef07410/go.mod h1:bC6axHOhabU15QhwfG7w5PipXdVtMXFTttgp+kVtyUA= +github.com/containerd/containerd v1.4.1/go.mod h1:bC6axHOhabU15QhwfG7w5PipXdVtMXFTttgp+kVtyUA= github.com/containerd/containerd v1.4.3/go.mod h1:bC6axHOhabU15QhwfG7w5PipXdVtMXFTttgp+kVtyUA= +github.com/containerd/containerd v1.5.0-beta.1/go.mod h1:5HfvG1V2FsKesEGQ17k5/T7V960Tmcumvqn8Mc+pCYQ= +github.com/containerd/containerd v1.5.0-beta.3/go.mod h1:/wr9AVtEM7x9c+n0+stptlo/uBBoBORwEx6ardVcmKU= +github.com/containerd/containerd v1.5.0-beta.4/go.mod h1:GmdgZd2zA2GYIBZ0w09ZvgqEq8EfBp/m3lcVZIvPHhI= +github.com/containerd/containerd v1.5.0-rc.0/go.mod h1:V/IXoMqNGgBlabz3tHD2TWDoTJseu1FGOKuoA4nNb2s= +github.com/containerd/containerd v1.5.1/go.mod h1:0DOxVqwDy2iZvrZp2JUx/E+hS0UNTVn7dJnIOwtYR4g= +github.com/containerd/containerd v1.5.7 h1:rQyoYtj4KddB3bxG6SAqd4+08gePNyJjRqvOIfV3rkM= +github.com/containerd/containerd v1.5.7/go.mod h1:gyvv6+ugqY25TiXxcZC3L5yOeYgEw0QMhscqVp1AR9c= +github.com/containerd/continuity v0.0.0-20190426062206-aaeac12a7ffc/go.mod h1:GL3xCUCBDV3CZiTSEKksMWbLE66hEyuu9qyDOOqM47Y= +github.com/containerd/continuity v0.0.0-20190815185530-f2a389ac0a02/go.mod h1:GL3xCUCBDV3CZiTSEKksMWbLE66hEyuu9qyDOOqM47Y= github.com/containerd/continuity v0.0.0-20190827140505-75bee3e2ccb6/go.mod h1:GL3xCUCBDV3CZiTSEKksMWbLE66hEyuu9qyDOOqM47Y= -github.com/containerd/continuity v0.0.0-20200413184840-d3ef23f19fbb h1:nXPkFq8X1a9ycY3GYQpFNxHh3j2JgY7zDZfq2EXMIzk= -github.com/containerd/continuity v0.0.0-20200413184840-d3ef23f19fbb/go.mod h1:Dq467ZllaHgAtVp4p1xUQWBrFXR9s/wyoTpG8zOJGkY= +github.com/containerd/continuity v0.0.0-20191127005431-f65d91d395eb/go.mod h1:GL3xCUCBDV3CZiTSEKksMWbLE66hEyuu9qyDOOqM47Y= +github.com/containerd/continuity v0.0.0-20200710164510-efbc4488d8fe/go.mod h1:cECdGN1O8G9bgKTlLhuPJimka6Xb/Gg7vYzCTNVxhvo= +github.com/containerd/continuity v0.0.0-20201208142359-180525291bb7/go.mod h1:kR3BEg7bDFaEddKm54WSmrol1fKWDU1nKYkgrcgZT7Y= +github.com/containerd/continuity v0.0.0-20210208174643-50096c924a4e/go.mod h1:EXlVlkqNba9rJe3j7w3Xa924itAMLgZH4UD/Q4PExuQ= +github.com/containerd/continuity v0.1.0 h1:UFRRY5JemiAhPZrr/uE0n8fMTLcZsUvySPr1+D7pgr8= +github.com/containerd/continuity v0.1.0/go.mod h1:ICJu0PwR54nI0yPEnJ6jcS+J7CZAUXrLh8lPo2knzsM= +github.com/containerd/fifo v0.0.0-20180307165137-3d5202aec260/go.mod h1:ODA38xgv3Kuk8dQz2ZQXpnv/UZZUHUCL7pnLehbXgQI= +github.com/containerd/fifo v0.0.0-20190226154929-a9fb20d87448/go.mod h1:ODA38xgv3Kuk8dQz2ZQXpnv/UZZUHUCL7pnLehbXgQI= +github.com/containerd/fifo v0.0.0-20200410184934-f15a3290365b/go.mod h1:jPQ2IAeZRCYxpS/Cm1495vGFww6ecHmMk1YJH2Q5ln0= +github.com/containerd/fifo v0.0.0-20201026212402-0724c46b320c/go.mod h1:jPQ2IAeZRCYxpS/Cm1495vGFww6ecHmMk1YJH2Q5ln0= +github.com/containerd/fifo v0.0.0-20210316144830-115abcc95a1d/go.mod h1:ocF/ME1SX5b1AOlWi9r677YJmCPSwwWnQ9O123vzpE4= +github.com/containerd/fifo v1.0.0/go.mod h1:ocF/ME1SX5b1AOlWi9r677YJmCPSwwWnQ9O123vzpE4= +github.com/containerd/go-cni v1.0.1/go.mod h1:+vUpYxKvAF72G9i1WoDOiPGRtQpqsNW/ZHtSlv++smU= +github.com/containerd/go-cni v1.0.2/go.mod h1:nrNABBHzu0ZwCug9Ije8hL2xBCYh/pjfMb1aZGrrohk= +github.com/containerd/go-runc v0.0.0-20180907222934-5a6d9f37cfa3/go.mod h1:IV7qH3hrUgRmyYrtgEeGWJfWbgcHL9CSRruz2Vqcph0= +github.com/containerd/go-runc v0.0.0-20190911050354-e029b79d8cda/go.mod h1:IV7qH3hrUgRmyYrtgEeGWJfWbgcHL9CSRruz2Vqcph0= +github.com/containerd/go-runc v0.0.0-20200220073739-7016d3ce2328/go.mod h1:PpyHrqVs8FTi9vpyHwPwiNEGaACDxT/N/pLcvMSRA9g= +github.com/containerd/go-runc v0.0.0-20201020171139-16b287bc67d0/go.mod h1:cNU0ZbCgCQVZK4lgG3P+9tn9/PaJNmoDXPpoJhDR+Ok= +github.com/containerd/go-runc v1.0.0/go.mod h1:cNU0ZbCgCQVZK4lgG3P+9tn9/PaJNmoDXPpoJhDR+Ok= +github.com/containerd/imgcrypt v1.0.1/go.mod h1:mdd8cEPW7TPgNG4FpuP3sGBiQ7Yi/zak9TYCG3juvb0= +github.com/containerd/imgcrypt v1.0.4-0.20210301171431-0ae5c75f59ba/go.mod h1:6TNsg0ctmizkrOgXRNQjAPFWpMYRWuiB6dSF4Pfa5SA= +github.com/containerd/imgcrypt v1.1.1-0.20210312161619-7ed62a527887/go.mod h1:5AZJNI6sLHJljKuI9IHnw1pWqo/F0nGDOuR9zgTs7ow= +github.com/containerd/imgcrypt v1.1.1/go.mod h1:xpLnwiQmEUJPvQoAapeb2SNCxz7Xr6PJrXQb0Dpc4ms= +github.com/containerd/nri v0.0.0-20201007170849-eb1350a75164/go.mod h1:+2wGSDGFYfE5+So4M5syatU0N0f0LbWpuqyMi4/BE8c= +github.com/containerd/nri v0.0.0-20210316161719-dbaa18c31c14/go.mod h1:lmxnXF6oMkbqs39FiCt1s0R2HSMhcLel9vNL3m4AaeY= +github.com/containerd/nri v0.1.0/go.mod h1:lmxnXF6oMkbqs39FiCt1s0R2HSMhcLel9vNL3m4AaeY= +github.com/containerd/ttrpc v0.0.0-20190828154514-0e0f228740de/go.mod h1:PvCDdDGpgqzQIzDW1TphrGLssLDZp2GuS+X5DkEJB8o= +github.com/containerd/ttrpc v0.0.0-20190828172938-92c8520ef9f8/go.mod h1:PvCDdDGpgqzQIzDW1TphrGLssLDZp2GuS+X5DkEJB8o= +github.com/containerd/ttrpc v0.0.0-20191028202541-4f1b8fe65a5c/go.mod h1:LPm1u0xBw8r8NOKoOdNMeVHSawSsltak+Ihv+etqsE8= +github.com/containerd/ttrpc v1.0.1/go.mod h1:UAxOpgT9ziI0gJrmKvgcZivgxOp8iFPSk8httJEt98Y= +github.com/containerd/ttrpc v1.0.2/go.mod h1:UAxOpgT9ziI0gJrmKvgcZivgxOp8iFPSk8httJEt98Y= +github.com/containerd/typeurl v0.0.0-20180627222232-a93fcdb778cd/go.mod h1:Cm3kwCdlkCfMSHURc+r6fwoGH6/F1hH3S4sg0rLFWPc= +github.com/containerd/typeurl v0.0.0-20190911142611-5eb25027c9fd/go.mod h1:GeKYzf2pQcqv7tJ0AoCuuhtnqhva5LNU3U+OyKxxJpk= +github.com/containerd/typeurl v1.0.1/go.mod h1:TB1hUtrpaiO88KEK56ijojHS1+NeF0izUACaJW2mdXg= +github.com/containerd/typeurl v1.0.2/go.mod h1:9trJWW2sRlGub4wZJRTW83VtbOLS6hwcDZXTn6oPz9s= +github.com/containerd/zfs v0.0.0-20200918131355-0a33824f23a2/go.mod h1:8IgZOBdv8fAgXddBT4dBXJPtxyRsejFIpXoklgxgEjw= +github.com/containerd/zfs v0.0.0-20210301145711-11e8f1707f62/go.mod h1:A9zfAbMlQwE+/is6hi0Xw8ktpL+6glmqZYtevJgaB8Y= +github.com/containerd/zfs v0.0.0-20210315114300-dde8f0fda960/go.mod h1:m+m51S1DvAP6r3FcmYCp54bQ34pyOwTieQDNRIRHsFY= +github.com/containerd/zfs v0.0.0-20210324211415-d5c4544f0433/go.mod h1:m+m51S1DvAP6r3FcmYCp54bQ34pyOwTieQDNRIRHsFY= +github.com/containerd/zfs v1.0.0/go.mod h1:m+m51S1DvAP6r3FcmYCp54bQ34pyOwTieQDNRIRHsFY= +github.com/containernetworking/cni v0.7.1/go.mod h1:LGwApLUm2FpoOfxTDEeq8T9ipbpZ61X79hmU3w8FmsY= +github.com/containernetworking/cni v0.8.0/go.mod h1:LGwApLUm2FpoOfxTDEeq8T9ipbpZ61X79hmU3w8FmsY= +github.com/containernetworking/cni v0.8.1/go.mod h1:LGwApLUm2FpoOfxTDEeq8T9ipbpZ61X79hmU3w8FmsY= +github.com/containernetworking/plugins v0.8.6/go.mod h1:qnw5mN19D8fIwkqW7oHHYDHVlzhJpcY6TQxn/fUyDDM= +github.com/containernetworking/plugins v0.9.1/go.mod h1:xP/idU2ldlzN6m4p5LmGiwRDjeJr6FLK6vuiUwoH7P8= +github.com/containers/ocicrypt v1.0.1/go.mod h1:MeJDzk1RJHv89LjsH0Sp5KTY3ZYkjXO/C+bKAeWFIrc= +github.com/containers/ocicrypt v1.1.0/go.mod h1:b8AOe0YR67uU8OqfVNcznfFpAzu3rdgUV4GP9qXPfu4= +github.com/containers/ocicrypt v1.1.1/go.mod h1:Dm55fwWm1YZAjYRaJ94z2mfZikIyIN4B0oB3dj3jFxY= github.com/coreos/bbolt v1.3.2/go.mod h1:iRUV2dpdMOn7Bo10OQBFzIJO9kkE559Wcmn+qkEiiKk= github.com/coreos/etcd v3.3.10+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= github.com/coreos/etcd v3.3.13+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= github.com/coreos/go-etcd v2.0.0+incompatible/go.mod h1:Jez6KQU2B/sWsbdaef3ED8NzMklzPG4d5KIOhIy30Tk= +github.com/coreos/go-iptables v0.4.5/go.mod h1:/mVI274lEDI2ns62jHCDnCyBF9Iwsmekav8Dbxlm1MU= +github.com/coreos/go-iptables v0.5.0/go.mod h1:/mVI274lEDI2ns62jHCDnCyBF9Iwsmekav8Dbxlm1MU= +github.com/coreos/go-oidc v2.1.0+incompatible/go.mod h1:CgnwVTmzoESiwO9qyAFEMiHoZ1nMCKZlZ9V6mm3/LKc= github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= github.com/coreos/go-semver v0.3.0 h1:wkHLiw0WNATZnSG7epLsujiMCgPAc9xhjJ4tgnAxmfM= github.com/coreos/go-semver v0.3.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= +github.com/coreos/go-systemd v0.0.0-20161114122254-48702e0da86b/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= github.com/coreos/go-systemd v0.0.0-20180511133405-39ca1b05acc7/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= github.com/coreos/go-systemd v0.0.0-20190321100706-95778dfbb74e/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= +github.com/coreos/go-systemd/v22 v22.0.0/go.mod h1:xO0FLkIi5MaZafQlIrOotqXZ90ih+1atmu1JpKERPPk= +github.com/coreos/go-systemd/v22 v22.1.0/go.mod h1:xO0FLkIi5MaZafQlIrOotqXZ90ih+1atmu1JpKERPPk= github.com/coreos/go-systemd/v22 v22.3.2 h1:D9/bQk5vlXQFZ6Kwuu6zaiXJ9oTPe68++AzAJc1DzSI= github.com/coreos/go-systemd/v22 v22.3.2/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/coreos/pkg v0.0.0-20160727233714-3ac0863d7acf/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= github.com/cpuguy83/go-md2man v1.0.10/go.mod h1:SmD6nW6nTyfqj6ABTjUi3V3JVMnlJmwcJI5acqYI6dE= github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= +github.com/cpuguy83/go-md2man/v2 v2.0.0/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= github.com/cpuguy83/go-md2man/v2 v2.0.1/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.7/go.mod h1:lj5s0c3V2DBrqTV7llrYr5NG6My20zk30Fl46Y7DoTY= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/creack/pty v1.1.11 h1:07n33Z8lZxZ2qwegKbObQohDhXDQxiMMz1NOUGYlesw= github.com/creack/pty v1.1.11/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= +github.com/cyberdelia/templates v0.0.0-20141128023046-ca7fffd4298c/go.mod h1:GyV+0YP4qX0UQ7r2MoYZ+AvYDp12OF5yg4q8rGnyNh4= +github.com/cyphar/filepath-securejoin v0.2.2/go.mod h1:FpkQEhXnPnOthhzymB7CGsFk2G9VLXONKD9G7QGMM+4= +github.com/d2g/dhcp4 v0.0.0-20170904100407-a1d1b6c41b1c/go.mod h1:Ct2BUK8SB0YC1SMSibvLzxjeJLnrYEVLULFNiHY9YfQ= +github.com/d2g/dhcp4client v1.0.0/go.mod h1:j0hNfjhrt2SxUOw55nL0ATM/z4Yt3t2Kd1mW34z5W5s= +github.com/d2g/dhcp4server v0.0.0-20181031114812-7d4a0a7f59a5/go.mod h1:Eo87+Kg/IX2hfWJfwxMzLyuSZyxSoAug2nGa1G2QAi8= +github.com/d2g/hardwareaddr v0.0.0-20190221164911-e7d9fbe030e4/go.mod h1:bMl4RjIciD2oAxI7DmWRx6gbeqrkoLqv3MV0vzNad+I= github.com/dave/jennifer v1.2.0/go.mod h1:fIb+770HOpJ2fmN9EPPKOqm1vMGhB+TwXKMZhrIygKg= 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/denisenkom/go-mssqldb v0.0.0-20200428022330-06a60b6afbbc/go.mod h1:xbL0rPBG9cCiLr28tMa8zpbdarY27NDyej4t/EjAShU= +github.com/deepmap/oapi-codegen v1.6.0/go.mod h1:ryDa9AgbELGeB+YEXE1dR53yAjHwFvE9iAUlWl9Al3M= +github.com/denisenkom/go-mssqldb v0.10.0/go.mod h1:xbL0rPBG9cCiLr28tMa8zpbdarY27NDyej4t/EjAShU= +github.com/dennwc/varint v1.0.0 h1:kGNFFSSw8ToIy3obO/kKr8U9GZYUAxQEVuix4zfDWzE= +github.com/dennwc/varint v1.0.0/go.mod h1:hnItb35rvZvJrbTALZtY/iQfDs48JKRG1RPpgziApxA= +github.com/denverdino/aliyungo v0.0.0-20190125010748-a747050bb1ba/go.mod h1:dV8lFg6daOBZbT6/BDGIz6Y3WFGn8juu6G+CQ6LHtl0= +github.com/dgrijalva/jwt-go v0.0.0-20170104182250-a601269ab70c/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ= github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ= -github.com/dgrijalva/jwt-go/v4 v4.0.0-preview1/go.mod h1:+hnT3ywWDTAFrW5aE+u2Sa/wT555ZqwoCS+pk3p6ry4= github.com/dgryski/go-bitstream v0.0.0-20180413035011-3522498ce2c8/go.mod h1:VMaSuZ+SZcx/wljOQKvp5srsbCiKDEb6K2wC4+PiBmQ= github.com/dgryski/go-sip13 v0.0.0-20181026042036-e10d5fee7954/go.mod h1:vAd38F8PWV+bWy6jNmig1y/TA+kYO4g3RSRF0IAv0no= github.com/dgryski/go-sip13 v0.0.0-20190329191031-25c5027a8c7b/go.mod h1:vAd38F8PWV+bWy6jNmig1y/TA+kYO4g3RSRF0IAv0no= github.com/dgryski/go-sip13 v0.0.0-20200911182023-62edffca9245/go.mod h1:vAd38F8PWV+bWy6jNmig1y/TA+kYO4g3RSRF0IAv0no= -github.com/digitalocean/godo v1.62.0 h1:7Gw2KFsWkxl36qJa0s50tgXaE0Cgm51JdRP+MFQvNnM= -github.com/digitalocean/godo v1.62.0/go.mod h1:p7dOjjtSBqCTUksqtA5Fd3uaKs9kyTq2xcz76ulEJRU= +github.com/digitalocean/godo v1.69.1 h1:aCyfwth8R3DeOaWB9J9E8v7cjlDIlF19eXTt8R3XhTE= +github.com/digitalocean/godo v1.69.1/go.mod h1:epPuOzTOOJujNo0nduDj2D5O1zu8cSpp9R+DdN0W9I0= github.com/dimchansky/utfbom v1.1.0/go.mod h1:rO41eb7gLfo8SF1jd9F8HplJm1Fewwi4mQvIirEdv+8= github.com/dnaeon/go-vcr v1.0.1/go.mod h1:aBB1+wY4s93YsC3HHjMBMrwTj2R9FHDzUr9KyGc8n1E= +github.com/docker/distribution v0.0.0-20190905152932-14b96e55d84c/go.mod h1:0+TTO4EOBfRPhZXAeF1Vu+W3hHZ8eLp8PgKVZlcvtFY= +github.com/docker/distribution v2.7.1-0.20190205005809-0d3efadf0154+incompatible/go.mod h1:J2gT2udsDAN96Uj4KfcMRqY0/ypR+oyYUYmja8H+y+w= github.com/docker/distribution v2.7.1+incompatible h1:a5mlkVzth6W5A4fOsS3D2EO5BUmsJpcB+cRlLU7cSug= github.com/docker/distribution v2.7.1+incompatible/go.mod h1:J2gT2udsDAN96Uj4KfcMRqY0/ypR+oyYUYmja8H+y+w= -github.com/docker/docker v20.10.7+incompatible h1:Z6O9Nhsjv+ayUEeI1IojKbYcsGdgYSNqxe1s2MYzUhQ= -github.com/docker/docker v20.10.7+incompatible/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk= +github.com/docker/docker v20.10.9+incompatible h1:JlsVnETOjM2RLQa0Cc1XCIspUdXW3Zenq9P54uXBm6k= +github.com/docker/docker v20.10.9+incompatible/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk= github.com/docker/go-connections v0.4.0 h1:El9xVISelRB7BuFusrZozjnkIM5YnzCViNKohAFqRJQ= github.com/docker/go-connections v0.4.0/go.mod h1:Gbd7IOopHjR8Iph03tsViu4nIes5XhDvyHbTtUxmeec= +github.com/docker/go-events v0.0.0-20170721190031-9461782956ad/go.mod h1:Uw6UezgYA44ePAFQYUehOuCzmy5zmg/+nl2ZfMWGkpA= +github.com/docker/go-events v0.0.0-20190806004212-e31b211e4f1c/go.mod h1:Uw6UezgYA44ePAFQYUehOuCzmy5zmg/+nl2ZfMWGkpA= +github.com/docker/go-metrics v0.0.0-20180209012529-399ea8c73916/go.mod h1:/u0gXw0Gay3ceNrsHubL3BtdOL2fHf93USgMTe0W5dI= +github.com/docker/go-metrics v0.0.1/go.mod h1:cG1hvH2utMXtqgqqYE9plW6lDxS3/5ayHzueweSI3Vw= github.com/docker/go-units v0.3.3/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk= github.com/docker/go-units v0.4.0 h1:3uh0PgVws3nIA0Q+MwDC8yjEPf9zjRfZZWXZYDct3Tw= github.com/docker/go-units v0.4.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk= +github.com/docker/libtrust v0.0.0-20150114040149-fa567046d9b1/go.mod h1:cyGadeNEkKy96OOhEzfZl+yxihPEzKnqJwvfuSUqbZE= github.com/docker/spdystream v0.0.0-20160310174837-449fdfce4d96/go.mod h1:Qh8CwZgvJUkLughtfhJv5dyTYa91l1fOUCrgjqmcifM= github.com/docopt/docopt-go v0.0.0-20180111231733-ee0de3bc6815/go.mod h1:WwZ+bS3ebgob9U8Nd0kOddGdZWjyMGR8Wziv+TBNwSE= github.com/dustin/go-humanize v0.0.0-20171111073723-bb3d318650d4/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= @@ -306,11 +456,18 @@ github.com/edsrzf/mmap-go v1.0.0/go.mod h1:YO35OhQPt3KJa3ryjFM5Bs14WD66h8eGKpfaB github.com/elazarl/goproxy v0.0.0-20170405201442-c4fc26588b6e/go.mod h1:/Zj4wYkgs4iZTTu3o/KG3Itv/qCCa8VVMlb3i9OVuzc= github.com/elazarl/goproxy v0.0.0-20180725130230-947c36da3153/go.mod h1:/Zj4wYkgs4iZTTu3o/KG3Itv/qCCa8VVMlb3i9OVuzc= github.com/emicklei/go-restful v0.0.0-20170410110728-ff4f55a20633/go.mod h1:otzb+WCGbkyDHkqmQmT5YD2WR4BBwUdeQoFo8l/7tVs= -github.com/envoyproxy/go-control-plane v0.9.7/go.mod h1:cwu0lG7PUMfa9snN8LXBig5ynNVH9qI8YYLbd1fK2po= +github.com/emicklei/go-restful v2.9.5+incompatible/go.mod h1:otzb+WCGbkyDHkqmQmT5YD2WR4BBwUdeQoFo8l/7tVs= +github.com/envoyproxy/go-control-plane v0.9.9-0.20210512163311-63b5d3c536b0/go.mod h1:hliV/p42l8fGbc6Y9bQ70uLwIvmJyVE5k4iMKlh8wCQ= +github.com/envoyproxy/go-control-plane v0.9.9/go.mod h1:hliV/p42l8fGbc6Y9bQ70uLwIvmJyVE5k4iMKlh8wCQ= +github.com/envoyproxy/go-control-plane v0.10.1 h1:cgDRLG7bs59Zd+apAWuzLQL95obVYAymNJek76W3mgw= +github.com/envoyproxy/go-control-plane v0.10.1/go.mod h1:AY7fTTXNdv/aJ2O5jwpxAPOWUZ7hQAEvzN5Pf27BkQQ= github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= +github.com/envoyproxy/protoc-gen-validate v0.6.1/go.mod h1:txg5va2Qkip90uYoSKH+nkAAmXrb2j3iq4FLwdrCbXQ= +github.com/envoyproxy/protoc-gen-validate v0.6.2 h1:JiO+kJTpmYGjEodY7O1Zk8oZcNz1+f30UtwtXoFUPzE= github.com/envoyproxy/protoc-gen-validate v0.6.2/go.mod h1:2t7qjJNvHPx8IjnBOzl9E9/baC+qXE/TeeyBRzgJDws= github.com/evanphx/json-patch v4.2.0+incompatible/go.mod h1:50XU6AFN0ol/bzJsmQLiYLvXMP4fmwYFNcr97nuDLSk= github.com/evanphx/json-patch v4.9.0+incompatible/go.mod h1:50XU6AFN0ol/bzJsmQLiYLvXMP4fmwYFNcr97nuDLSk= +github.com/evanphx/json-patch v4.11.0+incompatible/go.mod h1:50XU6AFN0ol/bzJsmQLiYLvXMP4fmwYFNcr97nuDLSk= github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= github.com/fatih/color v1.9.0/go.mod h1:eQcE1qtQxscV5RaZvpXrrb8Drkc3/DdQ+uUYCNjL+zU= github.com/fatih/color v1.13.0 h1:8LOYc1KYPPmyKMuN8QV2DNRWNbLo6LZ0iLs8+mlH53w= @@ -321,7 +478,6 @@ github.com/felixge/httpsnoop v1.0.2 h1:+nS9g82KMXccJ/wp0zyRW9ZBHFETmMGtkk+2CTTrW github.com/felixge/httpsnoop v1.0.2/go.mod h1:m8KPJKqk1gH5J9DgRY2ASl2lWCfGKXixSwevea8zH2U= github.com/fogleman/gg v1.2.1-0.20190220221249-0403632d5b90/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k= github.com/form3tech-oss/jwt-go v3.2.2+incompatible/go.mod h1:pbq4aXjuKjdthFRnoDwaVPLA+WlJuPGy+QneDUgJi2k= -github.com/form3tech-oss/jwt-go v3.2.3+incompatible h1:7ZaBxOI7TMoYBfyA3cQHErNNyAWIKUMIwqxEtgHOs5c= github.com/form3tech-oss/jwt-go v3.2.3+incompatible/go.mod h1:pbq4aXjuKjdthFRnoDwaVPLA+WlJuPGy+QneDUgJi2k= github.com/fortytw2/leaktest v1.3.0 h1:u8491cBMTQ8ft8aeV+adlcytMZylmA5nnwwkRZjI8vw= github.com/fortytw2/leaktest v1.3.0/go.mod h1:jDsjWgpAGjm2CA7WthBh/CdZYEPF31XHquHwclZch5g= @@ -329,15 +485,21 @@ github.com/foxcpp/go-mockdns v0.0.0-20201212160233-ede2f9158d15/go.mod h1:tPg4cp github.com/franela/goblin v0.0.0-20200105215937-c9ffbefa60db/go.mod h1:7dvUGVsVBjqR7JHJk0brhHOZYGmfBYOrK0ZhYMEtBr4= github.com/franela/goreq v0.0.0-20171204163338-bcd34c9993f8/go.mod h1:ZhphrRTfi2rbfLwlschooIH4+wKKDR4Pdxhh+TRoA20= github.com/frankban/quicktest v1.4.0/go.mod h1:36zfPVQyHxymz4cH7wlDmVwDrJuljRB60qkgn7rorfQ= +github.com/frankban/quicktest v1.11.3/go.mod h1:wRf/ReqHper53s+kmmSZizM8NamnL3IM0I9ntUbOk+k= github.com/frankban/quicktest v1.14.0/go.mod h1:NeW+ay9A/U67EYXNFA1nPE8e/tnQv/09mUdL/ijj8og= github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ= github.com/fsnotify/fsnotify v1.5.1 h1:mZcQUHVQUQWoPXXtuf9yuEXKudkV2sx1E06UadKWpgI= github.com/fsnotify/fsnotify v1.5.1/go.mod h1:T3375wBYaZdLLcVNkcVbzGHY7f1l/uK5T5Ai1i3InKU= +github.com/fullsailor/pkcs7 v0.0.0-20190404230743-d7302db945fa/go.mod h1:KnogPXtdwXqoenmZCw6S+25EAm2MkxbG0deNDu4cbSA= +github.com/garyburd/redigo v0.0.0-20150301180006-535138d7bcd7/go.mod h1:NR3MbYisc3/PwhQ00EMzDiPmrwpPxAn5GI05/YaO1SY= +github.com/getkin/kin-openapi v0.53.0/go.mod h1:7Yn5whZr5kJi6t+kShccXS8ae1APpYTW6yheSwk8Yi4= github.com/getsentry/raven-go v0.2.0 h1:no+xWJRb5ZI7eE8TWgIq1jLulQiIoLG0IfYxv5JYMGs= github.com/ghodss/yaml v0.0.0-20150909031657-73d445a93680/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/ghodss/yaml v1.0.0 h1:wQHKEahhL6wmXdzwWG11gIVCkOv05bNOh+Rxn0yngAk= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= +github.com/gin-contrib/sse v0.1.0/go.mod h1:RHrZQHXnP2xjPF+u1gW/2HnVO7nvIa9PG3Gm+fLHvGI= +github.com/gin-gonic/gin v1.5.0/go.mod h1:Nd6IXA8m5kNZdNEHMBd93KT+mdY3+bewLgRvmCsR2Do= github.com/globalsign/mgo v0.0.0-20180905125535-1ca0a4f7cbcb/go.mod h1:xkRDCp4j0OGD1HRkm4kmhM+pmpv3AKq5SU7GMg4oO/Q= github.com/globalsign/mgo v0.0.0-20181015135952-eeefdecb41b8/go.mod h1:xkRDCp4j0OGD1HRkm4kmhM+pmpv3AKq5SU7GMg4oO/Q= github.com/glycerine/go-unsnap-stream v0.0.0-20180323001048-9f0cb55181dd/go.mod h1:/20jfyN9Y5QPEAprSgKAUr+glWDY39ZiUEAYOEv5dsE= @@ -346,22 +508,28 @@ github.com/glycerine/go-unsnap-stream v0.0.0-20181221182339-f9677308dec2/go.mod github.com/glycerine/goconvey v0.0.0-20190410193231-58a59202ab31 h1:gclg6gY70GLy3PbkQ1AERPfmLMMagS60DKF78eWwLn8= github.com/glycerine/goconvey v0.0.0-20190410193231-58a59202ab31/go.mod h1:Ogl1Tioa0aV7gstGFO7KhffUsb9M4ydbEbbxpcEDc24= github.com/go-chi/chi v4.1.0+incompatible/go.mod h1:eB3wogJHnLi3x/kFX2A+IbTBlXxmMeXJVKy9tTv1XzQ= +github.com/go-chi/chi/v5 v5.0.0/go.mod h1:BBug9lr0cqtdAhsu6R4AAdvufI0/XBzAQSsUqJpoZOs= 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-ini/ini v1.25.4/go.mod h1:ByCAeIL28uOIIG0E3PJtZPDL8WnHpFKFOtgjp+3Ies8= 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/kit v0.10.0 h1:dXFJfIHVvUcpSgDOV+Ne6t7jXri8Tfv2uOLHUZ2XNuo= github.com/go-kit/kit v0.10.0/go.mod h1:xUsJbQ/Fp4kEt7AFgCuvyX4a71u8h9jB8tj/ORgOZ7o= -github.com/go-kit/log v0.1.0 h1:DGJh0Sm43HbOeYDNnVZFl8BvcYVvjD5bqYJvp0REbwQ= github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= +github.com/go-kit/log v0.2.0 h1:7i2K3eKTos3Vc0enKCfnVcgHh2olr/MyfboYq7cAcFw= +github.com/go-kit/log v0.2.0/go.mod h1:NwTd00d/i8cPZ3xOwwiv2PO5MOcx78fFErGNcVmBjv0= github.com/go-ldap/ldap v3.0.2+incompatible/go.mod h1:qfd9rJvER9Q0/D/Sqn1DfHRoBp40uXYvFoEVrNEPqRc= 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 h1:TrB8swr/68K7m9CcGut2g3UOihhbcbiMAYiuTXdEih4= github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= +github.com/go-logfmt/logfmt v0.5.1 h1:otpy5pqBCBZ1ng9RQ0dPu4PN7ba75Y/aA+UpowDyNVA= +github.com/go-logfmt/logfmt v0.5.1/go.mod h1:WYhtIu8zTZfxdn5+rREduYbwxfcBr/Vr6KEVveWlfTs= github.com/go-logr/logr v0.1.0/go.mod h1:ixOQHD9gLJUVQQ2ZOR7zLEifBX6tGkNJF4QyIY7sIas= +github.com/go-logr/logr v0.2.0/go.mod h1:z6/tIYblkpsD+a4lm/fGIIU9mZ+XfAiaFtq7xTgseGU= github.com/go-logr/logr v0.4.0/go.mod h1:z6/tIYblkpsD+a4lm/fGIIU9mZ+XfAiaFtq7xTgseGU= +github.com/go-logr/logr v1.0.0/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= github.com/go-logr/logr v1.2.0/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= github.com/go-logr/logr v1.2.1/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= github.com/go-logr/logr v1.2.2 h1:ahHml/yUpnlb96Rp8HCvtYVPY8ZYpxq3g7UYchIYwbs= @@ -419,7 +587,7 @@ github.com/go-openapi/runtime v0.19.4/go.mod h1:X277bwSUBxVlCYR3r7xgZZGKVvBd/29g github.com/go-openapi/runtime v0.19.15/go.mod h1:dhGWCTKRXlAfGnQG0ONViOZpjfg0m2gUt9nTQPQZuoo= github.com/go-openapi/runtime v0.19.16/go.mod h1:5P9104EJgYcizotuXhEuUrzVc+j1RiSjahULvYmlv98= github.com/go-openapi/runtime v0.19.24/go.mod h1:Lm9YGCeecBnUUkFTxPC4s1+lwrkJ0pthx8YvyjCfkgk= -github.com/go-openapi/runtime v0.19.28/go.mod h1:BvrQtn6iVb2QmiVXRsFAm6ZCAZBpbVKFfN6QWCp582M= +github.com/go-openapi/runtime v0.19.29/go.mod h1:BvrQtn6iVb2QmiVXRsFAm6ZCAZBpbVKFfN6QWCp582M= github.com/go-openapi/spec v0.0.0-20160808142527-6aced65f8501/go.mod h1:J8+jY1nAiCcj+friV/PDoE1/3eeccG9LYBs0tYvLOWc= github.com/go-openapi/spec v0.17.0/go.mod h1:XkF/MOi14NmjsfZ8VtAKf8pIlbZzyoTvZsdfssdxcBI= github.com/go-openapi/spec v0.18.0/go.mod h1:XkF/MOi14NmjsfZ8VtAKf8pIlbZzyoTvZsdfssdxcBI= @@ -442,6 +610,7 @@ github.com/go-openapi/strfmt v0.19.5/go.mod h1:eftuHTlB/dI8Uq8JJOyRlieZf+WkkxUuk github.com/go-openapi/strfmt v0.19.11/go.mod h1:UukAYgTaQfqJuAFlNxxMWNvMYiwiXtLsF2VwmoFtbtc= github.com/go-openapi/strfmt v0.20.0/go.mod h1:UukAYgTaQfqJuAFlNxxMWNvMYiwiXtLsF2VwmoFtbtc= github.com/go-openapi/strfmt v0.20.1/go.mod h1:43urheQI9dNtE5lTZQfuFJvjYJKPrxicATpEfZwHUNk= +github.com/go-openapi/strfmt v0.20.3/go.mod h1:43urheQI9dNtE5lTZQfuFJvjYJKPrxicATpEfZwHUNk= github.com/go-openapi/swag v0.0.0-20160704191624-1d0bd113de87/go.mod h1:DXUve3Dpr1UfpPtxFw+EFuQ41HhCWZfha5jSVRG7C7I= github.com/go-openapi/swag v0.17.0/go.mod h1:AByQ+nYG6gQg71GINrmuDXCPWdL640yX49/kXLo40Tg= github.com/go-openapi/swag v0.18.0/go.mod h1:AByQ+nYG6gQg71GINrmuDXCPWdL640yX49/kXLo40Tg= @@ -462,8 +631,10 @@ github.com/go-openapi/validate v0.19.12/go.mod h1:Rzou8hA/CBw8donlS6WNEUQupNvUZ0 github.com/go-openapi/validate v0.19.15/go.mod h1:tbn/fdOwYHgrhPBzidZfJC2MIVvs9GA7monOmWBbeCI= github.com/go-openapi/validate v0.20.1/go.mod h1:b60iJT+xNNLfaQJUqLI7946tYiFEOuE9E4k54HpKcJ0= github.com/go-openapi/validate v0.20.2/go.mod h1:e7OJoKNgd0twXZwIn0A43tHbvIcr/rZIVCbJBpTUoY0= +github.com/go-playground/locales v0.12.1/go.mod h1:IUMDtCfWo/w/mtMfIE/IG2K+Ey3ygWanZIBtBW0W2TM= github.com/go-playground/locales v0.13.0 h1:HyWk6mgj5qFqCT5fjGBuRArbVDfE4hi8+e8ceBS/t7Q= github.com/go-playground/locales v0.13.0/go.mod h1:taPMhCMXrRLJO55olJkUXHZBHCxTMfnGwq/HNwmWNS8= +github.com/go-playground/universal-translator v0.16.0/go.mod h1:1AnU7NaIRDWWzGEKwgtJRd2xk99HeFyHw3yid4rvQIY= github.com/go-playground/universal-translator v0.17.0 h1:icxd5fm+REJzpZx7ZfpaD876Lmtgy7VtROAbHHXk8no= github.com/go-playground/universal-translator v0.17.0/go.mod h1:UkSxE5sNxxRwHyU+Scu5vgOQjsIJAF8j9muTVoKLVtA= github.com/go-resty/resty/v2 v2.1.1-0.20191201195748-d7b97669fe48 h1:JVrqSeQfdhYRFk24TvhTZWU0q8lfCojxZQFi3Ou7+uY= @@ -500,19 +671,29 @@ github.com/gobuffalo/packd v0.1.0/go.mod h1:M2Juc+hhDXf/PnmBANFCqx4DM3wRbgDvnVWe github.com/gobuffalo/packr/v2 v2.0.9/go.mod h1:emmyGweYTm6Kdper+iywB6YK5YzuKchGtJQZ0Odn4pQ= github.com/gobuffalo/packr/v2 v2.2.0/go.mod h1:CaAwI0GPIAv+5wKLtv8Afwl+Cm78K/I/VCm/3ptBN+0= github.com/gobuffalo/syncx v0.0.0-20190224160051-33c29581e754/go.mod h1:HhnNqWY95UYwwW3uSASeV7vtgYkT2t16hJgV3AEPUpw= +github.com/godbus/dbus v0.0.0-20151105175453-c7fdd8b5cd55/go.mod h1:/YcGZj5zSblfDWMMoOzV4fas9FZnQYTkDnsGvmh2Grw= +github.com/godbus/dbus v0.0.0-20180201030542-885f9cc04c9c/go.mod h1:/YcGZj5zSblfDWMMoOzV4fas9FZnQYTkDnsGvmh2Grw= +github.com/godbus/dbus v0.0.0-20190422162347-ade71ed3457e/go.mod h1:bBOAhwG1umN6/6ZUMtDFBMQR8jRg9O75tm9K00oMsK4= +github.com/godbus/dbus/v5 v5.0.3/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= github.com/gofrs/uuid v3.3.0+incompatible/go.mod h1:b2aQJv3Z4Fp6yNu3cdSllBxTCLRxnplIgP/c0N/04lM= github.com/gofrs/uuid v4.0.0+incompatible/go.mod h1:b2aQJv3Z4Fp6yNu3cdSllBxTCLRxnplIgP/c0N/04lM= +github.com/gogo/googleapis v1.2.0/go.mod h1:Njal3psf3qN6dwBtQfUmBZh2ybovJ0tlu3o/AC7HYjU= +github.com/gogo/googleapis v1.4.0/go.mod h1:5YRNX2z1oM5gXdAkurHa942MDgEJyk02w4OecKY87+c= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.2.0/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zVXpSg4= github.com/gogo/protobuf v1.2.2-0.20190723190241-65acae22fc9d/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o= github.com/gogo/protobuf v1.2.2-0.20190730201129-28a6bbf47e48/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o= +github.com/gogo/protobuf v1.3.0/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o= github.com/gogo/protobuf v1.3.1/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= +github.com/golang-jwt/jwt v3.2.1+incompatible/go.mod h1:8pz2t5EyA70fFQQSrl6XZXzqecmYZeUEB8OUGHkxJ+I= github.com/golang-jwt/jwt v3.2.2+incompatible h1:IfV12K8xAKAnZqdXVzCZ+TOjboZ2keLg81eXfW3O+oY= github.com/golang-jwt/jwt v3.2.2+incompatible/go.mod h1:8pz2t5EyA70fFQQSrl6XZXzqecmYZeUEB8OUGHkxJ+I= +github.com/golang-jwt/jwt/v4 v4.0.0 h1:RAqyYixv1p7uEnocuy8P1nru5wprCh/MH2BIlW5z5/o= +github.com/golang-jwt/jwt/v4 v4.0.0/go.mod h1:/xlHOz8bRuivTWchD4jCa+NbatV+wEUSzwAxVc6locg= github.com/golang-sql/civil v0.0.0-20190719163853-cb61b32ac6fe/go.mod h1:8vg3r2VgvsThLBIFL93Qb5yWzgyZWhEmBwUJWevAkK0= github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0/go.mod h1:E/TSTwGwJL78qG/PmXZO1EjYhfJinVAhrmmHX6Z8B9k= github.com/golang/geo v0.0.0-20190916061304-5b978397cfec/go.mod h1:QZ0nwyI2jOfgRAoBvP+ab5aRr7c9x7lhGEJrKvBwjWI= @@ -557,6 +738,7 @@ github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEW github.com/golang/snappy v0.0.3/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golang/snappy v0.0.4 h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM= github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= +github.com/golangci/lint-1 v0.0.0-20181222135242-d2cdd8c08219/go.mod h1:/X8TswGSh1pIozq4ZwCfxS0WA5JGXguxk94ar/4c87Y= 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/btree v1.0.1 h1:gK4Kx5IaGY9CD5sPJ36FHiBJ6ZXl0kilRiiCj+jdYp4= @@ -600,10 +782,12 @@ github.com/google/pprof v0.0.0-20210226084205-cbba55b83ad5/go.mod h1:kpwsk12EmLe github.com/google/pprof v0.0.0-20210601050228-01bbb1931b22/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20210609004039-a478d1d731e9/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20210720184732-4bb14d4b1be1/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= +github.com/google/pprof v0.0.0-20211008130755-947d60d73cc0/go.mod h1:KgnwoLYCZ8IQu3XUZ8Nc/bM9CCZFOyjUNOSygVozoDg= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/google/uuid v1.2.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.3.0 h1:t6JiXgmwXMjEs8VusXIJk2BXHsn+wx8BZdTaoZ5fu7I= github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= @@ -612,24 +796,29 @@ github.com/googleapis/gax-go/v2 v2.1.0/go.mod h1:Q3nei7sK6ybPYH7twZdmQpAd1MKb7pf github.com/googleapis/gax-go/v2 v2.1.1/go.mod h1:hddJymUZASv3XPyGkUpKj8pPO47Rmb0eJc8R6ouapiM= github.com/googleapis/gnostic v0.0.0-20170729233727-0c5108395e2d/go.mod h1:sJBsCZ4ayReDTBIg8b9dl28c5xFWyhBTVRp3pOg5EKY= github.com/googleapis/gnostic v0.4.0/go.mod h1:on+2t9HRStVgn95RSsFWFz+6Q0Snyqv1awfrALZdbtU= -github.com/googleapis/gnostic v0.4.1 h1:DLJCy1n/vrD4HPjOvYcT8aYQXpPIzoRZONaYwyycI+I= github.com/googleapis/gnostic v0.4.1/go.mod h1:LRhVm6pbyptWbWbuZ38d1eyptfvIytN3ir6b65WBswg= +github.com/googleapis/gnostic v0.5.1/go.mod h1:6U4PtQXGIEt/Z3h5MAT7FNofLnw9vXk2cUuW7uA/OeU= +github.com/googleapis/gnostic v0.5.5 h1:9fHAtK0uDfpveeqqo1hkEZJcFvYXAiCN3UutL8F9xHw= +github.com/googleapis/gnostic v0.5.5/go.mod h1:7+EbHbldMins07ALC74bsA81Ovc97DwqyJO1AENw9kA= github.com/gophercloud/gophercloud v0.1.0/go.mod h1:vxM41WHh5uqHVBMZHzuwNOHh8XEoIEcSTewFxm1c5g8= github.com/gophercloud/gophercloud v0.10.0/go.mod h1:gmC5oQqMDOMO1t1gq5DquX/yAU808e/4mzjjDA76+Ss= -github.com/gophercloud/gophercloud v0.18.0 h1:V6hcuMPmjXg+js9flU8T3RIHDCjV7F5CG5GD0MRhP/w= -github.com/gophercloud/gophercloud v0.18.0/go.mod h1:wRtmUelyIIv3CSSDI47aUwbs075O6i+LY+pXsKCBsb4= +github.com/gophercloud/gophercloud v0.22.0 h1:9lFISNLafZcecT0xUveIMt3IafexC6DIV9ek1SZdSMw= +github.com/gophercloud/gophercloud v0.22.0/go.mod h1:wRtmUelyIIv3CSSDI47aUwbs075O6i+LY+pXsKCBsb4= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= github.com/gopherjs/gopherjs v0.0.0-20190910122728-9d188e94fb99 h1:twflg0XRTjwKpxb/jFExr4HGq6on2dEOmnL6FV+fgPw= github.com/gopherjs/gopherjs v0.0.0-20190910122728-9d188e94fb99/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= github.com/gorilla/context v1.1.1/go.mod h1:kBGZzfjB9CEq2AlWe17Uuf7NDRt0dE0s8S51q0aT7Yg= +github.com/gorilla/handlers v0.0.0-20150720190736-60c7bfde3e33/go.mod h1:Qkdc/uu4tH4g6mTK6auzZ766c4CA0Ng8+o/OAirnOIQ= github.com/gorilla/handlers v1.3.0/go.mod h1:Qkdc/uu4tH4g6mTK6auzZ766c4CA0Ng8+o/OAirnOIQ= github.com/gorilla/handlers v1.5.1 h1:9lRY6j8DEeeBT10CvO9hGW0gmky0BprnvDI5vfhUHH4= github.com/gorilla/handlers v1.5.1/go.mod h1:t8XrUpc4KVXb7HGyJ4/cEnwQiaxrX/hz1Zv/4g96P1Q= github.com/gorilla/mux v1.6.2/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2zaAs= +github.com/gorilla/mux v1.7.2/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2zaAs= github.com/gorilla/mux v1.7.3/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2zaAs= github.com/gorilla/mux v1.8.0 h1:i40aqfkR1h2SlN9hojwV5ZA91wcXFOvkdNIeFDP5koI= github.com/gorilla/mux v1.8.0/go.mod h1:DVbg23sWSpFRCP0SfiEN6jmj59UnW/n46BH5rLB71So= github.com/gorilla/websocket v0.0.0-20170926233335-4201258b820c/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= +github.com/gorilla/websocket v1.4.0/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= github.com/gorilla/websocket v1.4.2 h1:+/TMaTYc4QFitKJxsQ7Yye35DkWvkdLcvGKqM+x0Ufc= github.com/gorilla/websocket v1.4.2/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= github.com/gregjones/httpcache v0.0.0-20180305231024-9cad4c3443a7/go.mod h1:FecbI9+v66THATjSRHfNgh1IVFe/9kFxbXtjV0ctIMA= @@ -647,14 +836,13 @@ github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFb github.com/hashicorp/consul/api v1.1.0/go.mod h1:VmuI/Lkw1nC05EYQWNKwWGbkg+FbDBtguAZLlVdkD9Q= github.com/hashicorp/consul/api v1.3.0/go.mod h1:MmDNSzIMUjNpY/mQ398R4bk2FnqQLoPndWW5VkKPlCE= github.com/hashicorp/consul/api v1.4.0/go.mod h1:xc8u05kyMa3Wjr9eEAsIAo3dg8+LywT5E/Cl7cNS5nU= -github.com/hashicorp/consul/api v1.8.1/go.mod h1:sDjTOq0yUyv5G4h+BqSea7Fn6BU+XbolEz1952UB+mk= github.com/hashicorp/consul/api v1.11.0 h1:Hw/G8TtRvOElqxVIhBzXciiSTbapq8hZ2XKZsXk5ZCE= github.com/hashicorp/consul/api v1.11.0/go.mod h1:XjsvQN+RJGWI2TWy1/kqaE16HrR2J/FWgkYjdZQsX9M= github.com/hashicorp/consul/sdk v0.1.1/go.mod h1:VKf9jXwCTEY1QZP2MOLRhb5i/I/ssyNV1vwHyQBF0x8= github.com/hashicorp/consul/sdk v0.3.0/go.mod h1:VKf9jXwCTEY1QZP2MOLRhb5i/I/ssyNV1vwHyQBF0x8= github.com/hashicorp/consul/sdk v0.4.0/go.mod h1:fY08Y9z5SvJqevyZNy6WWPXiG3KwBPAvlcdx16zZ0fM= -github.com/hashicorp/consul/sdk v0.7.0/go.mod h1:fY08Y9z5SvJqevyZNy6WWPXiG3KwBPAvlcdx16zZ0fM= github.com/hashicorp/consul/sdk v0.8.0/go.mod h1:GBvyrGALthsZObzUGsfgHZQDXjg4lOjagTIwIR1vPms= +github.com/hashicorp/errwrap v0.0.0-20141028054710-7554cd9344ce/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= 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-cleanhttp v0.5.0/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80= @@ -674,6 +862,7 @@ github.com/hashicorp/go-immutable-radix v1.3.1/go.mod h1:0y9vanUI8NX6FsYoO3zeMjh github.com/hashicorp/go-msgpack v0.5.3/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM= github.com/hashicorp/go-msgpack v0.5.5 h1:i9R9JSrqIz0QVLz3sz+i3YJdT7TTSLcfLLzJi9aZTuI= github.com/hashicorp/go-msgpack v0.5.5/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM= +github.com/hashicorp/go-multierror v0.0.0-20161216184304-ed905158d874/go.mod h1:JMRHfdO9jKNzS/+BTlxCjKNQHg/jZAft8U7LloJvN7I= github.com/hashicorp/go-multierror v1.0.0/go.mod h1:dHtQlpGsu+cZNNAkkCN/P3hoUDHhCYQXV3UM06sGGrk= github.com/hashicorp/go-multierror v1.1.0 h1:B9UzwGQJehnUY1yNrnwREHc3fGbC2xefo8g4TbElacI= github.com/hashicorp/go-multierror v1.1.0/go.mod h1:spPvp8C1qA32ftKqdAHm4hHTbPw+vmowP0z+KUhOZdA= @@ -709,7 +898,7 @@ github.com/hashicorp/memberlist v0.1.3/go.mod h1:ajVTdAv/9Im8oMAAj5G31PhhMCZJV2p github.com/hashicorp/memberlist v0.1.4/go.mod h1:ajVTdAv/9Im8oMAAj5G31PhhMCZJV2pPBoIllUwCN7I= github.com/hashicorp/memberlist v0.2.0/go.mod h1:MS2lj3INKhZjWNqd3N0m3J+Jxf3DAOnAH9VT3Sh9MUE= github.com/hashicorp/memberlist v0.2.2/go.mod h1:MS2lj3INKhZjWNqd3N0m3J+Jxf3DAOnAH9VT3Sh9MUE= -github.com/hashicorp/memberlist v0.2.3/go.mod h1:MS2lj3INKhZjWNqd3N0m3J+Jxf3DAOnAH9VT3Sh9MUE= +github.com/hashicorp/memberlist v0.2.4/go.mod h1:MS2lj3INKhZjWNqd3N0m3J+Jxf3DAOnAH9VT3Sh9MUE= github.com/hashicorp/memberlist v0.3.0 h1:8+567mCcFDnS5ADl7lrpxPMWiFCElyUEeW0gtj34fMA= github.com/hashicorp/memberlist v0.3.0/go.mod h1:MS2lj3INKhZjWNqd3N0m3J+Jxf3DAOnAH9VT3Sh9MUE= github.com/hashicorp/serf v0.8.2/go.mod h1:6hOLApaqBFA1NXqRQAsxw9QxuDEvNxSQRwA/JwenrHc= @@ -721,42 +910,52 @@ github.com/hashicorp/vault/api v1.0.4/go.mod h1:gDcqh3WGcR1cpF5AJz/B1UFheUEneMoI github.com/hashicorp/vault/sdk v0.1.13/go.mod h1:B+hVj7TpuQY1Y/GPbCpffmgd+tSEwvhkWnjtSYCaS2M= github.com/hashicorp/yamux v0.0.0-20180604194846-3520598351bb/go.mod h1:+NfK9FKeTrX5uv1uIXGdwYDTeHna2qgaIlx54MXqjAM= github.com/hashicorp/yamux v0.0.0-20181012175058-2f1d1f20f75d/go.mod h1:+NfK9FKeTrX5uv1uIXGdwYDTeHna2qgaIlx54MXqjAM= -github.com/hetznercloud/hcloud-go v1.26.2 h1:fI8BXAGJI4EFeCDd2a/I4EhqyK32cDdxGeWfYMGUi50= -github.com/hetznercloud/hcloud-go v1.26.2/go.mod h1:2C5uMtBiMoFr3m7lBFPf7wXTdh33CevmZpQIIDPGYJI= +github.com/hetznercloud/hcloud-go v1.32.0 h1:7zyN2V7hMlhm3HZdxOarmOtvzKvkcYKjM0hcwYMQZz0= +github.com/hetznercloud/hcloud-go v1.32.0/go.mod h1:XX/TQub3ge0yWR2yHWmnDVIrB+MQbda1pHxkUmDlUME= github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= github.com/huandu/xstrings v1.0.0/go.mod h1:4qWG/gcEcfX4z/mBDHJ++3ReCw9ibxbsNJbcucJdbSo= github.com/hudl/fargo v1.3.0/go.mod h1:y3CKSmjA+wD2gak7sUSXTAoopbhU08POFhmITJgmKTg= github.com/hydrogen18/stalecucumber v0.0.0-20151102144322-9b38526d4bdf h1:d8mEMzY9ktqK+eVFDLIsYOcM19/yYvZlmR0kcb4MrSQ= github.com/hydrogen18/stalecucumber v0.0.0-20151102144322-9b38526d4bdf/go.mod h1:KE5xQoh/IqNckSFoQXL5o5nEkrBiUDxatgac7TSMQ8Y= +github.com/iancoleman/strcase v0.0.0-20180726023541-3605ed457bf7/go.mod h1:SK73tn/9oHe+/Y0h39VT4UCxmurVJkR5NA7kMEAOgSE= github.com/iancoleman/strcase v0.2.0/go.mod h1:iwCmte+B7n89clKwxIoIXy/HfoL7AsD47ZCWhYzw7ho= github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= github.com/ianlancetaylor/demangle v0.0.0-20200824232613-28f6c0f3b639/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= +github.com/ianlancetaylor/demangle v0.0.0-20210905161508-09a460cdf81d/go.mod h1:aYm2/VgdVmcIU8iMfdMvDMsRAQjcfZSKFby6HOFvi/w= github.com/imdario/mergo v0.3.4/go.mod h1:2EnlNZ0deacrJVfApfmtdGgDfMuh/nq6Ok1EcJh5FfA= -github.com/imdario/mergo v0.3.5 h1:JboBksRwiiAJWvIYJVo46AfV+IAIKZpfrSzVKj42R4Q= github.com/imdario/mergo v0.3.5/go.mod h1:2EnlNZ0deacrJVfApfmtdGgDfMuh/nq6Ok1EcJh5FfA= +github.com/imdario/mergo v0.3.8/go.mod h1:2EnlNZ0deacrJVfApfmtdGgDfMuh/nq6Ok1EcJh5FfA= +github.com/imdario/mergo v0.3.10/go.mod h1:jmQim1M+e3UYxmgPu/WyfjB3N3VflVyUjjjwH0dnCYA= +github.com/imdario/mergo v0.3.11/go.mod h1:jmQim1M+e3UYxmgPu/WyfjB3N3VflVyUjjjwH0dnCYA= +github.com/imdario/mergo v0.3.12 h1:b6R2BslTbIEToALKP7LxUvijTsNI9TAe80pLWN2g/HU= +github.com/imdario/mergo v0.3.12/go.mod h1:jmQim1M+e3UYxmgPu/WyfjB3N3VflVyUjjjwH0dnCYA= github.com/inconshreveable/mousetrap v1.0.0 h1:Z8tu5sraLXCXIcARxBp/8cbvlwVa7Z1NHg9XEKhtSvM= github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= github.com/influxdata/flux v0.65.0/go.mod h1:BwN2XG2lMszOoquQaFdPET8FRQfrXiZsWmcMO9rkaVY= -github.com/influxdata/flux v0.113.0/go.mod h1:3TJtvbm/Kwuo5/PEo5P6HUzwVg4bXWkb2wPQHPtQdlU= +github.com/influxdata/flux v0.131.0/go.mod h1:CKvnYe6FHpTj/E0YGI7TcOZdGiYHoToOPSnoa12RtKI= github.com/influxdata/httprouter v1.3.1-0.20191122104820-ee83e2772f69/go.mod h1:pwymjR6SrP3gD3pRj9RJwdl1j5s3doEEV8gS4X9qSzA= github.com/influxdata/influxdb v1.8.0/go.mod h1:SIzcnsjaHRFpmlxpJ4S3NT64qtEKYweNTUMb/vh0OMQ= -github.com/influxdata/influxdb v1.9.2 h1:yDwzIu/R8Q8sZF5R7+69/7/Qie5DrMetbCsjS5BnRtw= -github.com/influxdata/influxdb v1.9.2/go.mod h1:UEe3MeD9AaP5rlPIes102IhYua3FhIWZuOXNHxDjSrI= +github.com/influxdata/influxdb v1.9.5 h1:4O7AC5jOA9RoqtDuD2rysXbumcEwaqWlWXmwuyK+a2s= +github.com/influxdata/influxdb v1.9.5/go.mod h1:4uPVvcry9KWQVWLxyT9641qpkRXUBN+xa0MJFFNNLKo= +github.com/influxdata/influxdb-client-go/v2 v2.3.1-0.20210518120617-5d1fff431040/go.mod h1:vLNHdxTJkIf2mSLvGrpj8TCcISApPoXkaxP8g9uRlW8= github.com/influxdata/influxdb1-client v0.0.0-20191209144304-8bf82d3c094d/go.mod h1:qj24IKcXYK6Iy9ceXlo3Tc+vtHo9lIhSX5JddghvEPo= github.com/influxdata/influxql v1.1.0/go.mod h1:KpVI7okXjK6PRi3Z5B+mtKZli+R1DnZgb3N+tzevNgo= github.com/influxdata/influxql v1.1.1-0.20210223160523-b6ab99450c93/go.mod h1:gHp9y86a/pxhjJ+zMjNXiQAA197Xk9wLxaz+fGG+kWk= github.com/influxdata/line-protocol v0.0.0-20180522152040-32c6aa80de5e/go.mod h1:4kt73NQhadE3daL3WhR5EJ/J2ocX0PZzwxQ0gXJ7oFE= -github.com/influxdata/pkg-config v0.2.6/go.mod h1:EMS7Ll0S4qkzDk53XS3Z72/egBsPInt+BeRxb0WeSwk= -github.com/influxdata/pkg-config v0.2.7/go.mod h1:EMS7Ll0S4qkzDk53XS3Z72/egBsPInt+BeRxb0WeSwk= +github.com/influxdata/line-protocol v0.0.0-20200327222509-2487e7298839/go.mod h1:xaLFMmpvUxqXtVkUJfg9QmT88cDaCJ3ZKgdZ78oO8Qo= +github.com/influxdata/pkg-config v0.2.8/go.mod h1:EMS7Ll0S4qkzDk53XS3Z72/egBsPInt+BeRxb0WeSwk= github.com/influxdata/promql/v2 v2.12.0/go.mod h1:fxOPu+DY0bqCTCECchSRtWfc+0X19ybifQhZoQNF5D8= github.com/influxdata/roaring v0.4.13-0.20180809181101-fc520f41fab6/go.mod h1:bSgUQ7q5ZLSO+bKBGqJiCBGAl+9DxyW63zLTujjUlOE= github.com/influxdata/tdigest v0.0.0-20181121200506-bf2b5ad3c0a9/go.mod h1:Js0mqiSBE6Ffsg94weZZ2c+v/ciT8QRHFOap7EKDrR0= github.com/influxdata/tdigest v0.0.2-0.20210216194612-fc98d27c9e8b/go.mod h1:Z0kXnxzbTC2qrx4NaIzYkE1k66+6oEDQTvL95hQFh5Y= github.com/influxdata/usage-client v0.0.0-20160829180054-6d3895376368/go.mod h1:Wbbw6tYNvwa5dlB6304Sd+82Z3f7PmVZHVKU637d4po= +github.com/j-keck/arping v0.0.0-20160618110441-2cf9dc699c56/go.mod h1:ymszkNOg6tORTn+6F6j+Jc8TOr5osrynvN6ivFWZ2GA= github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= github.com/jessevdk/go-flags v1.5.0/go.mod h1:Fw0T6WPc1dYxT4mKEZRfG5kJhaTDP9pj1c2EWnYs/m4= github.com/jhump/protoreflect v1.6.1 h1:4/2yi5LyDPP7nN+Hiird1SAJ6YoxUm13/oxHGRnbPd8= github.com/jhump/protoreflect v1.6.1/go.mod h1:RZQ/lnuN+zqeRVpQigTwO6o0AJUkxbnSnpuG7toUTG4= +github.com/jmespath/go-jmespath v0.0.0-20160202185014-0b12d6b521d8/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k= +github.com/jmespath/go-jmespath v0.0.0-20160803190731-bd40a432e4c7/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k= github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k= github.com/jmespath/go-jmespath v0.3.0/go.mod h1:9QtRXoHjLGCJ5IBSaohpXITPlowMeeYCZ7fLUTSywik= github.com/jmespath/go-jmespath v0.4.0 h1:BEgLn5cpjn8UN1mAw4NjwDrS35OdebyEtFe+9YPoQUg= @@ -797,6 +996,8 @@ github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/compress v1.4.0/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.9.5/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= +github.com/klauspost/compress v1.11.3/go.mod h1:aoV0uJVorq1K+umq18yTdKaF57EivdYsUV+/s2qKfXs= +github.com/klauspost/compress v1.11.13/go.mod h1:aoV0uJVorq1K+umq18yTdKaF57EivdYsUV+/s2qKfXs= github.com/klauspost/compress v1.14.1/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= github.com/klauspost/compress v1.14.2 h1:S0OHlFk/Gbon/yauFJ4FfJJF5V0fc5HbBTJazi28pRw= github.com/klauspost/compress v1.14.2/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= @@ -805,6 +1006,8 @@ github.com/klauspost/crc32 v0.0.0-20161016154125-cb6bfca970f6/go.mod h1:+ZoRqAPR github.com/klauspost/pgzip v1.0.2-0.20170402124221-0bf5dcad4ada/go.mod h1:Ch1tH69qFZu15pkjo5kYi6mth2Zzwzt50oCQKQE9RUs= github.com/knadh/koanf v1.4.0 h1:/k0Bh49SqLyLNfte9r6cvuZWrApOQhglOmhIU3L/zDw= github.com/knadh/koanf v1.4.0/go.mod h1:1cfH5223ZeZUOs8FU2UdTmaNfHpqgtjV0+NHjRO43gs= +github.com/kolo/xmlrpc v0.0.0-20201022064351-38db28db192b h1:iNjcivnc6lhbvJA3LD622NPrUponluJrBWPIwGG/3Bg= +github.com/kolo/xmlrpc v0.0.0-20201022064351-38db28db192b/go.mod h1:pcaDhQK0/NJZEvtCO0qQPPropqV0sJOJ6YW7X+9kRwM= 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.2/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= @@ -822,8 +1025,11 @@ github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= github.com/kylelemons/godebug v0.0.0-20160406211939-eadb3ce320cb/go.mod h1:B69LEHPfb2qLo0BaaOLcbitczOKLWTsrBG9LczfCD4k= github.com/kylelemons/godebug v1.1.0/go.mod h1:9/0rRGxNHcop5bhtWyNeEfOS8JIWk580+fNqagV/RAw= +github.com/labstack/echo/v4 v4.2.1/go.mod h1:AA49e0DZ8kk5jTOOCKNuPR6oTnBS0dYiM4FW1e6jwpg= +github.com/labstack/gommon v0.3.0/go.mod h1:MULnywXg0yavhxWKc+lOruYdAhDwPK9wf0OL7NoOu+k= github.com/leanovate/gopter v0.2.8 h1:eFPtJ3aa5zLfbxGROSNY75T9Dume60CWBAqoWQ3h/ig= github.com/leanovate/gopter v0.2.8/go.mod h1:gNcbPWNEWRe4lm+bycKqxUYoH5uoVje5SkOJ3uoLer8= +github.com/leodido/go-urn v1.1.0/go.mod h1:+cyI34gQWZcE1eQU7NVgKkkzdXDQHr1dBMtdAPozLkw= github.com/leodido/go-urn v1.2.1 h1:BqpAaACuzVSgi/VLzGZIobT2z4v53pjosyNd9Yv6n/w= github.com/leodido/go-urn v1.2.1/go.mod h1:zt4jvISO2HfUBqxjfIshjdMTYS56ZS/qv49ictyFfxY= github.com/lib/pq v0.0.0-20180327071824-d34b9ff171c2/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= @@ -834,8 +1040,9 @@ github.com/lightstep/lightstep-tracer-common/golang/gogo v0.0.0-20190605223551-b github.com/lightstep/lightstep-tracer-common/golang/gogo v0.0.0-20190605223551-bc2310a04743/go.mod h1:qklhhLq1aX+mtWk9cPHPzaBjWImj5ULL6C7HFJtXQMM= github.com/lightstep/lightstep-tracer-go v0.18.1 h1:vi1F1IQ8N7hNWytK9DpJsUfQhGuNSc19z330K6vl4zk= github.com/lightstep/lightstep-tracer-go v0.18.1/go.mod h1:jlF1pusYV4pidLvZ+XD0UBX0ZE6WURAspgAczcDHrL4= -github.com/linode/linodego v0.28.5 h1:JaCziTxHJ7a01MYyjHqskRc8zXQxXOddwrDeoQ2rBnw= -github.com/linode/linodego v0.28.5/go.mod h1:BR0gVkCJffEdIGJSl6bHR80Ty+Uvg/2jkjmrWaFectM= +github.com/linode/linodego v1.1.0 h1:ZiFVUptlzuExtUbHZtXiN7I0dAOFQAyirBKb/6/n9n4= +github.com/linode/linodego v1.1.0/go.mod h1:x/7+BoaKd4unViBmS2umdjYyVAmpFtBtEXZ0wou7FYQ= +github.com/lyft/protoc-gen-star v0.5.1/go.mod h1:9toiA3cC7z5uVbODF7kEQ91Xn7XNFkVUl+SrEe+ZORU= github.com/lyft/protoc-gen-star v0.5.3/go.mod h1:V0xaHgaf5oCCqmcxYcWiDfTiKsZsRc87/1qhoTACD8w= github.com/m3db/bitset v2.0.0+incompatible h1:wMgri1Z2QSwJ8K/7ZuV7vE4feLOT7EofVC8RakIOybI= github.com/m3db/bitset v2.0.0+incompatible/go.mod h1:X8CCqZmZxs2O6d4qHhiqtAKCin4G5mScPhiwX9rsc5c= @@ -843,12 +1050,21 @@ github.com/m3db/bloom v3.0.1+incompatible h1:EILDlnoiPKJAGtg3RCm7Zf5VIpwh+hgcHok github.com/m3db/bloom v3.0.1+incompatible/go.mod h1:W6XzpFw4t+CIYq+NGyp5c2394YsUc1P1+W/KAWty2lU= github.com/m3db/bloom/v4 v4.0.0-20200901140942-52efb8544fe9 h1:H5Iznc9FI44Sekos8STE+Hj2cPohzaYfWTUHf77q6RY= github.com/m3db/bloom/v4 v4.0.0-20200901140942-52efb8544fe9/go.mod h1:JDmGHlO6ygyY1V9eOHtXiNl3+axznDTrBqwWEeWALlQ= -github.com/m3db/prometheus_client_golang v0.8.1 h1:t7w/tcFws81JL1j5sqmpqcOyQOpH4RDOmIe3A3fdN3w= -github.com/m3db/prometheus_client_golang v0.8.1/go.mod h1:8R/f1xYhXWq59KD/mbRqoBulXejss7vYtYzWmruNUwI= -github.com/m3db/prometheus_client_model v0.0.0-20180517145114-8b2299a4bf7d h1:BtNPRz2kmh42OAVYjsOBjty2wE0FaIh7aPVtFNPcF48= -github.com/m3db/prometheus_client_model v0.0.0-20180517145114-8b2299a4bf7d/go.mod h1:Qfsxn+LypxzF+lNhak7cF7k0zxK7uB/ynGYoj80zcD4= -github.com/m3db/prometheus_common v0.0.0-20180517030744-25aaa3dff79b h1:DEtcqizQ9PnY4xudqHyze5F89jijy33fnPYYSCCYPXY= -github.com/m3db/prometheus_common v0.0.0-20180517030744-25aaa3dff79b/go.mod h1:EBmDQaMAy4B8i+qsg1wMXAelLNVbp49i/JOeVszQ/rs= +github.com/m3db/prometheus_client_golang v0.9.0-pre1/go.mod h1:8R/f1xYhXWq59KD/mbRqoBulXejss7vYtYzWmruNUwI= +github.com/m3db/prometheus_client_golang v1.12.5-0.20220603164556-514aad459573/go.mod h1:MeqtPbBp1E+3tgmtttxmGnfCWBY7TO6pV//6of0qSZs= +github.com/m3db/prometheus_client_golang v1.12.5-0.20220603165239-04931ef271c6/go.mod h1:dAdsbJi18Ojkdapu5kjwmhDOFr5/y1yzE2IwjthkUeg= +github.com/m3db/prometheus_client_golang v1.12.6/go.mod h1:ufdbllybhIqHdJ29pveDS1nb8vOWzCYR/ziHhlCa2Go= +github.com/m3db/prometheus_client_golang v1.12.7/go.mod h1:1TrZ6IcZluZnYH6eXRi6Ovb877dHOY/I7+CfI6h3TCE= +github.com/m3db/prometheus_client_golang v1.12.8 h1:DAE5sN2IH0FJZx5KvnuVksB147ac9Dt8gmi8h4A7VcU= +github.com/m3db/prometheus_client_golang v1.12.8/go.mod h1:1TrZ6IcZluZnYH6eXRi6Ovb877dHOY/I7+CfI6h3TCE= +github.com/m3db/prometheus_client_model v0.2.1 h1:XsdCsa4GZl8oSbdIhmp+9EvD1G8ZzvStkf7uhi8wfNs= +github.com/m3db/prometheus_client_model v0.2.1/go.mod h1:Pi6e2aN58HDdgTT+1EsLoW0Qzb1Dqm5P6C2Esd36skk= +github.com/m3db/prometheus_common v0.34.5-0.20220603164409-76eb9b921650/go.mod h1:MADWGTus+fO9f/xDPgXyLCB2dFN3w7EpnbaMAI5nxr0= +github.com/m3db/prometheus_common v0.34.5-0.20220603164832-70f043edf99e/go.mod h1:Pf+7yPHChEsThLFaUB/ZD0/KxMLK+UYW/QVlWJ1uAyY= +github.com/m3db/prometheus_common v0.34.5/go.mod h1:PlGkYJxacZY9hjtZh9SyjK+CR3aHh9nm1jcHfUUvojI= +github.com/m3db/prometheus_common v0.34.6/go.mod h1:y0334xwXc0gsrWoQfffLgbmx9TTw2BkrrhBC7iz3zEs= +github.com/m3db/prometheus_common v0.34.7 h1:QMABInp3m8ySuoY0gkSJ9rsjZmEemD8Fui7Ggj4z0rs= +github.com/m3db/prometheus_common v0.34.7/go.mod h1:oQjNl5G6gyqrFlOiMKO911HUPgZQkmHsXrw0iBiuQJs= github.com/m3db/prometheus_procfs v0.8.1 h1:LsxWzVELhDU9sLsZTaFLCeAwCn7bC7qecZcK4zobs/g= github.com/m3db/prometheus_procfs v0.8.1/go.mod h1:N8lv8fLh3U3koZx1Bnisj60GYUMDpWb09x1R+dmMOJo= github.com/m3db/stackadler32 v0.0.0-20180104200216-bfebcd73ef6f h1:+FWCOZjB96lBc0L7lvz6/TW1gF5fEVd6JQNQG59CpZE= @@ -878,15 +1094,21 @@ github.com/mailru/easyjson v0.7.1/go.mod h1:KAzv3t3aY1NaHWoQz1+4F1ccyAH66Jk7yos7 github.com/mailru/easyjson v0.7.6/go.mod h1:xzfreul335JAWq5oZzymOObrkdz5UnU4kGfJJLY9Nlc= github.com/markbates/oncer v0.0.0-20181203154359-bf2de49a0be2/go.mod h1:Ld9puTsIW75CHf65OeIOkyKbteujpZVXDpWK6YGZbxE= github.com/markbates/safe v1.0.1/go.mod h1:nAqgmRi7cY2nqMc92/bSEeQA+R4OheNU2T1kNSCBdG0= +github.com/marstr/guid v1.1.0/go.mod h1:74gB1z2wpxxInTG6yaqA7KrtM0NZ+RbrcqDvYHefzho= +github.com/matryer/moq v0.0.0-20190312154309-6cfb0558e1bd/go.mod h1:9ELz6aaclSIGnZBoaSLZ3NAl1VTufbOrXBPvtcy6WiQ= github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= +github.com/mattn/go-colorable v0.1.2/go.mod h1:U0ppj6V5qS13XJ6of8GYAs25YV2eR4EVcfRqFIhoBtE= github.com/mattn/go-colorable v0.1.4/go.mod h1:U0ppj6V5qS13XJ6of8GYAs25YV2eR4EVcfRqFIhoBtE= github.com/mattn/go-colorable v0.1.6/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= +github.com/mattn/go-colorable v0.1.7/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= +github.com/mattn/go-colorable v0.1.8/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= github.com/mattn/go-colorable v0.1.9/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= github.com/mattn/go-colorable v0.1.12 h1:jF+Du6AlPIjs2BiUiQlKOX0rt3SujHxPnksPKZbaA40= github.com/mattn/go-colorable v0.1.12/go.mod h1:u5H1YNBxpqRaxsYJYSkiCWKzEfiAb1Gb520KVy5xxl4= github.com/mattn/go-isatty v0.0.3/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= github.com/mattn/go-isatty v0.0.4/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= github.com/mattn/go-isatty v0.0.8/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= +github.com/mattn/go-isatty v0.0.9/go.mod h1:YNRxwqDuOph6SZLI9vUUz6OYw3QyUt7WiY2yME+cCiQ= github.com/mattn/go-isatty v0.0.10/go.mod h1:qgIWMr58cqv1PHHyhnkY9lrL7etaEgOFcMEpPG5Rm84= github.com/mattn/go-isatty v0.0.11/go.mod h1:PhnuNfih5lzO57/f3n+odYbM4JtupLOxQOAqxQCu2WE= github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU= @@ -894,10 +1116,12 @@ github.com/mattn/go-isatty v0.0.14 h1:yVuAays6BHfxijgZPzw+3Zlu5yQgKGP2/hcQbHb7S9 github.com/mattn/go-isatty v0.0.14/go.mod h1:7GGIvUiUoEMVVmxf/4nioHXj79iQHKdU27kJ6hsGG94= github.com/mattn/go-runewidth v0.0.2/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= github.com/mattn/go-runewidth v0.0.3/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= +github.com/mattn/go-shellwords v1.0.3/go.mod h1:3xCvwCdWdlDJUrvuMn7Wuy9eWs4pE8vqg+NOMyg4B2o= github.com/mattn/go-sqlite3 v1.11.0/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= github.com/mattn/go-tty v0.0.0-20180907095812-13ff1204f104/go.mod h1:XPvLUNfbS4fJH25nqRHfWLMa1ONC8Amw+mIA639KxkE= -github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= +github.com/matttproud/golang_protobuf_extensions v1.0.2-0.20181231171920-c182affec369 h1:I0XW9+e1XWDxdcEniV4rQAIOPUGDq67JSCiRCgGCZLI= +github.com/matttproud/golang_protobuf_extensions v1.0.2-0.20181231171920-c182affec369/go.mod h1:BSXmuO+STAnVfrANrmjBb36TMTDstsz7MSK+HVaYKv4= github.com/mauricelam/genny v0.0.0-20180903214747-eb2c5232c885 h1:nCU/HIvsORu8nlebFTTkEpxao5zA/yt5Y4yQccm34bM= github.com/mauricelam/genny v0.0.0-20180903214747-eb2c5232c885/go.mod h1:wRyVMWiOZeVj+MieWS5tIBBtJ3RtqqMbPsA5Z+t5b5U= github.com/miekg/dns v1.0.14/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg= @@ -905,9 +1129,11 @@ github.com/miekg/dns v1.1.22/go.mod h1:bPDLeHnStXmXAq1m/Ch/hvfNHr14JKNPMBo3VZKju github.com/miekg/dns v1.1.26/go.mod h1:bPDLeHnStXmXAq1m/Ch/hvfNHr14JKNPMBo3VZKjuso= github.com/miekg/dns v1.1.29/go.mod h1:KNUDUusw/aVsxyTYZM1oqvCicbwhgbNgztCETuNZ7xM= github.com/miekg/dns v1.1.41/go.mod h1:p6aan82bvRIyn+zDIv9xYNUpwa73JcSh9BKwknJysuI= -github.com/miekg/dns v1.1.42 h1:gWGe42RGaIqXQZ+r3WUGEKBEtvPHY2SXo4dqixDNxuY= -github.com/miekg/dns v1.1.42/go.mod h1:+evo5L0630/F6ca/Z9+GAqzhjGyn8/c+TBaOyfEl0V4= +github.com/miekg/dns v1.1.43 h1:JKfpVSCB84vrAmHzyrsxB5NAr5kLoMXZArPSw7Qlgyg= +github.com/miekg/dns v1.1.43/go.mod h1:+evo5L0630/F6ca/Z9+GAqzhjGyn8/c+TBaOyfEl0V4= +github.com/miekg/pkcs11 v1.0.3/go.mod h1:XsNlhZGX73bx86s2hdc/FuaLm2CPZJemRLMA+WTFxgs= github.com/mileusna/useragent v0.0.0-20190129205925-3e331f0949a5/go.mod h1:JWhYAp2EXqUtsxTKdeGlY8Wp44M7VxThC9FEoNGi2IE= +github.com/mistifyio/go-zfs v2.1.2-0.20190413222219-f784269be439+incompatible/go.mod h1:8AuVvqP/mXw1px98n46wfvcGfQ4ci2FwoAjKYxuo3Z4= github.com/mitchellh/cli v1.0.0/go.mod h1:hNIlj7HEI86fIcpObd7a0FcrxTWetlwJDGcceTlRvqc= github.com/mitchellh/cli v1.1.0/go.mod h1:xcISNoH86gajksDmfB23e/pu+B+GeFRMYmoHXxx3xhI= github.com/mitchellh/colorstring v0.0.0-20190213212951-d06e56a500db/go.mod h1:l0dey0ia/Uv7NcFFVbCLtqEBQbrT4OCwCSKTEv6enCw= @@ -931,13 +1157,19 @@ github.com/mitchellh/mapstructure v1.4.0/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RR github.com/mitchellh/mapstructure v1.4.1/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo= github.com/mitchellh/mapstructure v1.4.3 h1:OVowDSCllw/YjdLkam3/sm7wEtOy59d8ndGgCcyj8cs= github.com/mitchellh/mapstructure v1.4.3/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo= +github.com/mitchellh/osext v0.0.0-20151018003038-5e2d6d41470f/go.mod h1:OkQIRizQZAeMln+1tSwduZz7+Af5oFlKirV/MSYes2A= github.com/mitchellh/reflectwalk v1.0.0/go.mod h1:mSTlrgnPZtwu0c4WaC2kGObEpuNDbx0jmZXqmk4esnw= github.com/mitchellh/reflectwalk v1.0.2 h1:G2LzWKi524PWgd3mLHV8Y5k7s6XUvT0Gef6zxSIeXaQ= github.com/mitchellh/reflectwalk v1.0.2/go.mod h1:mSTlrgnPZtwu0c4WaC2kGObEpuNDbx0jmZXqmk4esnw= +github.com/moby/locker v1.0.1/go.mod h1:S7SDdo5zpBK84bzzVlKr2V0hz+7x9hWbYC/kq7oQppc= github.com/moby/spdystream v0.2.0/go.mod h1:f7i0iNDQJ059oMTcWxx8MA/zKFIuD/lY+0GqbN2Wy8c= +github.com/moby/sys/mountinfo v0.4.0/go.mod h1:rEr8tzG/lsIZHBtN/JjGG+LMYx9eXgW2JI+6q0qou+A= +github.com/moby/sys/mountinfo v0.4.1/go.mod h1:rEr8tzG/lsIZHBtN/JjGG+LMYx9eXgW2JI+6q0qou+A= +github.com/moby/sys/symlink v0.1.0/go.mod h1:GGDODQmbFOjFsXvfLVn3+ZRxkch54RkSiGqsZeMYowQ= +github.com/moby/term v0.0.0-20200312100748-672ec06f55cd/go.mod h1:DdlQx2hp0Ss5/fLikoLlEeIYiATotOjgB//nb973jeo= github.com/moby/term v0.0.0-20200915141129-7f0af18e79f2/go.mod h1:TjQg8pa4iejrUrjiz0MCtMV38jdMNW4doKSiBrEvCQQ= -github.com/moby/term v0.0.0-20201216013528-df9cb8a40635 h1:rzf0wL0CHVc8CEsgyygG0Mn9CNCCPZqOPaz8RiiHYQk= -github.com/moby/term v0.0.0-20201216013528-df9cb8a40635/go.mod h1:FBS0z0QWA44HXygs7VXDUOGoN/1TV3RuWkLO04am3wc= +github.com/moby/term v0.0.0-20210619224110-3f7ff695adc6 h1:dcztxKSvZ4Id8iPpHERQBbIJfabdt4wUm5qy3wOL2Zc= +github.com/moby/term v0.0.0-20210619224110-3f7ff695adc6/go.mod h1:E2VnQOmVuvZB6UYnnDB0qG5Nq/1tD9acaOpo6xmt0Kw= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= @@ -949,9 +1181,11 @@ github.com/montanaflynn/stats v0.0.0-20171201202039-1bf9dbcd8cbe/go.mod h1:wL8QJ github.com/morikuni/aec v1.0.0/go.mod h1:BbKIizmSmc5MMPqRYbxO4ZU0S0+P200+tUnFx7PXmsc= github.com/mostynb/go-grpc-compression v1.1.16 h1:D9tGUINmcII049pxOj9dl32Fzhp26TrDVQXECoKJqQg= github.com/mostynb/go-grpc-compression v1.1.16/go.mod h1:xxa6UoYynYS2h+5HB/Hglu81iYAp87ARaNmhhwi0s1s= +github.com/mrunalp/fileutils v0.5.0/go.mod h1:M1WthSahJixYnrXQl/DFQuteStB1weuxD2QJNHXfbSQ= github.com/mschoch/smat v0.0.0-20160514031455-90eadee771ae h1:VeRdUYdCw49yizlSbMEn2SZ+gT+3IUKx8BqxyQdz+BY= github.com/mschoch/smat v0.0.0-20160514031455-90eadee771ae/go.mod h1:qAyveg+e4CE+eKJXWVjKXM4ck2QobLqTDytGJbLLhJg= github.com/munnerz/goautoneg v0.0.0-20120707110453-a547fc61f48d/go.mod h1:+n7T8mK8HuQTcFwEeznm/DIxMOiR9yIdICNftLE1DvQ= +github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822/go.mod h1:+n7T8mK8HuQTcFwEeznm/DIxMOiR9yIdICNftLE1DvQ= 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 h1:KUppIJq7/+SVif2QVs3tOP0zanoHgBEVAwHxUSIzRqU= github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= @@ -963,6 +1197,7 @@ github.com/nats-io/nats.go v1.9.1/go.mod h1:ZjDU1L/7fJ09jvUSRVBR2e7+RnLiiIQyqyzE github.com/nats-io/nkeys v0.1.0/go.mod h1:xpnFELMwJABBLVhffcfd1MZx6VsNRFpEugbxziKVo7w= github.com/nats-io/nkeys v0.1.3/go.mod h1:xpnFELMwJABBLVhffcfd1MZx6VsNRFpEugbxziKVo7w= github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OSON2c= +github.com/ncw/swift v1.0.47/go.mod h1:23YIA4yWVnGwv2dQlN4bB7egfYX6YLn0Yo/S6zZO/ZM= github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLANZcx1PVRCS0qkT7pwLkGfwJo4zjcN/Tysno= github.com/npillmayer/nestext v0.1.3/go.mod h1:h2lrijH8jpicr25dFY+oAJLyzlya6jhnuG+zWp9L0Uk= github.com/nxadm/tail v1.4.4 h1:DQuhQpB1tVlglWS2hLQ5OV6B5r8aGxSrPc5Qo6uTN78= @@ -973,31 +1208,55 @@ github.com/oklog/run v1.1.0/go.mod h1:sVPdnTZT1zYwAJeCMu2Th4T21pA3FPOQRfWjQlk7DV github.com/oklog/ulid v1.3.1 h1:EGfNDEx6MqHz8B3uNV6QAib1UR2Lm97sHi3ocA6ESJ4= github.com/oklog/ulid v1.3.1/go.mod h1:CirwcVhetQ6Lv90oh/F+FBtV6XMibvdAFo93nm5qn4U= github.com/olekukonko/tablewriter v0.0.0-20170122224234-a0225b3f23b5/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXWMclvtLoiIKAKIo= +github.com/onsi/ginkgo v0.0.0-20151202141238-7f8ab55aaf3b/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v0.0.0-20170829012221-11459a886d9c/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.7.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.8.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.10.1/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= +github.com/onsi/ginkgo v1.10.3/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.11.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.12.1/go.mod h1:zj2OWP4+oCPe1qIXoGWkgMRwljMUYCdkwsT2108oapk= +github.com/onsi/ginkgo v1.14.0/go.mod h1:iSB4RoI2tjJc9BBv4NKIKWKya62Rps+oPG/Lv9klQyY= github.com/onsi/ginkgo v1.14.2 h1:8mVmC9kjFFmA8H4pKMUhcblgifdkOIXPvbhN1T36q1M= github.com/onsi/ginkgo v1.14.2/go.mod h1:iSB4RoI2tjJc9BBv4NKIKWKya62Rps+oPG/Lv9klQyY= +github.com/onsi/gomega v0.0.0-20151007035656-2152b45fa28a/go.mod h1:C1qb7wdrVGGVU+Z6iS04AVkA3Q65CEZX59MT0QO5uiA= github.com/onsi/gomega v0.0.0-20170829124025-dcabb60a477c/go.mod h1:C1qb7wdrVGGVU+Z6iS04AVkA3Q65CEZX59MT0QO5uiA= github.com/onsi/gomega v1.4.3/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= github.com/onsi/gomega v1.5.0/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= github.com/onsi/gomega v1.7.0/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7JYyY= github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo= +github.com/onsi/gomega v1.10.3/go.mod h1:V9xEwhxec5O8UDM77eCW8vLymOMltsqPVYWrpDsH8xc= github.com/onsi/gomega v1.10.4 h1:NiTx7EEvBzu9sFOD1zORteLSt3o8gnlvZZwSE9TnY9U= github.com/onsi/gomega v1.10.4/go.mod h1:g/HbgYopi++010VEqkFgJHKC09uJiW9UkXvMUuKHUCQ= github.com/op/go-logging v0.0.0-20160315200505-970db520ece7/go.mod h1:HzydrMdWErDVzsI23lYNej1Htcns9BCg93Dk0bBINWk= +github.com/opencontainers/go-digest v0.0.0-20170106003457-a6d0ee40d420/go.mod h1:cMLVZDEM3+U2I4VmLI6N8jQYUd2OVphdqWwCJHrFt2s= +github.com/opencontainers/go-digest v0.0.0-20180430190053-c9281466c8b2/go.mod h1:cMLVZDEM3+U2I4VmLI6N8jQYUd2OVphdqWwCJHrFt2s= github.com/opencontainers/go-digest v1.0.0-rc1/go.mod h1:cMLVZDEM3+U2I4VmLI6N8jQYUd2OVphdqWwCJHrFt2s= +github.com/opencontainers/go-digest v1.0.0-rc1.0.20180430190053-c9281466c8b2/go.mod h1:cMLVZDEM3+U2I4VmLI6N8jQYUd2OVphdqWwCJHrFt2s= github.com/opencontainers/go-digest v1.0.0 h1:apOUWs51W5PlhuyGyz9FCeeBIOUDA/6nW8Oi/yOhh5U= github.com/opencontainers/go-digest v1.0.0/go.mod h1:0JzlMkj0TRzQZfJkVvzbP0HBR3IKzErnv2BNG4W4MAM= +github.com/opencontainers/image-spec v1.0.0/go.mod h1:BtxoFyWECRxE4U/7sNtV5W15zMzWCbyJoFRP3s7yZA0= github.com/opencontainers/image-spec v1.0.1 h1:JMemWkRwHx4Zj+fVxWoMCFm/8sYGGrUVojFA6h/TRcI= github.com/opencontainers/image-spec v1.0.1/go.mod h1:BtxoFyWECRxE4U/7sNtV5W15zMzWCbyJoFRP3s7yZA0= -github.com/opencontainers/runc v1.0.0-rc9 h1:/k06BMULKF5hidyoZymkoDCzdJzltZpz/UU4LguQVtc= +github.com/opencontainers/runc v0.0.0-20190115041553-12f6a991201f/go.mod h1:qT5XzbpPznkRYVz/mWwUaVBUv2rmF59PVA73FjuZG0U= +github.com/opencontainers/runc v0.1.1/go.mod h1:qT5XzbpPznkRYVz/mWwUaVBUv2rmF59PVA73FjuZG0U= +github.com/opencontainers/runc v1.0.0-rc8.0.20190926000215-3e425f80a8c9/go.mod h1:qT5XzbpPznkRYVz/mWwUaVBUv2rmF59PVA73FjuZG0U= github.com/opencontainers/runc v1.0.0-rc9/go.mod h1:qT5XzbpPznkRYVz/mWwUaVBUv2rmF59PVA73FjuZG0U= +github.com/opencontainers/runc v1.0.0-rc93/go.mod h1:3NOsor4w32B2tC0Zbl8Knk4Wg84SM2ImC1fxBuqJ/H0= +github.com/opencontainers/runc v1.0.2 h1:opHZMaswlyxz1OuGpBE53Dwe4/xF7EZTY0A2L/FpCOg= +github.com/opencontainers/runc v1.0.2/go.mod h1:aTaHFFwQXuA71CiyxOdFFIorAoemI04suvGRQFzWTD0= +github.com/opencontainers/runtime-spec v0.1.2-0.20190507144316-5b71a03e2700/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0= +github.com/opencontainers/runtime-spec v1.0.1/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0= +github.com/opencontainers/runtime-spec v1.0.2-0.20190207185410-29686dbc5559/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0= +github.com/opencontainers/runtime-spec v1.0.2/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0= +github.com/opencontainers/runtime-spec v1.0.3-0.20200929063507-e6143ca7d51d/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0= +github.com/opencontainers/runtime-spec v1.0.3-0.20210326190908-1c3f411f0417/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0= +github.com/opencontainers/runtime-tools v0.0.0-20181011054405-1d69bd0f9c39/go.mod h1:r3f7wjNzSs2extwzU3Y+6pKfobzPh+kKFJ3ofN+3nfs= +github.com/opencontainers/selinux v1.6.0/go.mod h1:VVGKuOLlE7v4PJyT6h7mNWvq1rzqiriPsEqVhc+svHE= +github.com/opencontainers/selinux v1.8.0/go.mod h1:RScLhm78qiWa2gbVCcGkC7tCGdgk3ogry1nUQF8Evvo= +github.com/opencontainers/selinux v1.8.2/go.mod h1:MUIHuUEvKB1wtJjQdOyYRgOnLD2xAPP8dBsCoU0KuF8= github.com/opentracing-contrib/go-observer v0.0.0-20170622124052-a52f23424492/go.mod h1:Ngi6UdF0k5OKD5t5wlmGhe/EDKPoUM3BXZSSfIuJbis= github.com/opentracing-contrib/go-stdlib v0.0.0-20190519235532-cf7a6c988dc9/go.mod h1:PLldrQSroqzH70Xl+1DQcGnefIbqsKR7UDaiux3zV+w= github.com/opentracing-contrib/go-stdlib v1.0.0 h1:TBS7YuVotp8myLon4Pv7BtCBzOTo1DeZCld0Z63mW2w= @@ -1025,6 +1284,7 @@ github.com/pborman/uuid v1.2.0/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtP github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic= github.com/pelletier/go-toml v1.4.0/go.mod h1:PN7xzY2wHTK0K9p34ErDQMlFxa51Fk0OUruD3k1mMwo= github.com/pelletier/go-toml v1.7.0/go.mod h1:vwGMzjaWMwyfHwgIBhI2YUM4fB6nL6lVAvS1LBMMhTE= +github.com/pelletier/go-toml v1.8.1/go.mod h1:T2/BmBdy8dvIRq1a/8aqjN41wvWlN4lrapLU/GW4pbc= github.com/pelletier/go-toml v1.9.4 h1:tjENF6MfZAg8e4ZmZTeWaWiT2vXtsoO6+iuOjFhECwM= github.com/pelletier/go-toml v1.9.4/go.mod h1:u1nR/EPcESfeI/szUZKdtJ0xRNbUoANCkoOuaOx1Y+c= github.com/performancecopilot/speed v3.0.0+incompatible/go.mod h1:/CLtqpZ5gBg1M9iaPbIdPPGyKcA8hKdoy6hAWba7Yac= @@ -1050,23 +1310,26 @@ github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZb github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/posener/complete v1.1.1/go.mod h1:em0nMJCgc9GFtwrmVmEMR/ZL6WyhyjMBndrE9hABlRI= github.com/posener/complete v1.2.3/go.mod h1:WZIdtGGp+qx0sLrYKtIRAruyNpv6hFCicSgv7Sy7s/s= +github.com/pquerna/cachecontrol v0.0.0-20171018203845-0dec1b30a021/go.mod h1:prYjPmNq4d1NPVmpShWobRqXY3q7Vp+80DqgxxUrUIA= github.com/prashantv/protectmem v0.0.0-20171002184600-e20412882b3a h1:AA9vgIBDjMHPC2McaGPojgV2dcI78ZC0TLNhYCXEKH8= github.com/prashantv/protectmem v0.0.0-20171002184600-e20412882b3a/go.mod h1:lzZQ3Noex5pfAy7mkAeCjcBDteYU85uWWnJ/y6gKU8k= github.com/prometheus/alertmanager v0.20.0/go.mod h1:9g2i48FAyZW6BtbsnvHtMHQXl2aVtrORKwKVCQ+nbrg= -github.com/prometheus/alertmanager v0.22.2/go.mod h1:rYinOWxFuCnNssc3iOjn2oMTlhLaPcUuqV5yk5JKUAE= +github.com/prometheus/alertmanager v0.23.0/go.mod h1:0MLTrjQI8EuVmvykEhcfr/7X0xmaDAZrqMgxIq3OXHk= +github.com/prometheus/client_golang v0.0.0-20180209125602-c332b6f63c06/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v0.9.3-0.20190127221311-3c4408c8b829/go.mod h1:p2iRAGwDERtqlqzRXnrOVns+ignqQo//hLXqYxZYVNs= github.com/prometheus/client_golang v0.9.3/go.mod h1:/TN21ttK/J9q6uSwhBd54HahCDft0ttaMvbicHlPoso= github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= +github.com/prometheus/client_golang v1.1.0/go.mod h1:I1FGZT9+L76gKKOs5djB6ezCbFQP1xR9D75/vuwEF3g= github.com/prometheus/client_golang v1.2.1/go.mod h1:XMU6Z2MjaRKVu/dC1qupJI9SiNkDYzz3xecMgSW/F+U= github.com/prometheus/client_golang v1.3.0/go.mod h1:hJaj2vgQTGQmVCsAACORcieXFeDPbaTKGT+JTgUa3og= github.com/prometheus/client_golang v1.4.0/go.mod h1:e9GMxYsXl05ICDXkRhurwBS4Q3OK1iX/F2sw+iXX5zU= github.com/prometheus/client_golang v1.5.1/go.mod h1:e9GMxYsXl05ICDXkRhurwBS4Q3OK1iX/F2sw+iXX5zU= github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP4j5+D6mVACh+pe2M= -github.com/prometheus/client_golang v1.10.0/go.mod h1:WJM3cc3yu7XKBKa/I8WeZm+V3eltZnBwfENSU7mdogU= github.com/prometheus/client_golang v1.11.0/go.mod h1:Z6t4BnS23TR94PD6BsDNk8yVqroYurpAkEiz0P2BEV0= github.com/prometheus/client_golang v1.12.1 h1:ZiaPsmm9uiBeaSMRznKsCDNtPCS0T3JVDGF+06gjBzk= github.com/prometheus/client_golang v1.12.1/go.mod h1:3Z9XVyYiZYEO+YQWt3RD2R3jrbd179Rt297l4aS6nDY= +github.com/prometheus/client_model v0.0.0-20171117100541-99fa1f4be8e5/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190115171406-56726106282f/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= @@ -1074,6 +1337,7 @@ github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1: github.com/prometheus/client_model v0.1.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.2.0 h1:uq5h0d+GuxiXLJLNABMgp2qUWDPiLvgCzz2dUR+/W/M= github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/prometheus/common v0.0.0-20180110214958-89604d197083/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= github.com/prometheus/common v0.0.0-20181113130724-41aa239b4cce/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= github.com/prometheus/common v0.2.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.4.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= @@ -1082,28 +1346,34 @@ github.com/prometheus/common v0.6.0/go.mod h1:eBmuwkDJBwy6iBfxCBob6t6dR6ENT/y+J+ github.com/prometheus/common v0.7.0/go.mod h1:DjGbpBbp5NYNiECxcL/VnbXCCaQpKd3tt26CguLLsqA= github.com/prometheus/common v0.9.1/go.mod h1:yhUN8i9wzaXS3w1O07YhxHEBxD+W35wd8bs7vj7HSQ4= github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo= -github.com/prometheus/common v0.15.0/go.mod h1:U+gB1OBLb1lF3O42bTCL+FK18tX9Oar16Clt/msog/s= -github.com/prometheus/common v0.18.0/go.mod h1:U+gB1OBLb1lF3O42bTCL+FK18tX9Oar16Clt/msog/s= -github.com/prometheus/common v0.23.0/go.mod h1:H6QK/N6XVT42whUeIdI3dp36w49c+/iMDk7UAI2qm7Q= github.com/prometheus/common v0.26.0/go.mod h1:M7rCNAaPfAosfx8veZJCuw84e35h3Cfd9VFqTh1DIvc= github.com/prometheus/common v0.29.0/go.mod h1:vu+V0TpY+O6vW9J44gczi3Ap/oXXR10b+M/gUGO4Hls= -github.com/prometheus/common v0.32.1 h1:hWIdL3N2HoUx3B8j3YN9mWor0qhY/NlEKZEaXxuIRh4= +github.com/prometheus/common v0.30.0/go.mod h1:vu+V0TpY+O6vW9J44gczi3Ap/oXXR10b+M/gUGO4Hls= github.com/prometheus/common v0.32.1/go.mod h1:vu+V0TpY+O6vW9J44gczi3Ap/oXXR10b+M/gUGO4Hls= -github.com/prometheus/exporter-toolkit v0.5.1/go.mod h1:OCkM4805mmisBhLmVFw858QYi3v0wKdY6/UxrT0pZVg= +github.com/prometheus/common v0.34.0 h1:RBmGO9d/FVjqHT0yUGQwBJhkwKV+wPCn7KGpvfab0uE= +github.com/prometheus/common v0.34.0/go.mod h1:gB3sOl7P0TvJabZpLY5uQMpUqRCPPCyRLCZYc7JZTNE= +github.com/prometheus/common/sigv4 v0.1.0 h1:qoVebwtwwEhS85Czm2dSROY5fTo2PAPEVdDeppTwGX4= +github.com/prometheus/common/sigv4 v0.1.0/go.mod h1:2Jkxxk9yYvCkE5G1sQT7GuEXm57JrvHu9k5YwTjsNtI= +github.com/prometheus/exporter-toolkit v0.6.1/go.mod h1:ZUBIj498ePooX9t/2xtDjeQYwvRpiPP2lh5u4iblj2g= +github.com/prometheus/exporter-toolkit v0.7.0/go.mod h1:ZUBIj498ePooX9t/2xtDjeQYwvRpiPP2lh5u4iblj2g= +github.com/prometheus/procfs v0.0.0-20180125133057-cb4147076ac7/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20190117184657-bf6a532e95b1/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20190507164030-5867b95ac084/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= +github.com/prometheus/procfs v0.0.0-20190522114515-bc1a522cf7b1/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= +github.com/prometheus/procfs v0.0.3/go.mod h1:4A/X28fw3Fc593LaREMrKMqOKvUAntwMDaekg4FpcdQ= github.com/prometheus/procfs v0.0.5/go.mod h1:4A/X28fw3Fc593LaREMrKMqOKvUAntwMDaekg4FpcdQ= github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= github.com/prometheus/procfs v0.0.11/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= +github.com/prometheus/procfs v0.2.0/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 h1:4jVXhlkAyzOScmCkXBTOLRLTz8EeU+eyjrwB/EPq0VU= github.com/prometheus/procfs v0.7.3/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= github.com/prometheus/prometheus v0.0.0-20200609090129-a6600f564e3c/go.mod h1:S5n0C6tSgdnwWshBUceRx5G1OsjLv/EeZ9t3wIfEtsY= -github.com/prometheus/prometheus v1.8.2-0.20210621150501-ff58416a0b02 h1:waKRn/b6LBaXHjQ3dlZd+0li1nIykM34r5XEYr4lTBM= -github.com/prometheus/prometheus v1.8.2-0.20210621150501-ff58416a0b02/go.mod h1:fC6ROpjS/2o+MQTO7X8NSZLhLBSNlDzxaeDMqQm+TUM= +github.com/prometheus/prometheus v0.0.0-20211110084043-4ef8c7c1d8e4 h1:2sburFnqLR9B7BGhl/KFf94fJF7PYfLwwPHEdXbyXPI= +github.com/prometheus/prometheus v0.0.0-20211110084043-4ef8c7c1d8e4/go.mod h1:07FWuvRzfovrwH/yP4gxJesTNGOj1RWoBDIkgWfthjk= github.com/prometheus/tsdb v0.7.1/go.mod h1:qhTCs0VvXwvX/y3TZrWD7rabWM+ijKTux40TwIPHuXU= github.com/rakyll/statik v0.1.6 h1:uICcfUXpgqtw2VopbIncslhAmE5hwc4g20TEyEENBNs= github.com/rakyll/statik v0.1.6/go.mod h1:OEi9wJV/fMUAGx1eNjq75DKDsJVuEv1U0oYdX6GX8Zs= @@ -1118,7 +1388,7 @@ github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFR github.com/rogpeppe/go-internal v1.6.1 h1:/FiVV8dS/e+YqF2JvO3yXRFbBLTIuSDkuC7aBOAvL+k= github.com/rogpeppe/go-internal v1.6.1/go.mod h1:xXDCJY+GAPziupqXw64V24skbSoqbTEfhy4qGm1nDQc= github.com/rs/cors v1.6.0/go.mod h1:gFx+x8UowdsKA9AchylcLynDq+nNFfI8FkUZdN/jGCU= -github.com/rs/cors v1.7.0/go.mod h1:gFx+x8UowdsKA9AchylcLynDq+nNFfI8FkUZdN/jGCU= +github.com/rs/cors v1.8.0/go.mod h1:EBwu+T5AvHOcXwvZIkQFjUN6s8Czyqw12GL/Y0tUyRM= github.com/rs/cors v1.8.2 h1:KCooALfAYGs415Cwu5ABvv9n9509fSiG5SQJn/AQo4U= github.com/rs/cors v1.8.2/go.mod h1:XyqrcTp5zjWr1wsJ8PIRZssZ8b/WMcMf71DJnit4EMU= github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g= @@ -1127,6 +1397,7 @@ github.com/russross/blackfriday/v2 v2.1.0/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQD github.com/ryanuber/columnize v0.0.0-20160712163229-9b3edd62028f/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= github.com/ryanuber/columnize v2.1.0+incompatible/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= github.com/ryanuber/go-glob v1.0.0/go.mod h1:807d1WSdnB0XRJzKNil9Om6lcp/3a0v4qIHxIXzX/Yc= +github.com/safchain/ethtool v0.0.0-20190326074333-42ed695e3de8/go.mod h1:Z0q5wiBQGYcxhMZ6gUqHn6pYNLypFAvaL3UvgZLR0U4= github.com/sagikazarmark/crypt v0.3.0/go.mod h1:uD/D+6UF4SrIR1uGEv7bBNkNqLGqUr43MRiaGWX1Nig= github.com/samuel/go-zookeeper v0.0.0-20190923202752-2cc03de413da/go.mod h1:gi+0XIa01GRL2eRQVjQkKGqKF3SF9vZR/HnPullcV2E= github.com/satori/go.uuid v0.0.0-20160603004225-b111a074d5ef/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0= @@ -1138,6 +1409,7 @@ github.com/scaleway/scaleway-sdk-go v1.0.0-beta.7.0.20210223165440-c65ae3540d44/ github.com/schollz/progressbar/v2 v2.13.2/go.mod h1:6YZjqdthH6SCZKv2rqGryrxPtfmRB/DWZxSMfCXPyD8= github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529 h1:nn5Wsu0esKSJiIVhscUtVbo7ada43DJhG55ua/hjS5I= github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529/go.mod h1:DxrIzT+xaE7yg65j358z/aeFdxmN0P9QXhEzd20vsDc= +github.com/seccomp/libseccomp-golang v0.9.1/go.mod h1:GbW5+tmTXfcxTToHLXlScSlAvWlF4P2Ca7zGrPiEpWo= github.com/segmentio/kafka-go v0.1.0/go.mod h1:X6itGqS9L4jDletMsxZ7Dz+JFWxM6JHfPOCvTvk+EJo= github.com/segmentio/kafka-go v0.2.0/go.mod h1:X6itGqS9L4jDletMsxZ7Dz+JFWxM6JHfPOCvTvk+EJo= github.com/sergi/go-diff v1.0.0/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= @@ -1152,16 +1424,19 @@ github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeV github.com/shurcooL/vfsgen v0.0.0-20181202132449-6a9ea43bcacd/go.mod h1:TrYk7fJVaAttu97ZZKrO9UbRa8izdowaMIZcxYMbVaw= github.com/shurcooL/vfsgen v0.0.0-20200824052919-0d455de96546/go.mod h1:TrYk7fJVaAttu97ZZKrO9UbRa8izdowaMIZcxYMbVaw= github.com/sirupsen/logrus v1.0.4-0.20170822132746-89742aefa4b2/go.mod h1:pMByvHTf9Beacp5x1UXfOR9xyW/9antXMhjMPG0dEzc= +github.com/sirupsen/logrus v1.0.6/go.mod h1:pMByvHTf9Beacp5x1UXfOR9xyW/9antXMhjMPG0dEzc= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/sirupsen/logrus v1.4.1/go.mod h1:ni0Sbl8bgC9z8RoU9G6nDWqqs/fq4eDPysMBDgk/93Q= github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= +github.com/sirupsen/logrus v1.7.0/go.mod h1:yWOB1SBYBC5VeMP7gHvWumXLIWorT60ONWic61uBYv0= github.com/sirupsen/logrus v1.8.1 h1:dJKuHgqk1NNQlqoA6BTlM1Wf9DOH3NBjQyu0h9+AZZE= github.com/sirupsen/logrus v1.8.1/go.mod h1:yWOB1SBYBC5VeMP7gHvWumXLIWorT60ONWic61uBYv0= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= +github.com/smartystreets/goconvey v0.0.0-20190330032615-68dc04aab96a/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= -github.com/snowflakedb/gosnowflake v1.3.4/go.mod h1:NsRq2QeiMUuoNUJhp5Q6xGC4uBrsS9g6LwZVEkTWgsE= +github.com/snowflakedb/gosnowflake v1.3.13/go.mod h1:6nfka9aTXkUNha1p1cjeeyjDvcyh7jfjp0l8kGpDBok= github.com/soheilhy/cmux v0.1.4/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4kGIyLM= github.com/soheilhy/cmux v0.1.5 h1:jjzc5WVemNEDTLwv9tlmemhC73tI08BNOIGwBOo10Js= github.com/soheilhy/cmux v0.1.5/go.mod h1:T7TcVDs9LWfQgPlPsdngu6I6QIoyIFZDDC6sNE1GqG0= @@ -1171,6 +1446,7 @@ github.com/spaolacci/murmur3 v1.1.0/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2 github.com/spf13/afero v1.1.2/go.mod h1:j4pytiNVoe2o6bmDsKpLACNPDBIoEAkihy7loJ1B0CQ= github.com/spf13/afero v1.2.2/go.mod h1:9ZxEEn6pIJ8Rxe320qSDBk6AsU0r9pR7Q4OcevTdifk= github.com/spf13/afero v1.3.3/go.mod h1:5KUK8ByomD5Ti5Artl0RtHeI5pTF7MIDuXL3yY520V4= +github.com/spf13/afero v1.3.4/go.mod h1:Ai8FlHk4v/PARR026UzYexafAt9roJ7LcLMAmO6Z93I= github.com/spf13/afero v1.6.0/go.mod h1:Ai8FlHk4v/PARR026UzYexafAt9roJ7LcLMAmO6Z93I= github.com/spf13/cast v1.3.0/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= github.com/spf13/cast v1.4.1 h1:s0hze+J0196ZfEMTs80N7UlFt0BDuQ7Q+JDnHiMWKdA= @@ -1178,6 +1454,7 @@ github.com/spf13/cast v1.4.1/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkU github.com/spf13/cobra v0.0.2-0.20171109065643-2da4a54c5cee/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3XqQ= github.com/spf13/cobra v0.0.3/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3XqQ= github.com/spf13/cobra v0.0.5/go.mod h1:3K3wKZymM7VvHMDS9+Akkh4K60UwM26emMESw8tLCHU= +github.com/spf13/cobra v1.0.0/go.mod h1:/6GTrnGXV9HjY+aR4k0oJ5tcvakLuG6EuKReYlHNrgE= github.com/spf13/cobra v1.3.0 h1:R7cSvGu+Vv+qX0gW5R/85dx2kmmJT5z5NM8ifdYjdn0= github.com/spf13/cobra v1.3.0/go.mod h1:BrRVncBjOJa/eUcVVm9CE+oC6as8k+VYr4NY7WCi9V4= github.com/spf13/jwalterweatherman v1.0.0/go.mod h1:cQK4TGJAtQXfYWX+Ddv3mKDzgVb68N+wFjFa4jdeBTo= @@ -1189,13 +1466,17 @@ github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnIn github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA= github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= github.com/spf13/viper v1.3.2/go.mod h1:ZiWeW+zYFKm7srdB9IoDzzZXaJaI5eL9QjNiN/DMA2s= +github.com/spf13/viper v1.4.0/go.mod h1:PTJ7Z/lr49W6bUbkmS1V3by4uWynFiR9p7+dSq/yZzE= github.com/spf13/viper v1.7.1/go.mod h1:8WkrPz2fc9jxqZNCJI/76HCieCp4Q8HaLFoCha5qpdg= github.com/spf13/viper v1.10.0/go.mod h1:SoyBPwAtKDzypXNDFKN5kzH7ppppbGZtls1UpIy5AsM= +github.com/stefanberger/go-pkcs11uri v0.0.0-20201008174630-78d3cae3a980/go.mod h1:AO3tvPzVZ/ayst6UlUKUv6rcPQInYe3IknH3jYhAKu8= +github.com/stoewer/go-strcase v1.2.0/go.mod h1:IBiWB2sKIp3wVVQ3Y035++gc+knqhUQag1KpM8ahLw8= github.com/streadway/amqp v0.0.0-20190404075320-75d898a42a94/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= github.com/streadway/amqp v0.0.0-20190827072141-edfb9018d271/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw= github.com/streadway/handy v0.0.0-20190108123426-d5acb3125c2a/go.mod h1:qNTQ5P5JnDBl6z3cMAg/SywNDC5ABu5ApDIw6lUbRmI= github.com/streadway/quantile v0.0.0-20150917103942-b0c588724d25 h1:7z3LSn867ex6VSaahyKadf4WtSsJIgne6A1WLOAGM8A= github.com/streadway/quantile v0.0.0-20150917103942-b0c588724d25/go.mod h1:lbP8tGiBjZ5YWIc2fzuRpTaz0b/53vT6PEs3QuAWzuU= +github.com/stretchr/objx v0.0.0-20180129172003-8a3f7159479f/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= 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.2.0 h1:Hbg2NidpLE8veEBkEZTL3CvlkUIVzuU9jDplZO54c48= @@ -1203,6 +1484,10 @@ github.com/stretchr/objx v0.2.0/go.mod h1:qt09Ya8vawLte6SNmTgCsAVtYtaKzEcn8ATUoH github.com/stretchr/testify v1.1.4-0.20160305165446-6fe211e49392 h1:7ubzBW6wJ46nWdWvZQlDjtGTnupA4Z1dyHY9Xbhq3us= github.com/stretchr/testify v1.1.4-0.20160305165446-6fe211e49392/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/subosito/gotenv v1.2.0/go.mod h1:N0PQaV/YGNqwC0u51sEeR/aUtSLEXKX9iv69rRypqCw= +github.com/syndtr/gocapability v0.0.0-20170704070218-db04d3cc01c8/go.mod h1:hkRG7XYTFWNJGYcbNJQlaLq0fg1yr4J4t/NcTQtrfww= +github.com/syndtr/gocapability v0.0.0-20180916011248-d98352740cb2/go.mod h1:hkRG7XYTFWNJGYcbNJQlaLq0fg1yr4J4t/NcTQtrfww= +github.com/syndtr/gocapability v0.0.0-20200815063812-42c35b437635/go.mod h1:hkRG7XYTFWNJGYcbNJQlaLq0fg1yr4J4t/NcTQtrfww= +github.com/tchap/go-patricia v2.2.6+incompatible/go.mod h1:bmLyhP68RS6kStMGxByiQ23RP/odRBOTVjwp2cDyi6I= github.com/tidwall/pretty v1.0.0/go.mod h1:XNkn88O1ChpSDQmQeStsy+sBenx6DDtFZJxhVysOjyk= github.com/tinylib/msgp v1.0.2/go.mod h1:+d+yLhGm8mzTaHzB+wgMYrodPfmZrzkirds8fDWklFE= github.com/tinylib/msgp v1.1.0 h1:9fQd+ICuRIu/ue4vxJZu6/LzxN0HwMds2nq/0cFvxHU= @@ -1220,11 +1505,12 @@ github.com/twmb/murmur3 v1.1.6/go.mod h1:Qq/R7NUyOfr65zD+6Q5IHKsJLwP7exErjN6lyyq github.com/twotwotwo/sorts v0.0.0-20160814051341-bf5c1f2b8553 h1:DRC1ubdb3ZmyyIeCSTxjZIQAnpLPfKVgYrLETQuOPjo= github.com/twotwotwo/sorts v0.0.0-20160814051341-bf5c1f2b8553/go.mod h1:Rj7Csq/tZ/egz+Ltc2IVpsA5309AmSMEswjkTZmq2Xc= github.com/uber-go/tally v3.3.15+incompatible/go.mod h1:YDTIBxdXyOU/sCWilKB4bgyufu1cEi0jdVnRdxvjnmU= -github.com/uber-go/tally v3.4.3+incompatible h1:Oq25FXV8cWHPRo+EPeNdbN3LfuozC9mDK2/4vZ1k38U= -github.com/uber-go/tally v3.4.3+incompatible/go.mod h1:YDTIBxdXyOU/sCWilKB4bgyufu1cEi0jdVnRdxvjnmU= +github.com/uber-go/tally v3.5.0+incompatible h1:2vIkqVrSaspifqcJh2yQjQqqpfavvmfj/ognDrBxuSg= +github.com/uber-go/tally v3.5.0+incompatible/go.mod h1:YDTIBxdXyOU/sCWilKB4bgyufu1cEi0jdVnRdxvjnmU= github.com/uber/athenadriver v1.1.4/go.mod h1:tQjho4NzXw55LGfSZEcETuYydpY1vtmixUabHkC1K/E= github.com/uber/jaeger-client-go v2.23.0+incompatible/go.mod h1:WVhlPFC8FDjOFMMWRy2pZqQJSXxYSwNYOkTr/Z6d3Kk= github.com/uber/jaeger-client-go v2.25.0+incompatible/go.mod h1:WVhlPFC8FDjOFMMWRy2pZqQJSXxYSwNYOkTr/Z6d3Kk= +github.com/uber/jaeger-client-go v2.28.0+incompatible/go.mod h1:WVhlPFC8FDjOFMMWRy2pZqQJSXxYSwNYOkTr/Z6d3Kk= github.com/uber/jaeger-client-go v2.29.1+incompatible h1:R9ec3zO3sGpzs0abd43Y+fBZRJ9uiH6lXyR/+u6brW4= github.com/uber/jaeger-client-go v2.29.1+incompatible/go.mod h1:WVhlPFC8FDjOFMMWRy2pZqQJSXxYSwNYOkTr/Z6d3Kk= github.com/uber/jaeger-lib v2.2.0+incompatible/go.mod h1:ComeNDZlWwrWnDv8aPp0Ba6+uUTzImX/AauajbLI56U= @@ -1232,23 +1518,42 @@ github.com/uber/jaeger-lib v2.4.1+incompatible h1:td4jdvLcExb4cBISKIpHuGoVXh+dVK github.com/uber/jaeger-lib v2.4.1+incompatible/go.mod h1:ComeNDZlWwrWnDv8aPp0Ba6+uUTzImX/AauajbLI56U= github.com/uber/tchannel-go v1.20.1 h1:lX+oiTaM1KV9DTaRZNox+uguJWh/tkAJO6b+BdywOTE= github.com/uber/tchannel-go v1.20.1/go.mod h1:Rrgz1eL8kMjW/nEzZos0t+Heq0O4LhnUJVA32OvWKHo= +github.com/ugorji/go v1.1.4/go.mod h1:uQMGLiO92mf5W77hV/PUCpI3pbzQx3CRekS0kk+RGrc= +github.com/ugorji/go v1.1.7/go.mod h1:kZn38zHttfInRq0xu/PH0az30d+z6vm202qpg1oXVMw= github.com/ugorji/go/codec v0.0.0-20181204163529-d75b2dcb6bc8/go.mod h1:VFNgLljTbGfSG7qAOspJ7OScBnGdDN/yBr0sguwnwf0= +github.com/ugorji/go/codec v1.1.7/go.mod h1:Ax+UKWsSmolVDwsd+7N3ZtXu+yMGCf907BLYF3GoBXY= +github.com/urfave/cli v0.0.0-20171014202726-7bc6a0acffa5/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijbERA= github.com/urfave/cli v1.20.0/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijbERA= github.com/urfave/cli v1.22.1/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0= +github.com/urfave/cli v1.22.2/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0= +github.com/valyala/bytebufferpool v1.0.0/go.mod h1:6bBcMArwyJ5K/AmCkWv1jt77kVWyCJ6HpOuEn7z0Csc= +github.com/valyala/fasttemplate v1.0.1/go.mod h1:UQGH1tvbgY+Nz5t2n7tXsz52dQxojPUpymEIMZ47gx8= +github.com/valyala/fasttemplate v1.2.1/go.mod h1:KHLXt3tVN2HBp8eijSv/kGJopbvo7S+qRAEEKiv+SiQ= github.com/valyala/tcplisten v0.0.0-20161114210144-ceec8f93295a h1:0R4NLDRDZX6JcmhJgXi5E4b8Wg84ihbmUKp/GvSPEzc= github.com/valyala/tcplisten v0.0.0-20161114210144-ceec8f93295a/go.mod h1:v3UYOV9WzVtRmSR+PDvWpU/qWl4Wa5LApYYX4ZtKbio= github.com/vektah/gqlparser v1.1.2/go.mod h1:1ycwN7Ij5njmMkPPAOaRFY4rET2Enx7IkVv3vaXspKw= +github.com/vishvananda/netlink v0.0.0-20181108222139-023a6dafdcdf/go.mod h1:+SR5DhBJrl6ZM7CoCKvpw5BKroDKQ+PJqOg65H/2ktk= +github.com/vishvananda/netlink v1.1.0/go.mod h1:cTgwzPIzzgDAYoQrMm0EdrjRUBkTqKYppBueQtXaqoE= +github.com/vishvananda/netlink v1.1.1-0.20201029203352-d40f9887b852/go.mod h1:twkDnbuQxJYemMlGd4JFIcuhgX83tXhKS2B/PRMpOho= +github.com/vishvananda/netns v0.0.0-20180720170159-13995c7128cc/go.mod h1:ZjcWmFBXmLKZu9Nxj3WKYEafiSqer2rnvPr0en9UNpI= +github.com/vishvananda/netns v0.0.0-20191106174202-0a2b9b5464df/go.mod h1:JP3t17pCcGlemwknint6hfoeCVQrEMVwxRLRjXpq+BU= +github.com/vishvananda/netns v0.0.0-20200728191858-db3c7e526aae/go.mod h1:DD4vA1DwXk04H54A1oHXtwZmA0grkVMdPxx/VGLCah0= github.com/vmihailenco/msgpack v2.8.3+incompatible h1:76LCLwxS08gKHRpGA10PBxfWk72JfUH6mgzp2+URwYM= github.com/vmihailenco/msgpack v2.8.3+incompatible/go.mod h1:fy3FlTQTDXWkZ7Bh6AcGMlsjHatGryHQYUTf1ShIgkk= github.com/willf/bitset v1.1.3/go.mod h1:RjeCKbqT1RxIR/KWY6phxZiaY1IyutSBfGjNPySAYV4= github.com/willf/bitset v1.1.9/go.mod h1:RjeCKbqT1RxIR/KWY6phxZiaY1IyutSBfGjNPySAYV4= -github.com/willf/bitset v1.1.10 h1:NotGKqX0KwQ72NUzqrjZq5ipPNDQex9lo3WpaS8L2sc= github.com/willf/bitset v1.1.10/go.mod h1:RjeCKbqT1RxIR/KWY6phxZiaY1IyutSBfGjNPySAYV4= +github.com/willf/bitset v1.1.11-0.20200630133818-d5bec3311243/go.mod h1:RjeCKbqT1RxIR/KWY6phxZiaY1IyutSBfGjNPySAYV4= +github.com/willf/bitset v1.1.11 h1:N7Z7E9UvjW+sGsEl7k/SJrvY2reP1A07MrGuCjIOjRE= +github.com/willf/bitset v1.1.11/go.mod h1:83CECat5yLh5zVOf4P1ErAgKA5UDvKtgyUABdr3+MjI= github.com/xdg-go/pbkdf2 v1.0.0/go.mod h1:jrpuAogTd400dnrH08LKmI/xc1MbPOebTwRqcT5RDeI= github.com/xdg-go/scram v1.0.2/go.mod h1:1WAq6h33pAW+iRreB34OORO2Nf7qel3VV3fjBj+hCSs= github.com/xdg-go/stringprep v1.0.2/go.mod h1:8F9zXuvzgwmyT5DUm4GUfZGDdT3W+LCvS6+da4O5kxM= github.com/xdg/scram v0.0.0-20180814205039-7eeb5667e42c/go.mod h1:lB8K/P019DLNhemzwFU4jHLhdvlE6uDZjXFejJXr49I= github.com/xdg/stringprep v0.0.0-20180714160509-73f8eece6fdc/go.mod h1:Jhud4/sHMO4oL310DaZAKk9ZaJ08SJfe+sJh0HrGL1Y= +github.com/xeipuuv/gojsonpointer v0.0.0-20180127040702-4e3ac2762d5f/go.mod h1:N2zxlSyiKSe5eX1tZViRH5QA0qijqEDrYZiPEAiq3wU= +github.com/xeipuuv/gojsonreference v0.0.0-20180127040603-bd5ef7bd5415/go.mod h1:GwrjFmJcFw6At/Gs6z4yjiIwzuJ1/+UwLxMQDVQXShQ= +github.com/xeipuuv/gojsonschema v0.0.0-20180618132009-1d523034197f/go.mod h1:5yf86TLmAcydyeJq5YvxkGPE2fm/u4myDekKRoLuqhs= github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2 h1:eY9dn8+vbi4tKz5Qo6v2eYzo7kUS51QINcR5jNpbZS8= github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU= github.com/xlab/treeprint v0.0.0-20180616005107-d6fb6747feb6/go.mod h1:ce1O1j6UtZfjr22oyGxGLbauSBp2YVXpARAosm7dHBg= @@ -1260,12 +1565,17 @@ github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9de github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= +github.com/yuin/goldmark v1.4.0/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= +github.com/yvasiyarov/go-metrics v0.0.0-20140926110328-57bccd1ccd43/go.mod h1:aX5oPXxHm3bOH+xeAttToC8pqch2ScQN/JoXYupl6xs= +github.com/yvasiyarov/gorelic v0.0.0-20141212073537-a9bba5b9ab50/go.mod h1:NUSPSUX/bi6SeDMUh6brw0nXpxHnc96TguQh0+r/ssA= +github.com/yvasiyarov/newrelic_platform_go v0.0.0-20140908184405-b21fdbd4370f/go.mod h1:GlGEuHIJweS1mbCqG+7vt2nvWLzLLnRHbXz5JKd/Qbg= go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/bbolt v1.3.5/go.mod h1:G5EMThwa9y8QZGBClrRx5EY+Yw9kAhnjy3bSjsnlVTQ= go.etcd.io/bbolt v1.3.6 h1:/ecaJf0sk1l4l6V4awd65v2C3ILy7MSj+s/x1ADCIMU= go.etcd.io/bbolt v1.3.6/go.mod h1:qXsaaIqmgQH0T+OPdb99Bf+PKfBBQVAdyD6TY9G8XM4= go.etcd.io/etcd v0.0.0-20191023171146-3cf2f69b5738/go.mod h1:dnLIgRNXwCJa5e+c6mIZCrds/GIG4ncV9HhK5PX7jPg= +go.etcd.io/etcd v0.5.0-alpha.5.0.20200910180754-dd1b699fc489/go.mod h1:yVHk9ub3CSBatqGNg7GRmsnfLWtoW60w4eDYfh7vHDg= go.etcd.io/etcd/api/v3 v3.5.0/go.mod h1:cbVKeC6lCfl7j/8jBhAK6aIYO9XOjdptoxU/nLQcPvs= go.etcd.io/etcd/api/v3 v3.5.1/go.mod h1:cbVKeC6lCfl7j/8jBhAK6aIYO9XOjdptoxU/nLQcPvs= go.etcd.io/etcd/api/v3 v3.6.0-alpha.0 h1:se+XckWlVTTfwjZSsAZJ2zGPzmIMq3j7fKBCmHoB9UA= @@ -1296,6 +1606,7 @@ go.mongodb.org/mongo-driver v1.4.3/go.mod h1:WcMNYLx/IlOxLe6JRJiv2uXuCz6zBLndR4S go.mongodb.org/mongo-driver v1.4.4/go.mod h1:WcMNYLx/IlOxLe6JRJiv2uXuCz6zBLndR4SoGjYphSc= go.mongodb.org/mongo-driver v1.4.6/go.mod h1:WcMNYLx/IlOxLe6JRJiv2uXuCz6zBLndR4SoGjYphSc= go.mongodb.org/mongo-driver v1.5.1/go.mod h1:gRXCHX4Jo7J0IJ1oDQyUxF7jfy19UfxniMS4xxMmUqw= +go.mozilla.org/pkcs7 v0.0.0-20200128120323-432b2356ecb1/go.mod h1:SNgMg+EgDFwmvSmLRTNKC5fegJjB7v23qTQ0XLGUNHk= go.opencensus.io v0.20.1/go.mod h1:6WKK9ahsWS3RSO+PY9ZHZUfv2irvY6gN279GOPZjmmk= go.opencensus.io v0.20.2/go.mod h1:6WKK9ahsWS3RSO+PY9ZHZUfv2irvY6gN279GOPZjmmk= go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= @@ -1338,6 +1649,7 @@ go.opentelemetry.io/otel/trace v1.3.0/go.mod h1:c/VDhno8888bvQYmbYLqe41/Ldmr/KKu go.opentelemetry.io/otel/trace v1.4.0/go.mod h1:uc3eRsqDfWs9R7b92xbQbU42/eTNz4N+gLP8qJCi4aE= go.opentelemetry.io/otel/trace v1.4.1 h1:O+16qcdTrT7zxv2J6GejTPFinSwA++cYerC5iSiF8EQ= go.opentelemetry.io/otel/trace v1.4.1/go.mod h1:iYEVbroFCNut9QkwEczV9vMRPHNKSSwYZjulEtsmhFc= +go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI= go.opentelemetry.io/proto/otlp v0.12.0 h1:CMJ/3Wp7iOWES+CYLfnBv+DVmPbB+kmy9PJ92XvlR6c= go.opentelemetry.io/proto/otlp v0.12.0/go.mod h1:TsIjwGWIx5VFYv9KGVlOpxoBl5Dy+63SUguV7GGvlSQ= go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= @@ -1346,12 +1658,10 @@ go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.5.1/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= -go.uber.org/atomic v1.8.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.9.0 h1:ECmE8Bn/WFTYwEW/bpKD3M8VtR/zQVbavAoalC1PYyE= go.uber.org/atomic v1.9.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/config v1.4.0 h1:upnMPpMm6WlbZtXoasNkK4f0FhxwS+W4Iqz5oNznehQ= go.uber.org/config v1.4.0/go.mod h1:aCyrMHmUAc/s2h9sv1koP84M9ZF/4K+g2oleyESO/Ig= -go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/goleak v1.1.11/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= go.uber.org/goleak v1.1.12 h1:gZAh5/EyT/HQwlpkCy6wTpqfH9H8Lz8zbm3dZh+OyzA= go.uber.org/goleak v1.1.12/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= @@ -1374,6 +1684,7 @@ go.uber.org/zap v1.21.0/go.mod h1:wjWOCqI0f2ZZrJF/UufIOkiC8ii6tm1iqIsLo76RfJw= golang.org/x/crypto v0.0.0-20171113213409-9f005a07e0d3/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20180505025534-4ec37c66abab/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= +golang.org/x/crypto v0.0.0-20181009213950-7c1a557ab941/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20181029021203-45a5f77698d3/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20181203042331-505ab145d0a9/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190211182817-74369b46fc67/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= @@ -1395,11 +1706,15 @@ golang.org/x/crypto v0.0.0-20191206172530-e9b2fee46413/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20200220183623-bac4c82f6975/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20200302210943-78000ba7a073/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20200422194213-44a606286825/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20200510223506-06a226fb4e37/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20200728195943-123391ffb6de/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20201002170205-7f63de1d35b0/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/crypto v0.0.0-20201208171446-5f87f3452ae9/go.mod h1:jdWPYTVW3xRLrWPugEBEK3UY2ZEsg3UU495nc5E+M+I= golang.org/x/crypto v0.0.0-20201221181555-eec23a3978ad/go.mod h1:jdWPYTVW3xRLrWPugEBEK3UY2ZEsg3UU495nc5E+M+I= golang.org/x/crypto v0.0.0-20210220033148-5ea612d1eb83/go.mod h1:jdWPYTVW3xRLrWPugEBEK3UY2ZEsg3UU495nc5E+M+I= +golang.org/x/crypto v0.0.0-20210322153248-0c34fe9e7dc2/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4= +golang.org/x/crypto v0.0.0-20210616213533-5ff15b29337e/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20210817164053-32db794688a5 h1:HWj/xjIHfjYU5nVXpTM0s39J9CbLn7Cc5a7IC5rwsMQ= golang.org/x/crypto v0.0.0-20210817164053-32db794688a5/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -1447,6 +1762,7 @@ golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73r golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181005035420-146acd28ed58/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20181011144130-49bb7cea24b1/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181023162649-9b4f9f5ad519/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-20181201002055-351d144fa1fc/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -1459,8 +1775,10 @@ golang.org/x/net v0.0.0-20190320064053-1272bf9dcd53/go.mod h1:t9HGtf8HONx5eT2rtn 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-20190522155817-f3200d17e092/go.mod h1:HSz+uSET+XFnRR8LxR5pz3Of3rY3CfYBVs4xY44aLks= 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-20190619014844-b5b0513f8c1b/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= @@ -1488,6 +1806,7 @@ golang.org/x/net v0.0.0-20200602114024-627f9648deb9/go.mod h1:qpuaurCH72eLCgpAm/ 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-20201006153459-a7d1128ccaa0/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20201031054903-ff519b6c9102/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= @@ -1495,18 +1814,21 @@ golang.org/x/net v0.0.0-20201202161906-c7110b5ffcbb/go.mod h1:sp8m0HH+o8qH0wwXwY golang.org/x/net v0.0.0-20201209123823-ac852fbbde11/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20201224014010-6772e930b67b/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210119194325-5f4716e94777/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= -golang.org/x/net v0.0.0-20210224082022-3d97a244fca7/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210316092652-d523dce5a7f4/go.mod h1:RBQZq4jEuRlivfhVLdyRGr576XBO4/greRjx4P4O3yc= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= golang.org/x/net v0.0.0-20210410081132-afb366fc7cd1/go.mod h1:9tjilg8BloeKEkVJvy7fQ90B1CfIiPueXVOjqfkSzI8= -golang.org/x/net v0.0.0-20210421230115-4e50805a0758/go.mod h1:72T/g9IO56b78aLF+1Kcs5dz7/ng1VjMUvfKvpfy+jM= golang.org/x/net v0.0.0-20210503060351-7fd8e65b6420/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= +golang.org/x/net v0.0.0-20210520170846-37e1c6afe023/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20210525063256-abc453219eb5/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.0.0-20210610132358-84b48f89b13b/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= +golang.org/x/net v0.0.0-20210614182718-04defd469f4e/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= +golang.org/x/net v0.0.0-20210726213435-c6fcb2dbf985/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= +golang.org/x/net v0.0.0-20210805182204-aaa1db679c0d/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20210813160813-60bc85c4be6d/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.0.0-20220105145211-5b0dc2dfae98 h1:+6WJMRLHlD7X7frgp7TUZ36RnQzSf9wVVTNakEp+nqY= -golang.org/x/net v0.0.0-20220105145211-5b0dc2dfae98/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= +golang.org/x/net v0.0.0-20211020060615-d418f374d309/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 h1:oA4XRj0qtSt8Yo1Zms0CUlsT3KG69V2UGQWPBxujDmc= +golang.org/x/net v0.0.0-20220225172249-27dd8689420f/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= 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= @@ -1523,8 +1845,9 @@ golang.org/x/oauth2 v0.0.0-20210628180205-a41e5a781914/go.mod h1:KelEdhl1UZF7XfJ golang.org/x/oauth2 v0.0.0-20210805134026-6f1e6394065a/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= golang.org/x/oauth2 v0.0.0-20210819190943-2bc19b11175f/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= golang.org/x/oauth2 v0.0.0-20211005180243-6b3c2da341f1/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= -golang.org/x/oauth2 v0.0.0-20211104180415-d3ed0bb246c8 h1:RerP+noqYHUQ8CMRcPlC2nvTa4dcBIjegkuWdcUDuqg= golang.org/x/oauth2 v0.0.0-20211104180415-d3ed0bb246c8/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= +golang.org/x/oauth2 v0.0.0-20220223155221-ee480838109b h1:clP8eMhB30EHdc0bd2Twtq6kgU7yl5ub2cQLSdrv1Dg= +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= @@ -1559,11 +1882,18 @@ golang.org/x/sys v0.0.0-20190419153524-e8e3143a4f4a/go.mod h1:h1NjWce9XRLGQEsW7w 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-20190514135907-3a4b5fb9f71f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190522044717-8097e1b27ff5/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190531175056-4c3a928424d2/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190602015325-4c4f7f33c9ed/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190606165138-5da285871e9c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190606203320-7fc4e5ec1444/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190616124812-15dcb6c0061f/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-20190801041406-cbf593c0f2f3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190812073006-9eafafc0a87e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190813064441-fde4db37ae7a/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190826190057-c7b8b68b1456/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190904154756-749cb33beabd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190916202348-b4ddaad3f8a3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -1573,20 +1903,25 @@ golang.org/x/sys v0.0.0-20191001151750-bb3f8db39f24/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20191005200804-aed5e4c7ecf9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191008105621-543471e840be/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191010194322-b09406accb47/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191022100944-742c48ecaeb7/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191026070338-33540a1f6037/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191115151921-52ab43148777/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191120155948-bd437916bb0e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191128015809-6d18c012aee9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191204072324-ce4227a45e2e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191210023423-ac6580df4449/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191220142924-d4481acd189f/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-20200107162124-548cf772de50/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200113162924-86b910548bc1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200116001909-b77594299b42/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200120151820-655fe14d7479/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200122134326-e047566fdf82/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200124204421-9fbb57f87de9/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-20200217220822-9197077df867/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= @@ -1598,52 +1933,65 @@ golang.org/x/sys v0.0.0-20200515095857-1151b9dac4a9/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20200519105757-fe76b779f299/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-20200622214017-ed371f2e16b4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200625212154-ddb9806d33ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200728102440-3e129f6d46b1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200803210538-64077c9b5642/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200817155316-9781c653f443/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200826173525-f9321e4c35a6/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200831180312-196b9ba8737a/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200905004654-be1d3432aa8f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200909081042-eff7692f9009/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200916030750-2334cc1a136f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200922070232-aee5d888a860/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200923182605-d9f96fdee20d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20201112073958-5cba982894dd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20201117170446-d9b008d0a637/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201201145000-ef89a241ccb3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20201202213521-69691e467435/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210104204734-6f8348627aad/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210119212857-b64e53b001e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210220050731-9a76102bfb43/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210225134936-a50acf3fe073/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210303074136-134d130e1a04/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210305230114-8fe3ee5dd75b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210309074719-68d13333faf2/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210315160823-c6e025ad8005/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210320140829-1e4c9ba3b0c4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210324051608-47abb6519492/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210330210617-4fbd30eecc44/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210403161142-5e06dd20ab57/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210420072515-93ed5bcd2bfe/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210423185535-09eb48e85fd7/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210426230700-d19ff857e887/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210514084401-e8d321eab015/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210603125802-9665404d3644/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20210611083646-a4fc73990273/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-20210616094352-59db8d763f22/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-20210806184541-e5e7981a1069/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210809222454-d867a43fc93e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210816183151-1e6c022a8912/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210823070655-63515b42dcdf/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210908233432-aa78b53d3365/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-20211007075335-d3039528d8ac/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20211020174200-9d6173849985/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211124211545-fe61309f8881/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20211205182925-97ca703d548d/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-20220328115105-d36c6a25d886/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220422013727-9388b58f7150 h1:xHms4gcpe1YE7A3yIllJXP16CMAGuqwO2lX1mTyyRRc= golang.org/x/sys v0.0.0-20220422013727-9388b58f7150/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201117132131-f5c789dd3221/go.mod h1:Nr5EML6q2oocZ2LXRh80K7BxOlk5/8JxuGnuhpl+muw= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= -golang.org/x/term v0.0.0-20210220032956-6a3ed077a48d h1:SZxvLBoTP5yHO3Frd4z4vrF+DBX9vMVanchswa69toE= golang.org/x/term v0.0.0-20210220032956-6a3ed077a48d/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= +golang.org/x/term v0.0.0-20210927222741-03fcf44c2211 h1:JGgROgKl9N8DuW20oFS5gxc+lE67/N3FcwmBPMe7ArY= +golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/text v0.0.0-20160726164857-2910a502d2bf/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -1661,9 +2009,11 @@ golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxb 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/time v0.0.0-20200416051211-89c76fbcd5d1/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.0.0-20200630173020-3af7569d3a1e/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.0.0-20201208040808-7e3f01d25324/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20210220033141-f8bda1e9f3ba/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/time v0.0.0-20210611083556-38a9dc6acbc6 h1:Vv0JUPWTyeqUq42B2WJ1FeIDjjvGKoA2Ss+Ts0lAVbs= -golang.org/x/time v0.0.0-20210611083556-38a9dc6acbc6/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.0.0-20210723032227-1f47c861a9ac h1:7zkz7BUtwNFFqcowJ+RIgu2MaV/MapERkDIy+mwPyjs= +golang.org/x/time v0.0.0-20210723032227-1f47c861a9ac/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180525024113-a5b4c53f6e8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= @@ -1697,7 +2047,6 @@ golang.org/x/tools v0.0.0-20191012152004-8de300cfc20a/go.mod h1:b+2E5dAYhXwXZwtn golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191029190741-b9c20aec41a5/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191104232314-dc038396d1f0/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= -golang.org/x/tools v0.0.0-20191108193012-7d206e10da11/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191112195655-aa38f8e97acc/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= @@ -1727,6 +2076,7 @@ golang.org/x/tools v0.0.0-20200426102838-f3a5411a4c3b/go.mod h1:EkVYQZoAsY45+roY 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-20200522201501-cb1345f3a375/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-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200721032237-77f530d86f9a/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= @@ -1744,8 +2094,9 @@ golang.org/x/tools v0.1.1/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.2/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.3/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.4/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= -golang.org/x/tools v0.1.5 h1:ouewzE6p+/VEB31YYnTbEJdi8pFqKp4P4n85vwo3DHA= golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= +golang.org/x/tools v0.1.7 h1:6j8CgantCy3yc8JGBqkDLMKWqZ0RDU2g1HVgacojGWQ= +golang.org/x/tools v0.1.7/go.mod h1:LGqMHiF4EqQNHR1JncWGqT5BVaXmza+X+BDGol+dOxo= 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= @@ -1754,9 +2105,11 @@ golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8T gonum.org/v1/gonum v0.0.0-20180816165407-929014505bf4/go.mod h1:Y+Yx5eoAFn32cQvJDxZx5Dpnq+c3wtXuadVZAcxbbBo= gonum.org/v1/gonum v0.0.0-20181121035319-3f7ecaa7e8ca/go.mod h1:Y+Yx5eoAFn32cQvJDxZx5Dpnq+c3wtXuadVZAcxbbBo= gonum.org/v1/gonum v0.6.0/go.mod h1:9mxDZsDKxgMAuccQkewq682L+0eCu4dCN2yonUJTCLU= +gonum.org/v1/gonum v0.8.2/go.mod h1:oe/vMfY3deqTw+1EZJhuvEW2iwGF1bW9wwu7XCu0+v0= gonum.org/v1/netlib v0.0.0-20181029234149-ec6d1f5cefe6/go.mod h1:wa6Ws7BG/ESfp6dHfk7C6KdzKA7wR7u/rKwOGE66zvw= gonum.org/v1/netlib v0.0.0-20190313105609-8cb42192e0e0/go.mod h1:wa6Ws7BG/ESfp6dHfk7C6KdzKA7wR7u/rKwOGE66zvw= gonum.org/v1/plot v0.0.0-20190515093506-e2840ee46a6b/go.mod h1:Wt8AAjI+ypCyYX3nZBvf6cAIx93T+c/OS2HFAYskSZc= +google.golang.org/api v0.0.0-20160322025152-9bf6e6e569ff/go.mod h1:4mhQ8q/RsB7i+udVvVy5NUi08OU8ZlA0gRVgrF7VFY0= google.golang.org/api v0.3.1/go.mod h1:6wY9I6uQWHQ8EM57III9mq/AjF+i8G65rmVagqKMtkk= 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= @@ -1798,6 +2151,7 @@ google.golang.org/appengine v1.6.5/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCID google.golang.org/appengine v1.6.6/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= google.golang.org/appengine v1.6.7 h1:FZR1q0exgwxzPzp/aF+VccGrSfxfPpkBqjIIEq3ru6c= google.golang.org/appengine v1.6.7/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= +google.golang.org/cloud v0.0.0-20151119220103-975617b05ea8/go.mod h1:0H1ncTHf11KCFhTc/+EFRbzSCOZx+VUbRMk55Yv5MYk= google.golang.org/genproto v0.0.0-20170818010345-ee236bd376b0/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= 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= @@ -1805,6 +2159,7 @@ google.golang.org/genproto v0.0.0-20190404172233-64821d5d2107/go.mod h1:VzzqZJRn 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-20190522204451-c2c4e71fbf69/go.mod h1:z3L6/3dTEVtUr6QSP8miRzeRqwQOioJ9I66odjN4I7s= google.golang.org/genproto v0.0.0-20190530194941-fb225487d101/go.mod h1:z3L6/3dTEVtUr6QSP8miRzeRqwQOioJ9I66odjN4I7s= google.golang.org/genproto v0.0.0-20190716160619-c506a9f90610/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= google.golang.org/genproto v0.0.0-20190801165951-fa694d86fc64/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= @@ -1817,6 +2172,7 @@ google.golang.org/genproto v0.0.0-20191216164720-4f79533eabd1/go.mod h1:n3cpQtvx google.golang.org/genproto v0.0.0-20191230161307-f3c370f40bfb/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= google.golang.org/genproto v0.0.0-20200108215221-bd8f9a0ef82f/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= google.golang.org/genproto v0.0.0-20200115191322-ca5a22157cba/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20200117163144-32f20d992d24/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= @@ -1837,7 +2193,9 @@ google.golang.org/genproto v0.0.0-20200729003335-053ba62fc06f/go.mod h1:FWY/as6D 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/genproto v0.0.0-20200904004341-0bd0a958aa1d/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20201019141844-1ed22bb0c154/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= google.golang.org/genproto v0.0.0-20201109203340-2640f1f9cdfb/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20201110150050-8816d57aaa9a/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= google.golang.org/genproto v0.0.0-20201201144952-b05cb90ed32e/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= google.golang.org/genproto v0.0.0-20201210142538-e3217bee35cc/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= google.golang.org/genproto v0.0.0-20201214200347-8c77b98c765d/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= @@ -1863,6 +2221,7 @@ google.golang.org/genproto v0.0.0-20210903162649-d08c68adba83/go.mod h1:eFjDcFEc google.golang.org/genproto v0.0.0-20210909211513-a8c4777a87af/go.mod h1:eFjDcFEctNawg4eG61bRv87N7iHBWyVhJu7u1kqDUXY= google.golang.org/genproto v0.0.0-20210924002016-3dee208752a0/go.mod h1:5CzLGKJ67TSI2B9POpiiyGha0AjJvZIUgRMt1dSmuhc= google.golang.org/genproto v0.0.0-20211008145708-270636b82663/go.mod h1:5CzLGKJ67TSI2B9POpiiyGha0AjJvZIUgRMt1dSmuhc= +google.golang.org/genproto v0.0.0-20211020151524-b7c3a969101a/go.mod h1:5CzLGKJ67TSI2B9POpiiyGha0AjJvZIUgRMt1dSmuhc= google.golang.org/genproto v0.0.0-20211028162531-8db9c33dc351/go.mod h1:5CzLGKJ67TSI2B9POpiiyGha0AjJvZIUgRMt1dSmuhc= google.golang.org/genproto v0.0.0-20211118181313-81c1377c94b1/go.mod h1:5CzLGKJ67TSI2B9POpiiyGha0AjJvZIUgRMt1dSmuhc= google.golang.org/genproto v0.0.0-20211129164237-f09f9a12af12/go.mod h1:5CzLGKJ67TSI2B9POpiiyGha0AjJvZIUgRMt1dSmuhc= @@ -1891,6 +2250,7 @@ gopkg.in/airbrake/gobrake.v2 v2.0.9/go.mod h1:/h5ZAUhDkGaJfjzjKLSjv6zCL6O0LLBxU4 gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/asn1-ber.v1 v1.0.0-20181015200546-f715ec2f112d/go.mod h1:cuepJuh7vyXfUyUwEgHQXw849cJrilpS5NeIjOWESAw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20141024133853-64131543e789/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-20200227125254-8fa46927fb4f/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= @@ -1914,7 +2274,9 @@ gopkg.in/ini.v1 v1.66.2/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= gopkg.in/natefinch/lumberjack.v2 v2.0.0 h1:1Lc07Kr7qY4U2YPouBjpCLxpiyxIVoxqXgkXLknAOE8= gopkg.in/natefinch/lumberjack.v2 v2.0.0/go.mod h1:l0ndWWf7gzL7RNwBG7wST/UCcT4T24xpD6X8LsfU/+k= gopkg.in/resty.v1 v1.12.0/go.mod h1:mDo4pnntr5jdWRML875a/NmxYqAlA73dVijT2AXvQQo= +gopkg.in/square/go-jose.v2 v2.2.2/go.mod h1:M9dMgbHiYLoDGQrXy7OpJDJWiKiU//h+vD76mk0e1AI= gopkg.in/square/go-jose.v2 v2.3.1/go.mod h1:M9dMgbHiYLoDGQrXy7OpJDJWiKiU//h+vD76mk0e1AI= +gopkg.in/square/go-jose.v2 v2.5.1/go.mod h1:M9dMgbHiYLoDGQrXy7OpJDJWiKiU//h+vD76mk0e1AI= gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 h1:uRGJdciOHaEIrze2W8Q3AKkepLTh2hOroT7a+7czfdQ= gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw= gopkg.in/validator.v2 v2.0.0-20160201165114-3e4f037f12a1 h1:1IZMbdoz1SZAQ4HMRwAP0FPSyXt7ywsiJ4q7OPTEu4A= @@ -1934,7 +2296,10 @@ 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.0-20200605160147-a5ece683394c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.0-20200615113413-eeeca48fe776/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b h1:h8qDotaEPuJATrMmW04NCwg7v22aHH28wwpauUhK9Oo= gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +gotest.tools v2.2.0+incompatible h1:VsBPFP1AI068pPrMxtb/S8Zkgf9xEmTLJjfM+P5UIEo= +gotest.tools v2.2.0+incompatible/go.mod h1:DsYFclhRJ6vuDpmuTbkuFWG+y2sxOXAzmJt81HFBacw= gotest.tools/v3 v3.0.2/go.mod h1:3SzNCllyD9/Y+b5r9JIKQ474KzkZyqLqEfYqMsX94Bk= gotest.tools/v3 v3.0.3 h1:4AuOwCGf4lLR9u3YOe2awrHygurzhO/HeQ6laiA6Sx0= gotest.tools/v3 v3.0.3/go.mod h1:Z7Lb0S5l+klDB31fvDQX8ss/FlKDxtlFlw3Oa8Ymbl8= @@ -1946,14 +2311,33 @@ honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt 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= k8s.io/api v0.17.5/go.mod h1:0zV5/ungglgy2Rlm3QK8fbxkXVs+BSJWpJP/+8gUVLY= -k8s.io/api v0.21.1 h1:94bbZ5NTjdINJEdzOkpS4vdPhkb1VFpTYC9zh43f75c= -k8s.io/api v0.21.1/go.mod h1:FstGROTmsSHBarKc8bylzXih8BLNYTiS3TZcsoEDg2s= +k8s.io/api v0.20.1/go.mod h1:KqwcCVogGxQY3nBlRpwt+wpAMF/KjaCc7RpywacvqUo= +k8s.io/api v0.20.4/go.mod h1:++lNL1AJMkDymriNniQsWRkMDzRaX2Y/POTUi8yvqYQ= +k8s.io/api v0.20.6/go.mod h1:X9e8Qag6JV/bL5G6bU8sdVRltWKmdHsFUGS3eVndqE8= +k8s.io/api v0.22.2 h1:M8ZzAD0V6725Fjg53fKeTJxGsJvRbk4TEm/fexHMtfw= +k8s.io/api v0.22.2/go.mod h1:y3ydYpLJAaDI+BbSe2xmGcqxiWHmWjkEeIbiwHvnPR8= k8s.io/apimachinery v0.17.5/go.mod h1:ioIo1G/a+uONV7Tv+ZmCbMG1/a3kVw5YcDdncd8ugQ0= -k8s.io/apimachinery v0.21.1 h1:Q6XuHGlj2xc+hlMCvqyYfbv3H7SRGn2c8NycxJquDVs= -k8s.io/apimachinery v0.21.1/go.mod h1:jbreFvJo3ov9rj7eWT7+sYiRx+qZuCYXwWT1bcDswPY= +k8s.io/apimachinery v0.20.1/go.mod h1:WlLqWAHZGg07AeltaI0MV5uk1Omp8xaN0JGLY6gkRpU= +k8s.io/apimachinery v0.20.4/go.mod h1:WlLqWAHZGg07AeltaI0MV5uk1Omp8xaN0JGLY6gkRpU= +k8s.io/apimachinery v0.20.6/go.mod h1:ejZXtW1Ra6V1O5H8xPBGz+T3+4gfkTCeExAHKU57MAc= +k8s.io/apimachinery v0.22.2 h1:ejz6y/zNma8clPVfNDLnPbleBo6MpoFy/HBiBqCouVk= +k8s.io/apimachinery v0.22.2/go.mod h1:O3oNtNadZdeOMxHFVxOreoznohCpy0z6mocxbZr7oJ0= +k8s.io/apiserver v0.20.1/go.mod h1:ro5QHeQkgMS7ZGpvf4tSMx6bBOgPfE+f52KwvXfScaU= +k8s.io/apiserver v0.20.4/go.mod h1:Mc80thBKOyy7tbvFtB4kJv1kbdD0eIH8k8vianJcbFM= +k8s.io/apiserver v0.20.6/go.mod h1:QIJXNt6i6JB+0YQRNcS0hdRHJlMhflFmsBDeSgT1r8Q= k8s.io/client-go v0.17.5/go.mod h1:S8uZpBpjJJdEH/fEyxcqg7Rn0P5jH+ilkgBHjriSmNo= -k8s.io/client-go v0.21.1 h1:bhblWYLZKUu+pm50plvQF8WpY6TXdRRtcS/K9WauOj4= -k8s.io/client-go v0.21.1/go.mod h1:/kEw4RgW+3xnBGzvp9IWxKSNA+lXn3A7AuH3gdOAzLs= +k8s.io/client-go v0.20.1/go.mod h1:/zcHdt1TeWSd5HoUe6elJmHSQ6uLLgp4bIJHVEuy+/Y= +k8s.io/client-go v0.20.4/go.mod h1:LiMv25ND1gLUdBeYxBIwKpkSC5IsozMMmOOeSJboP+k= +k8s.io/client-go v0.20.6/go.mod h1:nNQMnOvEUEsOzRRFIIkdmYOjAZrC8bgq0ExboWSU1I0= +k8s.io/client-go v0.22.2 h1:DaSQgs02aCC1QcwUdkKZWOeaVsQjYvWv8ZazcZ6JcHc= +k8s.io/client-go v0.22.2/go.mod h1:sAlhrkVDf50ZHx6z4K0S40wISNTarf1r800F+RlCF6U= +k8s.io/component-base v0.20.1/go.mod h1:guxkoJnNoh8LNrbtiQOlyp2Y2XFCZQmrcg2n/DeYNLk= +k8s.io/component-base v0.20.4/go.mod h1:t4p9EdiagbVCJKrQ1RsA5/V4rFQNDfRlevJajlGwgjI= +k8s.io/component-base v0.20.6/go.mod h1:6f1MPBAeI+mvuts3sIdtpjljHWBQ2cIy38oBIWMYnrM= +k8s.io/cri-api v0.17.3/go.mod h1:X1sbHmuXhwaHs9xxYffLqJogVsnI+f6cPRcgPel7ywM= +k8s.io/cri-api v0.20.1/go.mod h1:2JRbKt+BFLTjtrILYVqQK5jqhI+XNdF6UiGMgczeBCI= +k8s.io/cri-api v0.20.4/go.mod h1:2JRbKt+BFLTjtrILYVqQK5jqhI+XNdF6UiGMgczeBCI= +k8s.io/cri-api v0.20.6/go.mod h1:ew44AjNXwyn1s0U4xCKGodU7J1HzBeZ1MpGrpa5r8Yc= k8s.io/gengo v0.0.0-20190128074634-0689ccc1d7d6/go.mod h1:ezvh/TsK7cY6rbqRK0oQQ8IAqLxYwwyPxAX1Pzy0ii0= k8s.io/gengo v0.0.0-20200413195148-3a45101e95ac/go.mod h1:ezvh/TsK7cY6rbqRK0oQQ8IAqLxYwwyPxAX1Pzy0ii0= k8s.io/klog v0.0.0-20181102134211-b9b56d5dfc92/go.mod h1:Gq+BEi5rUBO/HRz0bTSXDUcqjScdoY3a9IHpCEIOOfk= @@ -1961,23 +2345,30 @@ k8s.io/klog v0.3.0/go.mod h1:Gq+BEi5rUBO/HRz0bTSXDUcqjScdoY3a9IHpCEIOOfk= k8s.io/klog v1.0.0 h1:Pt+yjF5aB1xDSVbau4VsWe+dQNzA0qv1LlXdC2dF6Q8= k8s.io/klog v1.0.0/go.mod h1:4Bi6QPql/J/LkTDqv7R/cd3hPo4k2DG6Ptcz060Ez5I= k8s.io/klog/v2 v2.0.0/go.mod h1:PBfzABfn139FHAV07az/IF9Wp1bkk3vpT2XSJ76fSDE= -k8s.io/klog/v2 v2.8.0/go.mod h1:hy9LJ/NvuK+iVyP4Ehqva4HxZG/oXyIS3n3Jmire4Ec= -k8s.io/klog/v2 v2.9.0 h1:D7HV+n1V57XeZ0m6tdRkfknthUaM06VFbWldOFh8kzM= +k8s.io/klog/v2 v2.4.0/go.mod h1:Od+F08eJP+W3HUb4pSrPpgp9DGU4GzlpG/TmITuYh/Y= k8s.io/klog/v2 v2.9.0/go.mod h1:hy9LJ/NvuK+iVyP4Ehqva4HxZG/oXyIS3n3Jmire4Ec= +k8s.io/klog/v2 v2.20.0 h1:tlyxlSvd63k7axjhuchckaRJm+a92z5GSOrTOQY5sHw= +k8s.io/klog/v2 v2.20.0/go.mod h1:Gm8eSIfQN6457haJuPaMxZw4wyP5k+ykPFlrhQDvhvw= k8s.io/kube-openapi v0.0.0-20200316234421-82d701f24f9d/go.mod h1:F+5wygcW0wmRTnM3cOgIqGivxkwSWIWT5YdsDbeAOaU= -k8s.io/kube-openapi v0.0.0-20210305001622-591a79e4bda7/go.mod h1:wXW5VT87nVfh/iLV8FpR2uDvrFyomxbtb1KivDbvPTE= +k8s.io/kube-openapi v0.0.0-20201113171705-d219536bb9fd/go.mod h1:WOJ3KddDSol4tAGcJo0Tvi+dK12EcqSLqcWsryKMpfM= +k8s.io/kube-openapi v0.0.0-20210421082810-95288971da7e/go.mod h1:vHXdDvt9+2spS2Rx9ql3I8tycm3H9FDfdUoIuKCefvw= +k8s.io/kubernetes v1.13.0/go.mod h1:ocZa8+6APFNC2tX1DZASIbocyYT5jHzqFVsY5aoB7Jk= k8s.io/utils v0.0.0-20191114184206-e782cd3c129f/go.mod h1:sZAwmy6armz5eXlNoLmJcl4F1QuKu7sr+mFQ0byX7Ew= k8s.io/utils v0.0.0-20200414100711-2df71ebbae66/go.mod h1:jPW/WVKK9YHAvNhRxK0md/EJ228hCsBRufyofKtW8HA= -k8s.io/utils v0.0.0-20201110183641-67b214c5f920 h1:CbnUZsM497iRC5QMVkHwyl8s2tB3g7yaSHkYPkpgelw= k8s.io/utils v0.0.0-20201110183641-67b214c5f920/go.mod h1:jPW/WVKK9YHAvNhRxK0md/EJ228hCsBRufyofKtW8HA= +k8s.io/utils v0.0.0-20210819203725-bdf08cb9a70a h1:8dYfu/Fc9Gz2rNJKB9IQRGgQOh2clmRzNIPPY1xLY5g= +k8s.io/utils v0.0.0-20210819203725-bdf08cb9a70a/go.mod h1:jPW/WVKK9YHAvNhRxK0md/EJ228hCsBRufyofKtW8HA= rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8= rsc.io/pdf v0.1.1/go.mod h1:n8OzWcQ6Sp37PL01nO98y4iUCRdTGarVfzxY20ICaU4= rsc.io/quote/v3 v3.1.0/go.mod h1:yEA65RcK8LyAZtP9Kv3t0HmxON59tX3rD+tICJqUlj0= rsc.io/sampler v1.3.0/go.mod h1:T1hPZKmBbMNahiBKFy5HrXp6adAjACjK9JXDnKaTXpA= +sigs.k8s.io/apiserver-network-proxy/konnectivity-client v0.0.14/go.mod h1:LEScyzhFmoF5pso/YSeBstl57mOzx9xlU9n85RGrDQg= +sigs.k8s.io/apiserver-network-proxy/konnectivity-client v0.0.15/go.mod h1:LEScyzhFmoF5pso/YSeBstl57mOzx9xlU9n85RGrDQg= sigs.k8s.io/structured-merge-diff/v2 v2.0.1/go.mod h1:Wb7vfKAodbKgf6tn1Kl0VvGj7mRH6DGaRcixXEJXTsE= sigs.k8s.io/structured-merge-diff/v4 v4.0.2/go.mod h1:bJZC9H9iH24zzfZ/41RGcq60oK1F7G282QMXDPYydCw= -sigs.k8s.io/structured-merge-diff/v4 v4.1.0 h1:C4r9BgJ98vrKnnVCjwCSXcWjWe0NKcUQkmzDXZXGwH8= -sigs.k8s.io/structured-merge-diff/v4 v4.1.0/go.mod h1:bJZC9H9iH24zzfZ/41RGcq60oK1F7G282QMXDPYydCw= +sigs.k8s.io/structured-merge-diff/v4 v4.0.3/go.mod h1:bJZC9H9iH24zzfZ/41RGcq60oK1F7G282QMXDPYydCw= +sigs.k8s.io/structured-merge-diff/v4 v4.1.2 h1:Hr/htKFmJEbtMgS/UD0N+gtgctAqz81t3nu+sPzynno= +sigs.k8s.io/structured-merge-diff/v4 v4.1.2/go.mod h1:j/nl6xW8vLS49O8YvXW1ocPhZawJtm+Yrr7PPRQ0Vg4= sigs.k8s.io/yaml v1.1.0/go.mod h1:UJmg0vDUVViEyp3mgSv9WPwZCDxu4rQW1olrI1uml+o= sigs.k8s.io/yaml v1.2.0 h1:kr/MCeFWJWTwyaHoR9c8EjH9OumOmoF9YGiZd7lFm/Q= sigs.k8s.io/yaml v1.2.0/go.mod h1:yfXDCHCao9+ENCvLSE62v9VSji2MKu5jeNfTrofGhJc= diff --git a/src/cmd/tools/query_index_segments/main/main.go b/src/cmd/tools/query_index_segments/main/main.go index d165306705..76db473e2d 100644 --- a/src/cmd/tools/query_index_segments/main/main.go +++ b/src/cmd/tools/query_index_segments/main/main.go @@ -42,7 +42,7 @@ import ( xsync "github.com/m3db/m3/src/x/sync" "github.com/pborman/getopt" - "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/model/labels" "go.uber.org/zap" ) diff --git a/src/query/api/v1/handler/prom/mocks.go b/src/query/api/v1/handler/prom/mocks.go index 87f25797e0..f001aebd61 100644 --- a/src/query/api/v1/handler/prom/mocks.go +++ b/src/query/api/v1/handler/prom/mocks.go @@ -29,7 +29,7 @@ import ( "github.com/go-kit/kit/log" kitlogzap "github.com/go-kit/kit/log/zap" - "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/promql" promstorage "github.com/prometheus/prometheus/storage" "go.uber.org/zap/zapcore" @@ -60,11 +60,11 @@ func (q *mockQuerier) Select( return &mockSeriesSet{mockOptions: q.mockOptions} } -func (*mockQuerier) LabelValues(name string, matchers ...*labels.Matcher) ([]string, promstorage.Warnings, error) { +func (*mockQuerier) LabelValues(string, ...*labels.Matcher) ([]string, promstorage.Warnings, error) { return nil, nil, errors.New("not implemented") } -func (*mockQuerier) LabelNames() ([]string, promstorage.Warnings, error) { +func (*mockQuerier) LabelNames(...*labels.Matcher) ([]string, promstorage.Warnings, error) { return nil, nil, errors.New("not implemented") } diff --git a/src/query/api/v1/handler/prom/read_test.go b/src/query/api/v1/handler/prom/read_test.go index 504d765c22..a644e9e637 100644 --- a/src/query/api/v1/handler/prom/read_test.go +++ b/src/query/api/v1/handler/prom/read_test.go @@ -39,7 +39,7 @@ import ( xerrors "github.com/m3db/m3/src/x/errors" "github.com/m3db/m3/src/x/instrument" - "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/promql" promstorage "github.com/prometheus/prometheus/storage" "github.com/stretchr/testify/require" diff --git a/src/query/api/v1/handler/prometheus/remote/read.go b/src/query/api/v1/handler/prometheus/remote/read.go index 74ad014b3a..a7c44d5245 100644 --- a/src/query/api/v1/handler/prometheus/remote/read.go +++ b/src/query/api/v1/handler/prometheus/remote/read.go @@ -51,7 +51,7 @@ import ( "github.com/golang/protobuf/proto" "github.com/golang/snappy" - "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/model/labels" promql "github.com/prometheus/prometheus/promql/parser" "github.com/uber-go/tally" "go.uber.org/zap" diff --git a/src/query/parser/promql/matcher_test.go b/src/query/parser/promql/matcher_test.go index 79f67d7384..b617cd89ec 100644 --- a/src/query/parser/promql/matcher_test.go +++ b/src/query/parser/promql/matcher_test.go @@ -27,7 +27,7 @@ import ( "github.com/m3db/m3/src/query/models" - "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/model/labels" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) diff --git a/src/query/parser/promql/matchers.go b/src/query/parser/promql/matchers.go index 6bcf32113b..f64ca1b6f0 100644 --- a/src/query/parser/promql/matchers.go +++ b/src/query/parser/promql/matchers.go @@ -37,7 +37,7 @@ import ( "github.com/m3db/m3/src/query/parser/common" "github.com/prometheus/common/model" - "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/model/labels" promql "github.com/prometheus/prometheus/promql/parser" ) diff --git a/src/query/parser/promql/options.go b/src/query/parser/promql/options.go index 2ecce2db0f..7c83f7edbf 100644 --- a/src/query/parser/promql/options.go +++ b/src/query/parser/promql/options.go @@ -23,12 +23,12 @@ package promql import ( "time" - "github.com/prometheus/prometheus/pkg/labels" - pql "github.com/prometheus/prometheus/promql/parser" - "github.com/m3db/m3/src/query/models" "github.com/m3db/m3/src/query/parser" xclock "github.com/m3db/m3/src/x/clock" + + "github.com/prometheus/prometheus/model/labels" + pql "github.com/prometheus/prometheus/promql/parser" ) // ParseFunctionExpr parses arguments to a function expression, returning diff --git a/src/query/storage/prometheus/prometheus_storage.go b/src/query/storage/prometheus/prometheus_storage.go index 86c39c1089..2f2bfb2dca 100644 --- a/src/query/storage/prometheus/prometheus_storage.go +++ b/src/query/storage/prometheus/prometheus_storage.go @@ -27,7 +27,7 @@ import ( "github.com/pkg/errors" "github.com/prometheus/common/model" - "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/model/labels" promstorage "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/uber-go/tally" @@ -161,13 +161,13 @@ func (q *querier) Select( return seriesSet } -func (q *querier) LabelValues(name string, matchers ...*labels.Matcher) ([]string, promstorage.Warnings, error) { +func (q *querier) LabelValues(string, ...*labels.Matcher) ([]string, promstorage.Warnings, error) { // TODO (@shreyas): Implement this. q.logger.Warn("calling unsupported LabelValues method") return nil, nil, errors.New("not implemented") } -func (q *querier) LabelNames() ([]string, promstorage.Warnings, error) { +func (q *querier) LabelNames(...*labels.Matcher) ([]string, promstorage.Warnings, error) { // TODO (@shreyas): Implement this. q.logger.Warn("calling unsupported LabelNames method") return nil, nil, errors.New("not implemented") diff --git a/src/query/storage/prometheus/prometheus_storage_test.go b/src/query/storage/prometheus/prometheus_storage_test.go index 4d0abb1417..89326c7a9b 100644 --- a/src/query/storage/prometheus/prometheus_storage_test.go +++ b/src/query/storage/prometheus/prometheus_storage_test.go @@ -27,6 +27,7 @@ import ( "time" "github.com/golang/mock/gomock" + "github.com/m3db/m3/src/query/block" "github.com/m3db/m3/src/query/generated/proto/prompb" "github.com/m3db/m3/src/query/models" @@ -34,7 +35,7 @@ import ( "github.com/m3db/m3/src/x/instrument" xtest "github.com/m3db/m3/src/x/test" - "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/model/labels" promstorage "github.com/prometheus/prometheus/storage" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" diff --git a/src/query/test/compatibility/test.go b/src/query/test/compatibility/test.go index 057245d922..743f35afe4 100644 --- a/src/query/test/compatibility/test.go +++ b/src/query/test/compatibility/test.go @@ -52,7 +52,7 @@ import ( "github.com/pkg/errors" "github.com/prometheus/common/model" - "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/promql" "github.com/prometheus/prometheus/promql/parser" "github.com/prometheus/prometheus/util/testutil" From f9379ea3dc41c57b243cbc2da8f54bc38b5164b7 Mon Sep 17 00:00:00 2001 From: Rob Skillington Date: Wed, 8 Jun 2022 14:25:43 -0400 Subject: [PATCH 8/8] [query] Add Graphite find integration test that verifies complex trees return valid results (#3636) --- src/dbnode/integration/graphite_find_test.go | 473 +++++++++++++++++++ src/dbnode/integration/setup.go | 107 ++++- src/dbnode/storage/coldflush.go | 9 +- src/query/server/query.go | 16 +- src/query/stores/m3db/async_session.go | 4 +- src/x/test/diff.go | 18 +- src/x/test/util.go | 22 + 7 files changed, 623 insertions(+), 26 deletions(-) create mode 100644 src/dbnode/integration/graphite_find_test.go diff --git a/src/dbnode/integration/graphite_find_test.go b/src/dbnode/integration/graphite_find_test.go new file mode 100644 index 0000000000..d655c1fd67 --- /dev/null +++ b/src/dbnode/integration/graphite_find_test.go @@ -0,0 +1,473 @@ +//go:build integration +// +build integration + +// Copyright (c) 2021 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package integration + +import ( + "context" + "encoding/json" + "fmt" + "math/rand" + "net/http" + "net/url" + "reflect" + "runtime" + "sort" + "strings" + "sync" + "testing" + "time" + + // nolint: gci + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "go.uber.org/atomic" + "go.uber.org/zap" + + "github.com/m3db/m3/src/dbnode/integration/generate" + "github.com/m3db/m3/src/dbnode/namespace" + "github.com/m3db/m3/src/dbnode/retention" + graphitehandler "github.com/m3db/m3/src/query/api/v1/handler/graphite" + "github.com/m3db/m3/src/query/graphite/graphite" + "github.com/m3db/m3/src/x/ident" + xhttp "github.com/m3db/m3/src/x/net/http" + xsync "github.com/m3db/m3/src/x/sync" + xtest "github.com/m3db/m3/src/x/test" +) + +type testGraphiteFindDatasetSize uint + +const ( + smallDatasetSize testGraphiteFindDatasetSize = iota + largeDatasetSize +) + +type testGraphiteFindOptions struct { + checkConcurrency int + datasetSize testGraphiteFindDatasetSize +} + +func TestGraphiteFindSequential(t *testing.T) { + // NB(rob): We need to investigate why using high concurrency (and hence + // need to use small dataset size since otherwise verification takes + // forever) encounters errors running on CI. + testGraphiteFind(t, testGraphiteFindOptions{ + checkConcurrency: 1, + datasetSize: smallDatasetSize, + }) +} + +func TestGraphiteFindParallel(t *testing.T) { + // Skip until investigation of why check concurrency encounters errors on CI. + t.SkipNow() + testGraphiteFind(t, testGraphiteFindOptions{ + checkConcurrency: runtime.NumCPU(), + datasetSize: largeDatasetSize, + }) +} + +func testGraphiteFind(tt *testing.T, testOpts testGraphiteFindOptions) { + if testing.Short() { + tt.SkipNow() // Just skip if we're doing a short run + } + + // Make sure that parallel assertions fail test immediately + // by using a TestingT that panics when FailNow is called. + t := xtest.FailNowPanicsTestingT(tt) + + const queryConfigYAML = ` +listenAddress: 127.0.0.1:7201 + +logging: + level: info + +metrics: + scope: + prefix: "coordinator" + prometheus: + handlerPath: /metrics + listenAddress: "127.0.0.1:0" + sanitization: prometheus + samplingRate: 1.0 + +local: + namespaces: + - namespace: default + type: unaggregated + retention: 12h + - namespace: testns + type: aggregated + retention: 12h + resolution: 1m +` + + var ( + blockSize = 2 * time.Hour + retentionPeriod = 6 * blockSize + rOpts = retention.NewOptions(). + SetRetentionPeriod(retentionPeriod). + SetBlockSize(blockSize) + idxOpts = namespace.NewIndexOptions(). + SetEnabled(true). + SetBlockSize(2 * blockSize) + nOpts = namespace.NewOptions(). + SetRetentionOptions(rOpts). + SetIndexOptions(idxOpts) + ) + ns, err := namespace.NewMetadata(ident.StringID("testns"), nOpts) + require.NoError(t, err) + + opts := NewTestOptions(tt). + SetNamespaces([]namespace.Metadata{ns}) + + // Test setup. + setup, err := NewTestSetup(tt, opts, nil) + require.NoError(t, err) + defer setup.Close() + + log := setup.StorageOpts().InstrumentOptions().Logger(). + With(zap.String("ns", ns.ID().String())) + + require.NoError(t, setup.InitializeBootstrappers(InitializeBootstrappersOptions{ + WithFileSystem: true, + })) + + // Write test data. + now := setup.NowFn()() + + // Create graphite node tree for tests. + var ( + // nolint: gosec + randConstSeedSrc = rand.NewSource(123456789) + // nolint: gosec + randGen = rand.New(randConstSeedSrc) + rootNode = &graphiteNode{} + buildNodes func(node *graphiteNode, level int) + generateSeries []generate.Series + levels int + entriesPerLevelMin int + entriesPerLevelMax int + ) + switch testOpts.datasetSize { + case smallDatasetSize: + levels = 4 + entriesPerLevelMin = 5 + entriesPerLevelMax = 7 + case largeDatasetSize: + // Ideally we'd always use a large dataset size, however you do need + // high concurrency to validate this entire dataset and CI can't seem + // to handle high concurrency without encountering errors. + levels = 5 + entriesPerLevelMin = 6 + entriesPerLevelMax = 9 + default: + require.FailNow(t, fmt.Sprintf("invalid test dataset size set: %d", testOpts.datasetSize)) + } + + buildNodes = func(node *graphiteNode, level int) { + entries := entriesPerLevelMin + + randGen.Intn(entriesPerLevelMax-entriesPerLevelMin) + for entry := 0; entry < entries; entry++ { + name := fmt.Sprintf("lvl%02d_entry%02d", level, entry) + + // Create a directory node and spawn more underneath. + if nextLevel := level + 1; nextLevel <= levels { + childDir := node.child(name+"_dir", graphiteNodeChildOptions{ + isLeaf: false, + }) + buildNodes(childDir, nextLevel) + } + + // Create a leaf node. + childLeaf := node.child(name+"_leaf", graphiteNodeChildOptions{ + isLeaf: true, + }) + + // Create series to generate data for the leaf node. + tags := make([]ident.Tag, 0, len(childLeaf.pathParts)) + for i, pathPartValue := range childLeaf.pathParts { + tags = append(tags, ident.Tag{ + Name: graphite.TagNameID(i), + Value: ident.StringID(pathPartValue), + }) + } + series := generate.Series{ + ID: ident.StringID(strings.Join(childLeaf.pathParts, ".")), + Tags: ident.NewTags(tags...), + } + generateSeries = append(generateSeries, series) + } + } + + // Build tree. + log.Info("building graphite data set series") + buildNodes(rootNode, 0) + + // Generate and write test data. + log.Info("generating graphite data set datapoints", + zap.Int("seriesSize", len(generateSeries))) + generateBlocks := make([]generate.BlockConfig, 0, len(generateSeries)) + for _, series := range generateSeries { + generateBlocks = append(generateBlocks, []generate.BlockConfig{ + { + IDs: []string{series.ID.String()}, + Tags: series.Tags, + NumPoints: 1, + Start: now.Add(-1 * blockSize), + }, + { + IDs: []string{series.ID.String()}, + Tags: series.Tags, + NumPoints: 1, + Start: now, + }, + }...) + } + seriesMaps := generate.BlocksByStart(generateBlocks) + log.Info("writing graphite data set to disk", + zap.Int("seriesMapSize", len(seriesMaps))) + require.NoError(t, writeTestDataToDisk(ns, setup, seriesMaps, 0)) + + // Start the server with filesystem bootstrapper. + log.Info("starting server") + require.NoError(t, setup.StartServer()) + log.Info("server is now up") + + // Stop the server. + defer func() { + require.NoError(t, setup.StopServer()) + log.Info("server is now down") + }() + + // Start the query server + log.Info("starting query server") + require.NoError(t, setup.StartQuery(queryConfigYAML)) + log.Info("started query server", zap.String("addr", setup.QueryAddress())) + + // Stop the query server. + defer func() { + require.NoError(t, setup.StopQuery()) + log.Info("query server is now down") + }() + + // Check each level of the tree can answer expected queries. + type checkResult struct { + leavesVerified int + } + type checkFailure struct { + expected graphiteFindResults + actual graphiteFindResults + failMsg string + } + var ( + verifyFindQueries func(node *graphiteNode, level int) (checkResult, *checkFailure, error) + parallelVerifyFindQueries func(node *graphiteNode, level int) + checkedSeriesAbort = atomic.NewBool(false) + numSeriesChecking = uint64(len(generateSeries)) + checkedSeriesLogEvery = numSeriesChecking / 10 + checkedSeries = atomic.NewUint64(0) + checkedSeriesLog = atomic.NewUint64(0) + // Use custom http client for higher number of max idle conns. + httpClient = xhttp.NewHTTPClient(xhttp.DefaultHTTPClientOptions()) + wg sync.WaitGroup + workerPool = xsync.NewWorkerPool(testOpts.checkConcurrency) + ) + workerPool.Init() + parallelVerifyFindQueries = func(node *graphiteNode, level int) { + // Verify this node at level. + wg.Add(1) + workerPool.Go(func() { + defer wg.Done() + + if checkedSeriesAbort.Load() { + // Do not execute if aborted. + return + } + + result, failure, err := verifyFindQueries(node, level) + if failure == nil && err == nil { + // Account for series checked (for progress report). + checkedSeries.Add(uint64(result.leavesVerified)) + return + } + + // Bail parallel execution (failed require/assert won't stop execution). + if checkedSeriesAbort.CAS(false, true) { + switch { + case failure != nil: + // Assert an error result and log once. + assert.Equal(t, failure.expected, failure.actual, failure.failMsg) + log.Error("aborting checks due to mismatch") + case err != nil: + assert.NoError(t, err) + log.Error("aborting checks due to error") + default: + require.FailNow(t, "unknown error condition") + log.Error("aborting checks due to unknown condition") + } + } + }) + + // Verify children of children. + for _, child := range node.children { + parallelVerifyFindQueries(child, level+1) + } + } + verifyFindQueries = func(node *graphiteNode, level int) (checkResult, *checkFailure, error) { + var r checkResult + + // Write progress report if progress made. + checked := checkedSeries.Load() + nextLog := checked - (checked % checkedSeriesLogEvery) + if lastLog := checkedSeriesLog.Swap(nextLog); lastLog < nextLog { + log.Info("checked series progressing", zap.Int("checked", int(checked))) + } + + // Verify at depth. + numPathParts := len(node.pathParts) + queryPathParts := make([]string, 0, 1+numPathParts) + if numPathParts > 0 { + queryPathParts = append(queryPathParts, node.pathParts...) + } + queryPathParts = append(queryPathParts, "*") + query := strings.Join(queryPathParts, ".") + + params := make(url.Values) + params.Set("query", query) + + url := fmt.Sprintf("http://%s%s?%s", setup.QueryAddress(), + graphitehandler.FindURL, params.Encode()) + + req, err := http.NewRequestWithContext(context.Background(), + http.MethodGet, url, nil) + require.NoError(t, err) + + res, err := httpClient.Do(req) + if err != nil { + return r, nil, err + } + if res.StatusCode != http.StatusOK { + return r, nil, fmt.Errorf("bad response code: expected=%d, actual=%d", + http.StatusOK, res.StatusCode) + } + + defer res.Body.Close() + + // Compare results. + var actual graphiteFindResults + if err := json.NewDecoder(res.Body).Decode(&actual); err != nil { + return r, nil, err + } + + expected := make(graphiteFindResults, 0, len(node.children)) + for _, child := range node.children { + leaf := 0 + if child.isLeaf { + leaf = 1 + r.leavesVerified++ + } + expected = append(expected, graphiteFindResult{ + Text: child.name, + Leaf: leaf, + }) + } + + sortGraphiteFindResults(actual) + sortGraphiteFindResults(expected) + + if !reflect.DeepEqual(expected, actual) { + failMsg := fmt.Sprintf("invalid results: level=%d, parts=%d, query=%s", + level, len(node.pathParts), query) + failMsg += fmt.Sprintf("\n\ndiff:\n%s\n\n", + xtest.Diff(xtest.MustPrettyJSONObject(t, expected), + xtest.MustPrettyJSONObject(t, actual))) + return r, &checkFailure{ + expected: expected, + actual: actual, + failMsg: failMsg, + }, nil + } + + return r, nil, nil + } + + // Check all top level entries and recurse. + log.Info("checking series", + zap.Int("checkConcurrency", testOpts.checkConcurrency), + zap.Uint64("numSeriesChecking", numSeriesChecking)) + parallelVerifyFindQueries(rootNode, 0) + + // Wait for execution. + wg.Wait() + + // Allow for debugging by issuing queries, etc. + if DebugTest() { + log.Info("debug test set, pausing for investigate") + <-make(chan struct{}) + } +} + +type graphiteFindResults []graphiteFindResult + +type graphiteFindResult struct { + Text string `json:"text"` + Leaf int `json:"leaf"` +} + +func sortGraphiteFindResults(r graphiteFindResults) { + sort.Slice(r, func(i, j int) bool { + if r[i].Leaf != r[j].Leaf { + return r[i].Leaf < r[j].Leaf + } + return r[i].Text < r[j].Text + }) +} + +type graphiteNode struct { + name string + pathParts []string + isLeaf bool + children []*graphiteNode +} + +type graphiteNodeChildOptions struct { + isLeaf bool +} + +func (n *graphiteNode) child( + name string, + opts graphiteNodeChildOptions, +) *graphiteNode { + pathParts := append(make([]string, 0, 1+len(n.pathParts)), n.pathParts...) + pathParts = append(pathParts, name) + + child := &graphiteNode{ + name: name, + pathParts: pathParts, + isLeaf: opts.isLeaf, + } + + n.children = append(n.children, child) + + return child +} diff --git a/src/dbnode/integration/setup.go b/src/dbnode/integration/setup.go index 9c3a3c1360..b0f7da8684 100644 --- a/src/dbnode/integration/setup.go +++ b/src/dbnode/integration/setup.go @@ -25,6 +25,7 @@ import ( "flag" "fmt" "io/ioutil" + "net" "os" "os/exec" "strings" @@ -32,8 +33,18 @@ import ( "testing" "time" + // nolint: gci + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/uber-go/tally" + "github.com/uber/tchannel-go" + "go.uber.org/zap" + "go.uber.org/zap/zapcore" + + clusterclient "github.com/m3db/m3/src/cluster/client" "github.com/m3db/m3/src/cluster/services" "github.com/m3db/m3/src/cluster/shard" + queryconfig "github.com/m3db/m3/src/cmd/services/m3query/config" "github.com/m3db/m3/src/dbnode/client" "github.com/m3db/m3/src/dbnode/generated/thrift/rpc" "github.com/m3db/m3/src/dbnode/integration/fake" @@ -58,17 +69,13 @@ import ( "github.com/m3db/m3/src/dbnode/testdata/prototest" "github.com/m3db/m3/src/dbnode/topology" "github.com/m3db/m3/src/dbnode/ts" + queryserver "github.com/m3db/m3/src/query/server" "github.com/m3db/m3/src/x/clock" + xconfig "github.com/m3db/m3/src/x/config" "github.com/m3db/m3/src/x/ident" "github.com/m3db/m3/src/x/instrument" xsync "github.com/m3db/m3/src/x/sync" xtime "github.com/m3db/m3/src/x/time" - - "github.com/stretchr/testify/require" - "github.com/uber-go/tally" - "github.com/uber/tchannel-go" - "go.uber.org/zap" - "go.uber.org/zap/zapcore" ) var ( @@ -127,6 +134,7 @@ type testSetup struct { m3dbAdminClient client.AdminClient m3dbVerificationAdminClient client.AdminClient workerPool xsync.WorkerPool + queryAddress string // compare expected with actual data function assertEqual assertTestDataEqual @@ -137,8 +145,10 @@ type testSetup struct { namespaces []namespace.Metadata // signals - doneCh chan struct{} - closedCh chan struct{} + doneCh chan struct{} + closedCh chan struct{} + queryInterruptCh chan error + queryDoneCh chan struct{} } type xNowFn func() xtime.UnixNano @@ -170,6 +180,9 @@ type TestSetup interface { StopServerAndVerifyOpenFilesAreClosed() error StartServer() error StartServerDontWaitBootstrap() error + StopQuery() error + StartQuery(configYAML string) error + QueryAddress() string NowFn() xNowFn ClockNowFn() clock.NowFn SetNowFn(xtime.UnixNano) @@ -843,6 +856,66 @@ func openFiles(parentDir string) []string { return strings.Split(string(out), "\n") } +func (ts *testSetup) StartQuery(configYAML string) error { + m3dbClient := ts.m3dbClient + if m3dbClient == nil { + return fmt.Errorf("dbnode admin client not set") + } + + configFile, cleanup := newTestFile(ts.t, "config.yaml", configYAML) + defer cleanup() + + var cfg queryconfig.Configuration + err := xconfig.LoadFile(&cfg, configFile.Name(), xconfig.Options{}) + if err != nil { + return err + } + + dbClientCh := make(chan client.Client, 1) + dbClientCh <- m3dbClient + clusterClientCh := make(chan clusterclient.Client, 1) + listenerCh := make(chan net.Listener, 1) + localSessionReadyCh := make(chan struct{}, 1) + + ts.queryInterruptCh = make(chan error, 1) + ts.queryDoneCh = make(chan struct{}, 1) + + go func() { + queryserver.Run(queryserver.RunOptions{ + Config: cfg, + InterruptCh: ts.queryInterruptCh, + ListenerCh: listenerCh, + LocalSessionReadyCh: localSessionReadyCh, + DBClient: dbClientCh, + ClusterClient: clusterClientCh, + }) + ts.queryDoneCh <- struct{}{} + }() + + // Wait for local session to connect. + <-localSessionReadyCh + + // Wait for listener. + listener := <-listenerCh + ts.queryAddress = listener.Addr().String() + + return nil +} + +func (ts *testSetup) StopQuery() error { + // Send interrupt. + ts.queryInterruptCh <- fmt.Errorf("interrupt") + + // Wait for done. + <-ts.queryDoneCh + + return nil +} + +func (ts *testSetup) QueryAddress() string { + return ts.queryAddress +} + func (ts *testSetup) TChannelClient() *TestTChannelClient { return ts.tchannelClient } @@ -1189,3 +1262,21 @@ func mustInspectFilesystem(fsOpts fs.Options) fs.Inspection { return inspection } + +func newTestFile(t *testing.T, fileName, contents string) (*os.File, closeFn) { + tmpFile, err := ioutil.TempFile("", fileName) + require.NoError(t, err) + + _, err = tmpFile.WriteString(contents) + require.NoError(t, err) + + return tmpFile, func() { + assert.NoError(t, tmpFile.Close()) + assert.NoError(t, os.Remove(tmpFile.Name())) + } +} + +// DebugTest allows testing to see if a standard debug test env var is set. +func DebugTest() bool { + return os.Getenv("DEBUG_TEST") == "true" +} diff --git a/src/dbnode/storage/coldflush.go b/src/dbnode/storage/coldflush.go index 23b7f4ae4b..9e341241f2 100644 --- a/src/dbnode/storage/coldflush.go +++ b/src/dbnode/storage/coldflush.go @@ -108,9 +108,8 @@ func (m *coldFlushManager) Run(t xtime.UnixNano) bool { m.Unlock() }() - debugLog := m.log.Check(zapcore.DebugLevel, "cold flush run") - if debugLog != nil { - debugLog.Write(zap.String("status", "starting cold flush"), zap.Time("time", t.ToTime())) + if log := m.log.Check(zapcore.DebugLevel, "cold flush run start"); log != nil { + log.Write(zap.Time("time", t.ToTime())) } // NB(xichen): perform data cleanup and flushing sequentially to minimize the impact of disk seeks. @@ -133,8 +132,8 @@ func (m *coldFlushManager) Run(t xtime.UnixNano) bool { }) } - if debugLog != nil { - debugLog.Write(zap.String("status", "completed cold flush"), zap.Time("time", t.ToTime())) + if log := m.log.Check(zapcore.DebugLevel, "cold flush run complete"); log != nil { + log.Write(zap.Time("time", t.ToTime())) } return true diff --git a/src/query/server/query.go b/src/query/server/query.go index bb4487b31f..f4ed82eb7c 100644 --- a/src/query/server/query.go +++ b/src/query/server/query.go @@ -136,6 +136,10 @@ type RunOptions struct { // ready signal once it is open. DownsamplerReadyCh chan<- struct{} + // LocalSessionReadyCh is a programmatic channel to receive the + // local DB session ready signal once it is open. + LocalSessionReadyCh chan struct{} + // InstrumentOptionsReadyCh is a programmatic channel to receive a set of // instrument options and metric reporters that is delivered when // constructed. @@ -472,7 +476,8 @@ func Run(runOpts RunOptions) RunResult { // which generates a session and use the storage with the session. m3dbClusters, m3dbPoolWrapper, err = initClusters(cfg, runOpts.DBConfig, clusterNamespacesWatcher, runOpts.DBClient, encodingOpts, - instrumentOptions, tsdbOpts.CustomAdminOptions()) + runOpts.LocalSessionReadyCh, instrumentOptions, + tsdbOpts.CustomAdminOptions()) if err != nil { logger.Fatal("unable to init clusters", zap.Error(err)) } @@ -968,6 +973,7 @@ func initClusters( clusterNamespacesWatcher m3.ClusterNamespacesWatcher, dbClientCh <-chan client.Client, encodingOpts encoding.Options, + localSessionReadyCh chan struct{}, instrumentOpts instrument.Options, customAdminOptions []client.CustomAdminOption, ) (m3.Clusters, *pools.PoolWrapper, error) { @@ -1018,10 +1024,12 @@ func initClusters( return nil, nil, errors.New("no clusters configured and not running local cluster") } - sessionInitChan := make(chan struct{}) + if localSessionReadyCh == nil { + localSessionReadyCh = make(chan struct{}) + } session := m3db.NewAsyncSession(func() (client.Client, error) { return <-dbClientCh, nil - }, sessionInitChan) + }, localSessionReadyCh) clusterStaticConfig := m3.ClusterStaticConfiguration{ Namespaces: localCfg.Namespaces, @@ -1065,7 +1073,7 @@ func initClusters( poolWrapper = pools.NewAsyncPoolsWrapper() go func() { - <-sessionInitChan + <-localSessionReadyCh poolWrapper.Init(session.IteratorPools()) }() } diff --git a/src/query/stores/m3db/async_session.go b/src/query/stores/m3db/async_session.go index 4c3fbf4bf0..316f0f7d22 100644 --- a/src/query/stores/m3db/async_session.go +++ b/src/query/stores/m3db/async_session.go @@ -64,9 +64,7 @@ func NewAsyncSession(fn NewClientFn, done chan<- struct{}) *AsyncSession { go func() { if asyncSession.done != nil { - defer func() { - asyncSession.done <- struct{}{} - }() + defer close(asyncSession.done) } c, err := fn() diff --git a/src/x/test/diff.go b/src/x/test/diff.go index 4cf1309d9d..2ff4833fa0 100644 --- a/src/x/test/diff.go +++ b/src/x/test/diff.go @@ -22,12 +22,11 @@ package test import ( "encoding/json" - "testing" - - xjson "github.com/m3db/m3/src/x/json" "github.com/sergi/go-diff/diffmatchpatch" "github.com/stretchr/testify/require" + + xjson "github.com/m3db/m3/src/x/json" ) // Diff is a helper method to print a terminal pretty diff of two strings @@ -39,21 +38,28 @@ func Diff(expected, actual string) string { } // MustPrettyJSONMap returns an indented JSON string of the object. -func MustPrettyJSONMap(t *testing.T, value xjson.Map) string { +func MustPrettyJSONMap(t require.TestingT, value xjson.Map) string { pretty, err := json.MarshalIndent(value, "", " ") require.NoError(t, err) return string(pretty) } // MustPrettyJSONArray returns an indented JSON string of the object. -func MustPrettyJSONArray(t *testing.T, value xjson.Array) string { +func MustPrettyJSONArray(t require.TestingT, value xjson.Array) string { + pretty, err := json.MarshalIndent(value, "", " ") + require.NoError(t, err) + return string(pretty) +} + +// MustPrettyJSONObject returns an indented JSON string of the object. +func MustPrettyJSONObject(t require.TestingT, value interface{}) string { pretty, err := json.MarshalIndent(value, "", " ") require.NoError(t, err) return string(pretty) } // MustPrettyJSONString returns an indented version of the JSON. -func MustPrettyJSONString(t *testing.T, str string) string { +func MustPrettyJSONString(t require.TestingT, str string) string { var unmarshalled map[string]interface{} err := json.Unmarshal([]byte(str), &unmarshalled) require.NoError(t, err) diff --git a/src/x/test/util.go b/src/x/test/util.go index f10f1cb5d6..8ee893eb2c 100644 --- a/src/x/test/util.go +++ b/src/x/test/util.go @@ -22,9 +22,31 @@ package test import ( "reflect" + "testing" "unsafe" + + "github.com/stretchr/testify/require" ) +// FailNowPanicsTestingT returns a TestingT that panics on a failed assertion. +// This is useful for aborting a test on failed assertions from an asynchronous +// goroutine (since stretchr calls FailNow on testing.T but it will not abort +// the test unless the goroutine is the one running the benchmark or test). +// For more info see: https://github.com/stretchr/testify/issues/652. +func FailNowPanicsTestingT(t *testing.T) require.TestingT { + return failNowPanicsTestingT{TestingT: t} +} + +var _ require.TestingT = failNowPanicsTestingT{} + +type failNowPanicsTestingT struct { + require.TestingT +} + +func (t failNowPanicsTestingT) FailNow() { + panic("failed assertion") +} + // ByteSlicesBackedBySameData returns a bool indicating if the raw backing bytes // under the []byte slice point to the same memory. func ByteSlicesBackedBySameData(a, b []byte) bool {