From 8ca03a2a3d73e5250ee26ac3c9218e254550560d Mon Sep 17 00:00:00 2001 From: George Robinson Date: Wed, 17 Jul 2024 06:55:14 +0100 Subject: [PATCH 01/23] feat: Remove flush loop and queue from Ingester RF-1 (#13538) --- pkg/ingester-rf1/flush.go | 68 +++++++++-------------- pkg/ingester-rf1/ingester.go | 94 +++++--------------------------- pkg/storage/wal/manager.go | 41 +++++++++++--- pkg/storage/wal/manager_test.go | 96 ++++++++++++++++++++++++++++++--- 4 files changed, 159 insertions(+), 140 deletions(-) diff --git a/pkg/ingester-rf1/flush.go b/pkg/ingester-rf1/flush.go index f577d0b8851f..4ab36713d392 100644 --- a/pkg/ingester-rf1/flush.go +++ b/pkg/ingester-rf1/flush.go @@ -2,9 +2,9 @@ package ingesterrf1 import ( "crypto/rand" + "errors" "fmt" "net/http" - "strconv" "time" "github.com/dustin/go-humanize" @@ -17,7 +17,6 @@ import ( "golang.org/x/net/context" "github.com/grafana/loki/v3/pkg/storage/wal" - "github.com/grafana/loki/v3/pkg/util" util_log "github.com/grafana/loki/v3/pkg/util/log" ) @@ -38,11 +37,10 @@ const ( // Note: this is called both during the WAL replay (zero or more times) // and then after replay as well. -func (i *Ingester) InitFlushQueues() { - i.flushQueuesDone.Add(i.cfg.ConcurrentFlushes) +func (i *Ingester) InitFlushWorkers() { + i.flushWorkersDone.Add(i.cfg.ConcurrentFlushes) for j := 0; j < i.cfg.ConcurrentFlushes; j++ { - i.flushQueues[j] = util.NewPriorityQueue(i.metrics.flushQueues) - go i.flushLoop(j) + go i.flushWorker(j) } } @@ -50,7 +48,8 @@ func (i *Ingester) InitFlushQueues() { // Flush triggers a flush of all the chunks and closes the flush queues. // Called from the Lifecycler as part of the ingester shutdown. func (i *Ingester) Flush() { - i.flush() + i.wal.Close() + i.flushWorkersDone.Wait() } // TransferOut implements ring.FlushTransferer @@ -60,57 +59,38 @@ func (i *Ingester) TransferOut(_ context.Context) error { return ring.ErrTransferDisabled } -func (i *Ingester) flush() { - // TODO: Flush the last chunks - // Close the flush queues, to unblock waiting workers. - for _, flushQueue := range i.flushQueues { - flushQueue.Close() - } - - i.flushQueuesDone.Wait() - level.Debug(i.logger).Log("msg", "flush queues have drained") -} - // FlushHandler triggers a flush of all in memory chunks. Mainly used for // local testing. func (i *Ingester) FlushHandler(w http.ResponseWriter, _ *http.Request) { w.WriteHeader(http.StatusNoContent) } -type flushOp struct { - it *wal.PendingItem - num int64 -} - -func (o *flushOp) Key() string { - return strconv.Itoa(int(o.num)) -} - -func (o *flushOp) Priority() int64 { - return -o.num -} - -func (i *Ingester) flushLoop(j int) { - l := log.With(i.logger, "loop", j) +func (i *Ingester) flushWorker(j int) { + l := log.With(i.logger, "worker", j) defer func() { - level.Debug(l).Log("msg", "Ingester.flushLoop() exited") - i.flushQueuesDone.Done() + level.Debug(l).Log("msg", "Ingester.flushWorker() exited") + i.flushWorkersDone.Done() }() for { - o := i.flushQueues[j].Dequeue() - if o == nil { + it, err := i.wal.NextPending() + if errors.Is(err, wal.ErrClosed) { return } - op := o.(*flushOp) + + if it == nil { + // TODO: Do something more clever here instead. + time.Sleep(100 * time.Millisecond) + continue + } start := time.Now() // We'll use this to log the size of the segment that was flushed. - n := op.it.Writer.InputSize() + n := it.Writer.InputSize() humanized := humanize.Bytes(uint64(n)) - err := i.flushOp(l, op) + err = i.flushItem(l, it) d := time.Since(start) if err != nil { level.Error(l).Log("msg", "failed to flush", "size", humanized, "duration", d, "err", err) @@ -118,18 +98,18 @@ func (i *Ingester) flushLoop(j int) { level.Debug(l).Log("msg", "flushed", "size", humanized, "duration", d) } - op.it.Result.SetDone(err) - i.wal.Put(op.it) + it.Result.SetDone(err) + i.wal.Put(it) } } -func (i *Ingester) flushOp(l log.Logger, op *flushOp) error { +func (i *Ingester) flushItem(l log.Logger, it *wal.PendingItem) error { ctx, cancelFunc := context.WithCancel(context.Background()) defer cancelFunc() b := backoff.New(ctx, i.cfg.FlushOpBackoff) for b.Ongoing() { - err := i.flushSegment(ctx, op.it.Writer) + err := i.flushSegment(ctx, it.Writer) if err == nil { break } diff --git a/pkg/ingester-rf1/ingester.go b/pkg/ingester-rf1/ingester.go index 3df47c598c13..c3e6afd052d5 100644 --- a/pkg/ingester-rf1/ingester.go +++ b/pkg/ingester-rf1/ingester.go @@ -5,7 +5,6 @@ import ( "flag" "fmt" "io" - "math/rand" "net/http" "os" "path" @@ -202,15 +201,11 @@ type Ingester struct { store Storage periodicConfigs []config.PeriodConfig - loopDone sync.WaitGroup - loopQuit chan struct{} tailersQuit chan struct{} // One queue per flush thread. Fingerprint is used to // pick a queue. - numOps int64 - flushQueues []*util.PriorityQueue - flushQueuesDone sync.WaitGroup + flushWorkersDone sync.WaitGroup wal *wal.Manager @@ -270,17 +265,16 @@ func New(cfg Config, clientConfig client.Config, } i := &Ingester{ - cfg: cfg, - logger: logger, - clientConfig: clientConfig, - tenantConfigs: configs, - instances: map[string]*instance{}, - store: storage, - periodicConfigs: periodConfigs, - loopQuit: make(chan struct{}), - flushQueues: make([]*util.PriorityQueue, cfg.ConcurrentFlushes), - tailersQuit: make(chan struct{}), - metrics: metrics, + cfg: cfg, + logger: logger, + clientConfig: clientConfig, + tenantConfigs: configs, + instances: map[string]*instance{}, + store: storage, + periodicConfigs: periodConfigs, + flushWorkersDone: sync.WaitGroup{}, + tailersQuit: make(chan struct{}), + metrics: metrics, // flushOnShutdownSwitch: &OnceSwitch{}, terminateOnShutdown: false, streamRateCalculator: NewStreamRateCalculator(), @@ -401,7 +395,7 @@ func (i *Ingester) ServeHTTP(w http.ResponseWriter, r *http.Request) { } func (i *Ingester) starting(ctx context.Context) error { - i.InitFlushQueues() + i.InitFlushWorkers() // pass new context to lifecycler, so that it doesn't stop automatically when Ingester's service context is done err := i.lifecycler.StartAsync(context.Background()) @@ -435,9 +429,6 @@ func (i *Ingester) starting(ctx context.Context) error { return fmt.Errorf("can not ensure recalculate owned streams service is running: %w", err) } - // start our loop - i.loopDone.Add(1) - go i.loop() return nil } @@ -457,8 +448,6 @@ func (i *Ingester) running(ctx context.Context) error { // instance.closeTailers() //} - close(i.loopQuit) - i.loopDone.Wait() return serviceError } @@ -474,10 +463,7 @@ func (i *Ingester) stopping(_ error) error { //} errs.Add(services.StopAndAwaitTerminated(context.Background(), i.lifecycler)) - for _, flushQueue := range i.flushQueues { - flushQueue.Close() - } - i.flushQueuesDone.Wait() + i.flushWorkersDone.Wait() // i.streamRateCalculator.Stop() @@ -518,60 +504,6 @@ func (i *Ingester) removeShutdownMarkerFile() { } } -func (i *Ingester) loop() { - defer i.loopDone.Done() - - // Delay the first flush operation by up to 0.8x the flush time period. - // This will ensure that multiple ingesters started at the same time do not - // flush at the same time. Flushing at the same time can cause concurrently - // writing the same chunk to object storage, which in AWS S3 leads to being - // rate limited. - jitter := time.Duration(rand.Int63n(int64(float64(i.cfg.FlushCheckPeriod.Nanoseconds()) * 0.8))) - initialDelay := time.NewTimer(jitter) - defer initialDelay.Stop() - - level.Info(i.logger).Log("msg", "sleeping for initial delay before starting periodic flushing", "delay", jitter) - - select { - case <-initialDelay.C: - // do nothing and continue with flush loop - case <-i.loopQuit: - // ingester stopped while waiting for initial delay - return - } - - // Add +/- 20% of flush interval as jitter. - // The default flush check period is 30s so max jitter will be 6s. - j := i.cfg.FlushCheckPeriod / 5 - flushTicker := util.NewTickerWithJitter(i.cfg.FlushCheckPeriod, j) - defer flushTicker.Stop() - - for { - select { - case <-flushTicker.C: - i.doFlushTick() - case <-i.loopQuit: - return - } - } -} - -func (i *Ingester) doFlushTick() { - for { - // Keep adding ops to the queue until there are no more. - it := i.wal.NextPending() - if it == nil { - break - } - i.numOps++ - flushQueueIndex := i.numOps % int64(i.cfg.ConcurrentFlushes) - i.flushQueues[flushQueueIndex].Enqueue(&flushOp{ - num: i.numOps, - it: it, - }) - } -} - // PrepareShutdown will handle the /ingester/prepare_shutdown endpoint. // // Internally, when triggered, this handler will configure the ingester service to release their resources whenever a SIGTERM is received. diff --git a/pkg/storage/wal/manager.go b/pkg/storage/wal/manager.go index d1359f80601e..cfc89605cb7c 100644 --- a/pkg/storage/wal/manager.go +++ b/pkg/storage/wal/manager.go @@ -25,6 +25,9 @@ const ( ) var ( + // ErrClosed is returned when the WAL is closed. + ErrClosed = errors.New("WAL is closed") + // ErrFull is returned when an append fails because the WAL is full. This // happens when all segments are either in the pending list waiting to be // flushed, or in the process of being flushed. @@ -111,9 +114,10 @@ type Manager struct { // pending is a list of segments that are waiting to be flushed. Once // flushed, the segment is reset and moved to the back of the available // list to accept writes again. - pending *list.List - shutdown chan struct{} - mu sync.Mutex + pending *list.List + + closed bool + mu sync.Mutex } // item is similar to PendingItem, but it is an internal struct used in the @@ -141,7 +145,6 @@ func NewManager(cfg Config, metrics *Metrics) (*Manager, error) { metrics: metrics.ManagerMetrics, available: list.New(), pending: list.New(), - shutdown: make(chan struct{}), } m.metrics.NumPending.Set(0) m.metrics.NumFlushing.Set(0) @@ -162,6 +165,9 @@ func NewManager(cfg Config, metrics *Metrics) (*Manager, error) { func (m *Manager) Append(r AppendRequest) (*AppendResult, error) { m.mu.Lock() defer m.mu.Unlock() + if m.closed { + return nil, ErrClosed + } if m.available.Len() == 0 { return nil, ErrFull } @@ -185,9 +191,25 @@ func (m *Manager) Append(r AppendRequest) (*AppendResult, error) { return it.r, nil } +func (m *Manager) Close() { + m.mu.Lock() + defer m.mu.Unlock() + m.closed = true + if m.available.Len() > 0 { + el := m.available.Front() + it := el.Value.(*item) + if it.w.InputSize() > 0 { + m.pending.PushBack(it) + m.metrics.NumPending.Inc() + m.available.Remove(el) + m.metrics.NumAvailable.Dec() + } + } +} + // NextPending returns the next segment to be flushed. It returns nil if the -// pending list is empty. -func (m *Manager) NextPending() *PendingItem { +// pending list is empty, and ErrClosed if the WAL is closed. +func (m *Manager) NextPending() (*PendingItem, error) { m.mu.Lock() defer m.mu.Unlock() if m.pending.Len() == 0 { @@ -205,7 +227,10 @@ func (m *Manager) NextPending() *PendingItem { } // If the pending list is still empty return nil. if m.pending.Len() == 0 { - return nil + if m.closed { + return nil, ErrClosed + } + return nil, nil } } el := m.pending.Front() @@ -213,7 +238,7 @@ func (m *Manager) NextPending() *PendingItem { m.pending.Remove(el) m.metrics.NumPending.Dec() m.metrics.NumFlushing.Inc() - return &PendingItem{Result: it.r, Writer: it.w} + return &PendingItem{Result: it.r, Writer: it.w}, nil } // Put resets the segment and puts it back in the available list to accept diff --git a/pkg/storage/wal/manager_test.go b/pkg/storage/wal/manager_test.go index 64c1056b395e..f5fee51ff166 100644 --- a/pkg/storage/wal/manager_test.go +++ b/pkg/storage/wal/manager_test.go @@ -171,6 +171,40 @@ func TestManager_AppendMaxSize(t *testing.T) { require.Equal(t, 1, m.pending.Len()) } +func TestManager_AppendWALClosed(t *testing.T) { + m, err := NewManager(Config{ + MaxAge: 30 * time.Second, + MaxSegments: 10, + MaxSegmentSize: 1024, // 1KB + }, NewMetrics(nil)) + require.NoError(t, err) + + // Append some data. + lbs := labels.Labels{{Name: "a", Value: "b"}} + entries := []*logproto.Entry{{Timestamp: time.Now(), Line: "c"}} + res, err := m.Append(AppendRequest{ + TenantID: "1", + Labels: lbs, + LabelsStr: lbs.String(), + Entries: entries, + }) + require.NoError(t, err) + require.NotNil(t, res) + + // Close the WAL. + m.Close() + + // Should not be able to append more data as the WAL is closed. + res, err = m.Append(AppendRequest{ + TenantID: "1", + Labels: lbs, + LabelsStr: lbs.String(), + Entries: entries, + }) + require.Nil(t, res) + require.ErrorIs(t, err, ErrClosed) +} + func TestManager_AppendWALFull(t *testing.T) { m, err := NewManager(Config{ MaxAge: 30 * time.Second, @@ -216,7 +250,8 @@ func TestManager_NextPending(t *testing.T) { // There should be no segments waiting to be flushed as no data has been // written. - it := m.NextPending() + it, err := m.NextPending() + require.NoError(t, err) require.Nil(t, it) // Append 1KB of data. @@ -231,11 +266,54 @@ func TestManager_NextPending(t *testing.T) { require.NoError(t, err) // There should be a segment waiting to be flushed. - it = m.NextPending() + it, err = m.NextPending() + require.NoError(t, err) require.NotNil(t, it) // There should be no more segments waiting to be flushed. - it = m.NextPending() + it, err = m.NextPending() + require.NoError(t, err) + require.Nil(t, it) +} + +func TestManager_NextPendingClosed(t *testing.T) { + m, err := NewManager(Config{ + MaxAge: 30 * time.Second, + MaxSegments: 10, + MaxSegmentSize: 1024, // 1KB + }, NewMetrics(nil)) + require.NoError(t, err) + + // Append some data. + lbs := labels.Labels{{Name: "a", Value: "b"}} + entries := []*logproto.Entry{{Timestamp: time.Now(), Line: "c"}} + res, err := m.Append(AppendRequest{ + TenantID: "1", + Labels: lbs, + LabelsStr: lbs.String(), + Entries: entries, + }) + require.NoError(t, err) + require.NotNil(t, res) + + // There should be no segments waiting to be flushed as neither the maximum + // age nor maximum size has been exceeded. + it, err := m.NextPending() + require.NoError(t, err) + require.Nil(t, it) + + // Close the WAL. + m.Close() + + // There should be one segment waiting to be flushed. + it, err = m.NextPending() + require.NoError(t, err) + require.NotNil(t, it) + + // There are no more segments waiting to be flushed, and since the WAL is + // closed, successive calls should return ErrClosed. + it, err = m.NextPending() + require.ErrorIs(t, err, ErrClosed) require.Nil(t, it) } @@ -261,7 +339,8 @@ func TestManager_NexPendingMaxAge(t *testing.T) { // The segment that was just appended to has neither reached the maximum // age nor maximum size to be flushed. - it := m.NextPending() + it, err := m.NextPending() + require.NoError(t, err) require.Nil(t, it) require.Equal(t, 1, m.available.Len()) require.Equal(t, 0, m.pending.Len()) @@ -269,7 +348,8 @@ func TestManager_NexPendingMaxAge(t *testing.T) { // Wait 100ms. The segment that was just appended to should have reached // the maximum age. time.Sleep(100 * time.Millisecond) - it = m.NextPending() + it, err = m.NextPending() + require.NoError(t, err) require.NotNil(t, it) require.Equal(t, 0, m.available.Len()) require.Equal(t, 0, m.pending.Len()) @@ -304,7 +384,8 @@ func TestManager_Put(t *testing.T) { require.Equal(t, 1, m.pending.Len()) // Getting the pending segment should remove it from the list. - it := m.NextPending() + it, err := m.NextPending() + require.NoError(t, err) require.NotNil(t, it) require.Equal(t, 0, m.available.Len()) require.Equal(t, 0, m.pending.Len()) @@ -373,7 +454,8 @@ wal_segments_pending 1 require.NoError(t, testutil.CollectAndCompare(r, strings.NewReader(expected), metricNames...)) // Get the segment from the pending list. - it := m.NextPending() + it, err := m.NextPending() + require.NoError(t, err) require.NotNil(t, it) expected = ` # HELP wal_segments_available The number of WAL segments accepting writes. From a03e3d39d4a38f24cfd392b7f34e842072df0722 Mon Sep 17 00:00:00 2001 From: George Robinson Date: Wed, 17 Jul 2024 07:08:43 +0100 Subject: [PATCH 02/23] chore: Use pool of bytes.Buffer instead of io.Pipe (#13543) --- pkg/ingester-rf1/flush.go | 24 ++++++++++++++---------- pkg/ingester-rf1/ingester.go | 2 ++ pkg/storage/wal/segment.go | 26 -------------------------- pkg/storage/wal/segment_test.go | 16 ---------------- 4 files changed, 16 insertions(+), 52 deletions(-) diff --git a/pkg/ingester-rf1/flush.go b/pkg/ingester-rf1/flush.go index 4ab36713d392..08229cef2be3 100644 --- a/pkg/ingester-rf1/flush.go +++ b/pkg/ingester-rf1/flush.go @@ -1,6 +1,7 @@ package ingesterrf1 import ( + "bytes" "crypto/rand" "errors" "fmt" @@ -12,12 +13,10 @@ import ( "github.com/go-kit/log/level" "github.com/grafana/dskit/backoff" "github.com/grafana/dskit/ring" - "github.com/grafana/dskit/runutil" "github.com/oklog/ulid" "golang.org/x/net/context" "github.com/grafana/loki/v3/pkg/storage/wal" - util_log "github.com/grafana/loki/v3/pkg/util/log" ) const ( @@ -40,6 +39,7 @@ const ( func (i *Ingester) InitFlushWorkers() { i.flushWorkersDone.Add(i.cfg.ConcurrentFlushes) for j := 0; j < i.cfg.ConcurrentFlushes; j++ { + i.flushBuffers[j] = new(bytes.Buffer) go i.flushWorker(j) } } @@ -90,7 +90,7 @@ func (i *Ingester) flushWorker(j int) { n := it.Writer.InputSize() humanized := humanize.Bytes(uint64(n)) - err = i.flushItem(l, it) + err = i.flushItem(l, j, it) d := time.Since(start) if err != nil { level.Error(l).Log("msg", "failed to flush", "size", humanized, "duration", d, "err", err) @@ -103,13 +103,13 @@ func (i *Ingester) flushWorker(j int) { } } -func (i *Ingester) flushItem(l log.Logger, it *wal.PendingItem) error { +func (i *Ingester) flushItem(l log.Logger, j int, it *wal.PendingItem) error { ctx, cancelFunc := context.WithCancel(context.Background()) defer cancelFunc() b := backoff.New(ctx, i.cfg.FlushOpBackoff) for b.Ongoing() { - err := i.flushSegment(ctx, it.Writer) + err := i.flushSegment(ctx, j, it.Writer) if err == nil { break } @@ -124,19 +124,23 @@ func (i *Ingester) flushItem(l log.Logger, it *wal.PendingItem) error { // If the flush is successful, metrics for this flush are to be reported. // If the flush isn't successful, the operation for this userID is requeued allowing this and all other unflushed // segments to have another opportunity to be flushed. -func (i *Ingester) flushSegment(ctx context.Context, ch *wal.SegmentWriter) error { +func (i *Ingester) flushSegment(ctx context.Context, j int, w *wal.SegmentWriter) error { id := ulid.MustNew(ulid.Timestamp(time.Now()), rand.Reader) - r := ch.Reader() start := time.Now() defer func() { - runutil.CloseWithLogOnErr(util_log.Logger, r, "flushSegment") i.metrics.flushDuration.Observe(time.Since(start).Seconds()) - ch.Observe() + w.Observe() }() + buf := i.flushBuffers[j] + defer buf.Reset() + if _, err := w.WriteTo(buf); err != nil { + return err + } + i.metrics.flushesTotal.Add(1) - if err := i.store.PutObject(ctx, fmt.Sprintf("loki-v2/wal/anon/"+id.String()), r); err != nil { + if err := i.store.PutObject(ctx, fmt.Sprintf("loki-v2/wal/anon/"+id.String()), buf); err != nil { i.metrics.flushFailuresTotal.Inc() return fmt.Errorf("store put chunk: %w", err) } diff --git a/pkg/ingester-rf1/ingester.go b/pkg/ingester-rf1/ingester.go index c3e6afd052d5..4abe260dbcc4 100644 --- a/pkg/ingester-rf1/ingester.go +++ b/pkg/ingester-rf1/ingester.go @@ -1,6 +1,7 @@ package ingesterrf1 import ( + "bytes" "context" "flag" "fmt" @@ -205,6 +206,7 @@ type Ingester struct { // One queue per flush thread. Fingerprint is used to // pick a queue. + flushBuffers []*bytes.Buffer flushWorkersDone sync.WaitGroup wal *wal.Manager diff --git a/pkg/storage/wal/segment.go b/pkg/storage/wal/segment.go index b39d751c3736..6a792988bdb4 100644 --- a/pkg/storage/wal/segment.go +++ b/pkg/storage/wal/segment.go @@ -299,32 +299,6 @@ func (b *SegmentWriter) Reset() { b.inputSize.Store(0) } -type EncodedSegmentReader struct { - *io.PipeReader - *io.PipeWriter -} - -func (e *EncodedSegmentReader) Close() error { - err := e.PipeWriter.Close() - if err != nil { - return err - } - err = e.PipeReader.Close() - if err != nil { - return err - } - return nil -} - -func (b *SegmentWriter) Reader() io.ReadCloser { - pr, pw := io.Pipe() - go func() { - _, err := b.WriteTo(pw) - pw.CloseWithError(err) - }() - return &EncodedSegmentReader{PipeReader: pr, PipeWriter: pw} -} - // InputSize returns the total size of the input data written to the writer. // It doesn't account for timestamps and labels. func (b *SegmentWriter) InputSize() int64 { diff --git a/pkg/storage/wal/segment_test.go b/pkg/storage/wal/segment_test.go index a96ef5954c24..51117e3b500f 100644 --- a/pkg/storage/wal/segment_test.go +++ b/pkg/storage/wal/segment_test.go @@ -4,7 +4,6 @@ import ( "bytes" "context" "fmt" - "io" "sort" "sync" "testing" @@ -510,19 +509,4 @@ func BenchmarkWrites(b *testing.B) { require.EqualValues(b, encodedLength, n) } }) - - bytesBuf := make([]byte, encodedLength) - b.Run("Reader", func(b *testing.B) { - b.ResetTimer() - b.ReportAllocs() - for i := 0; i < b.N; i++ { - var err error - reader := writer.Reader() - - n, err := io.ReadFull(reader, bytesBuf) - require.NoError(b, err) - require.EqualValues(b, encodedLength, n) - require.NoError(b, reader.Close()) - } - }) } From 4451d56d6b9a9d2eb54ed75d3d2c8fe0db6908eb Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Wed, 17 Jul 2024 08:56:26 +0200 Subject: [PATCH 03/23] fix: Fixes span name of serializeRounTripper (#13541) --- pkg/querier/queryrange/serialize.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/querier/queryrange/serialize.go b/pkg/querier/queryrange/serialize.go index d7a5eb125d30..6f292a229f7e 100644 --- a/pkg/querier/queryrange/serialize.go +++ b/pkg/querier/queryrange/serialize.go @@ -25,7 +25,7 @@ func NewSerializeRoundTripper(next queryrangebase.Handler, codec queryrangebase. func (rt *serializeRoundTripper) RoundTrip(r *http.Request) (*http.Response, error) { ctx := r.Context() - sp, ctx := opentracing.StartSpanFromContext(ctx, "limitedRoundTripper.do") + sp, ctx := opentracing.StartSpanFromContext(ctx, "serializeRoundTripper.do") defer sp.Finish() request, err := rt.codec.DecodeRequest(ctx, r, nil) From 47d057854f6bfc3ae9b50b248389f5307f3b1570 Mon Sep 17 00:00:00 2001 From: George Robinson Date: Wed, 17 Jul 2024 08:36:15 +0100 Subject: [PATCH 04/23] chore: Fix not ready error message in RF-1 Ingester (#13546) --- pkg/loki/loki.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/loki/loki.go b/pkg/loki/loki.go index 32555391dd3c..2e7e010b9980 100644 --- a/pkg/loki/loki.go +++ b/pkg/loki/loki.go @@ -625,7 +625,7 @@ func (t *Loki) readyHandler(sm *services.Manager, shutdownRequested *atomic.Bool // and that all other ring entries are OK too. if t.IngesterRF1 != nil { if err := t.IngesterRF1.CheckReady(r.Context()); err != nil { - http.Error(w, "Pattern Ingester not ready: "+err.Error(), http.StatusServiceUnavailable) + http.Error(w, "RF-1 Ingester not ready: "+err.Error(), http.StatusServiceUnavailable) return } } From e1f9eb5fa97d6ba906504cc3efb5e0426a64ae28 Mon Sep 17 00:00:00 2001 From: George Robinson Date: Wed, 17 Jul 2024 08:36:23 +0100 Subject: [PATCH 05/23] chore: Remove size and duration logs now we have metrics (#13547) --- pkg/ingester-rf1/flush.go | 12 +----------- 1 file changed, 1 insertion(+), 11 deletions(-) diff --git a/pkg/ingester-rf1/flush.go b/pkg/ingester-rf1/flush.go index 08229cef2be3..1082f8340f5e 100644 --- a/pkg/ingester-rf1/flush.go +++ b/pkg/ingester-rf1/flush.go @@ -8,7 +8,6 @@ import ( "net/http" "time" - "github.com/dustin/go-humanize" "github.com/go-kit/log" "github.com/go-kit/log/level" "github.com/grafana/dskit/backoff" @@ -84,18 +83,9 @@ func (i *Ingester) flushWorker(j int) { continue } - start := time.Now() - - // We'll use this to log the size of the segment that was flushed. - n := it.Writer.InputSize() - humanized := humanize.Bytes(uint64(n)) - err = i.flushItem(l, j, it) - d := time.Since(start) if err != nil { - level.Error(l).Log("msg", "failed to flush", "size", humanized, "duration", d, "err", err) - } else { - level.Debug(l).Log("msg", "flushed", "size", humanized, "duration", d) + level.Error(l).Log("msg", "failed to flush", "err", err) } it.Result.SetDone(err) From fac8ddafcdbd521bde3d3b96267bc0013cc9dcf6 Mon Sep 17 00:00:00 2001 From: George Robinson Date: Wed, 17 Jul 2024 08:36:40 +0100 Subject: [PATCH 06/23] chore: Remove unused const (#13544) --- pkg/ingester-rf1/flush.go | 15 --------------- 1 file changed, 15 deletions(-) diff --git a/pkg/ingester-rf1/flush.go b/pkg/ingester-rf1/flush.go index 1082f8340f5e..efd8f72df726 100644 --- a/pkg/ingester-rf1/flush.go +++ b/pkg/ingester-rf1/flush.go @@ -18,21 +18,6 @@ import ( "github.com/grafana/loki/v3/pkg/storage/wal" ) -const ( - // Backoff for retrying 'immediate' flushes. Only counts for queue - // position, not wallclock time. - flushBackoff = 1 * time.Second - - nameLabel = "__name__" - logsValue = "logs" - - flushReasonIdle = "idle" - flushReasonMaxAge = "max_age" - flushReasonForced = "forced" - flushReasonFull = "full" - flushReasonSynced = "synced" -) - // Note: this is called both during the WAL replay (zero or more times) // and then after replay as well. func (i *Ingester) InitFlushWorkers() { From 646f42f71d33cf14f63816697710fdf2ce0bb4e5 Mon Sep 17 00:00:00 2001 From: "renovate[bot]" <29139614+renovate[bot]@users.noreply.github.com> Date: Wed, 17 Jul 2024 08:42:36 +0100 Subject: [PATCH 07/23] chore(deps): update module github.com/azure/azure-sdk-for-go/sdk/azidentity to v1.6.0 [security] (main) (#13434) Co-authored-by: renovate[bot] <29139614+renovate[bot]@users.noreply.github.com> --- cmd/chunks-inspect/go.mod | 24 ++++++++-------- cmd/chunks-inspect/go.sum | 58 +++++++++++++++++++-------------------- 2 files changed, 40 insertions(+), 42 deletions(-) diff --git a/cmd/chunks-inspect/go.mod b/cmd/chunks-inspect/go.mod index 2801f174a1af..96cb79b2de85 100644 --- a/cmd/chunks-inspect/go.mod +++ b/cmd/chunks-inspect/go.mod @@ -11,10 +11,10 @@ require ( ) require ( - github.com/Azure/azure-sdk-for-go/sdk/azcore v1.10.0 // indirect - github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.5.1 // indirect - github.com/Azure/azure-sdk-for-go/sdk/internal v1.5.2 // indirect - github.com/AzureAD/microsoft-authentication-library-for-go v1.2.1 // indirect + github.com/Azure/azure-sdk-for-go/sdk/azcore v1.11.1 // indirect + github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.6.0 // indirect + github.com/Azure/azure-sdk-for-go/sdk/internal v1.8.0 // indirect + github.com/AzureAD/microsoft-authentication-library-for-go v1.2.2 // indirect github.com/HdrHistogram/hdrhistogram-go v1.1.2 // indirect github.com/Masterminds/goutils v1.1.1 // indirect github.com/Masterminds/semver/v3 v3.2.0 // indirect @@ -47,7 +47,7 @@ require ( github.com/gogo/googleapis v1.4.0 // indirect github.com/gogo/protobuf v1.3.2 // indirect github.com/gogo/status v1.1.1 // indirect - github.com/golang-jwt/jwt/v5 v5.2.0 // indirect + github.com/golang-jwt/jwt/v5 v5.2.1 // indirect github.com/golang/protobuf v1.5.3 // indirect github.com/google/btree v1.1.2 // indirect github.com/google/go-cmp v0.6.0 // indirect @@ -123,16 +123,16 @@ require ( go.uber.org/multierr v1.11.0 // indirect go.uber.org/zap v1.21.0 // indirect go4.org/netipx v0.0.0-20230125063823-8449b0a6169f // indirect - golang.org/x/crypto v0.21.0 // indirect + golang.org/x/crypto v0.24.0 // indirect golang.org/x/exp v0.0.0-20240325151524-a685a6edb6d8 // indirect - golang.org/x/mod v0.16.0 // indirect - golang.org/x/net v0.23.0 // indirect + golang.org/x/mod v0.17.0 // indirect + golang.org/x/net v0.26.0 // indirect golang.org/x/oauth2 v0.18.0 // indirect - golang.org/x/sync v0.6.0 // indirect - golang.org/x/sys v0.18.0 // indirect - golang.org/x/text v0.14.0 // indirect + golang.org/x/sync v0.7.0 // indirect + golang.org/x/sys v0.21.0 // indirect + golang.org/x/text v0.16.0 // indirect golang.org/x/time v0.5.0 // indirect - golang.org/x/tools v0.19.0 // indirect + golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d // indirect google.golang.org/appengine v1.6.8 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240304212257-790db918fca8 // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240304161311-37d4d3c04a78 // indirect diff --git a/cmd/chunks-inspect/go.sum b/cmd/chunks-inspect/go.sum index f90ea810759a..be5c16ea5dd5 100644 --- a/cmd/chunks-inspect/go.sum +++ b/cmd/chunks-inspect/go.sum @@ -31,18 +31,18 @@ cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohl cloud.google.com/go/storage v1.8.0/go.mod h1:Wv1Oy7z6Yz3DshWRJFhqM/UCfaWIRTdp0RXyy7KQOVs= cloud.google.com/go/storage v1.10.0/go.mod h1:FLPqc6j+Ki4BU591ie1oL6qBQGu2Bl/tZ9ullr3+Kg0= dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= -github.com/Azure/azure-sdk-for-go/sdk/azcore v1.10.0 h1:n1DH8TPV4qqPTje2RcUBYwtrTWlabVp4n46+74X2pn4= -github.com/Azure/azure-sdk-for-go/sdk/azcore v1.10.0/go.mod h1:HDcZnuGbiyppErN6lB+idp4CKhjbc8gwjto6OPpyggM= -github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.5.1 h1:sO0/P7g68FrryJzljemN+6GTssUXdANk6aJ7T1ZxnsQ= -github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.5.1/go.mod h1:h8hyGFDsU5HMivxiS2iYFZsgDbU9OnnJ163x5UGVKYo= -github.com/Azure/azure-sdk-for-go/sdk/internal v1.5.2 h1:LqbJ/WzJUwBf8UiaSzgX7aMclParm9/5Vgp+TY51uBQ= -github.com/Azure/azure-sdk-for-go/sdk/internal v1.5.2/go.mod h1:yInRyqWXAuaPrgI7p70+lDDgh3mlBohis29jGMISnmc= +github.com/Azure/azure-sdk-for-go/sdk/azcore v1.11.1 h1:E+OJmp2tPvt1W+amx48v1eqbjDYsgN+RzP4q16yV5eM= +github.com/Azure/azure-sdk-for-go/sdk/azcore v1.11.1/go.mod h1:a6xsAQUZg+VsS3TJ05SRp524Hs4pZ/AeFSr5ENf0Yjo= +github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.6.0 h1:U2rTu3Ef+7w9FHKIAXM6ZyqF3UOWJZ12zIm8zECAFfg= +github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.6.0/go.mod h1:9kIvujWAA58nmPmWB1m23fyWic1kYZMxD9CxaWn4Qpg= +github.com/Azure/azure-sdk-for-go/sdk/internal v1.8.0 h1:jBQA3cKT4L2rWMpgE7Yt3Hwh2aUj8KXjIGLxjHeYNNo= +github.com/Azure/azure-sdk-for-go/sdk/internal v1.8.0/go.mod h1:4OG6tQ9EOP/MT0NMjDlRzWoVFxfu9rN9B2X+tlSVktg= github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/compute/armcompute/v5 v5.5.0 h1:MxA59PGoCFb+vCwRQi3PhQEwHj4+r2dhuv9HG+vM7iM= github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/compute/armcompute/v5 v5.5.0/go.mod h1:uYt4CfhkJA9o0FN7jfE5minm/i4nUE4MjGUJkzB6Zs8= github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/network/armnetwork/v4 v4.3.0 h1:bXwSugBiSbgtz7rOtbfGf+woewp4f06orW9OP5BjHLA= github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/network/armnetwork/v4 v4.3.0/go.mod h1:Y/HgrePTmGy9HjdSGTqZNa+apUpTVIEVKXJyARP2lrk= -github.com/AzureAD/microsoft-authentication-library-for-go v1.2.1 h1:DzHpqpoJVaCgOUdVHxE8QB52S6NiVdDQvGlny1qvPqA= -github.com/AzureAD/microsoft-authentication-library-for-go v1.2.1/go.mod h1:wP83P5OoQ5p6ip3ScPr0BAq0BvuPAvacpEuSzyouqAI= +github.com/AzureAD/microsoft-authentication-library-for-go v1.2.2 h1:XHOnouVk1mxXfQidrMEnLlPk9UMeRtyBTnEFtxkV0kU= +github.com/AzureAD/microsoft-authentication-library-for-go v1.2.2/go.mod h1:wP83P5OoQ5p6ip3ScPr0BAq0BvuPAvacpEuSzyouqAI= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= github.com/Code-Hex/go-generics-cache v1.3.1 h1:i8rLwyhoyhaerr7JpjtYjJZUcCbWOdiYO3fZXLiEC4g= @@ -131,8 +131,6 @@ github.com/digitalocean/godo v1.109.0 h1:4W97RJLJSUQ3veRZDNbp1Ol3Rbn6Lmt9bKGvfqY github.com/digitalocean/godo v1.109.0/go.mod h1:R6EmmWI8CT1+fCtjWY9UCB+L5uufuZH13wk3YhxycCs= github.com/distribution/reference v0.5.0 h1:/FUIFXtfc/x2gpa5/VGfiGLuOIdYa1t65IKK2OFGvA0= github.com/distribution/reference v0.5.0/go.mod h1:BbU0aIcezP1/5jX/8MP0YiH4SdvB5Y4f/wlDRiLyi3E= -github.com/dnaeon/go-vcr v1.2.0 h1:zHCHvJYTMh1N7xnV7zf1m1GPBF9Ad0Jk/whtQ1663qI= -github.com/dnaeon/go-vcr v1.2.0/go.mod h1:R4UdLID7HZT3taECzJs4YgbbH6PIGXB6W/sc5OLb6RQ= github.com/docker/docker v25.0.3+incompatible h1:D5fy/lYmY7bvZa0XTZ5/UJPljor41F+vdyJG5luQLfQ= github.com/docker/docker v25.0.3+incompatible/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk= github.com/docker/go-connections v0.4.0 h1:El9xVISelRB7BuFusrZozjnkIM5YnzCViNKohAFqRJQ= @@ -209,8 +207,8 @@ github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= github.com/gogo/status v1.1.1 h1:DuHXlSFHNKqTQ+/ACf5Vs6r4X/dH2EgIzR9Vr+H65kg= github.com/gogo/status v1.1.1/go.mod h1:jpG3dM5QPcqu19Hg8lkUhBFBa3TcLs1DG7+2Jqci7oU= -github.com/golang-jwt/jwt/v5 v5.2.0 h1:d/ix8ftRUorsN+5eMIlF4T6J8CAt9rch3My2winC1Jw= -github.com/golang-jwt/jwt/v5 v5.2.0/go.mod h1:pqrtFR0X4osieyHYxtmOUWsAWrfe1Q5UVIyoH402zdk= +github.com/golang-jwt/jwt/v5 v5.2.1 h1:OuVbFODueb089Lh128TAcimifWaLhJwVflnrgM17wHk= +github.com/golang-jwt/jwt/v5 v5.2.1/go.mod h1:pqrtFR0X4osieyHYxtmOUWsAWrfe1Q5UVIyoH402zdk= github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0/go.mod h1:E/TSTwGwJL78qG/PmXZO1EjYhfJinVAhrmmHX6Z8B9k= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= @@ -529,8 +527,8 @@ github.com/prometheus/prometheus v0.51.0 h1:aRdjTnmHLved29ILtdzZN2GNvOjWATtA/z+3 github.com/prometheus/prometheus v0.51.0/go.mod h1:yv4MwOn3yHMQ6MZGHPg/U7Fcyqf+rxqiZfSur6myVtc= github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= -github.com/rogpeppe/go-internal v1.10.0 h1:TMyTOH3F/DB16zRVcYyreMH6GnZZrwQVAoYjRBZyWFQ= -github.com/rogpeppe/go-internal v1.10.0/go.mod h1:UQnix2H7Ngw/k4C5ijL5+65zddjncjaFoBhdsK/akog= +github.com/rogpeppe/go-internal v1.12.0 h1:exVL4IDcn6na9z1rAb56Vxr+CgyK3nn3O+epU5NdKM8= +github.com/rogpeppe/go-internal v1.12.0/go.mod h1:E+RYuTGaKKdloAfM02xzb0FW3Paa99yedzYV+kq4uf4= github.com/ryanuber/columnize v0.0.0-20160712163229-9b3edd62028f/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= github.com/scaleway/scaleway-sdk-go v1.0.0-beta.25 h1:/8rfZAdFfafRXOgz+ZpMZZWZ5pYggCY9t7e/BvjaBHM= github.com/scaleway/scaleway-sdk-go v1.0.0-beta.25/go.mod h1:fCa7OJZ/9DRTnOKmxvT6pn+LPWUptQAmHF/SBJUGEcg= @@ -627,8 +625,8 @@ golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8U golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.3.0/go.mod h1:hebNnKkNXi2UzZN1eVRvBB7co0a+JxK6XbPiWVs/3J4= -golang.org/x/crypto v0.21.0 h1:X31++rzVUdKhX5sWmSOFZxx8UW/ldWx55cbf08iNAMA= -golang.org/x/crypto v0.21.0/go.mod h1:0BP7YvVV9gBbVKyeTG0Gyn+gZm94bibOW5BjDEYAOMs= +golang.org/x/crypto v0.24.0 h1:mnl8DM0o513X8fdIkmyFE/5hTYxbwYOjDS/+rK6qpRI= +golang.org/x/crypto v0.24.0/go.mod h1:Z1PMYSOR5nyMcyAVAIQSKCDwalqy85Aqn1x3Ws4L5DM= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -668,8 +666,8 @@ golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= -golang.org/x/mod v0.16.0 h1:QX4fJ0Rr5cPQCF7O9lh9Se4pmwfwskqZfq5moyldzic= -golang.org/x/mod v0.16.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= +golang.org/x/mod v0.17.0 h1:zY54UmvipHiNd+pm+m0x9KhZ9hl1/7QNMyxXbc6ICqA= +golang.org/x/mod v0.17.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -709,8 +707,8 @@ golang.org/x/net v0.0.0-20210410081132-afb366fc7cd1/go.mod h1:9tjilg8BloeKEkVJvy golang.org/x/net v0.0.0-20210525063256-abc453219eb5/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/net v0.2.0/go.mod h1:KqCZLdyyvdV855qA2rE3GC2aiw5xGR5TEjj8smXukLY= -golang.org/x/net v0.23.0 h1:7EYJ93RZ9vYSZAIb2x3lnuvqO5zneoD6IvWjuhfxjTs= -golang.org/x/net v0.23.0/go.mod h1:JKghWKKOSdJwpW2GEx0Ja7fmaKnMsbu+MWVZTokSYmg= +golang.org/x/net v0.26.0 h1:soB7SVo0PWrY4vPW/+ay0jKDNScG2X9wFeYlXIvJsOQ= +golang.org/x/net v0.26.0/go.mod h1:5YKkiSynbBIh3p6iOc/vibscux0x38BZDkn8sCUPxHE= 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= @@ -731,8 +729,8 @@ golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.6.0 h1:5BMeUDZ7vkXGfEr1x9B4bRcTH4lpkTkpdh0T/J+qjbQ= -golang.org/x/sync v0.6.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.7.0 h1:YsImfSBoP9QPYL0xyKJPq0gcaJdG3rInoqxTWbfQu9M= +golang.org/x/sync v0.7.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180823144017-11551d06cbcc/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -790,13 +788,13 @@ golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.1.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.2.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.18.0 h1:DBdB3niSjOA/O0blCZBqDefyWNYveAYMNF1Wum0DYQ4= -golang.org/x/sys v0.18.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.21.0 h1:rF+pYz3DAGSQAxAu1CbC7catZg4ebC4UIeIhKxBZvws= +golang.org/x/sys v0.21.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.2.0/go.mod h1:TVmDHMZPmdnySmBfhjOoOdhjzdE1h4u1VwSiw2l1Nuc= -golang.org/x/term v0.18.0 h1:FcHjZXDMxI8mM3nwhX9HlKop4C0YQvCVCdwYl2wOtE8= -golang.org/x/term v0.18.0/go.mod h1:ILwASektA3OnRv7amZ1xhE/KTR+u50pbXfZ03+6Nx58= +golang.org/x/term v0.21.0 h1:WVXCp+/EBEHOj53Rvu+7KiT/iElMrO8ACK16SMZ3jaA= +golang.org/x/term v0.21.0/go.mod h1:ooXLefLobQVslOqselCNF4SxFAaoS6KujMbsGzSDmX0= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -807,8 +805,8 @@ golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= golang.org/x/text v0.3.8/go.mod h1:E6s5w1FMmriuDzIBO73fBruAKo1PCIq6d2Q6DHfQ8WQ= golang.org/x/text v0.4.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= -golang.org/x/text v0.14.0 h1:ScX5w1eTa3QqT8oi6+ziP7dTV1S2+ALU0bI+0zXKWiQ= -golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= +golang.org/x/text v0.16.0 h1:a94ExnEXNtEwYLGJSIUxnWoxoRz/ZcCsV63ROupILh4= +golang.org/x/text v0.16.0/go.mod h1:GhwF1Be+LQoKShO3cGOHzqOgRrGaYc9AvblQOmPVHnI= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -864,8 +862,8 @@ golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4f golang.org/x/tools v0.1.2/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= -golang.org/x/tools v0.19.0 h1:tfGCXNR1OsFG+sVdLAitlpjAvD/I6dHDKnYrpEZUHkw= -golang.org/x/tools v0.19.0/go.mod h1:qoJWxmGSIBmAeriMx19ogtrEPrGtDbPK634QFIcLAhc= +golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d h1:vU5i/LfpvrRCpgM/VPfJLg5KjxD3E+hfT1SH+d9zLwg= +golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d/go.mod h1:aiJjzUbINMkxbQROHiO6hDPo2LHcIPhhQsa9DLh0yGk= 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= From 6acb51d48518f247ec75b5b235cef57390f12811 Mon Sep 17 00:00:00 2001 From: George Robinson Date: Wed, 17 Jul 2024 08:44:27 +0100 Subject: [PATCH 08/23] chore: Use github.com/coder/quartz instead of time (#13542) --- go.mod | 5 +- go.sum | 2 + pkg/storage/wal/manager.go | 14 +- pkg/storage/wal/manager_test.go | 13 +- vendor/github.com/coder/quartz/.gitignore | 1 + vendor/github.com/coder/quartz/LICENSE | 121 ++++ vendor/github.com/coder/quartz/README.md | 632 +++++++++++++++++++++ vendor/github.com/coder/quartz/clock.go | 43 ++ vendor/github.com/coder/quartz/mock.go | 646 ++++++++++++++++++++++ vendor/github.com/coder/quartz/real.go | 80 +++ vendor/github.com/coder/quartz/ticker.go | 75 +++ vendor/github.com/coder/quartz/timer.go | 81 +++ vendor/modules.txt | 3 + 13 files changed, 1707 insertions(+), 9 deletions(-) create mode 100644 vendor/github.com/coder/quartz/.gitignore create mode 100644 vendor/github.com/coder/quartz/LICENSE create mode 100644 vendor/github.com/coder/quartz/README.md create mode 100644 vendor/github.com/coder/quartz/clock.go create mode 100644 vendor/github.com/coder/quartz/mock.go create mode 100644 vendor/github.com/coder/quartz/real.go create mode 100644 vendor/github.com/coder/quartz/ticker.go create mode 100644 vendor/github.com/coder/quartz/timer.go diff --git a/go.mod b/go.mod index 3d5aae7fd9e9..6387a6ebab06 100644 --- a/go.mod +++ b/go.mod @@ -1,8 +1,8 @@ module github.com/grafana/loki/v3 -go 1.21 +go 1.21.8 -toolchain go1.21.3 +toolchain go1.22.4 require ( cloud.google.com/go/bigtable v1.18.1 @@ -120,6 +120,7 @@ require ( github.com/IBM/ibm-cos-sdk-go v1.10.0 github.com/axiomhq/hyperloglog v0.0.0-20240124082744-24bca3a5b39b github.com/buger/jsonparser v1.1.1 + github.com/coder/quartz v0.1.0 github.com/d4l3k/messagediff v1.2.1 github.com/dolthub/swiss v0.2.1 github.com/efficientgo/core v1.0.0-rc.2 diff --git a/go.sum b/go.sum index 002786d54296..ea4dc9a96737 100644 --- a/go.sum +++ b/go.sum @@ -452,6 +452,8 @@ github.com/cncf/xds/go v0.0.0-20231128003011-0fa0005c9caa/go.mod h1:x/1Gn8zydmfq github.com/cockroachdb/apd v1.1.0/go.mod h1:8Sl8LxpKi29FqWXR16WEFZRNSz3SoPzUzeMeY4+DwBQ= github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa/go.mod h1:zn76sxSg3SzpJ0PPJaLDCu+Bu0Lg3sKTORVIj19EIF8= github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd/go.mod h1:sE/e/2PUdi/liOCUjSTXgM1o87ZssimdTWN964YiIeI= +github.com/coder/quartz v0.1.0 h1:cLL+0g5l7xTf6ordRnUMMiZtRE8Sq5LxpghS63vEXrQ= +github.com/coder/quartz v0.1.0/go.mod h1:vsiCc+AHViMKH2CQpGIpFgdHIEQsxwm8yCscqKmzbRA= github.com/containerd/containerd v1.4.1/go.mod h1:bC6axHOhabU15QhwfG7w5PipXdVtMXFTttgp+kVtyUA= github.com/containerd/fifo v1.0.0 h1:6PirWBr9/L7GDamKr+XM0IeUFXu5mf3M/BPpH9gaLBU= github.com/containerd/fifo v1.0.0/go.mod h1:ocF/ME1SX5b1AOlWi9r677YJmCPSwwWnQ9O123vzpE4= diff --git a/pkg/storage/wal/manager.go b/pkg/storage/wal/manager.go index cfc89605cb7c..196d02296b5f 100644 --- a/pkg/storage/wal/manager.go +++ b/pkg/storage/wal/manager.go @@ -6,6 +6,7 @@ import ( "sync" "time" + "github.com/coder/quartz" "github.com/prometheus/prometheus/model/labels" "github.com/grafana/loki/v3/pkg/logproto" @@ -115,9 +116,11 @@ type Manager struct { // flushed, the segment is reset and moved to the back of the available // list to accept writes again. pending *list.List + closed bool + mu sync.Mutex - closed bool - mu sync.Mutex + // Used in tests. + clock quartz.Clock } // item is similar to PendingItem, but it is an internal struct used in the @@ -145,6 +148,7 @@ func NewManager(cfg Config, metrics *Metrics) (*Manager, error) { metrics: metrics.ManagerMetrics, available: list.New(), pending: list.New(), + clock: quartz.NewReal(), } m.metrics.NumPending.Set(0) m.metrics.NumFlushing.Set(0) @@ -177,12 +181,12 @@ func (m *Manager) Append(r AppendRequest) (*AppendResult, error) { // This is the first append to the segment. This time will be used in // know when the segment has exceeded its maximum age and should be // moved to the pending list. - it.firstAppendedAt = time.Now() + it.firstAppendedAt = m.clock.Now() } it.w.Append(r.TenantID, r.LabelsStr, r.Labels, r.Entries) // If the segment exceeded the maximum age or the maximum size, move it to // the closed list to be flushed. - if time.Since(it.firstAppendedAt) >= m.cfg.MaxAge || it.w.InputSize() >= m.cfg.MaxSegmentSize { + if m.clock.Since(it.firstAppendedAt) >= m.cfg.MaxAge || it.w.InputSize() >= m.cfg.MaxSegmentSize { m.pending.PushBack(it) m.metrics.NumPending.Inc() m.available.Remove(el) @@ -218,7 +222,7 @@ func (m *Manager) NextPending() (*PendingItem, error) { // should be moved to the pending list. el := m.available.Front() it := el.Value.(*item) - if !it.firstAppendedAt.IsZero() && time.Since(it.firstAppendedAt) >= m.cfg.MaxAge { + if !it.firstAppendedAt.IsZero() && m.clock.Since(it.firstAppendedAt) >= m.cfg.MaxAge { m.pending.PushBack(it) m.metrics.NumPending.Inc() m.available.Remove(el) diff --git a/pkg/storage/wal/manager_test.go b/pkg/storage/wal/manager_test.go index f5fee51ff166..ef14b2420108 100644 --- a/pkg/storage/wal/manager_test.go +++ b/pkg/storage/wal/manager_test.go @@ -6,6 +6,7 @@ import ( "testing" "time" + "github.com/coder/quartz" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/testutil" "github.com/prometheus/prometheus/model/labels" @@ -94,6 +95,10 @@ func TestManager_AppendMaxAge(t *testing.T) { }, NewMetrics(nil)) require.NoError(t, err) + // Create a mock clock. + clock := quartz.NewMock(t) + m.clock = clock + // Append 1B of data. lbs := labels.Labels{{Name: "a", Value: "b"}} entries := []*logproto.Entry{{Timestamp: time.Now(), Line: "c"}} @@ -112,7 +117,7 @@ func TestManager_AppendMaxAge(t *testing.T) { require.Equal(t, 0, m.pending.Len()) // Wait 100ms and append some more data. - time.Sleep(100 * time.Millisecond) + clock.Advance(100 * time.Millisecond) entries = []*logproto.Entry{{Timestamp: time.Now(), Line: "c"}} res, err = m.Append(AppendRequest{ TenantID: "1", @@ -325,6 +330,10 @@ func TestManager_NexPendingMaxAge(t *testing.T) { }, NewMetrics(nil)) require.NoError(t, err) + // Create a mock clock. + clock := quartz.NewMock(t) + m.clock = clock + // Append 1B of data. lbs := labels.Labels{{Name: "a", Value: "b"}} entries := []*logproto.Entry{{Timestamp: time.Now(), Line: "c"}} @@ -347,7 +356,7 @@ func TestManager_NexPendingMaxAge(t *testing.T) { // Wait 100ms. The segment that was just appended to should have reached // the maximum age. - time.Sleep(100 * time.Millisecond) + clock.Advance(100 * time.Millisecond) it, err = m.NextPending() require.NoError(t, err) require.NotNil(t, it) diff --git a/vendor/github.com/coder/quartz/.gitignore b/vendor/github.com/coder/quartz/.gitignore new file mode 100644 index 000000000000..62c893550adb --- /dev/null +++ b/vendor/github.com/coder/quartz/.gitignore @@ -0,0 +1 @@ +.idea/ \ No newline at end of file diff --git a/vendor/github.com/coder/quartz/LICENSE b/vendor/github.com/coder/quartz/LICENSE new file mode 100644 index 000000000000..f7c5d7fee65c --- /dev/null +++ b/vendor/github.com/coder/quartz/LICENSE @@ -0,0 +1,121 @@ +Creative Commons Legal Code + +CC0 1.0 Universal + + CREATIVE COMMONS CORPORATION IS NOT A LAW FIRM AND DOES NOT PROVIDE + LEGAL SERVICES. DISTRIBUTION OF THIS DOCUMENT DOES NOT CREATE AN + ATTORNEY-CLIENT RELATIONSHIP. CREATIVE COMMONS PROVIDES THIS + INFORMATION ON AN "AS-IS" BASIS. CREATIVE COMMONS MAKES NO WARRANTIES + REGARDING THE USE OF THIS DOCUMENT OR THE INFORMATION OR WORKS + PROVIDED HEREUNDER, AND DISCLAIMS LIABILITY FOR DAMAGES RESULTING FROM + THE USE OF THIS DOCUMENT OR THE INFORMATION OR WORKS PROVIDED + HEREUNDER. + +Statement of Purpose + +The laws of most jurisdictions throughout the world automatically confer +exclusive Copyright and Related Rights (defined below) upon the creator +and subsequent owner(s) (each and all, an "owner") of an original work of +authorship and/or a database (each, a "Work"). + +Certain owners wish to permanently relinquish those rights to a Work for +the purpose of contributing to a commons of creative, cultural and +scientific works ("Commons") that the public can reliably and without fear +of later claims of infringement build upon, modify, incorporate in other +works, reuse and redistribute as freely as possible in any form whatsoever +and for any purposes, including without limitation commercial purposes. +These owners may contribute to the Commons to promote the ideal of a free +culture and the further production of creative, cultural and scientific +works, or to gain reputation or greater distribution for their Work in +part through the use and efforts of others. + +For these and/or other purposes and motivations, and without any +expectation of additional consideration or compensation, the person +associating CC0 with a Work (the "Affirmer"), to the extent that he or she +is an owner of Copyright and Related Rights in the Work, voluntarily +elects to apply CC0 to the Work and publicly distribute the Work under its +terms, with knowledge of his or her Copyright and Related Rights in the +Work and the meaning and intended legal effect of CC0 on those rights. + +1. Copyright and Related Rights. A Work made available under CC0 may be + protected by copyright and related or neighboring rights ("Copyright and + Related Rights"). Copyright and Related Rights include, but are not + limited to, the following: + +i. the right to reproduce, adapt, distribute, perform, display, +communicate, and translate a Work; +ii. moral rights retained by the original author(s) and/or performer(s); +iii. publicity and privacy rights pertaining to a person's image or +likeness depicted in a Work; +iv. rights protecting against unfair competition in regards to a Work, +subject to the limitations in paragraph 4(a), below; +v. rights protecting the extraction, dissemination, use and reuse of data +in a Work; +vi. database rights (such as those arising under Directive 96/9/EC of the +European Parliament and of the Council of 11 March 1996 on the legal +protection of databases, and under any national implementation +thereof, including any amended or successor version of such +directive); and +vii. other similar, equivalent or corresponding rights throughout the +world based on applicable law or treaty, and any national +implementations thereof. + +2. Waiver. To the greatest extent permitted by, but not in contravention + of, applicable law, Affirmer hereby overtly, fully, permanently, + irrevocably and unconditionally waives, abandons, and surrenders all of + Affirmer's Copyright and Related Rights and associated claims and causes + of action, whether now known or unknown (including existing as well as + future claims and causes of action), in the Work (i) in all territories + worldwide, (ii) for the maximum duration provided by applicable law or + treaty (including future time extensions), (iii) in any current or future + medium and for any number of copies, and (iv) for any purpose whatsoever, + including without limitation commercial, advertising or promotional + purposes (the "Waiver"). Affirmer makes the Waiver for the benefit of each + member of the public at large and to the detriment of Affirmer's heirs and + successors, fully intending that such Waiver shall not be subject to + revocation, rescission, cancellation, termination, or any other legal or + equitable action to disrupt the quiet enjoyment of the Work by the public + as contemplated by Affirmer's express Statement of Purpose. + +3. Public License Fallback. Should any part of the Waiver for any reason + be judged legally invalid or ineffective under applicable law, then the + Waiver shall be preserved to the maximum extent permitted taking into + account Affirmer's express Statement of Purpose. In addition, to the + extent the Waiver is so judged Affirmer hereby grants to each affected + person a royalty-free, non transferable, non sublicensable, non exclusive, + irrevocable and unconditional license to exercise Affirmer's Copyright and + Related Rights in the Work (i) in all territories worldwide, (ii) for the + maximum duration provided by applicable law or treaty (including future + time extensions), (iii) in any current or future medium and for any number + of copies, and (iv) for any purpose whatsoever, including without + limitation commercial, advertising or promotional purposes (the + "License"). The License shall be deemed effective as of the date CC0 was + applied by Affirmer to the Work. Should any part of the License for any + reason be judged legally invalid or ineffective under applicable law, such + partial invalidity or ineffectiveness shall not invalidate the remainder + of the License, and in such case Affirmer hereby affirms that he or she + will not (i) exercise any of his or her remaining Copyright and Related + Rights in the Work or (ii) assert any associated claims and causes of + action with respect to the Work, in either case contrary to Affirmer's + express Statement of Purpose. + +4. Limitations and Disclaimers. + +a. No trademark or patent rights held by Affirmer are waived, abandoned, +surrendered, licensed or otherwise affected by this document. +b. Affirmer offers the Work as-is and makes no representations or +warranties of any kind concerning the Work, express, implied, +statutory or otherwise, including without limitation warranties of +title, merchantability, fitness for a particular purpose, non +infringement, or the absence of latent or other defects, accuracy, or +the present or absence of errors, whether or not discoverable, all to +the greatest extent permissible under applicable law. +c. Affirmer disclaims responsibility for clearing rights of other persons +that may apply to the Work or any use thereof, including without +limitation any person's Copyright and Related Rights in the Work. +Further, Affirmer disclaims responsibility for obtaining any necessary +consents, permissions or other rights required for any use of the +Work. +d. Affirmer understands and acknowledges that Creative Commons is not a +party to this document and has no duty or obligation with respect to +this CC0 or use of the Work. \ No newline at end of file diff --git a/vendor/github.com/coder/quartz/README.md b/vendor/github.com/coder/quartz/README.md new file mode 100644 index 000000000000..8d467af12535 --- /dev/null +++ b/vendor/github.com/coder/quartz/README.md @@ -0,0 +1,632 @@ +# Quartz + +A Go time testing library for writing deterministic unit tests + +Our high level goal is to write unit tests that + +1. execute quickly +2. don't flake +3. are straightforward to write and understand + +For tests to execute quickly without flakes, we want to focus on _determinism_: the test should run +the same each time, and it should be easy to force the system into a known state (no races) before +executing test assertions. `time.Sleep`, `runtime.Gosched()`, and +polling/[Eventually](https://pkg.go.dev/github.com/stretchr/testify/assert#Eventually) are all +symptoms of an inability to do this easily. + +## Usage + +### `Clock` interface + +In your application code, maintain a reference to a `quartz.Clock` instance to start timers and +tickers, instead of the bare `time` standard library. + +```go +import "github.com/coder/quartz" + +type Component struct { + ... + + // for testing + clock quartz.Clock +} +``` + +Whenever you would call into `time` to start a timer or ticker, call `Component`'s `clock` instead. + +In production, set this clock to `quartz.NewReal()` to create a clock that just transparently passes +through to the standard `time` library. + +### Mocking + +In your tests, you can use a `*Mock` to control the tickers and timers your code under test gets. + +```go +import ( + "testing" + "github.com/coder/quartz" +) + +func TestComponent(t *testing.T) { + mClock := quartz.NewMock(t) + comp := &Component{ + ... + clock: mClock, + } +} +``` + +The `*Mock` clock starts at Jan 1, 2024, 00:00 UTC by default, but you can set any start time you'd like prior to your test. + +```go +mClock := quartz.NewMock(t) +mClock.Set(time.Date(2021, 6, 18, 12, 0, 0, 0, time.UTC)) // June 18, 2021 @ 12pm UTC +``` + +#### Advancing the clock + +Once you begin setting timers or tickers, you cannot change the time backward, only advance it +forward. You may continue to use `Set()`, but it is often easier and clearer to use `Advance()`. + +For example, with a timer: + +```go +fired := false + +tmr := mClock.Afterfunc(time.Second, func() { + fired = true +}) +mClock.Advance(time.Second) +``` + +When you call `Advance()` it immediately moves the clock forward the given amount, and triggers any +tickers or timers that are scheduled to happen at that time. Any triggered events happen on separate +goroutines, so _do not_ immediately assert the results: + +```go +fired := false + +tmr := mClock.Afterfunc(time.Second, func() { + fired = true +}) +mClock.Advance(time.Second) + +// RACE CONDITION, DO NOT DO THIS! +if !fired { + t.Fatal("didn't fire") +} +``` + +`Advance()` (and `Set()` for that matter) return an `AdvanceWaiter` object you can use to wait for +all triggered events to complete. + +```go +fired := false +// set a test timeout so we don't wait the default `go test` timeout for a failure +ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + +tmr := mClock.Afterfunc(time.Second, func() { + fired = true +}) + +w := mClock.Advance(time.Second) +err := w.Wait(ctx) +if err != nil { + t.Fatal("AfterFunc f never completed") +} +if !fired { + t.Fatal("didn't fire") +} +``` + +The construction of waiting for the triggered events and failing the test if they don't complete is +very common, so there is a shorthand: + +```go +w := mClock.Advance(time.Second) +err := w.Wait(ctx) +if err != nil { + t.Fatal("AfterFunc f never completed") +} +``` + +is equivalent to: + +```go +w := mClock.Advance(time.Second) +w.MustWait(ctx) +``` + +or even more briefly: + +```go +mClock.Advance(time.Second).MustWait(ctx) +``` + +### Advance only to the next event + +One important restriction on advancing the clock is that you may only advance forward to the next +timer or ticker event and no further. The following will result in a test failure: + +```go +func TestAdvanceTooFar(t *testing.T) { + ctx, cancel := context.WithTimeout(10*time.Second) + defer cancel() + mClock := quartz.NewMock(t) + var firedAt time.Time + mClock.AfterFunc(time.Second, func() { + firedAt := mClock.Now() + }) + mClock.Advance(2*time.Second).MustWait(ctx) +} +``` + +This is a deliberate design decision to allow `Advance()` to immediately and synchronously move the +clock forward (even without calling `Wait()` on returned waiter). This helps meet Quartz's design +goals of writing deterministic and easy to understand unit tests. It also allows the clock to be +advanced, deterministically _during_ the execution of a tick or timer function, as explained in the +next sections on Traps. + +Advancing multiple events can be accomplished via looping. E.g. if you have a 1-second ticker + +```go +for i := 0; i < 10; i++ { + mClock.Advance(time.Second).MustWait(ctx) +} +``` + +will advance 10 ticks. + +If you don't know or don't want to compute the time to the next event, you can use `AdvanceNext()`. + +```go +d, w := mClock.AdvanceNext() +w.MustWait(ctx) +// d contains the duration we advanced +``` + +`d, ok := Peek()` returns the duration until the next event, if any (`ok` is `true`). You can use +this to advance a specific time, regardless of the tickers and timer events: + +```go +desired := time.Minute // time to advance +for desired > 0 { + p, ok := mClock.Peek() + if !ok || p > desired { + mClock.Advance(desired).MustWait(ctx) + break + } + mClock.Advance(p).MustWait(ctx) + desired -= p +} +``` + +### Traps + +A trap allows you to match specific calls into the library while mocking, block their return, +inspect their arguments, then release them to allow them to return. They help you write +deterministic unit tests even when the code under test executes asynchronously from the test. + +You set your traps prior to executing code under test, and then wait for them to be triggered. + +```go +func TestTrap(t *testing.T) { + ctx, cancel := context.WithTimeout(10*time.Second) + defer cancel() + mClock := quartz.NewMock(t) + trap := mClock.Trap().AfterFunc() + defer trap.Close() // stop trapping AfterFunc calls + + count := 0 + go mClock.AfterFunc(time.Hour, func(){ + count++ + }) + call := trap.MustWait(ctx) + call.Release() + if call.Duration != time.Hour { + t.Fatal("wrong duration") + } + + // Now that the async call to AfterFunc has occurred, we can advance the clock to trigger it + mClock.Advance(call.Duration).MustWait(ctx) + if count != 1 { + t.Fatal("wrong count") + } +} +``` + +In this test, the trap serves 2 purposes. Firstly, it allows us to capture and assert the duration +passed to the `AfterFunc` call. Secondly, it prevents a race between setting the timer and advancing +it. Since these things happen on different goroutines, if `Advance()` completes before +`AfterFunc()` is called, then the timer never pops in this test. + +Any untrapped calls immediately complete using the current time, and calling `Close()` on a trap +causes the mock clock to stop trapping those calls. + +You may also `Advance()` the clock between trapping a call and releasing it. The call uses the +current (mocked) time at the moment it is released. + +```go +func TestTrap2(t *testing.T) { + ctx, cancel := context.WithTimeout(10*time.Second) + defer cancel() + mClock := quartz.NewMock(t) + trap := mClock.Trap().Now() + defer trap.Close() // stop trapping AfterFunc calls + + var logs []string + done := make(chan struct{}) + go func(clk quartz.Clock){ + defer close(done) + start := clk.Now() + phase1() + p1end := clk.Now() + logs = append(fmt.Sprintf("Phase 1 took %s", p1end.Sub(start).String())) + phase2() + p2end := clk.Now() + logs = append(fmt.Sprintf("Phase 2 took %s", p2end.Sub(p1end).String())) + }(mClock) + + // start + trap.MustWait(ctx).Release() + // phase 1 + call := trap.MustWait(ctx) + mClock.Advance(3*time.Second).MustWait(ctx) + call.Release() + // phase 2 + call = trap.MustWait(ctx) + mClock.Advance(5*time.Second).MustWait(ctx) + call.Release() + + <-done + // Now logs contains []string{"Phase 1 took 3s", "Phase 2 took 5s"} +} +``` + +### Tags + +When multiple goroutines in the code under test call into the Clock, you can use `tags` to +distinguish them in your traps. + +```go +trap := mClock.Trap.Now("foo") // traps any calls that contain "foo" +defer trap.Close() + +foo := make(chan time.Time) +go func(){ + foo <- mClock.Now("foo", "bar") +}() +baz := make(chan time.Time) +go func(){ + baz <- mClock.Now("baz") +}() +call := trap.MustWait(ctx) +mClock.Advance(time.Second).MustWait(ctx) +call.Release() +// call.Tags contains []string{"foo", "bar"} + +gotFoo := <-foo // 1s after start +gotBaz := <-baz // ?? never trapped, so races with Advance() +``` + +Tags appear as an optional suffix on all `Clock` methods (type `...string`) and are ignored entirely +by the real clock. They also appear on all methods on returned timers and tickers. + +## Recommended Patterns + +### Options + +We use the Option pattern to inject the mock clock for testing, keeping the call signature in +production clean. The option pattern is compatible with other optional fields as well. + +```go +type Option func(*Thing) + +// WithTestClock is used in tests to inject a mock Clock +func WithTestClock(clk quartz.Clock) Option { + return func(t *Thing) { + t.clock = clk + } +} + +func NewThing(, opts ...Option) *Thing { + t := &Thing{ + ... + clock: quartz.NewReal() + } + for _, o := range opts { + o(t) + } + return t +} +``` + +In tests, this becomes + +```go +func TestThing(t *testing.T) { + mClock := quartz.NewMock(t) + thing := NewThing(, WithTestClock(mClock)) + ... +} +``` + +### Tagging convention + +Tag your `Clock` method calls as: + +```go +func (c *Component) Method() { + now := c.clock.Now("Component", "Method") +} +``` + +or + +```go +func (c *Component) Method() { + start := c.clock.Now("Component", "Method", "start") + ... + end := c.clock.Now("Component", "Method", "end") +} +``` + +This makes it much less likely that code changes that introduce new components or methods will spoil +existing unit tests. + +## Why another time testing library? + +Writing good unit tests for components and functions that use the `time` package is difficult, even +though several open source libraries exist. In building Quartz, we took some inspiration from + +- [github.com/benbjohnson/clock](https://github.com/benbjohnson/clock) +- Tailscale's [tstest.Clock](https://github.com/coder/tailscale/blob/main/tstest/clock.go) +- [github.com/aspenmesh/tock](https://github.com/aspenmesh/tock) + +Quartz shares the high level design of a `Clock` interface that closely resembles the functions in +the `time` standard library, and a "real" clock passes thru to the standard library in production, +while a mock clock gives precise control in testing. + +As mentioned in our introduction, our high level goal is to write unit tests that + +1. execute quickly +2. don't flake +3. are straightforward to write and understand + +For several reasons, this is a tall order when it comes to code that depends on time, and we found +the existing libraries insufficient for our goals. + +### Preventing test flakes + +The following example comes from the README from benbjohnson/clock: + +```go +mock := clock.NewMock() +count := 0 + +// Kick off a timer to increment every 1 mock second. +go func() { + ticker := mock.Ticker(1 * time.Second) + for { + <-ticker.C + count++ + } +}() +runtime.Gosched() + +// Move the clock forward 10 seconds. +mock.Add(10 * time.Second) + +// This prints 10. +fmt.Println(count) +``` + +The first race condition is fairly obvious: moving the clock forward 10 seconds may generate 10 +ticks on the `ticker.C` channel, but there is no guarantee that `count++` executes before +`fmt.Println(count)`. + +The second race condition is more subtle, but `runtime.Gosched()` is the tell. Since the ticker +is started on a separate goroutine, there is no guarantee that `mock.Ticker()` executes before +`mock.Add()`. `runtime.Gosched()` is an attempt to get this to happen, but it makes no hard +promises. On a busy system, especially when running tests in parallel, this can flake, advance the +time 10 seconds first, then start the ticker and never generate a tick. + +Let's talk about how Quartz tackles these problems. + +In our experience, an extremely common use case is creating a ticker then doing a 2-arm `select` +with ticks in one and context expiring in another, i.e. + +```go +t := time.NewTicker(duration) +for { + select { + case <-ctx.Done(): + return ctx.Err() + case <-t.C: + err := do() + if err != nil { + return err + } + } +} +``` + +In Quartz, we refactor this to be more compact and testing friendly: + +```go +t := clock.TickerFunc(ctx, duration, do) +return t.Wait() +``` + +This affords the mock `Clock` the ability to explicitly know when processing of a tick is finished +because it's wrapped in the function passed to `TickerFunc` (`do()` in this example). + +In Quartz, when you advance the clock, you are returned an object you can `Wait()` on to ensure all +ticks and timers triggered are finished. This solves the first race condition in the example. + +(As an aside, we still support a traditional standard library-style `Ticker`. You may find it useful +if you want to keep your code as close as possible to the standard library, or if you need to use +the channel in a larger `select` block. In that case, you'll have to find some other mechanism to +sync tick processing to your test code.) + +To prevent race conditions related to the starting of the ticker, Quartz allows you to set "traps" +for calls that access the clock. + +```go +func TestTicker(t *testing.T) { + mClock := quartz.NewMock(t) + trap := mClock.Trap().TickerFunc() + defer trap.Close() // stop trapping at end + go runMyTicker(mClock) // async calls TickerFunc() + call := trap.Wait(context.Background()) // waits for a call and blocks its return + call.Release() // allow the TickerFunc() call to return + // optionally check the duration using call.Duration + // Move the clock forward 1 tick + mClock.Advance(time.Second).MustWait(context.Background()) + // assert results of the tick +} +``` + +Trapping and then releasing the call to `TickerFunc()` ensures the ticker is started at a +deterministic time, so our calls to `Advance()` will have a predictable effect. + +Take a look at `TestExampleTickerFunc` in `example_test.go` for a complete worked example. + +### Complex time dependence + +Another difficult issue to handle when unit testing is when some code under test makes multiple +calls that depend on the time, and you want to simulate some time passing between them. + +A very basic example is measuring how long something took: + +```go +var measurement time.Duration +go func(clock quartz.Clock) { + start := clock.Now() + doSomething() + measurement = clock.Since(start) +}(mClock) + +// how to get measurement to be, say, 5 seconds? +``` + +The two calls into the clock happen asynchronously, so we need to be able to advance the clock after +the first call to `Now()` but before the call to `Since()`. Doing this with the libraries we +mentioned above means that you have to be able to mock out or otherwise block the completion of +`doSomething()`. + +But, with the trap functionality we mentioned in the previous section, you can deterministically +control the time each call sees. + +```go +trap := mClock.Trap().Since() +var measurement time.Duration +go func(clock quartz.Clock) { + start := clock.Now() + doSomething() + measurement = clock.Since(start) +}(mClock) + +c := trap.Wait(ctx) +mClock.Advance(5*time.Second) +c.Release() +``` + +We wait until we trap the `clock.Since()` call, which implies that `clock.Now()` has completed, then +advance the mock clock 5 seconds. Finally, we release the `clock.Since()` call. Any changes to the +clock that happen _before_ we release the call will be included in the time used for the +`clock.Since()` call. + +As a more involved example, consider an inactivity timeout: we want something to happen if there is +no activity recorded for some period, say 10 minutes in the following example: + +```go +type InactivityTimer struct { + mu sync.Mutex + activity time.Time + clock quartz.Clock +} + +func (i *InactivityTimer) Start() { + i.mu.Lock() + defer i.mu.Unlock() + next := i.clock.Until(i.activity.Add(10*time.Minute)) + t := i.clock.AfterFunc(next, func() { + i.mu.Lock() + defer i.mu.Unlock() + next := i.clock.Until(i.activity.Add(10*time.Minute)) + if next == 0 { + i.timeoutLocked() + return + } + t.Reset(next) + }) +} +``` + +The actual contents of `timeoutLocked()` doesn't matter for this example, and assume there are other +functions that record the latest `activity`. + +We found that some time testing libraries hold a lock on the mock clock while calling the function +passed to `AfterFunc`, resulting in a deadlock if you made clock calls from within. + +Others allow this sort of thing, but don't have the flexibility to test edge cases. There is a +subtle bug in our `Start()` function. The timer may pop a little late, and/or some measurable real +time may elapse before `Until()` gets called inside the `AfterFunc`. If there hasn't been activity, +`next` might be negative. + +To test this in Quartz, we'll use a trap. We only want to trap the inner `Until()` call, not the +initial one, so to make testing easier we can "tag" the call we want. Like this: + +```go +func (i *InactivityTimer) Start() { + i.mu.Lock() + defer i.mu.Unlock() + next := i.clock.Until(i.activity.Add(10*time.Minute)) + t := i.clock.AfterFunc(next, func() { + i.mu.Lock() + defer i.mu.Unlock() + next := i.clock.Until(i.activity.Add(10*time.Minute), "inner") + if next == 0 { + i.timeoutLocked() + return + } + t.Reset(next) + }) +} +``` + +All Quartz `Clock` functions, and functions on returned timers and tickers support zero or more +string tags that allow traps to match on them. + +```go +func TestInactivityTimer_Late(t *testing.T) { + // set a timeout on the test itself, so that if Wait functions get blocked, we don't have to + // wait for the default test timeout of 10 minutes. + ctx, cancel := context.WithTimeout(10*time.Second) + defer cancel() + mClock := quartz.NewMock(t) + trap := mClock.Trap.Until("inner") + defer trap.Close() + + it := &InactivityTimer{ + activity: mClock.Now(), + clock: mClock, + } + it.Start() + + // Trigger the AfterFunc + w := mClock.Advance(10*time.Minute) + c := trap.Wait(ctx) + // Advance the clock a few ms to simulate a busy system + mClock.Advance(3*time.Millisecond) + c.Release() // Until() returns + w.MustWait(ctx) // Wait for the AfterFunc to wrap up + + // Assert that the timeoutLocked() function was called +} +``` + +This test case will fail with our bugged implementation, since the triggered AfterFunc won't call +`timeoutLocked()` and instead will reset the timer with a negative number. The fix is easy, use +`next <= 0` as the comparison. diff --git a/vendor/github.com/coder/quartz/clock.go b/vendor/github.com/coder/quartz/clock.go new file mode 100644 index 000000000000..729edfa5623d --- /dev/null +++ b/vendor/github.com/coder/quartz/clock.go @@ -0,0 +1,43 @@ +// Package quartz is a library for testing time related code. It exports an interface Clock that +// mimics the standard library time package functions. In production, an implementation that calls +// thru to the standard library is used. In testing, a Mock clock is used to precisely control and +// intercept time functions. +package quartz + +import ( + "context" + "time" +) + +type Clock interface { + // NewTicker returns a new Ticker containing a channel that will send the current time on the + // channel after each tick. The period of the ticks is specified by the duration argument. The + // ticker will adjust the time interval or drop ticks to make up for slow receivers. The + // duration d must be greater than zero; if not, NewTicker will panic. Stop the ticker to + // release associated resources. + NewTicker(d time.Duration, tags ...string) *Ticker + // TickerFunc is a convenience function that calls f on the interval d until either the given + // context expires or f returns an error. Callers may call Wait() on the returned Waiter to + // wait until this happens and obtain the error. The duration d must be greater than zero; if + // not, TickerFunc will panic. + TickerFunc(ctx context.Context, d time.Duration, f func() error, tags ...string) Waiter + // NewTimer creates a new Timer that will send the current time on its channel after at least + // duration d. + NewTimer(d time.Duration, tags ...string) *Timer + // AfterFunc waits for the duration to elapse and then calls f in its own goroutine. It returns + // a Timer that can be used to cancel the call using its Stop method. The returned Timer's C + // field is not used and will be nil. + AfterFunc(d time.Duration, f func(), tags ...string) *Timer + + // Now returns the current local time. + Now(tags ...string) time.Time + // Since returns the time elapsed since t. It is shorthand for Clock.Now().Sub(t). + Since(t time.Time, tags ...string) time.Duration + // Until returns the duration until t. It is shorthand for t.Sub(Clock.Now()). + Until(t time.Time, tags ...string) time.Duration +} + +// Waiter can be waited on for an error. +type Waiter interface { + Wait(tags ...string) error +} diff --git a/vendor/github.com/coder/quartz/mock.go b/vendor/github.com/coder/quartz/mock.go new file mode 100644 index 000000000000..5f97e67c4e00 --- /dev/null +++ b/vendor/github.com/coder/quartz/mock.go @@ -0,0 +1,646 @@ +package quartz + +import ( + "context" + "errors" + "fmt" + "slices" + "sync" + "testing" + "time" +) + +// Mock is the testing implementation of Clock. It tracks a time that monotonically increases +// during a test, triggering any timers or tickers automatically. +type Mock struct { + tb testing.TB + mu sync.Mutex + + // cur is the current time + cur time.Time + + all []event + nextTime time.Time + nextEvents []event + traps []*Trap +} + +type event interface { + next() time.Time + fire(t time.Time) +} + +func (m *Mock) TickerFunc(ctx context.Context, d time.Duration, f func() error, tags ...string) Waiter { + if d <= 0 { + panic("TickerFunc called with negative or zero duration") + } + m.mu.Lock() + defer m.mu.Unlock() + c := newCall(clockFunctionTickerFunc, tags, withDuration(d)) + m.matchCallLocked(c) + defer close(c.complete) + t := &mockTickerFunc{ + ctx: ctx, + d: d, + f: f, + nxt: m.cur.Add(d), + mock: m, + cond: sync.NewCond(&m.mu), + } + m.all = append(m.all, t) + m.recomputeNextLocked() + go t.waitForCtx() + return t +} + +func (m *Mock) NewTicker(d time.Duration, tags ...string) *Ticker { + if d <= 0 { + panic("NewTicker called with negative or zero duration") + } + m.mu.Lock() + defer m.mu.Unlock() + c := newCall(clockFunctionNewTicker, tags, withDuration(d)) + m.matchCallLocked(c) + defer close(c.complete) + // 1 element buffer follows standard library implementation + ticks := make(chan time.Time, 1) + t := &Ticker{ + C: ticks, + c: ticks, + d: d, + nxt: m.cur.Add(d), + mock: m, + } + m.addEventLocked(t) + return t +} + +func (m *Mock) NewTimer(d time.Duration, tags ...string) *Timer { + m.mu.Lock() + defer m.mu.Unlock() + c := newCall(clockFunctionNewTimer, tags, withDuration(d)) + defer close(c.complete) + m.matchCallLocked(c) + ch := make(chan time.Time, 1) + t := &Timer{ + C: ch, + c: ch, + nxt: m.cur.Add(d), + mock: m, + } + if d <= 0 { + // zero or negative duration timer means we should immediately fire + // it, rather than add it. + go t.fire(t.mock.cur) + return t + } + m.addEventLocked(t) + return t +} + +func (m *Mock) AfterFunc(d time.Duration, f func(), tags ...string) *Timer { + m.mu.Lock() + defer m.mu.Unlock() + c := newCall(clockFunctionAfterFunc, tags, withDuration(d)) + defer close(c.complete) + m.matchCallLocked(c) + t := &Timer{ + nxt: m.cur.Add(d), + fn: f, + mock: m, + } + if d <= 0 { + // zero or negative duration timer means we should immediately fire + // it, rather than add it. + go t.fire(t.mock.cur) + return t + } + m.addEventLocked(t) + return t +} + +func (m *Mock) Now(tags ...string) time.Time { + m.mu.Lock() + defer m.mu.Unlock() + c := newCall(clockFunctionNow, tags) + defer close(c.complete) + m.matchCallLocked(c) + return m.cur +} + +func (m *Mock) Since(t time.Time, tags ...string) time.Duration { + m.mu.Lock() + defer m.mu.Unlock() + c := newCall(clockFunctionSince, tags, withTime(t)) + defer close(c.complete) + m.matchCallLocked(c) + return m.cur.Sub(t) +} + +func (m *Mock) Until(t time.Time, tags ...string) time.Duration { + m.mu.Lock() + defer m.mu.Unlock() + c := newCall(clockFunctionUntil, tags, withTime(t)) + defer close(c.complete) + m.matchCallLocked(c) + return t.Sub(m.cur) +} + +func (m *Mock) addEventLocked(e event) { + m.all = append(m.all, e) + m.recomputeNextLocked() +} + +func (m *Mock) recomputeNextLocked() { + var best time.Time + var events []event + for _, e := range m.all { + if best.IsZero() || e.next().Before(best) { + best = e.next() + events = []event{e} + continue + } + if e.next().Equal(best) { + events = append(events, e) + continue + } + } + m.nextTime = best + m.nextEvents = events +} + +func (m *Mock) removeTimer(t *Timer) { + m.mu.Lock() + defer m.mu.Unlock() + m.removeTimerLocked(t) +} + +func (m *Mock) removeTimerLocked(t *Timer) { + t.stopped = true + m.removeEventLocked(t) +} + +func (m *Mock) removeEventLocked(e event) { + defer m.recomputeNextLocked() + for i := range m.all { + if m.all[i] == e { + m.all = append(m.all[:i], m.all[i+1:]...) + return + } + } +} + +func (m *Mock) matchCallLocked(c *Call) { + var traps []*Trap + for _, t := range m.traps { + if t.matches(c) { + traps = append(traps, t) + } + } + if len(traps) == 0 { + return + } + c.releases.Add(len(traps)) + m.mu.Unlock() + for _, t := range traps { + go t.catch(c) + } + c.releases.Wait() + m.mu.Lock() +} + +// AdvanceWaiter is returned from Advance and Set calls and allows you to wait for ticks and timers +// to complete. In the case of functions passed to AfterFunc or TickerFunc, it waits for the +// functions to return. For other ticks & timers, it just waits for the tick to be delivered to +// the channel. +// +// If multiple timers or tickers trigger simultaneously, they are all run on separate +// go routines. +type AdvanceWaiter struct { + tb testing.TB + ch chan struct{} +} + +// Wait for all timers and ticks to complete, or until context expires. +func (w AdvanceWaiter) Wait(ctx context.Context) error { + select { + case <-w.ch: + return nil + case <-ctx.Done(): + return ctx.Err() + } +} + +// MustWait waits for all timers and ticks to complete, and fails the test immediately if the +// context completes first. MustWait must be called from the goroutine running the test or +// benchmark, similar to `t.FailNow()`. +func (w AdvanceWaiter) MustWait(ctx context.Context) { + w.tb.Helper() + select { + case <-w.ch: + return + case <-ctx.Done(): + w.tb.Fatalf("context expired while waiting for clock to advance: %s", ctx.Err()) + } +} + +// Done returns a channel that is closed when all timers and ticks complete. +func (w AdvanceWaiter) Done() <-chan struct{} { + return w.ch +} + +// Advance moves the clock forward by d, triggering any timers or tickers. The returned value can +// be used to wait for all timers and ticks to complete. Advance sets the clock forward before +// returning, and can only advance up to the next timer or tick event. It will fail the test if you +// attempt to advance beyond. +// +// If you need to advance exactly to the next event, and don't know or don't wish to calculate it, +// consider AdvanceNext(). +func (m *Mock) Advance(d time.Duration) AdvanceWaiter { + m.tb.Helper() + w := AdvanceWaiter{tb: m.tb, ch: make(chan struct{})} + m.mu.Lock() + fin := m.cur.Add(d) + // nextTime.IsZero implies no events scheduled. + if m.nextTime.IsZero() || fin.Before(m.nextTime) { + m.cur = fin + m.mu.Unlock() + close(w.ch) + return w + } + if fin.After(m.nextTime) { + m.tb.Errorf(fmt.Sprintf("cannot advance %s which is beyond next timer/ticker event in %s", + d.String(), m.nextTime.Sub(m.cur))) + m.mu.Unlock() + close(w.ch) + return w + } + + m.cur = m.nextTime + go m.advanceLocked(w) + return w +} + +func (m *Mock) advanceLocked(w AdvanceWaiter) { + defer close(w.ch) + wg := sync.WaitGroup{} + for i := range m.nextEvents { + e := m.nextEvents[i] + t := m.cur + wg.Add(1) + go func() { + e.fire(t) + wg.Done() + }() + } + // release the lock and let the events resolve. This allows them to call back into the + // Mock to query the time or set new timers. Each event should remove or reschedule + // itself from nextEvents. + m.mu.Unlock() + wg.Wait() +} + +// Set the time to t. If the time is after the current mocked time, then this is equivalent to +// Advance() with the difference. You may only Set the time earlier than the current time before +// starting tickers and timers (e.g. at the start of your test case). +func (m *Mock) Set(t time.Time) AdvanceWaiter { + m.tb.Helper() + w := AdvanceWaiter{tb: m.tb, ch: make(chan struct{})} + m.mu.Lock() + if t.Before(m.cur) { + defer close(w.ch) + defer m.mu.Unlock() + // past + if !m.nextTime.IsZero() { + m.tb.Error("Set mock clock to the past after timers/tickers started") + } + m.cur = t + return w + } + // future + // nextTime.IsZero implies no events scheduled. + if m.nextTime.IsZero() || t.Before(m.nextTime) { + defer close(w.ch) + defer m.mu.Unlock() + m.cur = t + return w + } + if t.After(m.nextTime) { + defer close(w.ch) + defer m.mu.Unlock() + m.tb.Errorf("cannot Set time to %s which is beyond next timer/ticker event at %s", + t.String(), m.nextTime) + return w + } + + m.cur = m.nextTime + go m.advanceLocked(w) + return w +} + +// AdvanceNext advances the clock to the next timer or tick event. It fails the test if there are +// none scheduled. It returns the duration the clock was advanced and a waiter that can be used to +// wait for the timer/tick event(s) to finish. +func (m *Mock) AdvanceNext() (time.Duration, AdvanceWaiter) { + m.mu.Lock() + m.tb.Helper() + w := AdvanceWaiter{tb: m.tb, ch: make(chan struct{})} + if m.nextTime.IsZero() { + defer close(w.ch) + defer m.mu.Unlock() + m.tb.Error("cannot AdvanceNext because there are no timers or tickers running") + } + d := m.nextTime.Sub(m.cur) + m.cur = m.nextTime + go m.advanceLocked(w) + return d, w +} + +// Peek returns the duration until the next ticker or timer event and the value +// true, or, if there are no running tickers or timers, it returns zero and +// false. +func (m *Mock) Peek() (d time.Duration, ok bool) { + m.mu.Lock() + defer m.mu.Unlock() + if m.nextTime.IsZero() { + return 0, false + } + return m.nextTime.Sub(m.cur), true +} + +// Trapper allows the creation of Traps +type Trapper struct { + // mock is the underlying Mock. This is a thin wrapper around Mock so that + // we can have our interface look like mClock.Trap().NewTimer("foo") + mock *Mock +} + +func (t Trapper) NewTimer(tags ...string) *Trap { + return t.mock.newTrap(clockFunctionNewTimer, tags) +} + +func (t Trapper) AfterFunc(tags ...string) *Trap { + return t.mock.newTrap(clockFunctionAfterFunc, tags) +} + +func (t Trapper) TimerStop(tags ...string) *Trap { + return t.mock.newTrap(clockFunctionTimerStop, tags) +} + +func (t Trapper) TimerReset(tags ...string) *Trap { + return t.mock.newTrap(clockFunctionTimerReset, tags) +} + +func (t Trapper) TickerFunc(tags ...string) *Trap { + return t.mock.newTrap(clockFunctionTickerFunc, tags) +} + +func (t Trapper) TickerFuncWait(tags ...string) *Trap { + return t.mock.newTrap(clockFunctionTickerFuncWait, tags) +} + +func (t Trapper) NewTicker(tags ...string) *Trap { + return t.mock.newTrap(clockFunctionNewTicker, tags) +} + +func (t Trapper) TickerStop(tags ...string) *Trap { + return t.mock.newTrap(clockFunctionTickerStop, tags) +} + +func (t Trapper) TickerReset(tags ...string) *Trap { + return t.mock.newTrap(clockFunctionTickerReset, tags) +} + +func (t Trapper) Now(tags ...string) *Trap { + return t.mock.newTrap(clockFunctionNow, tags) +} + +func (t Trapper) Since(tags ...string) *Trap { + return t.mock.newTrap(clockFunctionSince, tags) +} + +func (t Trapper) Until(tags ...string) *Trap { + return t.mock.newTrap(clockFunctionUntil, tags) +} + +func (m *Mock) Trap() Trapper { + return Trapper{m} +} + +func (m *Mock) newTrap(fn clockFunction, tags []string) *Trap { + m.mu.Lock() + defer m.mu.Unlock() + tr := &Trap{ + fn: fn, + tags: tags, + mock: m, + calls: make(chan *Call), + done: make(chan struct{}), + } + m.traps = append(m.traps, tr) + return tr +} + +// NewMock creates a new Mock with the time set to midnight UTC on Jan 1, 2024. +// You may re-set the time earlier than this, but only before timers or tickers +// are created. +func NewMock(tb testing.TB) *Mock { + cur, err := time.Parse(time.RFC3339, "2024-01-01T00:00:00Z") + if err != nil { + panic(err) + } + return &Mock{ + tb: tb, + cur: cur, + } +} + +var _ Clock = &Mock{} + +type mockTickerFunc struct { + ctx context.Context + d time.Duration + f func() error + nxt time.Time + mock *Mock + + // cond is a condition Locked on the main Mock.mu + cond *sync.Cond + // done is true when the ticker exits + done bool + // err holds the error when the ticker exits + err error +} + +func (m *mockTickerFunc) next() time.Time { + return m.nxt +} + +func (m *mockTickerFunc) fire(_ time.Time) { + m.mock.mu.Lock() + defer m.mock.mu.Unlock() + if m.done { + return + } + m.nxt = m.nxt.Add(m.d) + m.mock.recomputeNextLocked() + + m.mock.mu.Unlock() + err := m.f() + m.mock.mu.Lock() + if err != nil { + m.exitLocked(err) + } +} + +func (m *mockTickerFunc) exitLocked(err error) { + if m.done { + return + } + m.done = true + m.err = err + m.mock.removeEventLocked(m) + m.cond.Broadcast() +} + +func (m *mockTickerFunc) waitForCtx() { + <-m.ctx.Done() + m.mock.mu.Lock() + defer m.mock.mu.Unlock() + m.exitLocked(m.ctx.Err()) +} + +func (m *mockTickerFunc) Wait(tags ...string) error { + m.mock.mu.Lock() + defer m.mock.mu.Unlock() + c := newCall(clockFunctionTickerFuncWait, tags) + m.mock.matchCallLocked(c) + defer close(c.complete) + for !m.done { + m.cond.Wait() + } + return m.err +} + +var _ Waiter = &mockTickerFunc{} + +type clockFunction int + +const ( + clockFunctionNewTimer clockFunction = iota + clockFunctionAfterFunc + clockFunctionTimerStop + clockFunctionTimerReset + clockFunctionTickerFunc + clockFunctionTickerFuncWait + clockFunctionNewTicker + clockFunctionTickerReset + clockFunctionTickerStop + clockFunctionNow + clockFunctionSince + clockFunctionUntil +) + +type callArg func(c *Call) + +type Call struct { + Time time.Time + Duration time.Duration + Tags []string + + fn clockFunction + releases sync.WaitGroup + complete chan struct{} +} + +func (c *Call) Release() { + c.releases.Done() + <-c.complete +} + +func withTime(t time.Time) callArg { + return func(c *Call) { + c.Time = t + } +} + +func withDuration(d time.Duration) callArg { + return func(c *Call) { + c.Duration = d + } +} + +func newCall(fn clockFunction, tags []string, args ...callArg) *Call { + c := &Call{ + fn: fn, + Tags: tags, + complete: make(chan struct{}), + } + for _, a := range args { + a(c) + } + return c +} + +type Trap struct { + fn clockFunction + tags []string + mock *Mock + calls chan *Call + done chan struct{} +} + +func (t *Trap) catch(c *Call) { + select { + case t.calls <- c: + case <-t.done: + c.Release() + } +} + +func (t *Trap) matches(c *Call) bool { + if t.fn != c.fn { + return false + } + for _, tag := range t.tags { + if !slices.Contains(c.Tags, tag) { + return false + } + } + return true +} + +func (t *Trap) Close() { + t.mock.mu.Lock() + defer t.mock.mu.Unlock() + for i, tr := range t.mock.traps { + if t == tr { + t.mock.traps = append(t.mock.traps[:i], t.mock.traps[i+1:]...) + } + } + close(t.done) +} + +var ErrTrapClosed = errors.New("trap closed") + +func (t *Trap) Wait(ctx context.Context) (*Call, error) { + select { + case <-ctx.Done(): + return nil, ctx.Err() + case <-t.done: + return nil, ErrTrapClosed + case c := <-t.calls: + return c, nil + } +} + +// MustWait calls Wait() and then if there is an error, immediately fails the +// test via tb.Fatalf() +func (t *Trap) MustWait(ctx context.Context) *Call { + t.mock.tb.Helper() + c, err := t.Wait(ctx) + if err != nil { + t.mock.tb.Fatalf("context expired while waiting for trap: %s", err.Error()) + } + return c +} diff --git a/vendor/github.com/coder/quartz/real.go b/vendor/github.com/coder/quartz/real.go new file mode 100644 index 000000000000..f39fb1163e82 --- /dev/null +++ b/vendor/github.com/coder/quartz/real.go @@ -0,0 +1,80 @@ +package quartz + +import ( + "context" + "time" +) + +type realClock struct{} + +func NewReal() Clock { + return realClock{} +} + +func (realClock) NewTicker(d time.Duration, _ ...string) *Ticker { + tkr := time.NewTicker(d) + return &Ticker{ticker: tkr, C: tkr.C} +} + +func (realClock) TickerFunc(ctx context.Context, d time.Duration, f func() error, _ ...string) Waiter { + ct := &realContextTicker{ + ctx: ctx, + tkr: time.NewTicker(d), + f: f, + err: make(chan error, 1), + } + go ct.run() + return ct +} + +type realContextTicker struct { + ctx context.Context + tkr *time.Ticker + f func() error + err chan error +} + +func (t *realContextTicker) Wait(_ ...string) error { + return <-t.err +} + +func (t *realContextTicker) run() { + defer t.tkr.Stop() + for { + select { + case <-t.ctx.Done(): + t.err <- t.ctx.Err() + return + case <-t.tkr.C: + err := t.f() + if err != nil { + t.err <- err + return + } + } + } +} + +func (realClock) NewTimer(d time.Duration, _ ...string) *Timer { + rt := time.NewTimer(d) + return &Timer{C: rt.C, timer: rt} +} + +func (realClock) AfterFunc(d time.Duration, f func(), _ ...string) *Timer { + rt := time.AfterFunc(d, f) + return &Timer{C: rt.C, timer: rt} +} + +func (realClock) Now(_ ...string) time.Time { + return time.Now() +} + +func (realClock) Since(t time.Time, _ ...string) time.Duration { + return time.Since(t) +} + +func (realClock) Until(t time.Time, _ ...string) time.Duration { + return time.Until(t) +} + +var _ Clock = realClock{} diff --git a/vendor/github.com/coder/quartz/ticker.go b/vendor/github.com/coder/quartz/ticker.go new file mode 100644 index 000000000000..5506750d5d1f --- /dev/null +++ b/vendor/github.com/coder/quartz/ticker.go @@ -0,0 +1,75 @@ +package quartz + +import "time" + +// A Ticker holds a channel that delivers “ticks” of a clock at intervals. +type Ticker struct { + C <-chan time.Time + //nolint: revive + c chan time.Time + ticker *time.Ticker // realtime impl, if set + d time.Duration // period, if set + nxt time.Time // next tick time + mock *Mock // mock clock, if set + stopped bool // true if the ticker is not running +} + +func (t *Ticker) fire(tt time.Time) { + t.mock.mu.Lock() + defer t.mock.mu.Unlock() + if t.stopped { + return + } + for !t.nxt.After(t.mock.cur) { + t.nxt = t.nxt.Add(t.d) + } + t.mock.recomputeNextLocked() + select { + case t.c <- tt: + default: + } +} + +func (t *Ticker) next() time.Time { + return t.nxt +} + +// Stop turns off a ticker. After Stop, no more ticks will be sent. Stop does +// not close the channel, to prevent a concurrent goroutine reading from the +// channel from seeing an erroneous "tick". +func (t *Ticker) Stop(tags ...string) { + if t.ticker != nil { + t.ticker.Stop() + return + } + t.mock.mu.Lock() + defer t.mock.mu.Unlock() + c := newCall(clockFunctionTickerStop, tags) + t.mock.matchCallLocked(c) + defer close(c.complete) + t.mock.removeEventLocked(t) + t.stopped = true +} + +// Reset stops a ticker and resets its period to the specified duration. The +// next tick will arrive after the new period elapses. The duration d must be +// greater than zero; if not, Reset will panic. +func (t *Ticker) Reset(d time.Duration, tags ...string) { + if t.ticker != nil { + t.ticker.Reset(d) + return + } + t.mock.mu.Lock() + defer t.mock.mu.Unlock() + c := newCall(clockFunctionTickerReset, tags, withDuration(d)) + t.mock.matchCallLocked(c) + defer close(c.complete) + t.nxt = t.mock.cur.Add(d) + t.d = d + if t.stopped { + t.stopped = false + t.mock.addEventLocked(t) + } else { + t.mock.recomputeNextLocked() + } +} diff --git a/vendor/github.com/coder/quartz/timer.go b/vendor/github.com/coder/quartz/timer.go new file mode 100644 index 000000000000..8d928ac6f609 --- /dev/null +++ b/vendor/github.com/coder/quartz/timer.go @@ -0,0 +1,81 @@ +package quartz + +import "time" + +// The Timer type represents a single event. When the Timer expires, the current time will be sent +// on C, unless the Timer was created by AfterFunc. A Timer must be created with NewTimer or +// AfterFunc. +type Timer struct { + C <-chan time.Time + //nolint: revive + c chan time.Time + timer *time.Timer // realtime impl, if set + nxt time.Time // next tick time + mock *Mock // mock clock, if set + fn func() // AfterFunc function, if set + stopped bool // True if stopped, false if running +} + +func (t *Timer) fire(tt time.Time) { + t.mock.removeTimer(t) + if t.fn != nil { + t.fn() + } else { + t.c <- tt + } +} + +func (t *Timer) next() time.Time { + return t.nxt +} + +// Stop prevents the Timer from firing. It returns true if the call stops the timer, false if the +// timer has already expired or been stopped. Stop does not close the channel, to prevent a read +// from the channel succeeding incorrectly. +// +// See https://pkg.go.dev/time#Timer.Stop for more information. +func (t *Timer) Stop(tags ...string) bool { + if t.timer != nil { + return t.timer.Stop() + } + t.mock.mu.Lock() + defer t.mock.mu.Unlock() + c := newCall(clockFunctionTimerStop, tags) + t.mock.matchCallLocked(c) + defer close(c.complete) + result := !t.stopped + t.mock.removeTimerLocked(t) + return result +} + +// Reset changes the timer to expire after duration d. It returns true if the timer had been active, +// false if the timer had expired or been stopped. +// +// See https://pkg.go.dev/time#Timer.Reset for more information. +func (t *Timer) Reset(d time.Duration, tags ...string) bool { + if t.timer != nil { + return t.timer.Reset(d) + } + t.mock.mu.Lock() + defer t.mock.mu.Unlock() + c := newCall(clockFunctionTimerReset, tags, withDuration(d)) + t.mock.matchCallLocked(c) + defer close(c.complete) + result := !t.stopped + select { + case <-t.c: + default: + } + if d <= 0 { + // zero or negative duration timer means we should immediately re-fire + // it, rather than remove and re-add it. + t.stopped = false + go t.fire(t.mock.cur) + return result + } + t.mock.removeTimerLocked(t) + t.stopped = false + t.nxt = t.mock.cur.Add(d) + t.mock.addEventLocked(t) + return result +} diff --git a/vendor/modules.txt b/vendor/modules.txt index 3987a73ef672..e513c15d0aae 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -499,6 +499,9 @@ github.com/cncf/xds/go/xds/data/orca/v3 github.com/cncf/xds/go/xds/service/orca/v3 github.com/cncf/xds/go/xds/type/matcher/v3 github.com/cncf/xds/go/xds/type/v3 +# github.com/coder/quartz v0.1.0 +## explicit; go 1.21.8 +github.com/coder/quartz # github.com/containerd/fifo v1.0.0 ## explicit; go 1.13 github.com/containerd/fifo From e2cbde884f65dc2193bbac7dc21af0639c6604b1 Mon Sep 17 00:00:00 2001 From: George Robinson Date: Wed, 17 Jul 2024 09:00:52 +0100 Subject: [PATCH 09/23] chore: Clean up manager_test.go (#13549) --- pkg/storage/wal/manager_test.go | 209 ++++++++++++++++---------------- 1 file changed, 105 insertions(+), 104 deletions(-) diff --git a/pkg/storage/wal/manager_test.go b/pkg/storage/wal/manager_test.go index ef14b2420108..9dc49dce4759 100644 --- a/pkg/storage/wal/manager_test.go +++ b/pkg/storage/wal/manager_test.go @@ -87,19 +87,15 @@ func TestManager_AppendFailed(t *testing.T) { require.EqualError(t, res.Err(), "failed to flush") } -func TestManager_AppendMaxAge(t *testing.T) { +func TestManager_AppendFailedWALClosed(t *testing.T) { m, err := NewManager(Config{ - MaxAge: 100 * time.Millisecond, - MaxSegments: 1, - MaxSegmentSize: 8 * 1024 * 1024, // 8MB + MaxAge: 30 * time.Second, + MaxSegments: 10, + MaxSegmentSize: 1024, // 1KB }, NewMetrics(nil)) require.NoError(t, err) - // Create a mock clock. - clock := quartz.NewMock(t) - m.clock = clock - - // Append 1B of data. + // Append some data. lbs := labels.Labels{{Name: "a", Value: "b"}} entries := []*logproto.Entry{{Timestamp: time.Now(), Line: "c"}} res, err := m.Append(AppendRequest{ @@ -111,40 +107,70 @@ func TestManager_AppendMaxAge(t *testing.T) { require.NoError(t, err) require.NotNil(t, res) - // The segment that was just appended to has neither reached the maximum - // age nor maximum size to be flushed. - require.Equal(t, 1, m.available.Len()) - require.Equal(t, 0, m.pending.Len()) + // Close the WAL. + m.Close() - // Wait 100ms and append some more data. - clock.Advance(100 * time.Millisecond) - entries = []*logproto.Entry{{Timestamp: time.Now(), Line: "c"}} + // Should not be able to append more data as the WAL is closed. res, err = m.Append(AppendRequest{ TenantID: "1", Labels: lbs, LabelsStr: lbs.String(), Entries: entries, }) + require.Nil(t, res) + require.ErrorIs(t, err, ErrClosed) +} + +func TestManager_AppendFailedWALFull(t *testing.T) { + m, err := NewManager(Config{ + MaxAge: 30 * time.Second, + MaxSegments: 10, + MaxSegmentSize: 1024, // 1KB + }, NewMetrics(nil)) require.NoError(t, err) - require.NotNil(t, res) - // The segment has reached the maximum age and should have been moved to - // pending list to be flushed. - require.Equal(t, 0, m.available.Len()) - require.Equal(t, 1, m.pending.Len()) + // Should be able to write 100KB of data, 10KB per segment. + lbs := labels.Labels{{Name: "a", Value: "b"}} + for i := 0; i < 10; i++ { + entries := []*logproto.Entry{{Timestamp: time.Now(), Line: strings.Repeat("c", 1024)}} + res, err := m.Append(AppendRequest{ + TenantID: "1", + Labels: lbs, + LabelsStr: lbs.String(), + Entries: entries, + }) + require.NoError(t, err) + require.NotNil(t, res) + } + + // However, appending more data should fail as all segments are full and + // waiting to be flushed. + entries := []*logproto.Entry{{Timestamp: time.Now(), Line: strings.Repeat("c", 1024)}} + res, err := m.Append(AppendRequest{ + TenantID: "1", + Labels: lbs, + LabelsStr: lbs.String(), + Entries: entries, + }) + require.ErrorIs(t, err, ErrFull) + require.Nil(t, res) } -func TestManager_AppendMaxSize(t *testing.T) { +func TestManager_AppendMaxAgeExceeded(t *testing.T) { m, err := NewManager(Config{ - MaxAge: 30 * time.Second, + MaxAge: 100 * time.Millisecond, MaxSegments: 1, - MaxSegmentSize: 1024, // 1KB + MaxSegmentSize: 8 * 1024 * 1024, // 8MB }, NewMetrics(nil)) require.NoError(t, err) - // Append 512B of data. + // Create a mock clock. + clock := quartz.NewMock(t) + m.clock = clock + + // Append 1B of data. lbs := labels.Labels{{Name: "a", Value: "b"}} - entries := []*logproto.Entry{{Timestamp: time.Now(), Line: strings.Repeat("c", 512)}} + entries := []*logproto.Entry{{Timestamp: time.Now(), Line: "c"}} res, err := m.Append(AppendRequest{ TenantID: "1", Labels: lbs, @@ -159,8 +185,9 @@ func TestManager_AppendMaxSize(t *testing.T) { require.Equal(t, 1, m.available.Len()) require.Equal(t, 0, m.pending.Len()) - // Append another 512B of data. - entries = []*logproto.Entry{{Timestamp: time.Now(), Line: strings.Repeat("c", 512)}} + // Wait 100ms and append some more data. + clock.Advance(100 * time.Millisecond) + entries = []*logproto.Entry{{Timestamp: time.Now(), Line: "c"}} res, err = m.Append(AppendRequest{ TenantID: "1", Labels: lbs, @@ -170,23 +197,23 @@ func TestManager_AppendMaxSize(t *testing.T) { require.NoError(t, err) require.NotNil(t, res) - // The segment has reached the maximum size and should have been moved to + // The segment has reached the maximum age and should have been moved to // pending list to be flushed. require.Equal(t, 0, m.available.Len()) require.Equal(t, 1, m.pending.Len()) } -func TestManager_AppendWALClosed(t *testing.T) { +func TestManager_AppendMaxSizeExceeded(t *testing.T) { m, err := NewManager(Config{ MaxAge: 30 * time.Second, - MaxSegments: 10, + MaxSegments: 1, MaxSegmentSize: 1024, // 1KB }, NewMetrics(nil)) require.NoError(t, err) - // Append some data. + // Append 512B of data. lbs := labels.Labels{{Name: "a", Value: "b"}} - entries := []*logproto.Entry{{Timestamp: time.Now(), Line: "c"}} + entries := []*logproto.Entry{{Timestamp: time.Now(), Line: strings.Repeat("c", 512)}} res, err := m.Append(AppendRequest{ TenantID: "1", Labels: lbs, @@ -196,53 +223,26 @@ func TestManager_AppendWALClosed(t *testing.T) { require.NoError(t, err) require.NotNil(t, res) - // Close the WAL. - m.Close() + // The segment that was just appended to has neither reached the maximum + // age nor maximum size to be flushed. + require.Equal(t, 1, m.available.Len()) + require.Equal(t, 0, m.pending.Len()) - // Should not be able to append more data as the WAL is closed. + // Append another 512B of data. + entries = []*logproto.Entry{{Timestamp: time.Now(), Line: strings.Repeat("c", 512)}} res, err = m.Append(AppendRequest{ TenantID: "1", Labels: lbs, LabelsStr: lbs.String(), Entries: entries, }) - require.Nil(t, res) - require.ErrorIs(t, err, ErrClosed) -} - -func TestManager_AppendWALFull(t *testing.T) { - m, err := NewManager(Config{ - MaxAge: 30 * time.Second, - MaxSegments: 10, - MaxSegmentSize: 1024, // 1KB - }, NewMetrics(nil)) require.NoError(t, err) + require.NotNil(t, res) - // Should be able to write 100KB of data, 10KB per segment. - lbs := labels.Labels{{Name: "a", Value: "b"}} - for i := 0; i < 10; i++ { - entries := []*logproto.Entry{{Timestamp: time.Now(), Line: strings.Repeat("c", 1024)}} - res, err := m.Append(AppendRequest{ - TenantID: "1", - Labels: lbs, - LabelsStr: lbs.String(), - Entries: entries, - }) - require.NoError(t, err) - require.NotNil(t, res) - } - - // However, appending more data should fail as all segments are full and - // waiting to be flushed. - entries := []*logproto.Entry{{Timestamp: time.Now(), Line: strings.Repeat("c", 1024)}} - res, err := m.Append(AppendRequest{ - TenantID: "1", - Labels: lbs, - LabelsStr: lbs.String(), - Entries: entries, - }) - require.ErrorIs(t, err, ErrFull) - require.Nil(t, res) + // The segment has reached the maximum size and should have been moved to + // pending list to be flushed. + require.Equal(t, 0, m.available.Len()) + require.Equal(t, 1, m.pending.Len()) } func TestManager_NextPending(t *testing.T) { @@ -281,15 +281,19 @@ func TestManager_NextPending(t *testing.T) { require.Nil(t, it) } -func TestManager_NextPendingClosed(t *testing.T) { +func TestManager_NextPendingMaxAgeExceeded(t *testing.T) { m, err := NewManager(Config{ - MaxAge: 30 * time.Second, - MaxSegments: 10, + MaxAge: 100 * time.Millisecond, + MaxSegments: 1, MaxSegmentSize: 1024, // 1KB }, NewMetrics(nil)) require.NoError(t, err) - // Append some data. + // Create a mock clock. + clock := quartz.NewMock(t) + m.clock = clock + + // Append 1B of data. lbs := labels.Labels{{Name: "a", Value: "b"}} entries := []*logproto.Entry{{Timestamp: time.Now(), Line: "c"}} res, err := m.Append(AppendRequest{ @@ -301,40 +305,33 @@ func TestManager_NextPendingClosed(t *testing.T) { require.NoError(t, err) require.NotNil(t, res) - // There should be no segments waiting to be flushed as neither the maximum - // age nor maximum size has been exceeded. + // The segment that was just appended to has neither reached the maximum + // age nor maximum size to be flushed. it, err := m.NextPending() require.NoError(t, err) require.Nil(t, it) + require.Equal(t, 1, m.available.Len()) + require.Equal(t, 0, m.pending.Len()) - // Close the WAL. - m.Close() - - // There should be one segment waiting to be flushed. + // Wait 100ms. The segment that was just appended to should have reached + // the maximum age. + clock.Advance(100 * time.Millisecond) it, err = m.NextPending() require.NoError(t, err) require.NotNil(t, it) - - // There are no more segments waiting to be flushed, and since the WAL is - // closed, successive calls should return ErrClosed. - it, err = m.NextPending() - require.ErrorIs(t, err, ErrClosed) - require.Nil(t, it) + require.Equal(t, 0, m.available.Len()) + require.Equal(t, 0, m.pending.Len()) } -func TestManager_NexPendingMaxAge(t *testing.T) { +func TestManager_NextPendingWALClosed(t *testing.T) { m, err := NewManager(Config{ - MaxAge: 100 * time.Millisecond, + MaxAge: 30 * time.Second, MaxSegments: 1, MaxSegmentSize: 1024, // 1KB }, NewMetrics(nil)) require.NoError(t, err) - // Create a mock clock. - clock := quartz.NewMock(t) - m.clock = clock - - // Append 1B of data. + // Append some data. lbs := labels.Labels{{Name: "a", Value: "b"}} entries := []*logproto.Entry{{Timestamp: time.Now(), Line: "c"}} res, err := m.Append(AppendRequest{ @@ -346,22 +343,27 @@ func TestManager_NexPendingMaxAge(t *testing.T) { require.NoError(t, err) require.NotNil(t, res) - // The segment that was just appended to has neither reached the maximum - // age nor maximum size to be flushed. + // There should be no segments waiting to be flushed as neither the maximum + // age nor maximum size has been exceeded. it, err := m.NextPending() require.NoError(t, err) require.Nil(t, it) - require.Equal(t, 1, m.available.Len()) - require.Equal(t, 0, m.pending.Len()) - // Wait 100ms. The segment that was just appended to should have reached - // the maximum age. - clock.Advance(100 * time.Millisecond) + // Close the WAL. + m.Close() + + // There should be one segment waiting to be flushed. it, err = m.NextPending() require.NoError(t, err) require.NotNil(t, it) - require.Equal(t, 0, m.available.Len()) - require.Equal(t, 0, m.pending.Len()) + + // There are no more segments waiting to be flushed, and since the WAL is + // closed, successive calls should return ErrClosed. + for i := 0; i < 10; i++ { + it, err = m.NextPending() + require.ErrorIs(t, err, ErrClosed) + require.Nil(t, it) + } } func TestManager_Put(t *testing.T) { @@ -493,5 +495,4 @@ wal_segments_flushing 0 wal_segments_pending 0 ` require.NoError(t, testutil.CollectAndCompare(r, strings.NewReader(expected), metricNames...)) - } From 8ef86f8d8bd6771f759cbba8dfc12dbec5263881 Mon Sep 17 00:00:00 2001 From: George Robinson Date: Wed, 17 Jul 2024 09:46:58 +0100 Subject: [PATCH 10/23] chore: refactor WAL Manager (#13551) --- pkg/storage/wal/manager.go | 179 ++++++++++++++++++------------------- 1 file changed, 89 insertions(+), 90 deletions(-) diff --git a/pkg/storage/wal/manager.go b/pkg/storage/wal/manager.go index 196d02296b5f..d5d8fff893f0 100644 --- a/pkg/storage/wal/manager.go +++ b/pkg/storage/wal/manager.go @@ -13,25 +13,19 @@ import ( ) const ( - // DefaultMaxAge is the default value for the maximum amount of time a - // segment can can be buffered in memory before it should be flushed. - DefaultMaxAge = 500 * time.Millisecond - // DefaultMaxSegments is the default value for the maximum number of - // segments that can be buffered in memory, including segments waiting to - // be flushed. - DefaultMaxSegments = 10 - // DefaultMaxSegmentSize is the default value for the maximum segment size - // (uncompressed). + DefaultMaxAge = 500 * time.Millisecond + DefaultMaxSegments = 10 DefaultMaxSegmentSize = 8 * 1024 * 1024 // 8MB. ) var ( - // ErrClosed is returned when the WAL is closed. + // ErrClosed is returned when the WAL is closed. It is a permanent error + // as once closed, a WAL cannot be re-opened. ErrClosed = errors.New("WAL is closed") - // ErrFull is returned when an append fails because the WAL is full. This + // ErrFull is returned when the WAL is full. It is a transient error that // happens when all segments are either in the pending list waiting to be - // flushed, or in the process of being flushed. + // flushed or in the process of being flushed. ErrFull = errors.New("WAL is full") ) @@ -42,19 +36,20 @@ type AppendRequest struct { Entries []*logproto.Entry } +// AppendResult contains the result of an AppendRequest. type AppendResult struct { done chan struct{} err error } -// Done returns a channel that is closed when the result of an append is -// available. Err() should be called to check if the operation was successful. +// Done returns a channel that is closed when the result for the AppendRequest +// is available. Use Err() to check if the request succeeded or failed. func (p *AppendResult) Done() <-chan struct{} { return p.done } -// Err returns a non-nil error if the operation failed, and nil if it was -// successful. It should not be called until Done() is closed to avoid data +// Err returns a non-nil error if the append request failed, and nil if it +// succeeded. It should not be called until Done() is closed to avoid data // races. func (p *AppendResult) Err() error { return p.err @@ -69,55 +64,58 @@ func (p *AppendResult) SetDone(err error) { type Config struct { // MaxAge is the maximum amount of time a segment can be buffered in memory // before it is moved to the pending list to be flushed. Increasing MaxAge - // allows more time for a segment to grow to MaxSegmentSize, but may increase - // latency if the write volume is too small. + // allows more time for a segment to grow to MaxSegmentSize, but may + // increase latency if appends cannot fill segments quickly enough. MaxAge time.Duration // MaxSegments is the maximum number of segments that can be buffered in - // memory. Increasing MaxSegments allows for large bursts of writes to be - // buffered in memory, but may increase latency if the write volume exceeds - // the rate at which segments can be flushed. + // memory. Increasing MaxSegments allows more data to be buffered, but may + // increase latency if the incoming volume of data exceeds the rate at + // which segments can be flushed. MaxSegments int64 - // MaxSegmentSize is the maximum size (uncompressed) of a segment. It is - // not a strict limit, and segments can exceed the maximum size when + // MaxSegmentSize is the maximum size of an uncompressed segment in bytes. + // It is not a strict limit, and segments can exceed the maximum size when // individual appends are larger than the remaining capacity. MaxSegmentSize int64 } -// Manager buffers segments in memory, and keeps track of which segments are -// available and which are waiting to be flushed. The maximum number of -// segments that can be buffered in memory, and their maximum age and maximum -// size before being flushed are configured when creating the Manager. +// Manager is a pool of in-memory segments. It keeps track of which segments +// are accepting writes and which are waiting to be flushed using two doubly +// linked lists called the available and pending lists. // -// By buffering segments in memory, the WAL can tolerate bursts of append -// requests that arrive faster than can be flushed. The amount of data that can -// be buffered is configured using MaxSegments and MaxSegmentSize. You must use -// caution when configuring these to avoid excessive latency. +// By buffering segments in memory, the WAL can tolerate bursts of writes that +// arrive faster than can be flushed. The amount of data that can be buffered +// is configured using MaxSegments and MaxSegmentSize. You must use caution +// when configuring these to avoid excessive latency. // -// The WAL is full when all segments are waiting to be flushed or in the process -// of being flushed. When the WAL is full, subsequent appends fail with ErrFull. -// It is not permitted to append more data until another segment has been flushed -// and returned to the available list. This allows the manager to apply back-pressure -// and avoid congestion collapse due to excessive timeouts and retries. +// The WAL is full when all segments are waiting to be flushed or in the +// process of being flushed. When the WAL is full, subsequent appends fail with +// the transient error ErrFull, and will not succeed until one or more other +// segments have been flushed and returned to the available list. Callers +// should back off and retry at a later time. +// +// On shutdown, the WAL must be closed to avoid losing data. This prevents +// additional appends to the WAL and allows all remaining segments to be +// flushed. type Manager struct { - cfg Config - metrics *ManagerMetrics - - // available is a list of segments that are available and accepting data. - // All segments other than the segment at the front of the list are empty, - // and only the segment at the front of the list is written to. When this - // segment has exceeded its maximum age or maximum size it is moved to the - // pending list to be flushed, and the next segment in the available list - // takes its place. + cfg Config + metrics *ManagerMetrics available *list.List // pending is a list of segments that are waiting to be flushed. Once // flushed, the segment is reset and moved to the back of the available // list to accept writes again. pending *list.List - closed bool - mu sync.Mutex + + // firstAppend is the time of the first append to the segment at the + // front of the available list. It is used to know when the segment has + // exceeded the maximum age and should be moved to the pending list. + // It is reset each time this happens. + firstAppend time.Time + + closed bool + mu sync.Mutex // Used in tests. clock quartz.Clock @@ -125,15 +123,11 @@ type Manager struct { // item is similar to PendingItem, but it is an internal struct used in the // available and pending lists. It contains a single-use result that is returned -// to callers of Manager.Append() and a re-usable segment that is reset after +// to callers appending to the WAL and a re-usable segment that is reset after // each flush. type item struct { r *AppendResult w *SegmentWriter - // firstAppendedAt is the time of the first append to the segment, and is - // used to know when the segment has exceeded the maximum age and should - // be moved to the pending list. It is reset after each flush. - firstAppendedAt time.Time } // PendingItem contains a result and the segment to be flushed. @@ -150,6 +144,7 @@ func NewManager(cfg Config, metrics *Metrics) (*Manager, error) { pending: list.New(), clock: quartz.NewReal(), } + m.metrics.NumAvailable.Set(0) m.metrics.NumPending.Set(0) m.metrics.NumFlushing.Set(0) for i := int64(0); i < cfg.MaxSegments; i++ { @@ -172,25 +167,22 @@ func (m *Manager) Append(r AppendRequest) (*AppendResult, error) { if m.closed { return nil, ErrClosed } - if m.available.Len() == 0 { + el := m.available.Front() + if el == nil { return nil, ErrFull } - el := m.available.Front() it := el.Value.(*item) - if it.firstAppendedAt.IsZero() { + if m.firstAppend.IsZero() { // This is the first append to the segment. This time will be used in // know when the segment has exceeded its maximum age and should be // moved to the pending list. - it.firstAppendedAt = m.clock.Now() + m.firstAppend = m.clock.Now() } it.w.Append(r.TenantID, r.LabelsStr, r.Labels, r.Entries) // If the segment exceeded the maximum age or the maximum size, move it to // the closed list to be flushed. - if m.clock.Since(it.firstAppendedAt) >= m.cfg.MaxAge || it.w.InputSize() >= m.cfg.MaxSegmentSize { - m.pending.PushBack(it) - m.metrics.NumPending.Inc() - m.available.Remove(el) - m.metrics.NumAvailable.Dec() + if m.clock.Since(m.firstAppend) >= m.cfg.MaxAge || it.w.InputSize() >= m.cfg.MaxSegmentSize { + m.move(el, it) } return it.r, nil } @@ -198,44 +190,27 @@ func (m *Manager) Append(r AppendRequest) (*AppendResult, error) { func (m *Manager) Close() { m.mu.Lock() defer m.mu.Unlock() - m.closed = true - if m.available.Len() > 0 { - el := m.available.Front() + if el := m.available.Front(); el != nil { it := el.Value.(*item) if it.w.InputSize() > 0 { - m.pending.PushBack(it) - m.metrics.NumPending.Inc() - m.available.Remove(el) - m.metrics.NumAvailable.Dec() + m.move(el, it) } } + m.closed = true } -// NextPending returns the next segment to be flushed. It returns nil if the -// pending list is empty, and ErrClosed if the WAL is closed. +// NextPending returns the next segment to be flushed from the pending list. +// It returns nil if there are no segments waiting to be flushed. If the WAL +// is closed it returns all remaining segments from the pending list and then +// ErrClosed. func (m *Manager) NextPending() (*PendingItem, error) { m.mu.Lock() defer m.mu.Unlock() - if m.pending.Len() == 0 { - if m.available.Len() > 0 { - // Check if the current segment has exceeded its maximum age and - // should be moved to the pending list. - el := m.available.Front() - it := el.Value.(*item) - if !it.firstAppendedAt.IsZero() && m.clock.Since(it.firstAppendedAt) >= m.cfg.MaxAge { - m.pending.PushBack(it) - m.metrics.NumPending.Inc() - m.available.Remove(el) - m.metrics.NumAvailable.Dec() - } - } - // If the pending list is still empty return nil. - if m.pending.Len() == 0 { - if m.closed { - return nil, ErrClosed - } - return nil, nil + if m.pending.Len() == 0 && !m.moveFrontIfExpired() { + if m.closed { + return nil, ErrClosed } + return nil, nil } el := m.pending.Front() it := el.Value.(*item) @@ -248,9 +223,9 @@ func (m *Manager) NextPending() (*PendingItem, error) { // Put resets the segment and puts it back in the available list to accept // writes. A PendingItem should not be put back until it has been flushed. func (m *Manager) Put(it *PendingItem) { + it.Writer.Reset() m.mu.Lock() defer m.mu.Unlock() - it.Writer.Reset() m.metrics.NumFlushing.Dec() m.metrics.NumAvailable.Inc() m.available.PushBack(&item{ @@ -258,3 +233,27 @@ func (m *Manager) Put(it *PendingItem) { w: it.Writer, }) } + +// move the element from the available list to the pending list and sets the +// relevant metrics. +func (m *Manager) move(el *list.Element, it *item) { + m.pending.PushBack(it) + m.metrics.NumPending.Inc() + m.available.Remove(el) + m.metrics.NumAvailable.Dec() + m.firstAppend = time.Time{} +} + +// moveFrontIfExpired moves the element from the front of the available list to +// the pending list if the segment has exceeded its maximum age and sets the +// relevant metrics. +func (m *Manager) moveFrontIfExpired() bool { + if el := m.available.Front(); el != nil { + it := el.Value.(*item) + if !m.firstAppend.IsZero() && m.clock.Since(m.firstAppend) >= m.cfg.MaxAge { + m.move(el, it) + return true + } + } + return false +} From c0f5fb6e4d7b83339601e8a4a82a6582e0185ec7 Mon Sep 17 00:00:00 2001 From: benclive Date: Wed, 17 Jul 2024 10:04:28 +0100 Subject: [PATCH 11/23] chore: Clean up old streams periodically in RF-1 ingester (#13511) --- docs/sources/shared/configuration.md | 73 ++++++++----------------- pkg/ingester-rf1/ingester.go | 81 ++++++++++++++-------------- pkg/ingester-rf1/ingester_test.go | 43 +++++++++++++++ pkg/ingester-rf1/stream.go | 12 +---- 4 files changed, 106 insertions(+), 103 deletions(-) create mode 100644 pkg/ingester-rf1/ingester_test.go diff --git a/docs/sources/shared/configuration.md b/docs/sources/shared/configuration.md index 38bc61ce6fab..0a35ce73518f 100644 --- a/docs/sources/shared/configuration.md +++ b/docs/sources/shared/configuration.md @@ -286,6 +286,25 @@ ingester_rf1: # CLI flag: -ingester-rf1.lifecycler.ID [id: | default = ""] + # The maximum age of a segment before it should be flushed. Increasing this + # value allows more time for a segment to grow to max-segment-size, but may + # increase latency if the write volume is too small. + # CLI flag: -ingester-rf1.max-segment-age + [max_segment_age: | default = 500ms] + + # The maximum size of a segment before it should be flushed. It is not a + # strict limit, and segments can exceed the maximum size when individual + # appends are larger than the remaining capacity. + # CLI flag: -ingester-rf1.max-segment-size + [max_segment_size: | default = 8388608] + + # The maximum number of segments to buffer in-memory. Increasing this value + # allows for large bursts of writes to be buffered in memory, but may increase + # latency if the write volume exceeds the rate at which segments can be + # flushed. + # CLI flag: -ingester-rf1.max-segments + [max_segments: | default = 10] + # How many flushes can happen concurrently from each stream. # CLI flag: -ingester-rf1.concurrent-flushes [concurrent_flushes: | default = 32] @@ -316,56 +335,6 @@ ingester_rf1: # CLI flag: -ingester-rf1.flush-op-timeout [flush_op_timeout: | default = 10m] - # The maximum age of a segment before it should be flushed. Increasing this - # value allows more time for a segment to grow to max-segment-size, but may - # increase latency if the write volume is too small. - # CLI flag: -ingester-rf1.max-segment-age - [max_segment_age: | default = 500ms] - - # The maximum size of a segment before it should be flushed. It is not a - # strict limit, and segments can exceed the maximum size when individual - # appends are larger than the remaining capacity. - # CLI flag: -ingester-rf1.max-segment-size - [max_segment_size: | default = 8388608] - - # The maximum number of segments to buffer in-memory. Increasing this value - # allows for large bursts of writes to be buffered in memory, but may increase - # latency if the write volume exceeds the rate at which segments can be - # flushed. - # CLI flag: -ingester-rf1.max-segments - [max_segments: | default = 10] - - # How long chunks should be retained in-memory after they've been flushed. - # CLI flag: -ingester-rf1.chunks-retain-period - [chunk_retain_period: | default = 0s] - - [chunk_idle_period: ] - - # The targeted _uncompressed_ size in bytes of a chunk block When this - # threshold is exceeded the head block will be cut and compressed inside the - # chunk. - # CLI flag: -ingester-rf1.chunks-block-size - [chunk_block_size: | default = 262144] - - # A target _compressed_ size in bytes for chunks. This is a desired size not - # an exact size, chunks may be slightly bigger or significantly smaller if - # they get flushed for other reasons (e.g. chunk_idle_period). A value of 0 - # creates chunks with a fixed 10 blocks, a non zero value will create chunks - # with a variable number of blocks to meet the target size. - # CLI flag: -ingester-rf1.chunk-target-size - [chunk_target_size: | default = 1572864] - - # The algorithm to use for compressing chunk. (none, gzip, lz4-64k, snappy, - # lz4-256k, lz4-1M, lz4, flate, zstd) - # CLI flag: -ingester-rf1.chunk-encoding - [chunk_encoding: | default = "gzip"] - - # The maximum duration of a timeseries chunk in memory. If a timeseries runs - # for longer than this, the current chunk will be flushed to the store and a - # new chunk created. - # CLI flag: -ingester-rf1.max-chunk-age - [max_chunk_age: | default = 2h] - # Forget about ingesters having heartbeat timestamps older than # `ring.kvstore.heartbeat_timeout`. This is equivalent to clicking on the # `/ring` `forget` button in the UI: the ingester is removed from the ring. @@ -400,6 +369,10 @@ ingester_rf1: # CLI flag: -ingester-rf1.owned-streams-check-interval [owned_streams_check_interval: | default = 30s] + # How long stream metadata is retained in memory after it was last seen. + # CLI flag: -ingester-rf1.stream-retain-period + [stream_retain_period: | default = 5m] + # Configures how the pattern ingester will connect to the ingesters. client_config: # Configures how connections are pooled. diff --git a/pkg/ingester-rf1/ingester.go b/pkg/ingester-rf1/ingester.go index 4abe260dbcc4..802934665af3 100644 --- a/pkg/ingester-rf1/ingester.go +++ b/pkg/ingester-rf1/ingester.go @@ -38,7 +38,6 @@ import ( "google.golang.org/grpc/health/grpc_health_v1" "github.com/grafana/loki/v3/pkg/analytics" - "github.com/grafana/loki/v3/pkg/chunkenc" "github.com/grafana/loki/v3/pkg/distributor/writefailures" "github.com/grafana/loki/v3/pkg/ingester/client" "github.com/grafana/loki/v3/pkg/logproto" @@ -68,23 +67,15 @@ var ( type Config struct { Enabled bool `yaml:"enabled" doc:"description=Whether the ingester is enabled."` - LifecyclerConfig ring.LifecyclerConfig `yaml:"lifecycler,omitempty" doc:"description=Configures how the lifecycle of the ingester will operate and where it will register for discovery."` - - ConcurrentFlushes int `yaml:"concurrent_flushes"` - FlushCheckPeriod time.Duration `yaml:"flush_check_period"` - FlushOpBackoff backoff.Config `yaml:"flush_op_backoff"` - FlushOpTimeout time.Duration `yaml:"flush_op_timeout"` - MaxSegmentAge time.Duration `yaml:"max_segment_age"` - MaxSegmentSize int `yaml:"max_segment_size"` - MaxSegments int `yaml:"max_segments"` - RetainPeriod time.Duration `yaml:"chunk_retain_period"` - MaxChunkIdle time.Duration `yaml:"chunk_idle_period"` - BlockSize int `yaml:"chunk_block_size"` - TargetChunkSize int `yaml:"chunk_target_size"` - ChunkEncoding string `yaml:"chunk_encoding"` - parsedEncoding chunkenc.Encoding `yaml:"-"` // placeholder for validated encoding - MaxChunkAge time.Duration `yaml:"max_chunk_age"` - AutoForgetUnhealthy bool `yaml:"autoforget_unhealthy"` + LifecyclerConfig ring.LifecyclerConfig `yaml:"lifecycler,omitempty" doc:"description=Configures how the lifecycle of the ingester will operate and where it will register for discovery."` + MaxSegmentAge time.Duration `yaml:"max_segment_age"` + MaxSegmentSize int `yaml:"max_segment_size"` + MaxSegments int `yaml:"max_segments"` + ConcurrentFlushes int `yaml:"concurrent_flushes"` + FlushCheckPeriod time.Duration `yaml:"flush_check_period"` + FlushOpBackoff backoff.Config `yaml:"flush_op_backoff"` + FlushOpTimeout time.Duration `yaml:"flush_op_timeout"` + AutoForgetUnhealthy bool `yaml:"autoforget_unhealthy"` MaxReturnedErrors int `yaml:"max_returned_stream_errors"` @@ -101,6 +92,7 @@ type Config struct { ShutdownMarkerPath string `yaml:"shutdown_marker_path"` OwnedStreamsCheckInterval time.Duration `yaml:"owned_streams_check_interval" doc:"description=Interval at which the ingester ownedStreamService checks for changes in the ring to recalculate owned streams."` + StreamRetainPeriod time.Duration `yaml:"stream_retain_period" doc:"description=How long stream metadata is retained in memory after it was last seen."` // Tee configs ClientConfig clientpool.Config `yaml:"client_config,omitempty" doc:"description=Configures how the pattern ingester will connect to the ingesters."` @@ -122,28 +114,17 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet) { f.IntVar(&cfg.MaxSegmentSize, "ingester-rf1.max-segment-size", 8*1024*1024, "The maximum size of a segment before it should be flushed. It is not a strict limit, and segments can exceed the maximum size when individual appends are larger than the remaining capacity.") f.IntVar(&cfg.MaxSegments, "ingester-rf1.max-segments", 10, "The maximum number of segments to buffer in-memory. Increasing this value allows for large bursts of writes to be buffered in memory, but may increase latency if the write volume exceeds the rate at which segments can be flushed.") - f.DurationVar(&cfg.RetainPeriod, "ingester-rf1.chunks-retain-period", 0, "How long chunks should be retained in-memory after they've been flushed.") - // f.DurationVar(&cfg.MaxChunkIdle, "ingester-rf1.chunks-idle-period", 30*time.Minute, "How long chunks should sit in-memory with no updates before being flushed if they don't hit the max block size. This means that half-empty chunks will still be flushed after a certain period as long as they receive no further activity.") - f.IntVar(&cfg.BlockSize, "ingester-rf1.chunks-block-size", 256*1024, "The targeted _uncompressed_ size in bytes of a chunk block When this threshold is exceeded the head block will be cut and compressed inside the chunk.") - f.IntVar(&cfg.TargetChunkSize, "ingester-rf1.chunk-target-size", 1572864, "A target _compressed_ size in bytes for chunks. This is a desired size not an exact size, chunks may be slightly bigger or significantly smaller if they get flushed for other reasons (e.g. chunk_idle_period). A value of 0 creates chunks with a fixed 10 blocks, a non zero value will create chunks with a variable number of blocks to meet the target size.") // 1.5 MB - f.StringVar(&cfg.ChunkEncoding, "ingester-rf1.chunk-encoding", chunkenc.EncGZIP.String(), fmt.Sprintf("The algorithm to use for compressing chunk. (%s)", chunkenc.SupportedEncoding())) f.IntVar(&cfg.MaxReturnedErrors, "ingester-rf1.max-ignored-stream-errors", 10, "The maximum number of errors a stream will report to the user when a push fails. 0 to make unlimited.") - f.DurationVar(&cfg.MaxChunkAge, "ingester-rf1.max-chunk-age", 2*time.Hour, "The maximum duration of a timeseries chunk in memory. If a timeseries runs for longer than this, the current chunk will be flushed to the store and a new chunk created.") f.BoolVar(&cfg.AutoForgetUnhealthy, "ingester-rf1.autoforget-unhealthy", false, "Forget about ingesters having heartbeat timestamps older than `ring.kvstore.heartbeat_timeout`. This is equivalent to clicking on the `/ring` `forget` button in the UI: the ingester is removed from the ring. This is a useful setting when you are sure that an unhealthy node won't return. An example is when not using stateful sets or the equivalent. Use `memberlist.rejoin_interval` > 0 to handle network partition cases when using a memberlist.") f.IntVar(&cfg.IndexShards, "ingester-rf1.index-shards", index.DefaultIndexShards, "Shard factor used in the ingesters for the in process reverse index. This MUST be evenly divisible by ALL schema shard factors or Loki will not start.") f.IntVar(&cfg.MaxDroppedStreams, "ingester-rf1.tailer.max-dropped-streams", 10, "Maximum number of dropped streams to keep in memory during tailing.") f.StringVar(&cfg.ShutdownMarkerPath, "ingester-rf1.shutdown-marker-path", "", "Path where the shutdown marker file is stored. If not set and common.path_prefix is set then common.path_prefix will be used.") f.DurationVar(&cfg.OwnedStreamsCheckInterval, "ingester-rf1.owned-streams-check-interval", 30*time.Second, "Interval at which the ingester ownedStreamService checks for changes in the ring to recalculate owned streams.") + f.DurationVar(&cfg.StreamRetainPeriod, "ingester-rf1.stream-retain-period", 5*time.Minute, "How long stream metadata should be retained in-memory after the last log was seen.") f.BoolVar(&cfg.Enabled, "ingester-rf1.enabled", false, "Flag to enable or disable the usage of the ingester-rf1 component.") } func (cfg *Config) Validate() error { - enc, err := chunkenc.ParseEncoding(cfg.ChunkEncoding) - if err != nil { - return err - } - cfg.parsedEncoding = enc - if cfg.FlushOpBackoff.MinBackoff > cfg.FlushOpBackoff.MaxBackoff { return errors.New("invalid flush op min backoff: cannot be larger than max backoff") } @@ -202,6 +183,7 @@ type Ingester struct { store Storage periodicConfigs []config.PeriodConfig + loopQuit chan struct{} tailersQuit chan struct{} // One queue per flush thread. Fingerprint is used to @@ -253,8 +235,6 @@ func New(cfg Config, clientConfig client.Config, if cfg.ingesterClientFactory == nil { cfg.ingesterClientFactory = client.New } - compressionStats.Set(cfg.ChunkEncoding) - targetSizeStats.Set(int64(cfg.TargetChunkSize)) metrics := newIngesterMetrics(registerer) walManager, err := wal.NewManager(wal.Config{ @@ -275,6 +255,7 @@ func New(cfg Config, clientConfig client.Config, store: storage, periodicConfigs: periodConfigs, flushWorkersDone: sync.WaitGroup{}, + loopQuit: make(chan struct{}), tailersQuit: make(chan struct{}), metrics: metrics, // flushOnShutdownSwitch: &OnceSwitch{}, @@ -431,6 +412,7 @@ func (i *Ingester) starting(ctx context.Context) error { return fmt.Errorf("can not ensure recalculate owned streams service is running: %w", err) } + go i.periodicStreamMaintenance() return nil } @@ -506,6 +488,31 @@ func (i *Ingester) removeShutdownMarkerFile() { } } +func (i *Ingester) periodicStreamMaintenance() { + streamRetentionTicker := time.NewTicker(i.cfg.StreamRetainPeriod) + defer streamRetentionTicker.Stop() + + for { + select { + case <-streamRetentionTicker.C: + i.cleanIdleStreams() + case <-i.loopQuit: + return + } + } +} + +func (i *Ingester) cleanIdleStreams() { + for _, instance := range i.getInstances() { + _ = instance.streams.ForEach(func(s *stream) (bool, error) { + if time.Since(s.highestTs) > i.cfg.StreamRetainPeriod { + instance.streams.Delete(s) + } + return true, nil + }) + } +} + // PrepareShutdown will handle the /ingester/prepare_shutdown endpoint. // // Internally, when triggered, this handler will configure the ingester service to release their resources whenever a SIGTERM is received. @@ -885,16 +892,6 @@ func (i *Ingester) getInstances() []*instance { // return &resp, nil //} -func adjustQueryStartTime(maxLookBackPeriod time.Duration, start, now time.Time) time.Time { - if maxLookBackPeriod > 0 { - oldestStartTime := now.Add(-maxLookBackPeriod) - if oldestStartTime.After(start) { - return oldestStartTime - } - } - return start -} - func (i *Ingester) GetDetectedFields(_ context.Context, r *logproto.DetectedFieldsRequest) (*logproto.DetectedFieldsResponse, error) { return &logproto.DetectedFieldsResponse{ Fields: []*logproto.DetectedField{ diff --git a/pkg/ingester-rf1/ingester_test.go b/pkg/ingester-rf1/ingester_test.go new file mode 100644 index 000000000000..182d95afef6d --- /dev/null +++ b/pkg/ingester-rf1/ingester_test.go @@ -0,0 +1,43 @@ +package ingesterrf1 + +import ( + "sync" + "testing" + "time" + + "github.com/stretchr/testify/require" +) + +func TestIngester_cleanIdleStreams(t *testing.T) { + i := &Ingester{ + instancesMtx: sync.RWMutex{}, + instances: make(map[string]*instance), + cfg: Config{StreamRetainPeriod: time.Minute}, + } + instance := &instance{ + instanceID: "test", + streams: newStreamsMap(), + } + stream := &stream{ + labelsString: "test,label", + highestTs: time.Now(), + } + instance.streams.Store(stream.labelsString, stream) + i.instances[instance.instanceID] = instance + + require.Len(t, i.instances, 1) + require.Equal(t, 1, instance.streams.Len()) + + // No-op + i.cleanIdleStreams() + + require.Len(t, i.instances, 1) + require.Equal(t, 1, instance.streams.Len()) + + // Pretend stream is old and retry + stream.highestTs = time.Now().Add(-time.Minute * 2) + i.cleanIdleStreams() + + require.Len(t, i.instances, 1) + require.Equal(t, 0, instance.streams.Len()) +} diff --git a/pkg/ingester-rf1/stream.go b/pkg/ingester-rf1/stream.go index 44057694ccf3..f0c556e34108 100644 --- a/pkg/ingester-rf1/stream.go +++ b/pkg/ingester-rf1/stream.go @@ -72,16 +72,6 @@ type stream struct { chunkHeadBlockFormat chunkenc.HeadBlockFmt } -type chunkDesc struct { - chunk *chunkenc.MemChunk - closed bool - synced bool - flushed time.Time - reason string - - lastUpdated time.Time -} - type entryWithError struct { entry *logproto.Entry e error @@ -269,7 +259,7 @@ func (s *stream) validateEntries(ctx context.Context, entries []logproto.Entry, } // The validity window for unordered writes is the highest timestamp present minus 1/2 * max-chunk-age. - cutoff := highestTs.Add(-s.cfg.MaxChunkAge / 2) + cutoff := highestTs.Add(-time.Hour) if s.unorderedWrites && !highestTs.IsZero() && cutoff.After(entries[i].Timestamp) { failedEntriesWithError = append(failedEntriesWithError, entryWithError{&entries[i], chunkenc.ErrTooFarBehind(entries[i].Timestamp, cutoff)}) s.writeFailures.Log(s.tenant, fmt.Errorf("%w for stream %s", failedEntriesWithError[len(failedEntriesWithError)-1].e, s.labels)) From 150e6539d175c232063aaa37c687c60d4809a08a Mon Sep 17 00:00:00 2001 From: benclive Date: Wed, 17 Jul 2024 11:15:46 +0100 Subject: [PATCH 12/23] feat: Add CLI to inspect RF=1 WAL segments (#13552) --- cmd/segment-inspect/.gitignore | 1 + cmd/segment-inspect/README.md | 34 ++ cmd/segment-inspect/go.mod | 148 +++++ cmd/segment-inspect/go.sum | 1024 ++++++++++++++++++++++++++++++++ cmd/segment-inspect/main.go | 108 ++++ 5 files changed, 1315 insertions(+) create mode 100644 cmd/segment-inspect/.gitignore create mode 100644 cmd/segment-inspect/README.md create mode 100644 cmd/segment-inspect/go.mod create mode 100644 cmd/segment-inspect/go.sum create mode 100644 cmd/segment-inspect/main.go diff --git a/cmd/segment-inspect/.gitignore b/cmd/segment-inspect/.gitignore new file mode 100644 index 000000000000..67208cd8d72c --- /dev/null +++ b/cmd/segment-inspect/.gitignore @@ -0,0 +1 @@ +segment-inspect diff --git a/cmd/segment-inspect/README.md b/cmd/segment-inspect/README.md new file mode 100644 index 000000000000..34e80360bd73 --- /dev/null +++ b/cmd/segment-inspect/README.md @@ -0,0 +1,34 @@ +# Tools for inspecting Loki chunks + +This tool can parse Loki WAL segments and print details from them. Useful for Loki developers. + +To build the tool, simply run `go build` in this directory. Running resulting program with chunks file name gives you some basic chunks information: + +```shell +$ ./segment-inspect 01J2F010G82AWFGAEC3WGKC84P + +Segment file: 01J2F010G82AWFGAEC3WGKC84P +Compressed Filesize: 2.0 MB +Series Size: 1.8 MB +Index Size: 255 kB +Stream count: 865 +Tenant count: 7 +From: 2024-07-10 18:55:47.006697 UTC +To: 2024-07-10 18:55:51.454554 UTC +Duration: 4.447857915s +``` + +To print the individual streams contained in this segment, use `-s` parameter: + +```shell script +$ ./segment-inspect -s 01J2F010G82AWFGAEC3WGKC84P + +... segment file info, see above ... + +{__loki_tenant__="29", cluster="dev-us-central-0", container="loki-canary", controller_revision_hash="9cbf8c4bb", job="promtail-ops/loki-canary", name="loki-canary", namespace="promtail-ops", pod="loki-canary-z6jrt", pod_template_generation="2008", service_name="loki/loki-canary", stream="stdout"} +{__loki_tenant__="29", cluster="dev-us-central-0", container="loki-canary", controller_revision_hash="9cbf8c4bb", job="promtail-ops/loki-canary", name="loki-canary", namespace="promtail-ops", pod="loki-canary-zpx5v", pod_template_generation="2008", service_name="loki/loki-canary", stream="stdout"} +{__loki_tenant__="29", cluster="dev-us-central-0", container="metadata-forwarder", job="metadata-forwarder/metadata-forwarder", name="metadata-forwarder", namespace="metadata-forwarder", pod="metadata-forwarder-676f57978c-dpd2s", pod_template_hash="676f57978c", service_name="metadata-forwarder", stream="stderr"} +{__loki_tenant__="29", cluster="dev-us-central-0", container="metadata-forwarder", job="metadata-forwarder/metadata-forwarder", name="metadata-forwarder", namespace="metadata-forwarder", pod="metadata-forwarder-676f57978c-sbnc9", pod_template_hash="676f57978c", service_name="metadata-forwarder", stream="stderr"} +{__loki_tenant__="29", cluster="dev-us-central-0", container="mimir-read", gossip_ring_member="true", job="mimir-dev-09/mimir-read", name="mimir-read", namespace="mimir-dev-09", pod="mimir-read-74d58cd64c-4kg55", pod_template_hash="74d58cd64c", service_name="mimir/mimir-read", stream="stderr"} +... and so on ... +``` diff --git a/cmd/segment-inspect/go.mod b/cmd/segment-inspect/go.mod new file mode 100644 index 000000000000..d441f6f046c2 --- /dev/null +++ b/cmd/segment-inspect/go.mod @@ -0,0 +1,148 @@ +module github.com/grafana/loki/cmd/segment-inspect + +go 1.22 + +toolchain go1.22.2 + +require ( + github.com/dustin/go-humanize v1.0.1 + github.com/grafana/loki/v3 v3.0.0-20240716103001-3ac130b8a152 +) + +require ( + github.com/Azure/azure-sdk-for-go/sdk/azcore v1.11.1 // indirect + github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.6.0 // indirect + github.com/Azure/azure-sdk-for-go/sdk/internal v1.8.0 // indirect + github.com/AzureAD/microsoft-authentication-library-for-go v1.2.2 // indirect + github.com/HdrHistogram/hdrhistogram-go v1.1.2 // indirect + github.com/Masterminds/goutils v1.1.1 // indirect + github.com/Masterminds/semver/v3 v3.2.0 // indirect + github.com/Masterminds/sprig/v3 v3.2.3 // indirect + github.com/Workiva/go-datastructures v1.1.0 // indirect + github.com/alecthomas/units v0.0.0-20231202071711-9a357b53e9c9 // indirect + github.com/armon/go-metrics v0.4.1 // indirect + github.com/aws/aws-sdk-go v1.50.32 // indirect + github.com/bboreham/go-loser v0.0.0-20230920113527-fcc2c21820a3 // indirect + github.com/beorn7/perks v1.0.1 // indirect + github.com/buger/jsonparser v1.1.1 // indirect + github.com/c2h5oh/datasize v0.0.0-20220606134207-859f65c6625b // indirect + github.com/cespare/xxhash v1.1.0 // indirect + github.com/cespare/xxhash/v2 v2.2.0 // indirect + github.com/coreos/go-semver v0.3.0 // indirect + github.com/coreos/go-systemd/v22 v22.5.0 // indirect + github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect + github.com/dennwc/varint v1.0.0 // indirect + github.com/dgryski/go-rendezvous v0.0.0-20200823014737-9f7001d12a5f // indirect + github.com/edsrzf/mmap-go v1.1.0 // indirect + github.com/facette/natsort v0.0.0-20181210072756-2cd4dd1e2dcb // indirect + github.com/fatih/color v1.16.0 // indirect + github.com/felixge/httpsnoop v1.0.4 // indirect + github.com/fsnotify/fsnotify v1.7.0 // indirect + github.com/go-kit/log v0.2.1 // indirect + github.com/go-logfmt/logfmt v0.6.0 // indirect + github.com/go-logr/logr v1.4.1 // indirect + github.com/go-logr/stdr v1.2.2 // indirect + github.com/go-redis/redis/v8 v8.11.5 // indirect + github.com/gogo/googleapis v1.4.0 // indirect + github.com/gogo/protobuf v1.3.2 // indirect + github.com/gogo/status v1.1.1 // indirect + github.com/golang-jwt/jwt/v5 v5.2.1 // indirect + github.com/golang/protobuf v1.5.3 // indirect + github.com/golang/snappy v0.0.4 // indirect + github.com/google/btree v1.1.2 // indirect + github.com/google/go-cmp v0.6.0 // indirect + github.com/google/uuid v1.6.0 // indirect + github.com/gorilla/mux v1.8.0 // indirect + github.com/grafana/dskit v0.0.0-20240626184720-35810fdf1c6d // indirect + github.com/grafana/gomemcache v0.0.0-20240229205252-cd6a66d6fb56 // indirect + github.com/grafana/jsonparser v0.0.0-20240209175146-098958973a2d // indirect + github.com/grafana/loki/pkg/push v0.0.0-20231124142027-e52380921608 // indirect + github.com/grafana/pyroscope-go/godeltaprof v0.1.6 // indirect + github.com/grafana/regexp v0.0.0-20221122212121-6b5c0a4cb7fd // indirect + github.com/hashicorp/consul/api v1.28.2 // indirect + github.com/hashicorp/errwrap v1.1.0 // indirect + github.com/hashicorp/go-cleanhttp v0.5.2 // indirect + github.com/hashicorp/go-hclog v1.6.3 // indirect + github.com/hashicorp/go-immutable-radix v1.3.1 // indirect + github.com/hashicorp/go-msgpack v0.5.5 // indirect + github.com/hashicorp/go-multierror v1.1.1 // indirect + github.com/hashicorp/go-rootcerts v1.0.2 // indirect + github.com/hashicorp/go-sockaddr v1.0.6 // indirect + github.com/hashicorp/golang-lru v0.6.0 // indirect + github.com/hashicorp/memberlist v0.5.0 // indirect + github.com/hashicorp/serf v0.10.1 // indirect + github.com/huandu/xstrings v1.3.3 // indirect + github.com/imdario/mergo v0.3.16 // indirect + github.com/jmespath/go-jmespath v0.4.0 // indirect + github.com/jpillora/backoff v1.0.0 // indirect + github.com/json-iterator/go v1.1.12 // indirect + github.com/klauspost/compress v1.17.7 // indirect + github.com/kylelemons/godebug v1.1.0 // indirect + github.com/mattn/go-colorable v0.1.13 // indirect + github.com/mattn/go-isatty v0.0.20 // indirect + github.com/miekg/dns v1.1.58 // indirect + github.com/mitchellh/copystructure v1.0.0 // indirect + github.com/mitchellh/go-homedir v1.1.0 // indirect + github.com/mitchellh/mapstructure v1.5.0 // indirect + github.com/mitchellh/reflectwalk v1.0.1 // indirect + github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect + github.com/modern-go/reflect2 v1.0.2 // indirect + github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f // indirect + github.com/oklog/ulid v1.3.1 // indirect + github.com/opentracing-contrib/go-grpc v0.0.0-20210225150812-73cb765af46e // indirect + github.com/opentracing-contrib/go-stdlib v1.0.0 // indirect + github.com/opentracing/opentracing-go v1.2.0 // indirect + github.com/pierrec/lz4/v4 v4.1.18 // indirect + github.com/pires/go-proxyproto v0.7.0 // indirect + github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c // indirect + github.com/pkg/errors v0.9.1 // indirect + github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect + github.com/prometheus/client_golang v1.19.0 // indirect + github.com/prometheus/client_model v0.6.0 // indirect + github.com/prometheus/common v0.49.1-0.20240306132007-4199f18c3e92 // indirect + github.com/prometheus/common/sigv4 v0.1.0 // indirect + github.com/prometheus/exporter-toolkit v0.11.0 // indirect + github.com/prometheus/procfs v0.12.0 // indirect + github.com/prometheus/prometheus v0.51.0 // indirect + github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529 // indirect + github.com/sercand/kuberesolver/v5 v5.1.1 // indirect + github.com/shopspring/decimal v1.2.0 // indirect + github.com/sony/gobreaker v0.5.0 // indirect + github.com/spf13/cast v1.3.1 // indirect + github.com/stretchr/objx v0.5.2 // indirect + github.com/stretchr/testify v1.9.0 // indirect + github.com/uber/jaeger-client-go v2.30.0+incompatible // indirect + github.com/uber/jaeger-lib v2.4.1+incompatible // indirect + go.etcd.io/etcd/api/v3 v3.5.4 // indirect + go.etcd.io/etcd/client/pkg/v3 v3.5.4 // indirect + go.etcd.io/etcd/client/v3 v3.5.4 // indirect + go.opentelemetry.io/otel v1.24.0 // indirect + go.opentelemetry.io/otel/metric v1.24.0 // indirect + go.opentelemetry.io/otel/trace v1.24.0 // indirect + go.uber.org/atomic v1.11.0 // indirect + go.uber.org/goleak v1.3.0 // indirect + go.uber.org/multierr v1.11.0 // indirect + go.uber.org/zap v1.21.0 // indirect + go4.org/netipx v0.0.0-20230125063823-8449b0a6169f // indirect + golang.org/x/crypto v0.24.0 // indirect + golang.org/x/exp v0.0.0-20240325151524-a685a6edb6d8 // indirect + golang.org/x/mod v0.17.0 // indirect + golang.org/x/net v0.26.0 // indirect + golang.org/x/oauth2 v0.18.0 // indirect + golang.org/x/sync v0.7.0 // indirect + golang.org/x/sys v0.21.0 // indirect + golang.org/x/text v0.16.0 // indirect + golang.org/x/time v0.5.0 // indirect + golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d // indirect + google.golang.org/appengine v1.6.8 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20240304212257-790db918fca8 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240304161311-37d4d3c04a78 // indirect + google.golang.org/grpc v1.62.1 // indirect + google.golang.org/protobuf v1.33.0 // indirect + gopkg.in/yaml.v2 v2.4.0 // indirect + gopkg.in/yaml.v3 v3.0.1 // indirect + k8s.io/apimachinery v0.29.2 // indirect + k8s.io/client-go v0.29.2 // indirect + k8s.io/klog/v2 v2.120.1 // indirect + k8s.io/utils v0.0.0-20230726121419-3b25d923346b // indirect +) diff --git a/cmd/segment-inspect/go.sum b/cmd/segment-inspect/go.sum new file mode 100644 index 000000000000..86a974c2072c --- /dev/null +++ b/cmd/segment-inspect/go.sum @@ -0,0 +1,1024 @@ +cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU= +cloud.google.com/go v0.44.1/go.mod h1:iSa0KzasP4Uvy3f1mN/7PiObzGgflwredwwASm/v6AU= +cloud.google.com/go v0.44.2/go.mod h1:60680Gw3Yr4ikxnPRS/oxxkBccT6SA1yMk63TGekxKY= +cloud.google.com/go v0.45.1/go.mod h1:RpBamKRgapWJb87xiFSdk4g1CME7QZg3uwTez+TSTjc= +cloud.google.com/go v0.46.3/go.mod h1:a6bKKbmY7er1mI7TEI4lsAkts/mkhTSZK8w33B4RAg0= +cloud.google.com/go v0.50.0/go.mod h1:r9sluTvynVuxRIOHXQEHMFffphuXHOMZMycpNR5e6To= +cloud.google.com/go v0.52.0/go.mod h1:pXajvRH/6o3+F9jDHZWQ5PbGhn+o8w9qiu/CffaVdO4= +cloud.google.com/go v0.53.0/go.mod h1:fp/UouUEsRkN6ryDKNW/Upv/JBKnv6WDthjR6+vze6M= +cloud.google.com/go v0.54.0/go.mod h1:1rq2OEkV3YMf6n/9ZvGWI3GWw0VoqH/1x2nd8Is/bPc= +cloud.google.com/go v0.56.0/go.mod h1:jr7tqZxxKOVYizybht9+26Z/gUq7tiRzu+ACVAMbKVk= +cloud.google.com/go v0.57.0/go.mod h1:oXiQ6Rzq3RAkkY7N6t3TcE6jE+CIBBbA36lwQ1JyzZs= +cloud.google.com/go v0.62.0/go.mod h1:jmCYTdRCQuc1PHIIJ/maLInMho30T/Y0M4hTdTShOYc= +cloud.google.com/go v0.65.0/go.mod h1:O5N8zS7uWy9vkA9vayVHs65eM1ubvY4h553ofrNHObY= +cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o= +cloud.google.com/go/bigquery v1.3.0/go.mod h1:PjpwJnslEMmckchkHFfq+HTD2DmtT67aNFKH1/VBDHE= +cloud.google.com/go/bigquery v1.4.0/go.mod h1:S8dzgnTigyfTmLBfrtrhyYhwRxG72rYxvftPBK2Dvzc= +cloud.google.com/go/bigquery v1.5.0/go.mod h1:snEHRnqQbz117VIFhE8bmtwIDY80NLUZUMb4Nv6dBIg= +cloud.google.com/go/bigquery v1.7.0/go.mod h1://okPTzCYNXSlb24MZs83e2Do+h+VXtc4gLoIoXIAPc= +cloud.google.com/go/bigquery v1.8.0/go.mod h1:J5hqkt3O0uAFnINi6JXValWIb1v0goeZM77hZzJN/fQ= +cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= +cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk= +cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= +cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+R3AArQw= +cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA= +cloud.google.com/go/pubsub v1.3.1/go.mod h1:i+ucay31+CNRpDW4Lu78I4xXG+O1r/MAHgjpRVR+TSU= +cloud.google.com/go/storage v1.0.0/go.mod h1:IhtSnM/ZTZV8YYJWCY8RULGVqBDmpoyjwiyrjsg+URw= +cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0ZeosJ0Rtdos= +cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohlUTyfDhBk= +cloud.google.com/go/storage v1.8.0/go.mod h1:Wv1Oy7z6Yz3DshWRJFhqM/UCfaWIRTdp0RXyy7KQOVs= +cloud.google.com/go/storage v1.10.0/go.mod h1:FLPqc6j+Ki4BU591ie1oL6qBQGu2Bl/tZ9ullr3+Kg0= +dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= +github.com/Azure/azure-sdk-for-go/sdk/azcore v1.11.1 h1:E+OJmp2tPvt1W+amx48v1eqbjDYsgN+RzP4q16yV5eM= +github.com/Azure/azure-sdk-for-go/sdk/azcore v1.11.1/go.mod h1:a6xsAQUZg+VsS3TJ05SRp524Hs4pZ/AeFSr5ENf0Yjo= +github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.6.0 h1:U2rTu3Ef+7w9FHKIAXM6ZyqF3UOWJZ12zIm8zECAFfg= +github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.6.0/go.mod h1:9kIvujWAA58nmPmWB1m23fyWic1kYZMxD9CxaWn4Qpg= +github.com/Azure/azure-sdk-for-go/sdk/internal v1.8.0 h1:jBQA3cKT4L2rWMpgE7Yt3Hwh2aUj8KXjIGLxjHeYNNo= +github.com/Azure/azure-sdk-for-go/sdk/internal v1.8.0/go.mod h1:4OG6tQ9EOP/MT0NMjDlRzWoVFxfu9rN9B2X+tlSVktg= +github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/compute/armcompute/v5 v5.5.0 h1:MxA59PGoCFb+vCwRQi3PhQEwHj4+r2dhuv9HG+vM7iM= +github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/compute/armcompute/v5 v5.5.0/go.mod h1:uYt4CfhkJA9o0FN7jfE5minm/i4nUE4MjGUJkzB6Zs8= +github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/network/armnetwork/v4 v4.3.0 h1:bXwSugBiSbgtz7rOtbfGf+woewp4f06orW9OP5BjHLA= +github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/network/armnetwork/v4 v4.3.0/go.mod h1:Y/HgrePTmGy9HjdSGTqZNa+apUpTVIEVKXJyARP2lrk= +github.com/AzureAD/microsoft-authentication-library-for-go v1.2.2 h1:XHOnouVk1mxXfQidrMEnLlPk9UMeRtyBTnEFtxkV0kU= +github.com/AzureAD/microsoft-authentication-library-for-go v1.2.2/go.mod h1:wP83P5OoQ5p6ip3ScPr0BAq0BvuPAvacpEuSzyouqAI= +github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= +github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= +github.com/Code-Hex/go-generics-cache v1.3.1 h1:i8rLwyhoyhaerr7JpjtYjJZUcCbWOdiYO3fZXLiEC4g= +github.com/Code-Hex/go-generics-cache v1.3.1/go.mod h1:qxcC9kRVrct9rHeiYpFWSoW1vxyillCVzX13KZG8dl4= +github.com/DataDog/datadog-go v3.2.0+incompatible/go.mod h1:LButxg5PwREeZtORoXG3tL4fMGNddJ+vMq1mwgfaqoQ= +github.com/HdrHistogram/hdrhistogram-go v1.1.2 h1:5IcZpTvzydCQeHzK4Ef/D5rrSqwxob0t8PQPMybUNFM= +github.com/HdrHistogram/hdrhistogram-go v1.1.2/go.mod h1:yDgFjdqOqDEKOvasDdhWNXYg9BVp4O+o5f6V/ehm6Oo= +github.com/Masterminds/goutils v1.1.1 h1:5nUrii3FMTL5diU80unEVvNevw1nH4+ZV4DSLVJLSYI= +github.com/Masterminds/goutils v1.1.1/go.mod h1:8cTjp+g8YejhMuvIA5y2vz3BpJxksy863GQaJW2MFNU= +github.com/Masterminds/semver/v3 v3.2.0 h1:3MEsd0SM6jqZojhjLWWeBY+Kcjy9i6MQAeY7YgDP83g= +github.com/Masterminds/semver/v3 v3.2.0/go.mod h1:qvl/7zhW3nngYb5+80sSMF+FG2BjYrf8m9wsX0PNOMQ= +github.com/Masterminds/sprig/v3 v3.2.3 h1:eL2fZNezLomi0uOLqjQoN6BfsDD+fyLtgbJMAj9n6YA= +github.com/Masterminds/sprig/v3 v3.2.3/go.mod h1:rXcFaZ2zZbLRJv/xSysmlgIM1u11eBaRMhvYXJNkGuM= +github.com/Microsoft/go-winio v0.6.1 h1:9/kr64B9VUZrLm5YYwbGtUJnMgqWVOdUAXu6Migciow= +github.com/Microsoft/go-winio v0.6.1/go.mod h1:LRdKpFKfdobln8UmuiYcKPot9D2v6svN5+sAH+4kjUM= +github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU= +github.com/OneOfOne/xxhash v1.2.6 h1:U68crOE3y3MPttCMQGywZOLrTeF5HHJ3/vDBCJn9/bA= +github.com/OneOfOne/xxhash v1.2.6/go.mod h1:eZbhyaAYD41SGSSsnmcpxVoRiQ/MPUTjUdIIOT9Um7Q= +github.com/Workiva/go-datastructures v1.1.0 h1:hu20UpgZneBhQ3ZvwiOGlqJSKIosin2Rd5wAKUHEO/k= +github.com/Workiva/go-datastructures v1.1.0/go.mod h1:1yZL+zfsztete+ePzZz/Zb1/t5BnDuE2Ya2MMGhzP6A= +github.com/ajstarks/svgo v0.0.0-20180226025133-644b8db467af/go.mod h1:K08gAheRH3/J6wwsYMMT4xOr94bZjxIelGM0+d/wbFw= +github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= +github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= +github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= +github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= +github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho= +github.com/alecthomas/units v0.0.0-20231202071711-9a357b53e9c9 h1:ez/4by2iGztzR4L0zgAOR8lTQK9VlyBVVd7G4omaOQs= +github.com/alecthomas/units v0.0.0-20231202071711-9a357b53e9c9/go.mod h1:OMCwj8VM1Kc9e19TLln2VL61YJF0x1XFtfdL4JdbSyE= +github.com/alicebob/gopher-json v0.0.0-20200520072559-a9ecdc9d1d3a h1:HbKu58rmZpUGpz5+4FfNmIU+FmZg2P3Xaj2v2bfNWmk= +github.com/alicebob/gopher-json v0.0.0-20200520072559-a9ecdc9d1d3a/go.mod h1:SGnFV6hVsYE877CKEZ6tDNTjaSXYUk6QqoIK6PrAtcc= +github.com/alicebob/miniredis v2.5.0+incompatible h1:yBHoLpsyjupjz3NL3MhKMVkR41j82Yjf3KFv7ApYzUI= +github.com/alicebob/miniredis/v2 v2.30.4 h1:8S4/o1/KoUArAGbGwPxcwf0krlzceva2XVOSchFS7Eo= +github.com/alicebob/miniredis/v2 v2.30.4/go.mod h1:b25qWj4fCEsBeAAR2mlb0ufImGC6uH3VlUfb/HS5zKg= +github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= +github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e/go.mod h1:3U/XgcO3hCbHZ8TKRvWD2dDTCfh9M9ya+I9JpbB7O8o= +github.com/armon/go-metrics v0.0.0-20180917152333-f0300d1749da/go.mod h1:Q73ZrmVTwzkszR9V5SSuryQ31EELlFMUz1kKyl939pY= +github.com/armon/go-metrics v0.4.1 h1:hR91U9KYmb6bLBYLQjyM+3j+rcd/UhE+G78SFnF8gJA= +github.com/armon/go-metrics v0.4.1/go.mod h1:E6amYzXo6aW1tqzoZGT755KkbgrJsSdpwZ+3JqfkOG4= +github.com/armon/go-radix v0.0.0-20180808171621-7fddfc383310/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8= +github.com/armon/go-radix v1.0.0/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8= +github.com/aws/aws-sdk-go v1.38.35/go.mod h1:hcU610XS61/+aQV88ixoOzUoG7v3b31pl2zKMmprdro= +github.com/aws/aws-sdk-go v1.50.32 h1:POt81DvegnpQKM4DMDLlHz1CO6OBnEoQ1gRhYFd7QRY= +github.com/aws/aws-sdk-go v1.50.32/go.mod h1:LF8svs817+Nz+DmiMQKTO3ubZ/6IaTpq3TjupRn3Eqk= +github.com/bboreham/go-loser v0.0.0-20230920113527-fcc2c21820a3 h1:6df1vn4bBlDDo4tARvBm7l6KA9iVMnE3NWizDeWSrps= +github.com/bboreham/go-loser v0.0.0-20230920113527-fcc2c21820a3/go.mod h1:CIWtjkly68+yqLPbvwwR/fjNJA/idrtULjZWh2v1ys0= +github.com/benbjohnson/clock v1.1.0 h1:Q92kusRqC1XV2MjkWETPvjJVqKetz1OzxZB7mHJLju8= +github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= +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/buger/jsonparser v1.1.1 h1:2PnMjfWD7wBILjqQbt530v576A/cAbQvEW9gGIpYMUs= +github.com/buger/jsonparser v1.1.1/go.mod h1:6RYKKt7H4d4+iWqouImQ9R2FZql3VbhNgx27UK13J/0= +github.com/c2h5oh/datasize v0.0.0-20220606134207-859f65c6625b h1:6+ZFm0flnudZzdSE0JxlhR2hKnGPcNB35BjQf4RYQDY= +github.com/c2h5oh/datasize v0.0.0-20220606134207-859f65c6625b/go.mod h1:S/7n9copUssQ56c7aAgHqftWO4LTf4xY6CGWt8Bc+3M= +github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= +github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko= +github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc= +github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= +github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= +github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= +github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= +github.com/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/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= +github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= +github.com/cncf/udpa/go v0.0.0-20201120205902-5459f2c99403/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk= +github.com/cncf/xds/go v0.0.0-20231128003011-0fa0005c9caa h1:jQCWAUqqlij9Pgj2i/PB79y4KOPYVyFYdROxgaCwdTQ= +github.com/cncf/xds/go v0.0.0-20231128003011-0fa0005c9caa/go.mod h1:x/1Gn8zydmfq8dk6e9PdstVsDgu9RuyIIJqAaF//0IM= +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/v22 v22.3.2/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= +github.com/coreos/go-systemd/v22 v22.5.0 h1:RrqgGjYQKalulkV8NGVIfkXQf6YYmOyiJKk8iXXhfZs= +github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= +github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= +github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc h1:U9qPSI2PIWSS1VwoXQT9A3Wy9MM3WgvqSxFWenqJduM= +github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/dennwc/varint v1.0.0 h1:kGNFFSSw8ToIy3obO/kKr8U9GZYUAxQEVuix4zfDWzE= +github.com/dennwc/varint v1.0.0/go.mod h1:hnItb35rvZvJrbTALZtY/iQfDs48JKRG1RPpgziApxA= +github.com/dgryski/go-rendezvous v0.0.0-20200823014737-9f7001d12a5f h1:lO4WD4F/rVNCu3HqELle0jiPLLBs70cWOduZpkS1E78= +github.com/dgryski/go-rendezvous v0.0.0-20200823014737-9f7001d12a5f/go.mod h1:cuUVRXasLTGF7a8hSLbxyZXjz+1KgoB3wDUb6vlszIc= +github.com/digitalocean/godo v1.109.0 h1:4W97RJLJSUQ3veRZDNbp1Ol3Rbn6Lmt9bKGvfqYI5SU= +github.com/digitalocean/godo v1.109.0/go.mod h1:R6EmmWI8CT1+fCtjWY9UCB+L5uufuZH13wk3YhxycCs= +github.com/distribution/reference v0.5.0 h1:/FUIFXtfc/x2gpa5/VGfiGLuOIdYa1t65IKK2OFGvA0= +github.com/distribution/reference v0.5.0/go.mod h1:BbU0aIcezP1/5jX/8MP0YiH4SdvB5Y4f/wlDRiLyi3E= +github.com/docker/docker v25.0.3+incompatible h1:D5fy/lYmY7bvZa0XTZ5/UJPljor41F+vdyJG5luQLfQ= +github.com/docker/docker v25.0.3+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-units v0.5.0 h1:69rxXcBk27SvSaaxTtLh/8llcHD8vYHT7WSdRZ/jvr4= +github.com/docker/go-units v0.5.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk= +github.com/dustin/go-humanize v1.0.0/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= +github.com/dustin/go-humanize v1.0.1 h1:GzkhY7T5VNhEkwH0PVJgjz+fX1rhBrR7pRT3mDkpeCY= +github.com/dustin/go-humanize v1.0.1/go.mod h1:Mu1zIs6XwVuF/gI1OepvI0qD18qycQx+mFykh5fBlto= +github.com/edsrzf/mmap-go v1.1.0 h1:6EUwBLQ/Mcr1EYLE4Tn1VdW1A4ckqCQWZBw8Hr0kjpQ= +github.com/edsrzf/mmap-go v1.1.0/go.mod h1:19H/e8pUPLicwkyNgOykDXkJ9F0MHE+Z52B8EIth78Q= +github.com/emicklei/go-restful/v3 v3.11.0 h1:rAQeMHw1c7zTmncogyy8VvRZwtkmkZ4FxERmMY4rD+g= +github.com/emicklei/go-restful/v3 v3.11.0/go.mod h1:6n3XBCmQQb25CM2LCACGz8ukIrRry+4bhvbpWn3mrbc= +github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= +github.com/envoyproxy/go-control-plane v0.9.9-0.20210217033140-668b12f5399d/go.mod h1:cXg6YxExXjJnVBQHBLXeUAgxn2UodCpnH306RInaBQk= +github.com/envoyproxy/go-control-plane v0.12.0 h1:4X+VP1GHd1Mhj6IB5mMeGbLCleqxjletLK6K0rbxyZI= +github.com/envoyproxy/go-control-plane v0.12.0/go.mod h1:ZBTaoJ23lqITozF0M6G4/IragXCQKCnYbmlmtHvwRG0= +github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= +github.com/envoyproxy/protoc-gen-validate v1.0.4 h1:gVPz/FMfvh57HdSJQyvBtF00j8JU4zdyUgIUNhlgg0A= +github.com/envoyproxy/protoc-gen-validate v1.0.4/go.mod h1:qys6tmnRsYrQqIhm2bvKZH4Blx/1gTIZ2UKVY1M+Yew= +github.com/facette/natsort v0.0.0-20181210072756-2cd4dd1e2dcb h1:IT4JYU7k4ikYg1SCxNI1/Tieq/NFvh6dzLdgi7eu0tM= +github.com/facette/natsort v0.0.0-20181210072756-2cd4dd1e2dcb/go.mod h1:bH6Xx7IW64qjjJq8M2u4dxNaBiDfKK+z/3eGDpXEQhc= +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/go.mod h1:kLAiJbzzSOZDVNGyDpeOxJ47H46qBXwg5ILebYFFOfk= +github.com/fatih/color v1.16.0 h1:zmkK9Ngbjj+K0yRhTVONQh1p/HknKYSlNT+vZCzyokM= +github.com/fatih/color v1.16.0/go.mod h1:fL2Sau1YI5c0pdGEVCbKQbLXB6edEj1ZgiY4NijnWvE= +github.com/felixge/httpsnoop v1.0.4 h1:NFTV2Zj1bL4mc9sqWACXbQFVBBg2W3GPvqp8/ESS2Wg= +github.com/felixge/httpsnoop v1.0.4/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/fsnotify/fsnotify v1.7.0 h1:8JEhPFa5W2WU7YfeZzPNqzMP6Lwt7L2715Ggo0nosvA= +github.com/fsnotify/fsnotify v1.7.0/go.mod h1:40Bi/Hjc2AVfZrqy+aj+yEI+/bRxZnMJyTJwOpGvigM= +github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= +github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= +github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= +github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= +github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= +github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= +github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= +github.com/go-kit/log v0.2.1 h1:MRVx0/zhvdseW+Gza6N9rVzU/IVzaeE1SFI4raAhmBU= +github.com/go-kit/log v0.2.1/go.mod h1:NwTd00d/i8cPZ3xOwwiv2PO5MOcx78fFErGNcVmBjv0= +github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= +github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= +github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= +github.com/go-logfmt/logfmt v0.6.0 h1:wGYYu3uicYdqXVgoYbvnkrPVXkuLM1p1ifugDMEdRi4= +github.com/go-logfmt/logfmt v0.6.0/go.mod h1:WYhtIu8zTZfxdn5+rREduYbwxfcBr/Vr6KEVveWlfTs= +github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= +github.com/go-logr/logr v1.4.1 h1:pKouT5E8xu9zeFC39JXRDukb6JFQPXM5p5I91188VAQ= +github.com/go-logr/logr v1.4.1/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY= +github.com/go-logr/stdr v1.2.2 h1:hSWxHoqTgW2S2qGc0LTAI563KZ5YKYRhT3MFKZMbjag= +github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE= +github.com/go-openapi/jsonpointer v0.20.2 h1:mQc3nmndL8ZBzStEo3JYF8wzmeWffDH4VbXz58sAx6Q= +github.com/go-openapi/jsonpointer v0.20.2/go.mod h1:bHen+N0u1KEO3YlmqOjTT9Adn1RfD91Ar825/PuiRVs= +github.com/go-openapi/jsonreference v0.20.4 h1:bKlDxQxQJgwpUSgOENiMPzCTBVuc7vTdXSSgNeAhojU= +github.com/go-openapi/jsonreference v0.20.4/go.mod h1:5pZJyJP2MnYCpoeoMAql78cCHauHj0V9Lhc506VOpw4= +github.com/go-openapi/swag v0.22.9 h1:XX2DssF+mQKM2DHsbgZK74y/zj4mo9I99+89xUmuZCE= +github.com/go-openapi/swag v0.22.9/go.mod h1:3/OXnFfnMAwBD099SwYRk7GD3xOrr1iL7d/XNLXVVwE= +github.com/go-redis/redis/v8 v8.11.5 h1:AcZZR7igkdvfVmQTPnu9WE37LRrO/YrBH5zWyjDC0oI= +github.com/go-redis/redis/v8 v8.11.5/go.mod h1:gREzHqY1hg6oD9ngVRbLStwAWKhA0FEgq8Jd4h5lpwo= +github.com/go-resty/resty/v2 v2.11.0 h1:i7jMfNOJYMp69lq7qozJP+bjgzfAzeOhuGlyDrqxT/8= +github.com/go-resty/resty/v2 v2.11.0/go.mod h1:iiP/OpA0CkcL3IGt1O0+/SIItFUbkkyw5BGXiVdTu+A= +github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= +github.com/go-zookeeper/zk v1.0.3 h1:7M2kwOsc//9VeeFiPtf+uSJlVpU66x9Ba5+8XK7/TDg= +github.com/go-zookeeper/zk v1.0.3/go.mod h1:nOB03cncLtlp4t+UAkGSV+9beXP/akpekBwL+UX1Qcw= +github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= +github.com/gogo/googleapis v0.0.0-20180223154316-0cd9801be74a/go.mod h1:gf4bu3Q80BeJ6H1S1vYPm8/ELATdvryBaNFGgqEef3s= +github.com/gogo/googleapis v1.4.0 h1:zgVt4UpGxcqVOw97aRGxT4svlcmdK35fynLNctY32zI= +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.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/gogo/status v1.1.1 h1:DuHXlSFHNKqTQ+/ACf5Vs6r4X/dH2EgIzR9Vr+H65kg= +github.com/gogo/status v1.1.1/go.mod h1:jpG3dM5QPcqu19Hg8lkUhBFBa3TcLs1DG7+2Jqci7oU= +github.com/golang-jwt/jwt/v5 v5.2.1 h1:OuVbFODueb089Lh128TAcimifWaLhJwVflnrgM17wHk= +github.com/golang-jwt/jwt/v5 v5.2.1/go.mod h1:pqrtFR0X4osieyHYxtmOUWsAWrfe1Q5UVIyoH402zdk= +github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0/go.mod h1:E/TSTwGwJL78qG/PmXZO1EjYhfJinVAhrmmHX6Z8B9k= +github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= +github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= +github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= +github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y= +github.com/golang/mock v1.4.0/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= +github.com/golang/mock v1.4.1/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= +github.com/golang/mock v1.4.3/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= +github.com/golang/mock v1.4.4/go.mod h1:l3mdAwkq5BuhzHwde/uurv3sEJeZMXNpwsxVWU71h+4= +github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= +github.com/golang/protobuf v1.3.4/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= +github.com/golang/protobuf v1.3.5/go.mod h1:6O5/vntMXwX2lRkT1hjjk0nAC1IDOTvTlVgjlRvqsdk= +github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8= +github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA= +github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs= +github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w= +github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= +github.com/golang/protobuf v1.4.1/go.mod h1:U8fpvMrcmy5pZrNK1lt4xCsGvpyWQ/VVv6QDs8UjoX8= +github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= +github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= +github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= +github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= +github.com/golang/protobuf v1.5.3 h1:KhyjKVUg7Usr/dYsdSqoFveMYd5ko72D+zANwlG1mmg= +github.com/golang/protobuf v1.5.3/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= +github.com/golang/snappy v0.0.4 h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM= +github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= +github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= +github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= +github.com/google/btree v1.1.2 h1:xf4v41cLI2Z6FxbKm+8Bu+m8ifhj15JuZ9sa0jZCMUU= +github.com/google/btree v1.1.2/go.mod h1:qOPhT0dTNdNzV6Z/lhRX0YXUafgPLFUh+gZMl761Gm4= +github.com/google/gnostic-models v0.6.8 h1:yo/ABAfM5IMRsS1VnXjTBvUb61tFIHozhlYvRgGre9I= +github.com/google/gnostic-models v0.6.8/go.mod h1:5n7qKqH0f5wFt+aWF8CW6pZLLNOfYuF5OpfBSENuI8U= +github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= +github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.4.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.1/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= +github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= +github.com/google/go-querystring v1.1.0 h1:AnCroh3fv4ZBgVIf1Iwtovgjaw/GiKJo8M8yD/fhyJ8= +github.com/google/go-querystring v1.1.0/go.mod h1:Kcdr2DB4koayq7X8pmAG4sNG59So17icRSOU623lUBU= +github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= +github.com/google/gofuzz v1.2.0 h1:xRy4A+RhZaiKjJ1bPfwQ8sedCA+YS2YcCHW6ec7JMi0= +github.com/google/gofuzz v1.2.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= +github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= +github.com/google/martian/v3 v3.0.0/go.mod h1:y5Zk1BBys9G+gd6Jrk0W3cC1+ELVxBWuIGO+w/tUAp0= +github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= +github.com/google/pprof v0.0.0-20190515194954-54271f7e092f/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= +github.com/google/pprof v0.0.0-20191218002539-d4f498aebedc/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/pprof v0.0.0-20200212024743-f11f1df84d12/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/pprof v0.0.0-20200229191704-1ebb73c60ed3/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/pprof v0.0.0-20200430221834-fc25d7d30c6d/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/pprof v0.0.0-20200708004538-1a94d8640e99/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= +github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= +github.com/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.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= +github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= +github.com/googleapis/gax-go/v2 v2.0.5/go.mod h1:DWXyrwAJ9X0FpwwEdw+IPEYBICEFu5mhpdKc/us6bOk= +github.com/gophercloud/gophercloud v1.8.0 h1:TM3Jawprb2NrdOnvcHhWJalmKmAmOGgfZElM/3oBYCk= +github.com/gophercloud/gophercloud v1.8.0/go.mod h1:aAVqcocTSXh2vYFZ1JTvx4EQmfgzxRcNupUfxZbBNDM= +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 v1.5.0 h1:PPwGk2jz7EePpoHN/+ClbZu8SPxiqlu12wZP/3sWmnc= +github.com/gorilla/websocket v1.5.0/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= +github.com/grafana/dskit v0.0.0-20240626184720-35810fdf1c6d h1:CD8PWWX+9lYdgeMquSofmLErvCtk7jb+3/W/SH6oo/k= +github.com/grafana/dskit v0.0.0-20240626184720-35810fdf1c6d/go.mod h1:HvSf3uf8Ps2vPpzHeAFyZTdUcbVr+Rxpq1xcx7J/muc= +github.com/grafana/gomemcache v0.0.0-20240229205252-cd6a66d6fb56 h1:X8IKQ0wu40wpvYcKfBcc5T4QnhdQjUhtUtB/1CY89lE= +github.com/grafana/gomemcache v0.0.0-20240229205252-cd6a66d6fb56/go.mod h1:PGk3RjYHpxMM8HFPhKKo+vve3DdlPUELZLSDEFehPuU= +github.com/grafana/jsonparser v0.0.0-20240209175146-098958973a2d h1:YwbJJ/PrVWVdnR+j/EAVuazdeP+Za5qbiH1Vlr+wFXs= +github.com/grafana/jsonparser v0.0.0-20240209175146-098958973a2d/go.mod h1:796sq+UcONnSlzA3RtlBZ+b/hrerkZXiEmO8oMjyRwY= +github.com/grafana/loki/pkg/push v0.0.0-20231124142027-e52380921608 h1:ZYk42718kSXOiIKdjZKljWLgBpzL5z1yutKABksQCMg= +github.com/grafana/loki/pkg/push v0.0.0-20231124142027-e52380921608/go.mod h1:f3JSoxBTPXX5ec4FxxeC19nTBSxoTz+cBgS3cYLMcr0= +github.com/grafana/loki/v3 v3.0.0-20240716103001-3ac130b8a152 h1:JxdN9YUeK/cWWWqOR+Dq1imdJzLFsIUBfcFcKq6kRx0= +github.com/grafana/loki/v3 v3.0.0-20240716103001-3ac130b8a152/go.mod h1:Y3tWPBwgnIvazqbuEyjjiGudaIX+7eH8l86LcFZeZ4o= +github.com/grafana/pyroscope-go/godeltaprof v0.1.6 h1:nEdZ8louGAplSvIJi1HVp7kWvFvdiiYg3COLlTwJiFo= +github.com/grafana/pyroscope-go/godeltaprof v0.1.6/go.mod h1:Tk376Nbldo4Cha9RgiU7ik8WKFkNpfds98aUzS8omLE= +github.com/grafana/regexp v0.0.0-20221122212121-6b5c0a4cb7fd h1:PpuIBO5P3e9hpqBD0O/HjhShYuM6XE0i/lbE6J94kww= +github.com/grafana/regexp v0.0.0-20221122212121-6b5c0a4cb7fd/go.mod h1:M5qHK+eWfAv8VR/265dIuEpL3fNfeC21tXXp9itM24A= +github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 h1:+9834+KizmvFV7pXQGSXQTsaWhq2GjuNUt0aUU0YBYw= +github.com/grpc-ecosystem/go-grpc-middleware v1.3.0/go.mod h1:z0ButlSOZa5vEBq9m2m2hlwIgKw+rp3sdCBRoJY+30Y= +github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk= +github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw= +github.com/grpc-ecosystem/grpc-opentracing v0.0.0-20180507213350-8e809c8a8645 h1:MJG/KsmcqMwFAkh8mTnAwhyKoB+sTAnY4CACC110tbU= +github.com/grpc-ecosystem/grpc-opentracing v0.0.0-20180507213350-8e809c8a8645/go.mod h1:6iZfnjpejD4L/4DwD7NryNaJyCQdzwWwH2MWhCA90Kw= +github.com/hashicorp/consul/api v1.28.2 h1:mXfkRHrpHN4YY3RqL09nXU1eHKLNiuAN4kHvDQ16k/8= +github.com/hashicorp/consul/api v1.28.2/go.mod h1:KyzqzgMEya+IZPcD65YFoOVAgPpbfERu4I/tzG6/ueE= +github.com/hashicorp/consul/sdk v0.16.0 h1:SE9m0W6DEfgIVCJX7xU+iv/hUl4m/nxqMTnCdMxDpJ8= +github.com/hashicorp/consul/sdk v0.16.0/go.mod h1:7pxqqhqoaPqnBnzXD1StKed62LqJeClzVsUEy85Zr0A= +github.com/hashicorp/cronexpr v1.1.2 h1:wG/ZYIKT+RT3QkOdgYc+xsKWVRgnxJ1OJtjjy84fJ9A= +github.com/hashicorp/cronexpr v1.1.2/go.mod h1:P4wA0KBl9C5q2hABiMO7cp6jcIg96CDh1Efb3g1PWA4= +github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= +github.com/hashicorp/errwrap v1.1.0 h1:OxrOeh75EUXMY8TBjag2fzXGZ40LB6IKw45YeGUDY2I= +github.com/hashicorp/errwrap v1.1.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= +github.com/hashicorp/go-cleanhttp v0.5.0/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80= +github.com/hashicorp/go-cleanhttp v0.5.2 h1:035FKYIWjmULyFRBKPs8TBQoi0x6d9G4xc9neXJWAZQ= +github.com/hashicorp/go-cleanhttp v0.5.2/go.mod h1:kO/YDlP8L1346E6Sodw+PrpBSV4/SoxCXGY6BqNFT48= +github.com/hashicorp/go-hclog v1.6.3 h1:Qr2kF+eVWjTiYmU7Y31tYlP1h0q/X3Nl3tPGdaB11/k= +github.com/hashicorp/go-hclog v1.6.3/go.mod h1:W4Qnvbt70Wk/zYJryRzDRU/4r0kIg0PVHBcfoyhpF5M= +github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= +github.com/hashicorp/go-immutable-radix v1.3.1 h1:DKHmCUm2hRBK510BaiZlwvpD40f8bJFeZnpfm2KLowc= +github.com/hashicorp/go-immutable-radix v1.3.1/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= +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 v1.0.0/go.mod h1:dHtQlpGsu+cZNNAkkCN/P3hoUDHhCYQXV3UM06sGGrk= +github.com/hashicorp/go-multierror v1.1.0/go.mod h1:spPvp8C1qA32ftKqdAHm4hHTbPw+vmowP0z+KUhOZdA= +github.com/hashicorp/go-multierror v1.1.1 h1:H5DkEtf6CXdFp0N0Em5UCwQpXMWke8IA0+lD48awMYo= +github.com/hashicorp/go-multierror v1.1.1/go.mod h1:iw975J/qwKPdAO1clOe2L8331t/9/fmwbPZ6JB6eMoM= +github.com/hashicorp/go-retryablehttp v0.5.3/go.mod h1:9B5zBasrRhHXnJnui7y6sL7es7NDiJgTc6Er0maI1Xs= +github.com/hashicorp/go-retryablehttp v0.7.7 h1:C8hUCYzor8PIfXHa4UrZkU4VvK8o9ISHxT2Q8+VepXU= +github.com/hashicorp/go-retryablehttp v0.7.7/go.mod h1:pkQpWZeYWskR+D1tR2O5OcBFOxfA7DoAO6xtkuQnHTk= +github.com/hashicorp/go-rootcerts v1.0.2 h1:jzhAVGtqPKbwpyCPELlgNWhE1znq+qwJtW5Oi2viEzc= +github.com/hashicorp/go-rootcerts v1.0.2/go.mod h1:pqUvnprVnM5bf7AOirdbb01K4ccR319Vf4pU3K5EGc8= +github.com/hashicorp/go-sockaddr v1.0.0/go.mod h1:7Xibr9yA9JjQq1JpNB2Vw7kxv8xerXegt+ozgdvDeDU= +github.com/hashicorp/go-sockaddr v1.0.6 h1:RSG8rKU28VTUTvEKghe5gIhIQpv8evvNpnDEyqO4u9I= +github.com/hashicorp/go-sockaddr v1.0.6/go.mod h1:uoUUmtwU7n9Dv3O4SNLeFvg0SxQ3lyjsj6+CCykpaxI= +github.com/hashicorp/go-syslog v1.0.0/go.mod h1:qPfqrKkXGihmCqbJM2mZgkZGvKG1dFdvsLplgctolz4= +github.com/hashicorp/go-uuid v1.0.0/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= +github.com/hashicorp/go-uuid v1.0.1/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= +github.com/hashicorp/go-uuid v1.0.3 h1:2gKiV6YVmrJ1i2CKKa9obLvRieoRGviZFL26PcT/Co8= +github.com/hashicorp/go-uuid v1.0.3/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= +github.com/hashicorp/go-version v1.6.0 h1:feTTfFNnjP967rlCxM/I9g701jU+RN74YKx2mOkIeek= +github.com/hashicorp/go-version v1.6.0/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09ZGVZPK5anwXA= +github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= +github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= +github.com/hashicorp/golang-lru v0.6.0 h1:uL2shRDx7RTrOrTCUZEGP/wJUFiUI8QT6E7z5o8jga4= +github.com/hashicorp/golang-lru v0.6.0/go.mod h1:iADmTwqILo4mZ8BN3D2Q6+9jd8WM5uGBxy+E8yxSoD4= +github.com/hashicorp/logutils v1.0.0/go.mod h1:QIAnNjmIWmVIIkWDTG1z5v++HQmx9WQRO+LraFDTW64= +github.com/hashicorp/mdns v1.0.4/go.mod h1:mtBihi+LeNXGtG8L9dX59gAEa12BDtBQSp4v/YAJqrc= +github.com/hashicorp/memberlist v0.5.0 h1:EtYPN8DpAURiapus508I4n9CzHs2W+8NZGbmmR/prTM= +github.com/hashicorp/memberlist v0.5.0/go.mod h1:yvyXLpo0QaGE59Y7hDTsTzDD25JYBZ4mHgHUZ8lrOI0= +github.com/hashicorp/nomad/api v0.0.0-20240306004928-3e7191ccb702 h1:fI1LXuBaS1d9z1kmb++Og6YD8uMRwadXorCwE+xgOFA= +github.com/hashicorp/nomad/api v0.0.0-20240306004928-3e7191ccb702/go.mod h1:z71gkJdrkAt/Rl6C7Q79VE7AwJ5lUF+M+fzFTyIHYB0= +github.com/hashicorp/serf v0.10.1 h1:Z1H2J60yRKvfDYAOZLd2MU0ND4AH/WDz7xYHDWQsIPY= +github.com/hashicorp/serf v0.10.1/go.mod h1:yL2t6BqATOLGc5HF7qbFkTfXoPIY0WZdWHfEvMqbG+4= +github.com/hetznercloud/hcloud-go/v2 v2.6.0 h1:RJOA2hHZ7rD1pScA4O1NF6qhkHyUdbbxjHgFNot8928= +github.com/hetznercloud/hcloud-go/v2 v2.6.0/go.mod h1:4J1cSE57+g0WS93IiHLV7ubTHItcp+awzeBp5bM9mfA= +github.com/huandu/xstrings v1.3.3 h1:/Gcsuc1x8JVbJ9/rlye4xZnVAbEkGauT8lbebqcQws4= +github.com/huandu/xstrings v1.3.3/go.mod h1:y5/lhBue+AyNmUVz9RLU9xbLR0o4KIIExikq4ovT0aE= +github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= +github.com/imdario/mergo v0.3.11/go.mod h1:jmQim1M+e3UYxmgPu/WyfjB3N3VflVyUjjjwH0dnCYA= +github.com/imdario/mergo v0.3.16 h1:wwQJbIsHYGMUyLSPrEq1CT16AhnhNJQ51+4fdHUnCl4= +github.com/imdario/mergo v0.3.16/go.mod h1:WBLT9ZmE3lPoWsEzCh9LPo3TiwVN+ZKEjmz+hD27ysY= +github.com/ionos-cloud/sdk-go/v6 v6.1.11 h1:J/uRN4UWO3wCyGOeDdMKv8LWRzKu6UIkLEaes38Kzh8= +github.com/ionos-cloud/sdk-go/v6 v6.1.11/go.mod h1:EzEgRIDxBELvfoa/uBN0kOQaqovLjUWEB7iW4/Q+t4k= +github.com/jmespath/go-jmespath v0.4.0 h1:BEgLn5cpjn8UN1mAw4NjwDrS35OdebyEtFe+9YPoQUg= +github.com/jmespath/go-jmespath v0.4.0/go.mod h1:T8mJZnbsbmF+m6zOOFylbeCJqk5+pHWvzYPziyZiYoo= +github.com/jmespath/go-jmespath/internal/testify v1.5.1 h1:shLQSRRSCCPj3f2gpwzGwWFoC7ycTf1rcQZHOlsJ6N8= +github.com/jmespath/go-jmespath/internal/testify v1.5.1/go.mod h1:L3OGu8Wl2/fWfCI6z80xFu9LTZmf1ZRjMHUOPmWr69U= +github.com/josharian/intern v1.0.0 h1:vlS4z54oSdjm0bgjRigI+G1HpF+tI+9rE5LLzOg8HmY= +github.com/josharian/intern v1.0.0/go.mod h1:5DoeVV0s6jJacbCEi61lwdGj/aVlrQvzHFFd8Hwg//Y= +github.com/jpillora/backoff v1.0.0 h1:uvFg412JmmHBHw7iwprIxkPMI+sGQ4kzOWsMeHnm2EA= +github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= +github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= +github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= +github.com/json-iterator/go v1.1.10/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= +github.com/json-iterator/go v1.1.11/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= +github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnrnM= +github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= +github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= +github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= +github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= +github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= +github.com/jung-kurt/gofpdf v1.0.3-0.20190309125859-24315acbbda5/go.mod h1:7Id9E/uU8ce6rXgefFLlgrJj/GYY22cpxn+r32jIOes= +github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00= +github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= +github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= +github.com/klauspost/compress v1.17.3/go.mod h1:/dCuZOvVtNoHsyb+cuJD3itjs3NbnF6KH9zAO4BDxPM= +github.com/klauspost/compress v1.17.7 h1:ehO88t2UGzQK66LMdE8tibEd1ErmzZjNEqWkjLAKQQg= +github.com/klauspost/compress v1.17.7/go.mod h1:Di0epgTjJY877eYKx5yC51cX2A2Vl2ibi7bDH9ttBbw= +github.com/kolo/xmlrpc v0.0.0-20220921171641-a4b6fa1dd06b h1:udzkj9S/zlT5X367kqJis0QP7YMxobob6zhzq6Yre00= +github.com/kolo/xmlrpc v0.0.0-20220921171641-a4b6fa1dd06b/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.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= +github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= +github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= +github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= +github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= +github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= +github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= +github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= +github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= +github.com/kylelemons/godebug v1.1.0 h1:RPNrshWIDI6G2gRW9EHilWtl7Z6Sb1BR0xunSBf0SNc= +github.com/kylelemons/godebug v1.1.0/go.mod h1:9/0rRGxNHcop5bhtWyNeEfOS8JIWk580+fNqagV/RAw= +github.com/linode/linodego v1.29.0 h1:gDSQWAbKMAQX8db9FDCXHhodQPrJmLcmthjx6m+PyV4= +github.com/linode/linodego v1.29.0/go.mod h1:3k6WvCM10gillgYcnoLqIL23ST27BD9HhMsCJWb3Bpk= +github.com/mailru/easyjson v0.7.7 h1:UGYAvKxe3sBsEDzO8ZeWOSlIQfWFlxbzLZe7hwFURr0= +github.com/mailru/easyjson v0.7.7/go.mod h1:xzfreul335JAWq5oZzymOObrkdz5UnU4kGfJJLY9Nlc= +github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= +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.9/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= +github.com/mattn/go-colorable v0.1.12/go.mod h1:u5H1YNBxpqRaxsYJYSkiCWKzEfiAb1Gb520KVy5xxl4= +github.com/mattn/go-colorable v0.1.13 h1:fFA4WZxdEF4tXPZVKMLwD8oUnCTTo08duU7wxecdEvA= +github.com/mattn/go-colorable v0.1.13/go.mod h1:7S9/ev0klgBDR4GtXTXX8a3vIGJpMovkB8vQcUbaXHg= +github.com/mattn/go-isatty v0.0.3/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.11/go.mod h1:PhnuNfih5lzO57/f3n+odYbM4JtupLOxQOAqxQCu2WE= +github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU= +github.com/mattn/go-isatty v0.0.14/go.mod h1:7GGIvUiUoEMVVmxf/4nioHXj79iQHKdU27kJ6hsGG94= +github.com/mattn/go-isatty v0.0.16/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/yFXSvRLM= +github.com/mattn/go-isatty v0.0.20 h1:xfD0iDuEKnDkl03q4limB+vH+GxLEtL/jb4xVJSWWEY= +github.com/mattn/go-isatty v0.0.20/go.mod h1:W+V8PltTTMOvKvAeJH7IuucS94S2C6jfK/D7dTCTo3Y= +github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= +github.com/miekg/dns v1.1.26/go.mod h1:bPDLeHnStXmXAq1m/Ch/hvfNHr14JKNPMBo3VZKjuso= +github.com/miekg/dns v1.1.41/go.mod h1:p6aan82bvRIyn+zDIv9xYNUpwa73JcSh9BKwknJysuI= +github.com/miekg/dns v1.1.58 h1:ca2Hdkz+cDg/7eNF6V56jjzuZ4aCAE+DbVkILdQWG/4= +github.com/miekg/dns v1.1.58/go.mod h1:Ypv+3b/KadlvW9vJfXOTf300O4UqaHFzFCuHz+rPkBY= +github.com/mitchellh/cli v1.1.0/go.mod h1:xcISNoH86gajksDmfB23e/pu+B+GeFRMYmoHXxx3xhI= +github.com/mitchellh/copystructure v1.0.0 h1:Laisrj+bAB6b/yJwB5Bt3ITZhGJdqmxquMKeZ+mmkFQ= +github.com/mitchellh/copystructure v1.0.0/go.mod h1:SNtv71yrdKgLRyLFxmLdkAbkKEFWgYaq1OVrnRcwhnw= +github.com/mitchellh/go-homedir v1.1.0 h1:lukF9ziXFxDFPkA1vsr5zpc1XuPDn/wFntq5mG+4E0Y= +github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= +github.com/mitchellh/mapstructure v0.0.0-20160808181253-ca63d7c062ee/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= +github.com/mitchellh/mapstructure v1.5.0 h1:jeMsZIYE/09sWLaz43PL7Gy6RuMjD2eJVyuac5Z2hdY= +github.com/mitchellh/mapstructure v1.5.0/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo= +github.com/mitchellh/reflectwalk v1.0.0/go.mod h1:mSTlrgnPZtwu0c4WaC2kGObEpuNDbx0jmZXqmk4esnw= +github.com/mitchellh/reflectwalk v1.0.1 h1:FVzMWA5RllMAKIdUSC8mdWo3XtwoecrH79BY70sEEpE= +github.com/mitchellh/reflectwalk v1.0.1/go.mod h1:mSTlrgnPZtwu0c4WaC2kGObEpuNDbx0jmZXqmk4esnw= +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= +github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= +github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= +github.com/modern-go/reflect2 v1.0.2 h1:xBagoLtFs94CBntxluKeaWgTMpvLxC4ur3nMaC9Gz0M= +github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= +github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 h1:C3w9PqII01/Oq1c1nUAm88MOHcQC9l5mIlSMApZMrHA= +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= +github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLANZcx1PVRCS0qkT7pwLkGfwJo4zjcN/Tysno= +github.com/nxadm/tail v1.4.8 h1:nPr65rt6Y5JFSKQO7qToXr7pePgD6Gwiw05lkbyAQTE= +github.com/nxadm/tail v1.4.8/go.mod h1:+ncqLTQzXmGhMZNUePPaPqPvBxHAIsmXswZKocGu+AU= +github.com/oklog/ulid v1.3.1 h1:EGfNDEx6MqHz8B3uNV6QAib1UR2Lm97sHi3ocA6ESJ4= +github.com/oklog/ulid v1.3.1/go.mod h1:CirwcVhetQ6Lv90oh/F+FBtV6XMibvdAFo93nm5qn4U= +github.com/onsi/ginkgo v1.16.5 h1:8xi0RTUf59SOSfEtZMvwTvXYMzG4gV23XVHOZiXNtnE= +github.com/onsi/ginkgo v1.16.5/go.mod h1:+E8gABHa3K6zRBolWtd+ROzc/U5bkGt0FwiG042wbpU= +github.com/onsi/gomega v1.29.0 h1:KIA/t2t5UBzoirT4H9tsML45GEbo3ouUnBHsCfD2tVg= +github.com/onsi/gomega v1.29.0/go.mod h1:9sxs+SwGrKI0+PWe4Fxa9tFQQBG5xSsSbMXOI8PPpoQ= +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.2 h1:9yCKha/T5XdGtO0q9Q9a6T5NUCsTn/DrBg0D7ufOcFM= +github.com/opencontainers/image-spec v1.0.2/go.mod h1:BtxoFyWECRxE4U/7sNtV5W15zMzWCbyJoFRP3s7yZA0= +github.com/opentracing-contrib/go-grpc v0.0.0-20210225150812-73cb765af46e h1:4cPxUYdgaGzZIT5/j0IfqOrrXmq6bG8AwvwisMXpdrg= +github.com/opentracing-contrib/go-grpc v0.0.0-20210225150812-73cb765af46e/go.mod h1:DYR5Eij8rJl8h7gblRrOZ8g0kW1umSpKqYIBTgeDtLo= +github.com/opentracing-contrib/go-stdlib v1.0.0 h1:TBS7YuVotp8myLon4Pv7BtCBzOTo1DeZCld0Z63mW2w= +github.com/opentracing-contrib/go-stdlib v1.0.0/go.mod h1:qtI1ogk+2JhVPIXVc6q+NHziSmy2W5GbdQZFUHADCBU= +github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= +github.com/opentracing/opentracing-go v1.2.0 h1:uEJPy/1a5RIPAJ0Ov+OIO8OxWu77jEv+1B0VhjKrZUs= +github.com/opentracing/opentracing-go v1.2.0/go.mod h1:GxEUsuufX4nBwe+T+Wl9TAgYrxe9dPLANfrWvHYVTgc= +github.com/ovh/go-ovh v1.4.3 h1:Gs3V823zwTFpzgGLZNI6ILS4rmxZgJwJCz54Er9LwD0= +github.com/ovh/go-ovh v1.4.3/go.mod h1:AkPXVtgwB6xlKblMjRKJJmjRp+ogrE7fz2lVgcQY8SY= +github.com/pascaldekloe/goe v0.0.0-20180627143212-57f6aae5913c/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc= +github.com/pascaldekloe/goe v0.1.0 h1:cBOtyMzM9HTpWjXfbbunk26uA6nG3a8n06Wieeh0MwY= +github.com/pascaldekloe/goe v0.1.0/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc= +github.com/philhofer/fwd v1.1.1/go.mod h1:gk3iGcWd9+svBvR0sR+KPcfE+RNWozjowpeBVG3ZVNU= +github.com/pierrec/lz4/v4 v4.1.18 h1:xaKrnTkyoqfh1YItXl56+6KJNVYWlEEPuAQW9xsplYQ= +github.com/pierrec/lz4/v4 v4.1.18/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= +github.com/pires/go-proxyproto v0.7.0 h1:IukmRewDQFWC7kfnb66CSomk2q/seBuilHBYFwyq0Hs= +github.com/pires/go-proxyproto v0.7.0/go.mod h1:Vz/1JPY/OACxWGQNIRY2BeyDmpoaWmEP40O9LbuiFR4= +github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c h1:+mdjkGKdHQG3305AYmdv1U2eRNDiU2ErMBj1gwrq8eQ= +github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c/go.mod h1:7rwL4CYBLnjLxUqIJNnCWiEdr3bn6IUYi15bNlnbCCU= +github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= +github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= +github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= +github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= +github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 h1:Jamvg5psRIccs7FGNTlIRMkT8wgtp5eCXdBlqhYGL6U= +github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2/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/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= +github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= +github.com/prometheus/client_golang v1.4.0/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.11.0/go.mod h1:Z6t4BnS23TR94PD6BsDNk8yVqroYurpAkEiz0P2BEV0= +github.com/prometheus/client_golang v1.11.1/go.mod h1:Z6t4BnS23TR94PD6BsDNk8yVqroYurpAkEiz0P2BEV0= +github.com/prometheus/client_golang v1.19.0 h1:ygXvpU1AoN1MhdzckN+PyD9QJOSD4x7kmXYlnfbA6JU= +github.com/prometheus/client_golang v1.19.0/go.mod h1:ZRM9uEAypZakd+q/x7+gmsvXdURP+DABIEIjnmDdp+k= +github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= +github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/prometheus/client_model v0.6.0 h1:k1v3CzpSRUTrKMppY35TLwPvxHqBu0bYgxZzqGIgaos= +github.com/prometheus/client_model v0.6.0/go.mod h1:NTQHnmxFpouOD0DpvP4XujX3CdOAGQPoaGhyTchlyt8= +github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= +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.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.49.1-0.20240306132007-4199f18c3e92 h1:nuwTDY/15McImfuXcUD6AA3alpUNEXfWws8K/8SXr68= +github.com/prometheus/common v0.49.1-0.20240306132007-4199f18c3e92/go.mod h1:Kxm+EULxRbUkjGU6WFsQqo3ORzB4tyKvlWFOE9mB2sE= +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.11.0 h1:yNTsuZ0aNCNFQ3aFTD2uhPOvr4iD7fdBvKPAEGkNf+g= +github.com/prometheus/exporter-toolkit v0.11.0/go.mod h1:BVnENhnNecpwoTLiABx7mrPB/OLRIgN74qlQbV+FK1Q= +github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= +github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= +github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= +github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= +github.com/prometheus/procfs v0.6.0/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= +github.com/prometheus/procfs v0.12.0 h1:jluTpSng7V9hY0O2R9DzzJHYb2xULk9VTR1V1R/k6Bo= +github.com/prometheus/procfs v0.12.0/go.mod h1:pcuDEFsWDnvcgNzo4EEweacyhjeA9Zk3cnaOZAZEfOo= +github.com/prometheus/prometheus v0.51.0 h1:aRdjTnmHLved29ILtdzZN2GNvOjWATtA/z+3fYuexOc= +github.com/prometheus/prometheus v0.51.0/go.mod h1:yv4MwOn3yHMQ6MZGHPg/U7Fcyqf+rxqiZfSur6myVtc= +github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= +github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= +github.com/rogpeppe/go-internal v1.12.0 h1:exVL4IDcn6na9z1rAb56Vxr+CgyK3nn3O+epU5NdKM8= +github.com/rogpeppe/go-internal v1.12.0/go.mod h1:E+RYuTGaKKdloAfM02xzb0FW3Paa99yedzYV+kq4uf4= +github.com/ryanuber/columnize v0.0.0-20160712163229-9b3edd62028f/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= +github.com/scaleway/scaleway-sdk-go v1.0.0-beta.25 h1:/8rfZAdFfafRXOgz+ZpMZZWZ5pYggCY9t7e/BvjaBHM= +github.com/scaleway/scaleway-sdk-go v1.0.0-beta.25/go.mod h1:fCa7OJZ/9DRTnOKmxvT6pn+LPWUptQAmHF/SBJUGEcg= +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/segmentio/fasthash v1.0.3 h1:EI9+KE1EwvMLBWwjpRDc+fEM+prwxDYbslddQGtrmhM= +github.com/segmentio/fasthash v1.0.3/go.mod h1:waKX8l2N8yckOgmSsXJi7x1ZfdKZ4x7KRMzBtS3oedY= +github.com/sercand/kuberesolver/v5 v5.1.1 h1:CYH+d67G0sGBj7q5wLK61yzqJJ8gLLC8aeprPTHb6yY= +github.com/sercand/kuberesolver/v5 v5.1.1/go.mod h1:Fs1KbKhVRnB2aDWN12NjKCB+RgYMWZJ294T3BtmVCpQ= +github.com/shopspring/decimal v1.2.0 h1:abSATXmQEYyShuxI4/vyW3tV1MrKAJzCZ/0zLUXYbsQ= +github.com/shopspring/decimal v1.2.0/go.mod h1:DKyhrW/HYNuLGql+MJL6WCR6knT2jwCFRcu2hWCYk4o= +github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= +github.com/sirupsen/logrus v1.4.2/go.mod h1:tLMulIdttU9McNUspp0xgXVQah82FyeX6MwdIuYE2rE= +github.com/sirupsen/logrus v1.6.0/go.mod h1:7uNnSEd1DgxDLC74fIahvMZmmYsHGZGEOFrfsX/uA88= +github.com/sony/gobreaker v0.5.0 h1:dRCvqm0P490vZPmy7ppEk2qCnCieBooFJ+YoXGYB+yg= +github.com/sony/gobreaker v0.5.0/go.mod h1:ZKptC7FHNvhBz7dN2LGjPVBz2sZJmc0/PkyDJOjmxWY= +github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= +github.com/spaolacci/murmur3 v1.1.0 h1:7c1g84S4BPRrfL5Xrdp6fOJ206sU9y293DDHaoy0bLI= +github.com/spaolacci/murmur3 v1.1.0/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= +github.com/spf13/cast v1.3.1 h1:nFm6S0SMdyzrzcmThSipiEubIDy8WEXKNZ0UOgiRpng= +github.com/spf13/cast v1.3.1/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= +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/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.5.2 h1:xuMeJ0Sdp5ZMRXx/aWO6RZxdr3beISkG5/G/aIRr3pY= +github.com/stretchr/objx v0.5.2/go.mod h1:FRsXN1f5AsAjCGJKqEizvkpNtU+EGNCLh3NxZ/8L+MA= +github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= +github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= +github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= +github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= +github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= +github.com/stretchr/testify v1.7.2/go.mod h1:R6va5+xMeoiuVRoj+gSkQ7d3FALtqAAGI1FQKckRals= +github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= +github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= +github.com/tinylib/msgp v1.1.5/go.mod h1:eQsjooMTnV42mHu917E26IogZ2930nFyBQdofk10Udg= +github.com/ttacon/chalk v0.0.0-20160626202418-22c06c80ed31/go.mod h1:onvgF043R+lC5RZ8IT9rBXDaEDnpnw/Cl+HFiw+v/7Q= +github.com/tv42/httpunix v0.0.0-20150427012821-b75d8614f926/go.mod h1:9ESjWnEqriFuLhtthL60Sar/7RFoluCcXsuvEwTV5KM= +github.com/uber/jaeger-client-go v2.30.0+incompatible h1:D6wyKGCecFaSRUpo8lCVbaOOb6ThwMmTEbhRwtKR97o= +github.com/uber/jaeger-client-go v2.30.0+incompatible/go.mod h1:WVhlPFC8FDjOFMMWRy2pZqQJSXxYSwNYOkTr/Z6d3Kk= +github.com/uber/jaeger-lib v2.4.1+incompatible h1:td4jdvLcExb4cBISKIpHuGoVXh+dVKhn2Um6rjCsSsg= +github.com/uber/jaeger-lib v2.4.1+incompatible/go.mod h1:ComeNDZlWwrWnDv8aPp0Ba6+uUTzImX/AauajbLI56U= +github.com/vultr/govultr/v2 v2.17.2 h1:gej/rwr91Puc/tgh+j33p/BLR16UrIPnSr+AIwYWZQs= +github.com/vultr/govultr/v2 v2.17.2/go.mod h1:ZFOKGWmgjytfyjeyAdhQlSWwTjh2ig+X49cAp50dzXI= +github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/yuin/goldmark v1.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.13/go.mod h1:6yULJ656Px+3vBD8DxQVa3kxgyrAnzto9xy5taEt/CY= +github.com/yuin/gopher-lua v1.1.0 h1:BojcDhfyDWgU2f2TOzYK/g5p2gxMrku8oupLDqlnSqE= +github.com/yuin/gopher-lua v1.1.0/go.mod h1:GBR0iDaNXjAgGg9zfCvksxSRnQx76gclCIb7kdAd1Pw= +go.etcd.io/etcd/api/v3 v3.5.4 h1:OHVyt3TopwtUQ2GKdd5wu3PmmipR4FTwCqoEjSyRdIc= +go.etcd.io/etcd/api/v3 v3.5.4/go.mod h1:5GB2vv4A4AOn3yk7MftYGHkUfGtDHnEraIjym4dYz5A= +go.etcd.io/etcd/client/pkg/v3 v3.5.4 h1:lrneYvz923dvC14R54XcA7FXoZ3mlGZAgmwhfm7HqOg= +go.etcd.io/etcd/client/pkg/v3 v3.5.4/go.mod h1:IJHfcCEKxYu1Os13ZdwCwIUTUVGYTSAM3YSwc9/Ac1g= +go.etcd.io/etcd/client/v3 v3.5.4 h1:p83BUL3tAYS0OT/r0qglgc3M1JjhM0diV8DSWAhVXv4= +go.etcd.io/etcd/client/v3 v3.5.4/go.mod h1:ZaRkVgBZC+L+dLCjTcF1hRXpgZXQPOvnA/Ak/gq3kiY= +go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= +go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= +go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= +go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= +go.opencensus.io v0.22.4/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= +go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.49.0 h1:jq9TW8u3so/bN+JPT166wjOI6/vQPF6Xe7nMNIltagk= +go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.49.0/go.mod h1:p8pYQP+m5XfbZm9fxtSKAbM6oIllS7s2AfxrChvc7iw= +go.opentelemetry.io/otel v1.24.0 h1:0LAOdjNmQeSTzGBzduGe/rU4tZhMwL5rWgtp9Ku5Jfo= +go.opentelemetry.io/otel v1.24.0/go.mod h1:W7b9Ozg4nkF5tWI5zsXkaKKDjdVjpD4oAt9Qi/MArHo= +go.opentelemetry.io/otel/metric v1.24.0 h1:6EhoGWWK28x1fbpA4tYTOWBkPefTDQnb8WSGXlc88kI= +go.opentelemetry.io/otel/metric v1.24.0/go.mod h1:VYhLe1rFfxuTXLgj4CBiyz+9WYBA8pNGJgDcSFRKBco= +go.opentelemetry.io/otel/trace v1.24.0 h1:CsKnnL4dUAr/0llH9FKuc698G04IrpWV0MQA/Y1YELI= +go.opentelemetry.io/otel/trace v1.24.0/go.mod h1:HPc3Xr/cOApsBI154IU0OI0HJexz+aw5uPdbs3UCjNU= +go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= +go.uber.org/atomic v1.11.0 h1:ZvwS0R+56ePWxUNi+Atn9dWONBPp/AUETXlHW0DxSjE= +go.uber.org/atomic v1.11.0/go.mod h1:LUxbIzbOniOlMKjJjyPfpl4v+PKK2cNJn91OQbhoJI0= +go.uber.org/goleak v1.1.11/go.mod h1:cwTWslyiVhfpKIDGSZEM2HlOvcqm+tG4zioyIeLoqMQ= +go.uber.org/goleak v1.3.0 h1:2K3zAYmnTNqV73imy9J1T3WC+gmCePx2hEGkimedGto= +go.uber.org/goleak v1.3.0/go.mod h1:CoHD4mav9JJNrW/WLlf7HGZPjdw8EucARQHekz1X6bE= +go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= +go.uber.org/multierr v1.11.0 h1:blXXJkSxSSfBVBlC76pxqeO+LN3aDfLQo+309xJstO0= +go.uber.org/multierr v1.11.0/go.mod h1:20+QtiLqy0Nd6FdQB9TLXag12DsQkrbs3htMFfDN80Y= +go.uber.org/zap v1.17.0/go.mod h1:MXVU+bhUf/A7Xi2HNOnopQOrmycQ5Ih87HtOu4q5SSo= +go.uber.org/zap v1.21.0 h1:WefMeulhovoZ2sYXz7st6K0sLj7bBhpiFaud4r4zST8= +go.uber.org/zap v1.21.0/go.mod h1:wjWOCqI0f2ZZrJF/UufIOkiC8ii6tm1iqIsLo76RfJw= +go4.org/netipx v0.0.0-20230125063823-8449b0a6169f h1:ketMxHg+vWm3yccyYiq+uK8D3fRmna2Fcj+awpQp84s= +go4.org/netipx v0.0.0-20230125063823-8449b0a6169f/go.mod h1:tgPU4N2u9RByaTN3NC2p9xOzyFpte4jYwsIIRF7XlSc= +golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= +golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20190605123033-f99c8df09eb5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20190923035154-9ee001bba392/go.mod h1:/lpIB1dKB+9EgE3H3cr1v9wB50oz8l4C4h62xy7jSTY= +golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= +golang.org/x/crypto v0.3.0/go.mod h1:hebNnKkNXi2UzZN1eVRvBB7co0a+JxK6XbPiWVs/3J4= +golang.org/x/crypto v0.24.0 h1:mnl8DM0o513X8fdIkmyFE/5hTYxbwYOjDS/+rK6qpRI= +golang.org/x/crypto v0.24.0/go.mod h1:Z1PMYSOR5nyMcyAVAIQSKCDwalqy85Aqn1x3Ws4L5DM= +golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20190125153040-c74c464bbbf2/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= +golang.org/x/exp v0.0.0-20190829153037-c13cbed26979/go.mod h1:86+5VVa7VpoJ4kLfm080zCjGlMRFzhUhsZKEZO7MGek= +golang.org/x/exp v0.0.0-20191030013958-a1ab85dbe136/go.mod h1:JXzH8nQsPlswgeRAPE3MuO9GYsAcnJvJ4vnMwN/5qkY= +golang.org/x/exp v0.0.0-20191129062945-2f5052295587/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= +golang.org/x/exp v0.0.0-20191227195350-da58074b4299/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= +golang.org/x/exp v0.0.0-20200119233911-0405dc783f0a/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= +golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EHIKF9dgMWnmCNThgcyBT1FY9mM= +golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= +golang.org/x/exp v0.0.0-20240325151524-a685a6edb6d8 h1:aAcj0Da7eBAtrTp03QXWvm88pSyOt+UgdZw2BFZ+lEw= +golang.org/x/exp v0.0.0-20240325151524-a685a6edb6d8/go.mod h1:CQ1k9gNrJ50XIzaKCRR2hssIjF07kZFEiieALBM/ARQ= +golang.org/x/image v0.0.0-20180708004352-c73c2afc3b81/go.mod h1:ux5Hcp/YLpHSI86hEcLt0YII63i6oz57MZXIpbrjZUs= +golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= +golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= +golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= +golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= +golang.org/x/lint v0.0.0-20190301231843-5614ed5bae6f/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= +golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20190409202823-959b441ac422/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20190909230951-414d861bb4ac/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20191125180803-fdd1cda4f05f/go.mod h1:5qLYkcX4OjUUV8bRuDixDT3tpyyb+LUpUlRWLxfhWrs= +golang.org/x/lint v0.0.0-20200130185559-910be7a94367/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= +golang.org/x/lint v0.0.0-20200302205851-738671d3881b/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= +golang.org/x/lint v0.0.0-20210508222113-6edffad5e616/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= +golang.org/x/mobile v0.0.0-20190312151609-d3739f865fa6/go.mod h1:z+o9i4GpDbdi3rU15maQ/Ox0txvL9dWGYEHz965HBQE= +golang.org/x/mobile v0.0.0-20190719004257-d2bd2a29d028/go.mod h1:E/iHnbuqvinMTCcRqshq8CkpyQDoeVncDDYHnLhea+o= +golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKGUJ2LatrhH/nqhxcFungHvyanc= +golang.org/x/mod v0.1.0/go.mod h1:0QHyrYULN0/3qlju5TqG8bIK38QM8yzMo5ekMj3DlcY= +golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= +golang.org/x/mod v0.1.1-0.20191107180719-034126e5016b/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= +golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= +golang.org/x/mod v0.17.0 h1:zY54UmvipHiNd+pm+m0x9KhZ9hl1/7QNMyxXbc6ICqA= +golang.org/x/mod v0.17.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= +golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190501004415-9ce7a6920f09/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190603091049-60506f45cf65/go.mod h1:HSz+uSET+XFnRR8LxR5pz3Of3rY3CfYBVs4xY44aLks= +golang.org/x/net v0.0.0-20190613194153-d28f0bde5980/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20190628185345-da137c7871d7/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20190724013045-ca1201d0de80/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20190921015927-1a5e07d1ff72/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20190923162816-aa69164e4478/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20191209160850-c0dbc17a3553/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200114155413-6afb5195e5aa/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200202094626-16171245cfb2/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200222125558-5a598a2470a0/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200301022130-244492dfa37a/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200324143707-d3edc9973b7e/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= +golang.org/x/net v0.0.0-20200501053045-e0ff5e5a1de5/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= +golang.org/x/net v0.0.0-20200506145744-7e3656a0809f/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= +golang.org/x/net v0.0.0-20200513185701-a91f0712d120/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= +golang.org/x/net v0.0.0-20200520182314-0ba52f642ac2/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= +golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= +golang.org/x/net v0.0.0-20200707034311-ab3426394381/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= +golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= +golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= +golang.org/x/net v0.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= +golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= +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-20210525063256-abc453219eb5/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= +golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= +golang.org/x/net v0.2.0/go.mod h1:KqCZLdyyvdV855qA2rE3GC2aiw5xGR5TEjj8smXukLY= +golang.org/x/net v0.26.0 h1:soB7SVo0PWrY4vPW/+ay0jKDNScG2X9wFeYlXIvJsOQ= +golang.org/x/net v0.26.0/go.mod h1:5YKkiSynbBIh3p6iOc/vibscux0x38BZDkn8sCUPxHE= +golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= +golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20191202225959-858c2ad4c8b6/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= +golang.org/x/oauth2 v0.0.0-20210514164344-f6687ab2804c/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= +golang.org/x/oauth2 v0.18.0 h1:09qnuIAgzdx1XplqJvW6CQqMCtGZykZWcXzPMPUusvI= +golang.org/x/oauth2 v0.18.0/go.mod h1:Wf7knwG0MPoWIMMBgFlEaSUDaKskp0dCfrlJRJXbBi8= +golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20200317015054-43a5402ce75a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.7.0 h1:YsImfSBoP9QPYL0xyKJPq0gcaJdG3rInoqxTWbfQu9M= +golang.org/x/sync v0.7.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sys v0.0.0-20180823144017-11551d06cbcc/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190222072716-a9d3bda3a223/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190502145724-3ef323f4f1fd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190507160741-ecd444e8653b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190606165138-5da285871e9c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190624142023-c5567b49c5d0/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190726091711-fc99dfbffb4e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190922100055-0a153f010e69/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190924154521-2837fb4f24fe/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191001151750-bb3f8db39f24/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191026070338-33540a1f6037/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191204072324-ce4227a45e2e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191228213918-04cbcbbfeed8/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200106162015-b016eb3dc98e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200113162924-86b910548bc1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200116001909-b77594299b42/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200122134326-e047566fdf82/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200202164722-d101bd2416d5/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200212091648-12a6c2dcc1e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200223170610-d5e6a3e2c0ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200302150141-5c8b2ff67527/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200331124033-c3d80250170d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200501052902-10377860bb8e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200511232937-7e40ca221e25/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200515095857-1151b9dac4a9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200523222454-059865788121/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200615200032-f1bc736245b1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200625212154-ddb9806d33ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200803210538-64077c9b5642/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210303074136-134d130e1a04/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-20210423082822-04245dca01da/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-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210927094055-39ccf1dd6fa6/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220503163025-988cb79eb6c6/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220728004956-3c1f35247d10/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.1.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.2.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.21.0 h1:rF+pYz3DAGSQAxAu1CbC7catZg4ebC4UIeIhKxBZvws= +golang.org/x/sys v0.21.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= +golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= +golang.org/x/term v0.2.0/go.mod h1:TVmDHMZPmdnySmBfhjOoOdhjzdE1h4u1VwSiw2l1Nuc= +golang.org/x/term v0.21.0 h1:WVXCp+/EBEHOj53Rvu+7KiT/iElMrO8ACK16SMZ3jaA= +golang.org/x/term v0.21.0/go.mod h1:ooXLefLobQVslOqselCNF4SxFAaoS6KujMbsGzSDmX0= +golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= +golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= +golang.org/x/text v0.3.8/go.mod h1:E6s5w1FMmriuDzIBO73fBruAKo1PCIq6d2Q6DHfQ8WQ= +golang.org/x/text v0.4.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= +golang.org/x/text v0.16.0 h1:a94ExnEXNtEwYLGJSIUxnWoxoRz/ZcCsV63ROupILh4= +golang.org/x/text v0.16.0/go.mod h1:GhwF1Be+LQoKShO3cGOHzqOgRrGaYc9AvblQOmPVHnI= +golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.5.0 h1:o7cqy6amK/52YcAKIPlM3a+Fpj35zvRj2TP+e1xFSfk= +golang.org/x/time v0.5.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= +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= +golang.org/x/tools v0.0.0-20181030221726-6c7e314b6563/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190206041539-40960b6deb8e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= +golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190312151545-0bb0c0a6e846/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190312170243-e65039ee4138/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190425150028-36563e24a262/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20190506145303-2d16b83fe98c/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20190606124116-d0a3d012864b/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= +golang.org/x/tools v0.0.0-20190621195816-6e04913cbbac/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= +golang.org/x/tools v0.0.0-20190628153133-6cdbf07be9d0/go.mod h1:/rFqwRUd4F7ZHNgwSSTFct+R/Kf4OFW1sUzUTQQTgfc= +golang.org/x/tools v0.0.0-20190816200558-6889da9d5479/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20190907020128-2ca718005c18/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20190911174233-4f2ddba30aff/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191012152004-8de300cfc20a/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191113191852-77e3bb0ad9e7/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191115202509-3a792d9c32b2/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191125144606-a911d9008d1f/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191130070609-6e064ea0cf2d/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191216173652-a0e659d51361/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20191227053925-7b8e75db28f4/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200117161641-43d50277825c/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200122220014-bf1340f18c4a/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200130002326-2f3ba24bd6e7/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200204074204-1cc6d1ef6c74/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200207183749-b753a1ba74fa/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200212150539-ea181f53ac56/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200224181240-023911ca70b2/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200227222343-706bc42d1f0d/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= +golang.org/x/tools v0.0.0-20200304193943-95d2e580d8eb/go.mod h1:o4KQGtdN14AW+yjsvvwRTJJuXz8XRtIHtEnmAXLyFUw= +golang.org/x/tools v0.0.0-20200312045724-11d5b4c81c7d/go.mod h1:o4KQGtdN14AW+yjsvvwRTJJuXz8XRtIHtEnmAXLyFUw= +golang.org/x/tools v0.0.0-20200331025713-a30bf2db82d4/go.mod h1:Sl4aGygMT6LrqrWclx+PTx3U+LnKx/seiNR+3G19Ar8= +golang.org/x/tools v0.0.0-20200501065659-ab2804fb9c9d/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= +golang.org/x/tools v0.0.0-20200512131952-2bc93b1c0c88/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= +golang.org/x/tools v0.0.0-20200515010526-7d3b6ebf133d/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= +golang.org/x/tools v0.0.0-20200618134242-20370b0cb4b2/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= +golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= +golang.org/x/tools v0.0.0-20200729194436-6467de6f59a7/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= +golang.org/x/tools v0.0.0-20200804011535-6c149bb5ef0d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= +golang.org/x/tools v0.0.0-20200825202427-b303f430e36d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= +golang.org/x/tools v0.0.0-20201022035929-9cf592e881e9/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= +golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= +golang.org/x/tools v0.1.2/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= +golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= +golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= +golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d h1:vU5i/LfpvrRCpgM/VPfJLg5KjxD3E+hfT1SH+d9zLwg= +golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d/go.mod h1:aiJjzUbINMkxbQROHiO6hDPo2LHcIPhhQsa9DLh0yGk= +golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +gonum.org/v1/gonum v0.0.0-20180816165407-929014505bf4/go.mod h1:Y+Yx5eoAFn32cQvJDxZx5Dpnq+c3wtXuadVZAcxbbBo= +gonum.org/v1/gonum v0.8.2/go.mod h1:oe/vMfY3deqTw+1EZJhuvEW2iwGF1bW9wwu7XCu0+v0= +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.4.0/go.mod h1:8k5glujaEP+g9n7WNsDg8QP6cUVNI86fCNMcbazEtwE= +google.golang.org/api v0.7.0/go.mod h1:WtwebWUNSVBH/HAw79HIFXZNqEvBhG+Ra+ax0hx3E3M= +google.golang.org/api v0.8.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= +google.golang.org/api v0.9.0/go.mod h1:o4eAsZoiT+ibD93RtjEohWalFOjRDx6CVaqeizhEnKg= +google.golang.org/api v0.13.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= +google.golang.org/api v0.14.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= +google.golang.org/api v0.15.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= +google.golang.org/api v0.17.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/api v0.18.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/api v0.19.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/api v0.20.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/api v0.22.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= +google.golang.org/api v0.24.0/go.mod h1:lIXQywCXRcnZPGlsd8NbLnOjtAoL6em04bJ9+z0MncE= +google.golang.org/api v0.28.0/go.mod h1:lIXQywCXRcnZPGlsd8NbLnOjtAoL6em04bJ9+z0MncE= +google.golang.org/api v0.29.0/go.mod h1:Lcubydp8VUV7KeIHD9z2Bys/sm/vGKnG1UHuDBSrHWM= +google.golang.org/api v0.30.0/go.mod h1:QGmEvQ87FHZNiUVJkT14jQNYJ4ZJjdRF23ZXz5138Fc= +google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= +google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/appengine v1.6.1/go.mod h1:i06prIuMbXzDqacNJfV5OdTW448YApPu5ww/cMBSeb0= +google.golang.org/appengine v1.6.5/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= +google.golang.org/appengine v1.6.6/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc= +google.golang.org/appengine v1.6.8 h1:IhEN5q69dyKagZPYMSdIjS2HqprW324FRQZJcGqPAsM= +google.golang.org/appengine v1.6.8/go.mod h1:1jJ3jBArFh5pcgW8gCtRJnepW8FzD1V44FJffLiz/Ds= +google.golang.org/genproto v0.0.0-20180518175338-11a468237815/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= +google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190425155659-357c62f0e4bb/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190502173448-54afdca5d873/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= +google.golang.org/genproto v0.0.0-20190801165951-fa694d86fc64/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +google.golang.org/genproto v0.0.0-20190911173649-1774047e7e51/go.mod h1:IbNlFCBrqXvoKpeg0TB2l7cyZUmoaFKYIwrEpbDKLA8= +google.golang.org/genproto v0.0.0-20191108220845-16a3f7862a1a/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20191115194625-c23dd37a84c9/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20191216164720-4f79533eabd1/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20191230161307-f3c370f40bfb/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20200115191322-ca5a22157cba/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20200122232147-0452cf42e150/go.mod h1:n3cpQtvxv34hfy77yVDNjmbRyujviMdxYliBSkLhpCc= +google.golang.org/genproto v0.0.0-20200204135345-fa8e72b47b90/go.mod h1:GmwEX6Z4W5gMy59cAlVYjN9JhxgbQH6Gn+gFDQe2lzA= +google.golang.org/genproto v0.0.0-20200212174721-66ed5ce911ce/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200224152610-e50cd9704f63/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200228133532-8c2c7df3a383/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200305110556-506484158171/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200312145019-da6875a35672/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200331122359-1ee6d9798940/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200430143042-b979b6f78d84/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200511104702-f5ebc3bea380/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200513103714-09dca8ec2884/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200515170657-fc4c6c6a6587/go.mod h1:YsZOwe1myG/8QRHRsmBRE1LrgQY60beZKjly0O1fX9U= +google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= +google.golang.org/genproto v0.0.0-20200618031413-b414f8b61790/go.mod h1:jDfRM7FcilCzHH/e9qn6dsT145K34l5v+OpcnNgKAAA= +google.golang.org/genproto v0.0.0-20200729003335-053ba62fc06f/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20200804131852-c06518451d9c/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20200825200019-8632dd797987/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no= +google.golang.org/genproto v0.0.0-20210602131652-f16073e35f0c/go.mod h1:UODoCrxHCcBojKKwX1terBiRUaqAsFqJiF615XL43r0= +google.golang.org/genproto/googleapis/api v0.0.0-20240304212257-790db918fca8 h1:8eadJkXbwDEMNwcB5O0s5Y5eCfyuCLdvaiOIaGTrWmQ= +google.golang.org/genproto/googleapis/api v0.0.0-20240304212257-790db918fca8/go.mod h1:O1cOfN1Cy6QEYr7VxtjOyP5AdAuR0aJ/MYZaaof623Y= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240304161311-37d4d3c04a78 h1:Xs9lu+tLXxLIfuci70nG4cpwaRC+mRQPUL7LoIeDJC4= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240304161311-37d4d3c04a78/go.mod h1:UCOku4NytXMJuLQE5VuqA5lX3PcHCBo8pxNyvkf4xBs= +google.golang.org/grpc v1.12.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= +google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= +google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= +google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= +google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= +google.golang.org/grpc v1.23.1/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= +google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= +google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.27.1/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.28.0/go.mod h1:rpkK4SK4GF4Ach/+MFLZUBavHOvF2JJB5uozKKal+60= +google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= +google.golang.org/grpc v1.30.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= +google.golang.org/grpc v1.31.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak= +google.golang.org/grpc v1.33.1/go.mod h1:fr5YgcSWrqhRRxogOsw7RzIpsmvOZ6IcH4kBYTpR3n0= +google.golang.org/grpc v1.38.0/go.mod h1:NREThFqKR1f3iQ6oBuvc5LadQuXVGo9rkm5ZGrQdJfM= +google.golang.org/grpc v1.62.1 h1:B4n+nfKzOICUXMgyrNd19h/I9oH0L1pizfk1d4zSgTk= +google.golang.org/grpc v1.62.1/go.mod h1:IWTG0VlJLCh1SkC58F7np9ka9mx/WNkjl4PGJaiq+QE= +google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= +google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= +google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= +google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE= +google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= +google.golang.org/protobuf v1.22.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= +google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= +google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= +google.golang.org/protobuf v1.24.0/go.mod h1:r/3tXBNzIEhYS9I1OUVjXDlt8tc493IdKGjtUeSXeh4= +google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= +google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= +google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= +google.golang.org/protobuf v1.33.0 h1:uNO2rsAINq/JlFpSdYEKIZ0uKD/R9cpdv0T+yoGwGmI= +google.golang.org/protobuf v1.33.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= +gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= +gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20200227125254-8fa46927fb4f/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= +gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= +gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= +gopkg.in/inf.v0 v0.9.1 h1:73M5CoZyi3ZLMOyDlQh031Cx6N9NDJ2Vvfl76EDAgDc= +gopkg.in/inf.v0 v0.9.1/go.mod h1:cWUDdTG/fYaXco+Dcufb5Vnc6Gp2YChqWtbxRZE0mXw= +gopkg.in/ini.v1 v1.67.0 h1:Dgnx+6+nfE+IfzjUEISNeydPJh9AXNNsWbGP9KzCsOA= +gopkg.in/ini.v1 v1.67.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= +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/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.3/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.5/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= +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-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= +gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= +honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= +honnef.co/go/tools v0.0.1-2020.1.4/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= +k8s.io/api v0.29.2 h1:hBC7B9+MU+ptchxEqTNW2DkUosJpp1P+Wn6YncZ474A= +k8s.io/api v0.29.2/go.mod h1:sdIaaKuU7P44aoyyLlikSLayT6Vb7bvJNCX105xZXY0= +k8s.io/apimachinery v0.29.2 h1:EWGpfJ856oj11C52NRCHuU7rFDwxev48z+6DSlGNsV8= +k8s.io/apimachinery v0.29.2/go.mod h1:6HVkd1FwxIagpYrHSwJlQqZI3G9LfYWRPAkUvLnXTKU= +k8s.io/client-go v0.29.2 h1:FEg85el1TeZp+/vYJM7hkDlSTFZ+c5nnK44DJ4FyoRg= +k8s.io/client-go v0.29.2/go.mod h1:knlvFZE58VpqbQpJNbCbctTVXcd35mMyAAwBdpt4jrA= +k8s.io/klog/v2 v2.120.1 h1:QXU6cPEOIslTGvZaXvFWiP9VKyeet3sawzTOvdXb4Vw= +k8s.io/klog/v2 v2.120.1/go.mod h1:3Jpz1GvMt720eyJH1ckRHK1EDfpxISzJ7I9OYgaDtPE= +k8s.io/kube-openapi v0.0.0-20231010175941-2dd684a91f00 h1:aVUu9fTY98ivBPKR9Y5w/AuzbMm96cd3YHRTU83I780= +k8s.io/kube-openapi v0.0.0-20231010175941-2dd684a91f00/go.mod h1:AsvuZPBlUDVuCdzJ87iajxtXuR9oktsTctW/R9wwouA= +k8s.io/utils v0.0.0-20230726121419-3b25d923346b h1:sgn3ZU783SCgtaSJjpcVVlRqd6GSnlTLKgpAAttJvpI= +k8s.io/utils v0.0.0-20230726121419-3b25d923346b/go.mod h1:OLgZIPagt7ERELqWJFomSt595RzquPNLL48iOWgYOg0= +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/json v0.0.0-20221116044647-bc3834ca7abd h1:EDPBXCAspyGV4jQlpZSudPeMmr1bNJefnuqLsRAsHZo= +sigs.k8s.io/json v0.0.0-20221116044647-bc3834ca7abd/go.mod h1:B8JuhiUyNFVKdsE8h686QcCxMaH6HrOAZj4vswFpcB0= +sigs.k8s.io/structured-merge-diff/v4 v4.4.1 h1:150L+0vs/8DA78h1u02ooW1/fFq/Lwr+sGiqlzvrtq4= +sigs.k8s.io/structured-merge-diff/v4 v4.4.1/go.mod h1:N8hJocpFajUSSeSJ9bOZ77VzejKZaXsTtZo4/u7Io08= +sigs.k8s.io/yaml v1.2.0/go.mod h1:yfXDCHCao9+ENCvLSE62v9VSji2MKu5jeNfTrofGhJc= +sigs.k8s.io/yaml v1.3.0 h1:a2VclLzOGrwOHDiV8EfBGhvjHvP46CtW5j6POvhYGGo= +sigs.k8s.io/yaml v1.3.0/go.mod h1:GeOyir5tyXNByN85N/dRIT9es5UQNerPYEKK56eTBm8= diff --git a/cmd/segment-inspect/main.go b/cmd/segment-inspect/main.go new file mode 100644 index 000000000000..118f2937da78 --- /dev/null +++ b/cmd/segment-inspect/main.go @@ -0,0 +1,108 @@ +package main + +import ( + "context" + "flag" + "fmt" + "io" + "log" + "os" + "time" + + "github.com/dustin/go-humanize" + + "github.com/grafana/loki/v3/pkg/storage/wal" +) + +const format = "2006-01-02 15:04:05.000000 MST" + +func main() { + streams := flag.Bool("s", false, "print streams") + flag.Parse() + + for _, f := range flag.Args() { + printFile(f, *streams) + } +} + +func printFile(filename string, segmentDetails bool) { + f, err := os.Open(filename) + if err != nil { + log.Printf("%s: %v", filename, err) + return + } + defer func() { _ = f.Close() }() + + segmentBytes, err := io.ReadAll(f) + if err != nil { + log.Printf("failed to read file: %v", err) + return + } + reader, err := wal.NewReader(segmentBytes) + if err != nil { + log.Printf("failed to open segment reader: %v", err) + return + } + + iter, err := reader.Series(context.Background()) + if err != nil { + log.Printf("failed to open series iterator: %v", err) + return + } + + segmentFrom := time.Now().Add(time.Hour) + var segmentTo time.Time + + var actualSeries []string + tenants := make(map[string]int) + + for iter.Next() { + actualSeries = append(actualSeries, iter.At().String()) + tenant := iter.At().Get("__loki_tenant__") + tenants[tenant]++ + + chk, err := iter.ChunkReader(nil) + if err != nil { + log.Printf("failed to open chunk reader: %v", err) + return + } + for chk.Next() { + ts, _ := chk.At() + if segmentFrom.After(time.Unix(0, ts)) { + segmentFrom = time.Unix(0, ts) + } + if segmentTo.Before(time.Unix(0, ts)) { + segmentTo = time.Unix(0, ts) + } + } + } + + sizes, err := reader.Sizes() + if err != nil { + log.Printf("failed to get segment sizes: %v", err) + return + } + seriesSize := int64(0) + indexSize := sizes.Index + for _, size := range sizes.Series { + seriesSize += size + } + + fmt.Println() + fmt.Println("Segment file:", filename) + fmt.Println("Compressed Filesize:", humanize.Bytes(uint64(len(segmentBytes)))) + fmt.Println("Series Size:", humanize.Bytes(uint64(seriesSize))) + fmt.Println("Index Size:", humanize.Bytes(uint64(indexSize))) + fmt.Println("Stream count:", len(actualSeries)) + fmt.Println("Tenant count:", len(tenants)) + fmt.Println("From:", segmentFrom.UTC().Format(format)) + fmt.Println("To:", segmentTo.UTC().Format(format)) + fmt.Println("Duration:", segmentTo.Sub(segmentFrom)) + + if segmentDetails { + fmt.Println() + for _, s := range actualSeries { + fmt.Println(s) + } + } +} From aeb23bb7fc3d33327060828ddf97cb7da7b3c8f8 Mon Sep 17 00:00:00 2001 From: Salva Corts Date: Wed, 17 Jul 2024 13:21:34 +0200 Subject: [PATCH 13/23] fix: Fix log line for fingerprint not found (#13555) --- pkg/storage/bloom/v1/fuse.go | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/pkg/storage/bloom/v1/fuse.go b/pkg/storage/bloom/v1/fuse.go index 2a6f74d590b8..ace67b0496c2 100644 --- a/pkg/storage/bloom/v1/fuse.go +++ b/pkg/storage/bloom/v1/fuse.go @@ -237,7 +237,12 @@ func (fq *FusedQuerier) Run() error { series := fq.bq.At() if series.Fingerprint != fp { // fingerprint not found, can't remove chunks - level.Debug(fq.logger).Log("msg", "fingerprint not found", "fp", series.Fingerprint, "err", fq.bq.Err()) + level.Debug(fq.logger).Log( + "msg", "fingerprint not found", + "fp", fp, + "foundFP", series.Fingerprint, + "err", fq.bq.Err(), + ) fq.recordMissingFp(nextBatch, fp) continue } From 7aa7c15f21f43805a0244a47d308b3dd02a10b7a Mon Sep 17 00:00:00 2001 From: Christian Haudum Date: Wed, 17 Jul 2024 13:48:02 +0200 Subject: [PATCH 14/23] chore: Use filesystem backed writer for blooms (#13522) The DirectoryBlockWriter and DirectoryBlockReader are used to avoid OOMing of compactors/builders. The tradeoff is that the writer/reader needs to be cleaned up and that it is I/O bound. Signed-off-by: Christian Haudum --- integration/bloom_building_test.go | 1 - pkg/bloombuild/builder/builder.go | 25 +++++++++++---- pkg/bloombuild/builder/config.go | 2 ++ pkg/bloombuild/builder/spec.go | 48 +++++++++++++++------------- pkg/bloomcompactor/controller.go | 15 +++++---- pkg/bloomcompactor/spec.go | 46 +++++++++++++------------- pkg/storage/bloom/v1/block_writer.go | 19 +++++++++++ pkg/storage/bloom/v1/reader.go | 31 +++++++++++++++++- 8 files changed, 125 insertions(+), 62 deletions(-) diff --git a/integration/bloom_building_test.go b/integration/bloom_building_test.go index 0a96ee5702ac..46e8570c4771 100644 --- a/integration/bloom_building_test.go +++ b/integration/bloom_building_test.go @@ -170,7 +170,6 @@ func checkForTimestampMetric(t *testing.T, cliPlanner *client.Client, metricName func createBloomStore(t *testing.T, sharedPath string) *bloomshipper.BloomStore { logger := log.NewNopLogger() - //logger := log.NewLogfmtLogger(os.Stdout) schemaCfg := config.SchemaConfig{ Configs: []config.PeriodConfig{ diff --git a/pkg/bloombuild/builder/builder.go b/pkg/bloombuild/builder/builder.go index 5957ad4925a8..982d14388e28 100644 --- a/pkg/bloombuild/builder/builder.go +++ b/pkg/bloombuild/builder/builder.go @@ -1,9 +1,9 @@ package builder import ( - "bytes" "context" "fmt" + "os" "sync" "time" @@ -356,7 +356,7 @@ func (b *Builder) processTask( seriesItrWithCounter, b.chunkLoader, blocksIter, - b.rwFn, + b.writerReaderFunc, nil, // TODO(salvacorts): Pass reporter or remove when we address tracking b.bloomStore.BloomMetrics(), logger, @@ -372,6 +372,9 @@ func (b *Builder) processTask( built, err := bloomshipper.BlockFrom(tenant, task.Table.Addr(), blk) if err != nil { level.Error(logger).Log("msg", "failed to build block", "err", err) + if err = blk.Reader().Cleanup(); err != nil { + level.Error(logger).Log("msg", "failed to cleanup block directory", "err", err) + } return nil, fmt.Errorf("failed to build block: %w", err) } @@ -382,10 +385,17 @@ func (b *Builder) processTask( built, ); err != nil { level.Error(logger).Log("msg", "failed to write block", "err", err) + if err = blk.Reader().Cleanup(); err != nil { + level.Error(logger).Log("msg", "failed to cleanup block directory", "err", err) + } return nil, fmt.Errorf("failed to write block: %w", err) } b.metrics.blocksCreated.Inc() + if err := blk.Reader().Cleanup(); err != nil { + level.Error(logger).Log("msg", "failed to cleanup block directory", "err", err) + } + totalGapKeyspace := gap.Bounds.Max - gap.Bounds.Min progress := built.Bounds.Max - gap.Bounds.Min pct := float64(progress) / float64(totalGapKeyspace) * 100 @@ -477,9 +487,10 @@ func (b *Builder) loadWorkForGap( return seriesItr, blocksIter, nil } -// TODO(owen-d): pool, evaluate if memory-only is the best choice -func (b *Builder) rwFn() (v1.BlockWriter, v1.BlockReader) { - indexBuf := bytes.NewBuffer(nil) - bloomsBuf := bytes.NewBuffer(nil) - return v1.NewMemoryBlockWriter(indexBuf, bloomsBuf), v1.NewByteReader(indexBuf, bloomsBuf) +func (b *Builder) writerReaderFunc() (v1.BlockWriter, v1.BlockReader) { + dir, err := os.MkdirTemp(b.cfg.WorkingDir, "bloom-block-") + if err != nil { + panic(err) + } + return v1.NewDirectoryBlockWriter(dir), v1.NewDirectoryBlockReader(dir) } diff --git a/pkg/bloombuild/builder/config.go b/pkg/bloombuild/builder/config.go index d0c553104b09..deeeb951465a 100644 --- a/pkg/bloombuild/builder/config.go +++ b/pkg/bloombuild/builder/config.go @@ -13,6 +13,7 @@ type Config struct { GrpcConfig grpcclient.Config `yaml:"grpc_config"` PlannerAddress string `yaml:"planner_address"` BackoffConfig backoff.Config `yaml:"backoff_config"` + WorkingDir string `yaml:"working_directory" doc:"hidden"` } // RegisterFlagsWithPrefix registers flags for the bloom-planner configuration. @@ -20,6 +21,7 @@ func (cfg *Config) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) { f.StringVar(&cfg.PlannerAddress, prefix+".planner-address", "", "Hostname (and port) of the bloom planner") cfg.GrpcConfig.RegisterFlagsWithPrefix(prefix+".grpc", f) cfg.BackoffConfig.RegisterFlagsWithPrefix(prefix+".backoff", f) + f.StringVar(&cfg.WorkingDir, prefix+".working-directory", "", "Working directory to which blocks are temporarily written to. Empty string defaults to the operating system's temp directory.") } func (cfg *Config) Validate() error { diff --git a/pkg/bloombuild/builder/spec.go b/pkg/bloombuild/builder/spec.go index abb6cef1447f..82457cf92b84 100644 --- a/pkg/bloombuild/builder/spec.go +++ b/pkg/bloombuild/builder/spec.go @@ -53,8 +53,8 @@ type SimpleBloomGenerator struct { metrics *v1.Metrics logger log.Logger - readWriterFn func() (v1.BlockWriter, v1.BlockReader) - reporter func(model.Fingerprint) + writerReaderFunc func() (v1.BlockWriter, v1.BlockReader) + reporter func(model.Fingerprint) tokenizer *v1.BloomTokenizer } @@ -69,7 +69,7 @@ func NewSimpleBloomGenerator( store iter.Iterator[*v1.Series], chunkLoader ChunkLoader, blocksIter iter.ResetIterator[*v1.SeriesWithBlooms], - readWriterFn func() (v1.BlockWriter, v1.BlockReader), + writerReaderFunc func() (v1.BlockWriter, v1.BlockReader), reporter func(model.Fingerprint), metrics *v1.Metrics, logger log.Logger, @@ -85,9 +85,9 @@ func NewSimpleBloomGenerator( "component", "bloom_generator", "org_id", userID, ), - readWriterFn: readWriterFn, - metrics: metrics, - reporter: reporter, + writerReaderFunc: writerReaderFunc, + metrics: metrics, + reporter: reporter, tokenizer: v1.NewBloomTokenizer( opts.Schema.NGramLen(), @@ -161,19 +161,19 @@ func (s *SimpleBloomGenerator) Generate(ctx context.Context) *LazyBlockBuilderIt ) } - return NewLazyBlockBuilderIterator(ctx, s.opts, s.metrics, s.populator(ctx), s.readWriterFn, series, s.blocksIter) + return NewLazyBlockBuilderIterator(ctx, s.opts, s.metrics, s.populator(ctx), s.writerReaderFunc, series, s.blocksIter) } // LazyBlockBuilderIterator is a lazy iterator over blocks that builds // each block by adding series to them until they are full. type LazyBlockBuilderIterator struct { - ctx context.Context - opts v1.BlockOptions - metrics *v1.Metrics - populate v1.BloomPopulatorFunc - readWriterFn func() (v1.BlockWriter, v1.BlockReader) - series iter.PeekIterator[*v1.Series] - blocks iter.ResetIterator[*v1.SeriesWithBlooms] + ctx context.Context + opts v1.BlockOptions + metrics *v1.Metrics + populate v1.BloomPopulatorFunc + writerReaderFunc func() (v1.BlockWriter, v1.BlockReader) + series iter.PeekIterator[*v1.Series] + blocks iter.ResetIterator[*v1.SeriesWithBlooms] bytesAdded int curr *v1.Block @@ -185,18 +185,18 @@ func NewLazyBlockBuilderIterator( opts v1.BlockOptions, metrics *v1.Metrics, populate v1.BloomPopulatorFunc, - readWriterFn func() (v1.BlockWriter, v1.BlockReader), + writerReaderFunc func() (v1.BlockWriter, v1.BlockReader), series iter.PeekIterator[*v1.Series], blocks iter.ResetIterator[*v1.SeriesWithBlooms], ) *LazyBlockBuilderIterator { return &LazyBlockBuilderIterator{ - ctx: ctx, - opts: opts, - metrics: metrics, - populate: populate, - readWriterFn: readWriterFn, - series: series, - blocks: blocks, + ctx: ctx, + opts: opts, + metrics: metrics, + populate: populate, + writerReaderFunc: writerReaderFunc, + series: series, + blocks: blocks, } } @@ -221,9 +221,10 @@ func (b *LazyBlockBuilderIterator) Next() bool { } mergeBuilder := v1.NewMergeBuilder(b.blocks, b.series, b.populate, b.metrics) - writer, reader := b.readWriterFn() + writer, reader := b.writerReaderFunc() blockBuilder, err := v1.NewBlockBuilder(b.opts, writer) if err != nil { + _ = writer.Cleanup() b.err = errors.Wrap(err, "failed to create bloom block builder") return false } @@ -231,6 +232,7 @@ func (b *LazyBlockBuilderIterator) Next() bool { b.bytesAdded += sourceBytes if err != nil { + _ = writer.Cleanup() b.err = errors.Wrap(err, "failed to build bloom block") return false } diff --git a/pkg/bloomcompactor/controller.go b/pkg/bloomcompactor/controller.go index fffd67f7f2f4..b852896bfd27 100644 --- a/pkg/bloomcompactor/controller.go +++ b/pkg/bloomcompactor/controller.go @@ -1,10 +1,10 @@ package bloomcompactor import ( - "bytes" "context" "fmt" "math" + "os" "sort" "sync" @@ -49,11 +49,12 @@ func NewSimpleBloomController( } } -// TODO(owen-d): pool, evaluate if memory-only is the best choice -func (s *SimpleBloomController) rwFn() (v1.BlockWriter, v1.BlockReader) { - indexBuf := bytes.NewBuffer(nil) - bloomsBuf := bytes.NewBuffer(nil) - return v1.NewMemoryBlockWriter(indexBuf, bloomsBuf), v1.NewByteReader(indexBuf, bloomsBuf) +func (s *SimpleBloomController) writerReaderFunc() (v1.BlockWriter, v1.BlockReader) { + dir, err := os.MkdirTemp("", "bloom-block-") + if err != nil { + panic(err) + } + return v1.NewDirectoryBlockWriter(dir), v1.NewDirectoryBlockReader(dir) } /* @@ -409,7 +410,7 @@ func (s *SimpleBloomController) buildGaps( seriesItrWithCounter, s.chunkLoader, blocksIter, - s.rwFn, + s.writerReaderFunc, reporter, s.metrics, logger, diff --git a/pkg/bloomcompactor/spec.go b/pkg/bloomcompactor/spec.go index 61cd8f1d06a4..696f192970b6 100644 --- a/pkg/bloomcompactor/spec.go +++ b/pkg/bloomcompactor/spec.go @@ -53,8 +53,8 @@ type SimpleBloomGenerator struct { metrics *Metrics logger log.Logger - readWriterFn func() (v1.BlockWriter, v1.BlockReader) - reporter func(model.Fingerprint) + writerReaderFunc func() (v1.BlockWriter, v1.BlockReader) + reporter func(model.Fingerprint) tokenizer *v1.BloomTokenizer } @@ -69,7 +69,7 @@ func NewSimpleBloomGenerator( store iter.Iterator[*v1.Series], chunkLoader ChunkLoader, blocksIter iter.ResetIterator[*v1.SeriesWithBlooms], - readWriterFn func() (v1.BlockWriter, v1.BlockReader), + writerReaderFunc func() (v1.BlockWriter, v1.BlockReader), reporter func(model.Fingerprint), metrics *Metrics, logger log.Logger, @@ -85,9 +85,9 @@ func NewSimpleBloomGenerator( "component", "bloom_generator", "org_id", userID, ), - readWriterFn: readWriterFn, - metrics: metrics, - reporter: reporter, + writerReaderFunc: writerReaderFunc, + metrics: metrics, + reporter: reporter, tokenizer: v1.NewBloomTokenizer( opts.Schema.NGramLen(), @@ -161,19 +161,19 @@ func (s *SimpleBloomGenerator) Generate(ctx context.Context) *LazyBlockBuilderIt ) } - return NewLazyBlockBuilderIterator(ctx, s.opts, s.metrics, s.populator(ctx), s.readWriterFn, series, s.blocksIter) + return NewLazyBlockBuilderIterator(ctx, s.opts, s.metrics, s.populator(ctx), s.writerReaderFunc, series, s.blocksIter) } // LazyBlockBuilderIterator is a lazy iterator over blocks that builds // each block by adding series to them until they are full. type LazyBlockBuilderIterator struct { - ctx context.Context - opts v1.BlockOptions - metrics *Metrics - populate v1.BloomPopulatorFunc - readWriterFn func() (v1.BlockWriter, v1.BlockReader) - series iter.PeekIterator[*v1.Series] - blocks iter.ResetIterator[*v1.SeriesWithBlooms] + ctx context.Context + opts v1.BlockOptions + metrics *Metrics + populate v1.BloomPopulatorFunc + writerReaderFunc func() (v1.BlockWriter, v1.BlockReader) + series iter.PeekIterator[*v1.Series] + blocks iter.ResetIterator[*v1.SeriesWithBlooms] bytesAdded int curr *v1.Block @@ -185,18 +185,18 @@ func NewLazyBlockBuilderIterator( opts v1.BlockOptions, metrics *Metrics, populate v1.BloomPopulatorFunc, - readWriterFn func() (v1.BlockWriter, v1.BlockReader), + writerReaderFunc func() (v1.BlockWriter, v1.BlockReader), series iter.PeekIterator[*v1.Series], blocks iter.ResetIterator[*v1.SeriesWithBlooms], ) *LazyBlockBuilderIterator { return &LazyBlockBuilderIterator{ - ctx: ctx, - opts: opts, - metrics: metrics, - populate: populate, - readWriterFn: readWriterFn, - series: series, - blocks: blocks, + ctx: ctx, + opts: opts, + metrics: metrics, + populate: populate, + writerReaderFunc: writerReaderFunc, + series: series, + blocks: blocks, } } @@ -221,7 +221,7 @@ func (b *LazyBlockBuilderIterator) Next() bool { } mergeBuilder := v1.NewMergeBuilder(b.blocks, b.series, b.populate, b.metrics.bloomMetrics) - writer, reader := b.readWriterFn() + writer, reader := b.writerReaderFunc() blockBuilder, err := v1.NewBlockBuilder(b.opts, writer) if err != nil { b.err = errors.Wrap(err, "failed to create bloom block builder") diff --git a/pkg/storage/bloom/v1/block_writer.go b/pkg/storage/bloom/v1/block_writer.go index 70ed868235a7..a50c2f81e4b8 100644 --- a/pkg/storage/bloom/v1/block_writer.go +++ b/pkg/storage/bloom/v1/block_writer.go @@ -8,6 +8,8 @@ import ( "github.com/pkg/errors" + "github.com/grafana/dskit/multierror" + "github.com/grafana/loki/v3/pkg/storage/chunk/client/util" ) @@ -22,6 +24,7 @@ type BlockWriter interface { Blooms() (io.WriteCloser, error) Size() (int, error) // byte size of accumualted index & blooms Full(maxSize uint64) (full bool, size int, err error) + Cleanup() error } // in memory impl @@ -39,6 +42,7 @@ func NewMemoryBlockWriter(index, blooms *bytes.Buffer) MemoryBlockWriter { func (b MemoryBlockWriter) Index() (io.WriteCloser, error) { return NewNoopCloser(b.index), nil } + func (b MemoryBlockWriter) Blooms() (io.WriteCloser, error) { return NewNoopCloser(b.blooms), nil } @@ -60,6 +64,12 @@ func (b MemoryBlockWriter) Full(maxSize uint64) (full bool, size int, err error) return uint64(size) >= maxSize, size, nil } +func (b MemoryBlockWriter) Cleanup() error { + b.index.Reset() + b.blooms.Reset() + return nil +} + // Directory based impl type DirectoryBlockWriter struct { dir string @@ -139,3 +149,12 @@ func (b *DirectoryBlockWriter) Full(maxSize uint64) (full bool, size int, err er return uint64(size) >= maxSize, size, nil } + +func (b *DirectoryBlockWriter) Cleanup() error { + b.initialized = false + err := multierror.New() + err.Add(os.Remove(b.index.Name())) + err.Add(os.Remove(b.blooms.Name())) + err.Add(os.RemoveAll(b.dir)) + return err.Err() +} diff --git a/pkg/storage/bloom/v1/reader.go b/pkg/storage/bloom/v1/reader.go index d402ee1fd971..d589aa19c492 100644 --- a/pkg/storage/bloom/v1/reader.go +++ b/pkg/storage/bloom/v1/reader.go @@ -8,6 +8,8 @@ import ( "github.com/pkg/errors" + "github.com/grafana/dskit/multierror" + iter "github.com/grafana/loki/v3/pkg/iter/v2" ) @@ -15,6 +17,7 @@ type BlockReader interface { Index() (io.ReadSeeker, error) Blooms() (io.ReadSeeker, error) TarEntries() (iter.Iterator[TarEntry], error) + Cleanup() error } // In memory reader @@ -61,6 +64,12 @@ func (r *ByteReader) TarEntries() (iter.Iterator[TarEntry], error) { return iter.NewSliceIter[TarEntry](entries), err } +func (r *ByteReader) Cleanup() error { + r.index.Reset() + r.blooms.Reset() + return nil +} + // File reader type DirectoryBlockReader struct { dir string @@ -113,17 +122,28 @@ func (r *DirectoryBlockReader) Blooms() (io.ReadSeeker, error) { } func (r *DirectoryBlockReader) TarEntries() (iter.Iterator[TarEntry], error) { + var err error if !r.initialized { - if err := r.Init(); err != nil { + if err = r.Init(); err != nil { return nil, err } } + _, err = r.index.Seek(0, io.SeekStart) + if err != nil { + return nil, errors.Wrap(err, "error seeking series file") + } + idxInfo, err := r.index.Stat() if err != nil { return nil, errors.Wrap(err, "error stat'ing series file") } + _, err = r.blooms.Seek(0, io.SeekStart) + if err != nil { + return nil, errors.Wrap(err, "error seeking bloom file") + } + bloomInfo, err := r.blooms.Stat() if err != nil { return nil, errors.Wrap(err, "error stat'ing bloom file") @@ -144,3 +164,12 @@ func (r *DirectoryBlockReader) TarEntries() (iter.Iterator[TarEntry], error) { return iter.NewSliceIter[TarEntry](entries), nil } + +func (r *DirectoryBlockReader) Cleanup() error { + r.initialized = false + err := multierror.New() + err.Add(os.Remove(r.index.Name())) + err.Add(os.Remove(r.blooms.Name())) + err.Add(os.RemoveAll(r.dir)) + return err.Err() +} From d8eb9dd9be7e66029aa71085e3c5efc4c3ed4657 Mon Sep 17 00:00:00 2001 From: George Robinson Date: Wed, 17 Jul 2024 13:49:31 +0100 Subject: [PATCH 15/23] chore: Remove redundant comments (#13553) --- pkg/storage/wal/manager.go | 13 +++---------- 1 file changed, 3 insertions(+), 10 deletions(-) diff --git a/pkg/storage/wal/manager.go b/pkg/storage/wal/manager.go index d5d8fff893f0..57ac8b3207fa 100644 --- a/pkg/storage/wal/manager.go +++ b/pkg/storage/wal/manager.go @@ -102,21 +102,14 @@ type Manager struct { cfg Config metrics *ManagerMetrics available *list.List - - // pending is a list of segments that are waiting to be flushed. Once - // flushed, the segment is reset and moved to the back of the available - // list to accept writes again. - pending *list.List - + pending *list.List // firstAppend is the time of the first append to the segment at the // front of the available list. It is used to know when the segment has // exceeded the maximum age and should be moved to the pending list. // It is reset each time this happens. firstAppend time.Time - - closed bool - mu sync.Mutex - + closed bool + mu sync.Mutex // Used in tests. clock quartz.Clock } From d3d9923410c3a724d3dea7472407b977f50d9f69 Mon Sep 17 00:00:00 2001 From: George Robinson Date: Wed, 17 Jul 2024 13:57:07 +0100 Subject: [PATCH 16/23] chore: Rename PendingItem to PendingSegment and clean up flush.go (#13554) --- pkg/ingester-rf1/flush.go | 20 ++++++++------------ pkg/storage/wal/manager.go | 14 +++++++------- pkg/storage/wal/segment.go | 6 +++--- 3 files changed, 18 insertions(+), 22 deletions(-) diff --git a/pkg/ingester-rf1/flush.go b/pkg/ingester-rf1/flush.go index efd8f72df726..631fc9728738 100644 --- a/pkg/ingester-rf1/flush.go +++ b/pkg/ingester-rf1/flush.go @@ -68,7 +68,7 @@ func (i *Ingester) flushWorker(j int) { continue } - err = i.flushItem(l, j, it) + err = i.flush(l, j, it) if err != nil { level.Error(l).Log("msg", "failed to flush", "err", err) } @@ -78,7 +78,7 @@ func (i *Ingester) flushWorker(j int) { } } -func (i *Ingester) flushItem(l log.Logger, j int, it *wal.PendingItem) error { +func (i *Ingester) flush(l log.Logger, j int, it *wal.PendingSegment) error { ctx, cancelFunc := context.WithCancel(context.Background()) defer cancelFunc() @@ -94,30 +94,26 @@ func (i *Ingester) flushItem(l log.Logger, j int, it *wal.PendingItem) error { return b.Err() } -// flushChunk flushes the given chunk to the store. -// -// If the flush is successful, metrics for this flush are to be reported. -// If the flush isn't successful, the operation for this userID is requeued allowing this and all other unflushed -// segments to have another opportunity to be flushed. func (i *Ingester) flushSegment(ctx context.Context, j int, w *wal.SegmentWriter) error { - id := ulid.MustNew(ulid.Timestamp(time.Now()), rand.Reader) - start := time.Now() defer func() { i.metrics.flushDuration.Observe(time.Since(start).Seconds()) - w.Observe() + w.ReportMetrics() }() + i.metrics.flushesTotal.Add(1) + buf := i.flushBuffers[j] defer buf.Reset() if _, err := w.WriteTo(buf); err != nil { + i.metrics.flushFailuresTotal.Inc() return err } - i.metrics.flushesTotal.Add(1) + id := ulid.MustNew(ulid.Timestamp(time.Now()), rand.Reader) if err := i.store.PutObject(ctx, fmt.Sprintf("loki-v2/wal/anon/"+id.String()), buf); err != nil { i.metrics.flushFailuresTotal.Inc() - return fmt.Errorf("store put chunk: %w", err) + return fmt.Errorf("failed to put object: %w", err) } return nil diff --git a/pkg/storage/wal/manager.go b/pkg/storage/wal/manager.go index 57ac8b3207fa..6a4ef7f052ee 100644 --- a/pkg/storage/wal/manager.go +++ b/pkg/storage/wal/manager.go @@ -114,7 +114,7 @@ type Manager struct { clock quartz.Clock } -// item is similar to PendingItem, but it is an internal struct used in the +// item is similar to PendingSegment, but it is an internal struct used in the // available and pending lists. It contains a single-use result that is returned // to callers appending to the WAL and a re-usable segment that is reset after // each flush. @@ -123,8 +123,8 @@ type item struct { w *SegmentWriter } -// PendingItem contains a result and the segment to be flushed. -type PendingItem struct { +// PendingSegment contains a result and the segment to be flushed. +type PendingSegment struct { Result *AppendResult Writer *SegmentWriter } @@ -196,7 +196,7 @@ func (m *Manager) Close() { // It returns nil if there are no segments waiting to be flushed. If the WAL // is closed it returns all remaining segments from the pending list and then // ErrClosed. -func (m *Manager) NextPending() (*PendingItem, error) { +func (m *Manager) NextPending() (*PendingSegment, error) { m.mu.Lock() defer m.mu.Unlock() if m.pending.Len() == 0 && !m.moveFrontIfExpired() { @@ -210,12 +210,12 @@ func (m *Manager) NextPending() (*PendingItem, error) { m.pending.Remove(el) m.metrics.NumPending.Dec() m.metrics.NumFlushing.Inc() - return &PendingItem{Result: it.r, Writer: it.w}, nil + return &PendingSegment{Result: it.r, Writer: it.w}, nil } // Put resets the segment and puts it back in the available list to accept -// writes. A PendingItem should not be put back until it has been flushed. -func (m *Manager) Put(it *PendingItem) { +// writes. A PendingSegment should not be put back until it has been flushed. +func (m *Manager) Put(it *PendingSegment) { it.Writer.Reset() m.mu.Lock() defer m.mu.Unlock() diff --git a/pkg/storage/wal/segment.go b/pkg/storage/wal/segment.go index 6a792988bdb4..b21e477c0d69 100644 --- a/pkg/storage/wal/segment.go +++ b/pkg/storage/wal/segment.go @@ -147,9 +147,9 @@ func (b *SegmentWriter) Append(tenantID, labelsString string, lbls labels.Labels } } -// Observe updates metrics for the writer. If called before WriteTo then the -// output size histogram will observe 0. -func (b *SegmentWriter) Observe() { +// ReportMetrics for the writer. If called before WriteTo then the output size +// histogram will observe 0. +func (b *SegmentWriter) ReportMetrics() { b.consistencyMtx.Lock() defer b.consistencyMtx.Unlock() From dbff69a2e92f3ce34f7d58a7418cd0456d644be3 Mon Sep 17 00:00:00 2001 From: George Robinson Date: Wed, 17 Jul 2024 14:11:29 +0100 Subject: [PATCH 17/23] fix: Fix panic in ingester.go (#13557) --- pkg/ingester-rf1/ingester.go | 1 + 1 file changed, 1 insertion(+) diff --git a/pkg/ingester-rf1/ingester.go b/pkg/ingester-rf1/ingester.go index 802934665af3..1304a58c1dbb 100644 --- a/pkg/ingester-rf1/ingester.go +++ b/pkg/ingester-rf1/ingester.go @@ -254,6 +254,7 @@ func New(cfg Config, clientConfig client.Config, instances: map[string]*instance{}, store: storage, periodicConfigs: periodConfigs, + flushBuffers: make([]*bytes.Buffer, cfg.ConcurrentFlushes), flushWorkersDone: sync.WaitGroup{}, loopQuit: make(chan struct{}), tailersQuit: make(chan struct{}), From 5fa9c4bd56ec996fa2cf018b4b74b992fedca26a Mon Sep 17 00:00:00 2001 From: benclive Date: Wed, 17 Jul 2024 14:15:04 +0100 Subject: [PATCH 18/23] fix: Retain original values in logfmt pattern tokenizer (#13535) --- pkg/pattern/drain/line_tokenizer.go | 2 +- pkg/pattern/drain/line_tokenizer_test.go | 8 ++++++++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/pkg/pattern/drain/line_tokenizer.go b/pkg/pattern/drain/line_tokenizer.go index 04b998b8dba5..533f1f92931c 100644 --- a/pkg/pattern/drain/line_tokenizer.go +++ b/pkg/pattern/drain/line_tokenizer.go @@ -233,7 +233,7 @@ func (t *logfmtTokenizer) Join(tokens []string, _ interface{}) string { buf := bytes.NewBuffer(make([]byte, 0, 1024)) enc := gologfmt.NewEncoder(buf) for i := 0; i < len(tokens); i += 2 { - k, v := tokens[i], tokens[i+1] + k, v := tokens[i], unsafeBytes(tokens[i+1]) if err := enc.EncodeKeyval(k, v); err != nil { return "" } diff --git a/pkg/pattern/drain/line_tokenizer_test.go b/pkg/pattern/drain/line_tokenizer_test.go index 106d5d44b487..200d1a8f510e 100644 --- a/pkg/pattern/drain/line_tokenizer_test.go +++ b/pkg/pattern/drain/line_tokenizer_test.go @@ -209,6 +209,10 @@ func TestLogFmtTokenizer(t *testing.T) { line: `ts=2024-05-30T12:50:36.648377186Z caller=scheduler_processor.go:143 level=warn msg="error contacting scheduler" err="rpc error: code = Unavailable desc = connection error: desc = \"error reading server preface: EOF\"" addr=10.0.151.101:9095`, want: []string{"ts", param, "caller", "scheduler_processor.go:143", "level", "warn", "msg", "error contacting scheduler", "err", "rpc error: code = Unavailable desc = connection error: desc = \"error reading server preface: EOF\"", "addr", "10.0.151.101:9095"}, }, + { + line: `logger=sqlstore.metrics traceID=c933fefbe893411d3be8e1648d6bcf37 t=2024-07-10T16:00:15.564896897Z level=debug msg="query finished" status=success elapsedtime=1.324305ms error=null`, + want: []string{"logger", "sqlstore.metrics", "traceID", "<_>", "t", "<_>", "level", "debug", "msg", "query finished", "status", "success", "elapsedtime", "1.324305ms", "", "", "error", "null"}, + }, } tokenizer := newLogfmtTokenizer(param) @@ -258,6 +262,10 @@ func TestLogFmtTokenizerJoin(t *testing.T) { want: `ts=2024-05-30T12:50:36.648377186Z caller=scheduler_processor.go:143 level=warn msg="error contacting scheduler" err="rpc error: code = Unavailable desc = connection error: desc = \"error reading server preface: EOF\"" addr=10.0.151.101:9095`, tokens: []string{"ts", "2024-05-30T12:50:36.648377186Z", "caller", "scheduler_processor.go:143", "level", "warn", "msg", "error contacting scheduler", "err", "rpc error: code = Unavailable desc = connection error: desc = \"error reading server preface: EOF\"", "addr", "10.0.151.101:9095"}, }, + { + want: `error=null`, + tokens: []string{"error", "null"}, + }, } tokenizer := newLogfmtTokenizer("") From e23598d710a231213a40f5bfb7d99fe2ee409ba2 Mon Sep 17 00:00:00 2001 From: benclive Date: Wed, 17 Jul 2024 14:15:41 +0100 Subject: [PATCH 19/23] fix: Dedup pattern tokens on output (#13534) --- pkg/pattern/drain/drain.go | 14 +++++--------- pkg/pattern/drain/drain_test.go | 2 +- pkg/pattern/drain/line_tokenizer.go | 9 +++++++++ 3 files changed, 15 insertions(+), 10 deletions(-) diff --git a/pkg/pattern/drain/drain.go b/pkg/pattern/drain/drain.go index 64ef81e0e8a8..5c48d2980e02 100644 --- a/pkg/pattern/drain/drain.go +++ b/pkg/pattern/drain/drain.go @@ -158,6 +158,7 @@ func New(config *Config, format string, metrics *Metrics) *Drain { default: tokenizer = newPunctuationTokenizer() } + d.idToCluster = createLogClusterCache(config.MaxClusters, func(int, *LogCluster) { if metrics != nil { if d.pruning { @@ -170,7 +171,10 @@ func New(config *Config, format string, metrics *Metrics) *Drain { limiter.Evict() } }) - d.tokenizer = tokenizer + d.tokenizer = &DedupingTokenizer{ + LineTokenizer: tokenizer, + dedupParam: config.ParamString, + } d.limiter = limiter return d } @@ -297,14 +301,6 @@ func deduplicatePlaceholders(line string, placeholder string) string { return unsafeString(builder) } -func (d *Drain) PatternString(c *LogCluster) string { - s := deduplicatePlaceholders(d.tokenizer.Join(c.Tokens, c.TokenState), d.config.ParamString) - if s == d.config.ParamString { - return "" - } - return s -} - func (d *Drain) Prune() { d.pruneTree(d.rootNode) } diff --git a/pkg/pattern/drain/drain_test.go b/pkg/pattern/drain/drain_test.go index 944fd5ccdcb9..1aabea548aed 100644 --- a/pkg/pattern/drain/drain_test.go +++ b/pkg/pattern/drain/drain_test.go @@ -260,7 +260,7 @@ func TestDrain_TrainExtractsPatterns(t *testing.T) { `I0507 <_> 1 defaultevictor.go:202] "Pod fails the following checks" pod="<_>" checks="[pod is a mirror pod, pod is a static pod, pod has system critical priority, pod has higher priority than specified priority class threshold, pod has local storage and descheduler is not configured with evictLocalStoragePods]"`, `I0507 <_> 1 defaultevictor.go:202] "Pod fails the following checks" pod="<_>" checks="pod has local storage and descheduler is not configured with evictLocalStoragePods"`, `I0507 <_> 1 defaultevictor.go:202] "Pod fails the following checks" pod="<_>" checks="pod is a DaemonSet pod"`, - `I0507 <_> 1 node.go:157] "Pod does not fit on any other node" pod:="<_>" node:="<_>" error:="[pod node selector does not match the node label, <_> <_><_> <_> <_><_> <_> <_>]"`, + `I0507 <_> 1 node.go:157] "Pod does not fit on any other node" pod:="<_>" node:="<_>" error:="[pod node selector does not match the node label, <_> <_> <_> <_> <_> <_>]"`, `I0507 <_> 1 node.go:157] "Pod does not fit on any other node" pod:="<_>" node:="<_>" error:="[pod node selector does not match the node label, insufficient <_>, insufficient <_>]"`, `I0507 <_> 1 node.go:157] "Pod does not fit on any other node" pod:="<_>" node:="<_>" error:="[pod node selector does not match the node label, insufficient <_>]"`, `I0507 <_> 1 node.go:157] "Pod does not fit on any other node" pod:="<_>" node:="<_>" error:="[pod node selector does not match the node label, pod does not tolerate taints on the node, insufficient <_>, insufficient <_>]"`, diff --git a/pkg/pattern/drain/line_tokenizer.go b/pkg/pattern/drain/line_tokenizer.go index 533f1f92931c..e2acd8228bcf 100644 --- a/pkg/pattern/drain/line_tokenizer.go +++ b/pkg/pattern/drain/line_tokenizer.go @@ -286,3 +286,12 @@ func isVariableField(key []byte) bool { bytes.EqualFold(key, []byte("time")) || bytes.EqualFold(key, []byte("timestamp")) } + +type DedupingTokenizer struct { + LineTokenizer + dedupParam string +} + +func (d DedupingTokenizer) Join(tokens []string, state interface{}) string { + return deduplicatePlaceholders(d.LineTokenizer.Join(tokens, state), d.dedupParam) +} From a88a0d3f6ceaba0082c557ab773b7fd45537ac64 Mon Sep 17 00:00:00 2001 From: Jack Baldry Date: Wed, 17 Jul 2024 14:43:49 +0100 Subject: [PATCH 20/23] feat: Update doc-validator version (#13558) --- .github/workflows/doc-validator.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/doc-validator.yml b/.github/workflows/doc-validator.yml index bb074949e29e..a496861e6e73 100644 --- a/.github/workflows/doc-validator.yml +++ b/.github/workflows/doc-validator.yml @@ -7,7 +7,7 @@ jobs: doc-validator: runs-on: "ubuntu-latest" container: - image: "grafana/doc-validator:v5.0.0" + image: "grafana/doc-validator:v5.1.0" steps: - name: "Checkout code" uses: "actions/checkout@v4" From 46de4c1bc839ef682798bec5003123f7d5f4404b Mon Sep 17 00:00:00 2001 From: Robert Jacob Date: Wed, 17 Jul 2024 20:53:10 +0200 Subject: [PATCH 21/23] fix(operator): Set object storage for delete requests when using retention (#13562) --- operator/internal/manifests/internal/config/build_test.go | 1 + .../internal/manifests/internal/config/loki-config.yaml | 7 +++++-- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/operator/internal/manifests/internal/config/build_test.go b/operator/internal/manifests/internal/config/build_test.go index 1d52d98692df..5e119e6661d5 100644 --- a/operator/internal/manifests/internal/config/build_test.go +++ b/operator/internal/manifests/internal/config/build_test.go @@ -1833,6 +1833,7 @@ compactor: retention_enabled: true retention_delete_delay: 4h retention_delete_worker_count: 50 + delete_request_store: s3 frontend: tail_proxy_url: http://loki-querier-http-lokistack-dev.default.svc.cluster.local:3100 compress_responses: true diff --git a/operator/internal/manifests/internal/config/loki-config.yaml b/operator/internal/manifests/internal/config/loki-config.yaml index da13920b770e..dfdf5c900325 100644 --- a/operator/internal/manifests/internal/config/loki-config.yaml +++ b/operator/internal/manifests/internal/config/loki-config.yaml @@ -99,11 +99,14 @@ common: compactor: compaction_interval: 2h working_directory: {{ .StorageDirectory }}/compactor -{{- if .Retention.Enabled }}{{- with .Retention }} +{{- if .Retention.Enabled }} +{{- with .Retention }} retention_enabled: true retention_delete_delay: 4h retention_delete_worker_count: {{.DeleteWorkerCount}} -{{- end }}{{- end }} +{{- end }} + delete_request_store: {{.ObjectStorage.SharedStore}} +{{- end }} frontend: tail_proxy_url: {{ .Querier.Protocol }}://{{ .Querier.FQDN }}:{{ .Querier.Port }} {{- if .Gates.HTTPEncryption }} From 29110feb991ad32660fc6d9a1055365543bf78f6 Mon Sep 17 00:00:00 2001 From: Paul Rogers <129207811+paul1r@users.noreply.github.com> Date: Wed, 17 Jul 2024 16:19:59 -0400 Subject: [PATCH 22/23] build: Loki build and Docker updates (#13325) --- Makefile | 31 ++++++++++--------- clients/cmd/promtail/Dockerfile | 3 +- clients/cmd/promtail/Dockerfile.cross | 3 +- cmd/logcli/Dockerfile | 8 +++-- cmd/logql-analyzer/Dockerfile | 9 +++--- cmd/loki-canary-boringcrypto/Dockerfile | 9 +++--- cmd/loki-canary/Dockerfile | 9 ++++-- cmd/loki-canary/Dockerfile.cross | 10 +++--- cmd/loki/Dockerfile | 13 ++++---- cmd/loki/Dockerfile.cross | 17 +++++----- cmd/loki/Dockerfile.debug | 13 ++++---- cmd/migrate/Dockerfile | 12 ++++--- cmd/querytee/Dockerfile | 10 ++++-- cmd/querytee/Dockerfile.cross | 10 +++--- loki-build-image/Dockerfile | 20 ++++++------ production/helm/loki/src/helm-test/Dockerfile | 6 ++-- 16 files changed, 102 insertions(+), 81 deletions(-) diff --git a/Makefile b/Makefile index 8f345f96d37a..cadce23fe262 100644 --- a/Makefile +++ b/Makefile @@ -38,6 +38,7 @@ BUILD_IN_CONTAINER ?= true # ensure you run `make drone` and `make release-workflows` after changing this BUILD_IMAGE_VERSION ?= 0.33.4 +GO_VERSION := 1.22.5 # Docker image info IMAGE_PREFIX ?= grafana @@ -141,7 +142,7 @@ logcli: cmd/logcli/logcli ## build logcli executable logcli-debug: cmd/logcli/logcli-debug ## build debug logcli executable logcli-image: ## build logcli docker image - $(SUDO) docker build -t $(IMAGE_PREFIX)/logcli:$(IMAGE_TAG) -f cmd/logcli/Dockerfile . + $(SUDO) docker build --build-arg=GO_VERSION=$(GO_VERSION) -t $(IMAGE_PREFIX)/logcli:$(IMAGE_TAG) -f cmd/logcli/Dockerfile . cmd/logcli/logcli: CGO_ENABLED=0 go build $(GO_FLAGS) -o $@ ./cmd/logcli @@ -597,9 +598,9 @@ endef # promtail promtail-image: ## build the promtail docker image - $(SUDO) docker build -t $(IMAGE_PREFIX)/promtail:$(IMAGE_TAG) -f clients/cmd/promtail/Dockerfile . + $(SUDO) docker build --build-arg=GO_VERSION=$(GO_VERSION) -t $(IMAGE_PREFIX)/promtail:$(IMAGE_TAG) -f clients/cmd/promtail/Dockerfile . promtail-image-cross: - $(SUDO) $(BUILD_OCI) -t $(IMAGE_PREFIX)/promtail:$(IMAGE_TAG) -f clients/cmd/promtail/Dockerfile.cross . + $(SUDO) $(BUILD_OCI) --build-arg=GO_VERSION=$(GO_VERSION) -t $(IMAGE_PREFIX)/promtail:$(IMAGE_TAG) -f clients/cmd/promtail/Dockerfile.cross . promtail-debug-image: ## build the promtail debug docker image $(SUDO) $(BUILD_OCI) -t $(IMAGE_PREFIX)/promtail:$(IMAGE_TAG)-debug -f clients/cmd/promtail/Dockerfile.debug . @@ -609,47 +610,47 @@ promtail-push: promtail-image-cross # loki loki-image: ## build the loki docker image - $(SUDO) docker build -t $(IMAGE_PREFIX)/loki:$(IMAGE_TAG) -f cmd/loki/Dockerfile . + $(SUDO) docker build --build-arg=GO_VERSION=$(GO_VERSION) -t $(IMAGE_PREFIX)/loki:$(IMAGE_TAG) -f cmd/loki/Dockerfile . loki-image-cross: - $(SUDO) $(BUILD_OCI) -t $(IMAGE_PREFIX)/loki:$(IMAGE_TAG) -f cmd/loki/Dockerfile.cross . + $(SUDO) $(BUILD_OCI) --build-arg=GO_VERSION=$(GO_VERSION) -t $(IMAGE_PREFIX)/loki:$(IMAGE_TAG) -f cmd/loki/Dockerfile.cross . loki-debug-image: ## build the debug loki docker image - $(SUDO) $(BUILD_OCI) -t $(IMAGE_PREFIX)/loki:$(IMAGE_TAG)-debug -f cmd/loki/Dockerfile.debug . + $(SUDO) $(BUILD_OCI) --build-arg=GO_VERSION=$(GO_VERSION) -t $(IMAGE_PREFIX)/loki:$(IMAGE_TAG)-debug -f cmd/loki/Dockerfile.debug . loki-push: loki-image-cross $(call push-image,loki) # loki-canary loki-canary-image: ## build the loki canary docker image - $(SUDO) docker build -t $(IMAGE_PREFIX)/loki-canary:$(IMAGE_TAG) -f cmd/loki-canary/Dockerfile . + $(SUDO) docker build --build-arg=GO_VERSION=$(GO_VERSION) -t $(IMAGE_PREFIX)/loki-canary:$(IMAGE_TAG) -f cmd/loki-canary/Dockerfile . loki-canary-image-cross: - $(SUDO) $(BUILD_OCI) -t $(IMAGE_PREFIX)/loki-canary:$(IMAGE_TAG) -f cmd/loki-canary/Dockerfile.cross . + $(SUDO) $(BUILD_OCI) --build-arg=GO_VERSION=$(GO_VERSION) -t $(IMAGE_PREFIX)/loki-canary:$(IMAGE_TAG) -f cmd/loki-canary/Dockerfile.cross . loki-canary-image-cross-boringcrypto: - $(SUDO) $(BUILD_OCI) -t $(IMAGE_PREFIX)/loki-canary-boringcrypto:$(IMAGE_TAG) -f cmd/loki-canary-boringcrypto/Dockerfile . + $(SUDO) $(BUILD_OCI) --build-arg=GO_VERSION=$(GO_VERSION) -t $(IMAGE_PREFIX)/loki-canary-boringcrypto:$(IMAGE_TAG) -f cmd/loki-canary-boringcrypto/Dockerfile . loki-canary-push: loki-canary-image-cross $(SUDO) $(PUSH_OCI) $(IMAGE_PREFIX)/loki-canary:$(IMAGE_TAG) loki-canary-push-boringcrypto: loki-canary-image-cross-boringcrypto $(SUDO) $(PUSH_OCI) $(IMAGE_PREFIX)/loki-canary-boringcrypto:$(IMAGE_TAG) helm-test-image: ## build the helm test image - $(SUDO) docker build -t $(IMAGE_PREFIX)/loki-helm-test:$(IMAGE_TAG) -f production/helm/loki/src/helm-test/Dockerfile . + $(SUDO) docker build --build-arg=GO_VERSION=$(GO_VERSION) -t $(IMAGE_PREFIX)/loki-helm-test:$(IMAGE_TAG) -f production/helm/loki/src/helm-test/Dockerfile . helm-test-push: helm-test-image ## push the helm test image $(SUDO) $(PUSH_OCI) $(IMAGE_PREFIX)/loki-helm-test:$(IMAGE_TAG) # loki-querytee loki-querytee-image: - $(SUDO) docker build -t $(IMAGE_PREFIX)/loki-query-tee:$(IMAGE_TAG) -f cmd/querytee/Dockerfile . + $(SUDO) docker build --build-arg=GO_VERSION=$(GO_VERSION) -t $(IMAGE_PREFIX)/loki-query-tee:$(IMAGE_TAG) -f cmd/querytee/Dockerfile . loki-querytee-image-cross: - $(SUDO) $(BUILD_OCI) -t $(IMAGE_PREFIX)/loki-query-tee:$(IMAGE_TAG) -f cmd/querytee/Dockerfile.cross . + $(SUDO) $(BUILD_OCI) --build-arg=GO_VERSION=$(GO_VERSION) -t $(IMAGE_PREFIX)/loki-query-tee:$(IMAGE_TAG) -f cmd/querytee/Dockerfile.cross . loki-querytee-push: loki-querytee-image-cross $(SUDO) $(PUSH_OCI) $(IMAGE_PREFIX)/loki-query-tee:$(IMAGE_TAG) # migrate-image migrate-image: - $(SUDO) docker build -t $(IMAGE_PREFIX)/loki-migrate:$(IMAGE_TAG) -f cmd/migrate/Dockerfile . + $(SUDO) docker build --build-arg=GO_VERSION=$(GO_VERSION) -t $(IMAGE_PREFIX)/loki-migrate:$(IMAGE_TAG) -f cmd/migrate/Dockerfile . # LogQL Analyzer logql-analyzer-image: ## build the LogQL Analyzer image - $(SUDO) docker build -t $(IMAGE_PREFIX)/logql-analyzer:$(IMAGE_TAG) -f cmd/logql-analyzer/Dockerfile . + $(SUDO) docker build --build-arg=GO_VERSION=$(GO_VERSION) -t $(IMAGE_PREFIX)/logql-analyzer:$(IMAGE_TAG) -f cmd/logql-analyzer/Dockerfile . logql-analyzer-push: logql-analyzer-image ## push the LogQL Analyzer image $(call push-image,logql-analyzer) @@ -663,7 +664,7 @@ else endif build-image: ensure-buildx-builder - $(SUDO) $(BUILD_OCI) -t $(IMAGE_PREFIX)/loki-build-image:$(IMAGE_TAG) ./loki-build-image + $(SUDO) $(BUILD_OCI) --build-arg=GO_VERSION=$(GO_VERSION) -t $(IMAGE_PREFIX)/loki-build-image:$(IMAGE_TAG) ./loki-build-image build-image-push: build-image ## push the docker build image ifneq (,$(findstring WIP,$(IMAGE_TAG))) @echo "Cannot push a WIP image, commit changes first"; \ diff --git a/clients/cmd/promtail/Dockerfile b/clients/cmd/promtail/Dockerfile index 0ac16613ea51..3c9088bb83ba 100644 --- a/clients/cmd/promtail/Dockerfile +++ b/clients/cmd/promtail/Dockerfile @@ -1,4 +1,5 @@ -FROM golang:1.22.5-bookworm as build +ARG GO_VERSION=1.22 +FROM golang:${GO_VERSION}-bookworm as build COPY . /src/loki WORKDIR /src/loki diff --git a/clients/cmd/promtail/Dockerfile.cross b/clients/cmd/promtail/Dockerfile.cross index ff9f3ac565e8..572e6394f459 100644 --- a/clients/cmd/promtail/Dockerfile.cross +++ b/clients/cmd/promtail/Dockerfile.cross @@ -1,8 +1,9 @@ ARG BUILD_IMAGE=grafana/loki-build-image:0.33.4 +ARG GO_VERSION=1.22 # Directories in this file are referenced from the root of the project not this folder # This file is intended to be called from the root like so: # docker build -t grafana/promtail -f clients/cmd/promtail/Dockerfile . -FROM 1.22.2-alpine as goenv +FROM golang:${GO_VERSION}-alpine as goenv RUN go env GOARCH > /goarch && \ go env GOARM > /goarm diff --git a/cmd/logcli/Dockerfile b/cmd/logcli/Dockerfile index ffcbef10ecfb..999434d075a8 100644 --- a/cmd/logcli/Dockerfile +++ b/cmd/logcli/Dockerfile @@ -1,13 +1,15 @@ -FROM golang:1.22.5 as build +ARG GO_VERSION=1.22 +FROM golang:${GO_VERSION} as build COPY . /src/loki WORKDIR /src/loki RUN make clean && make BUILD_IN_CONTAINER=false logcli -FROM alpine:3.18.5 -RUN apk add --no-cache ca-certificates +FROM gcr.io/distroless/base-nossl:debug COPY --from=build /src/loki/cmd/logcli/logcli /usr/bin/logcli +SHELL [ "/busybox/sh", "-c" ] +RUN ln -s /busybox/sh /bin/sh ENTRYPOINT [ "/usr/bin/logcli" ] diff --git a/cmd/logql-analyzer/Dockerfile b/cmd/logql-analyzer/Dockerfile index 4908bbb7da1e..53ba7bee94b4 100644 --- a/cmd/logql-analyzer/Dockerfile +++ b/cmd/logql-analyzer/Dockerfile @@ -1,13 +1,14 @@ -FROM golang:1.22.5 as build +ARG GO_VERSION=1.22 +FROM golang:${GO_VERSION} as build COPY . /src/loki WORKDIR /src/loki RUN make clean && CGO_ENABLED=0 go build ./cmd/logql-analyzer/ -FROM alpine:3.18.5 - -RUN apk add --no-cache ca-certificates +FROM gcr.io/distroless/base-nossl:debug COPY --from=build /src/loki/logql-analyzer /usr/bin/logql-analyzer +SHELL [ "/busybox/sh", "-c" ] +RUN ln -s /busybox/sh /bin/sh ENTRYPOINT [ "/usr/bin/logql-analyzer" ] diff --git a/cmd/loki-canary-boringcrypto/Dockerfile b/cmd/loki-canary-boringcrypto/Dockerfile index c1217cc3f68e..e69be2c0aeb6 100644 --- a/cmd/loki-canary-boringcrypto/Dockerfile +++ b/cmd/loki-canary-boringcrypto/Dockerfile @@ -1,12 +1,13 @@ -FROM golang:1.22.5 as build +ARG GO_VERSION=1.22 +FROM golang:${GO_VERSION} as build COPY . /src/loki WORKDIR /src/loki RUN go env GOARCH > /goarch RUN make clean && make GOARCH=$(cat /goarch) BUILD_IN_CONTAINER=true GOEXPERIMENT=boringcrypto loki-canary-boringcrypto -FROM alpine:3.18.5 -RUN apk add --update --no-cache ca-certificates -RUN apk add --no-cache libc6-compat +FROM gcr.io/distroless/base-nossl:debug COPY --from=build /src/loki/cmd/loki-canary-boringcrypto/loki-canary-boringcrypto /usr/bin/loki-canary +SHELL [ "/busybox/sh", "-c" ] +RUN ln -s /busybox/sh /bin/sh ENTRYPOINT [ "/usr/bin/loki-canary" ] diff --git a/cmd/loki-canary/Dockerfile b/cmd/loki-canary/Dockerfile index cf0f60c695e3..f0dcf02d5d81 100644 --- a/cmd/loki-canary/Dockerfile +++ b/cmd/loki-canary/Dockerfile @@ -1,10 +1,13 @@ -FROM golang:1.22.5 as build +ARG GO_VERSION=1.22 +FROM golang:${GO_VERSION} as build COPY . /src/loki WORKDIR /src/loki RUN make clean && make BUILD_IN_CONTAINER=false loki-canary -FROM alpine:3.18.5 -RUN apk add --update --no-cache ca-certificates +FROM gcr.io/distroless/base-nossl:debug + COPY --from=build /src/loki/cmd/loki-canary/loki-canary /usr/bin/loki-canary +SHELL [ "/busybox/sh", "-c" ] +RUN ln -s /busybox/sh /bin/sh ENTRYPOINT [ "/usr/bin/loki-canary" ] diff --git a/cmd/loki-canary/Dockerfile.cross b/cmd/loki-canary/Dockerfile.cross index b56f292e5561..e1f7b133313a 100644 --- a/cmd/loki-canary/Dockerfile.cross +++ b/cmd/loki-canary/Dockerfile.cross @@ -1,18 +1,20 @@ ARG BUILD_IMAGE=grafana/loki-build-image:0.33.4 +ARG GO_VERSION=1.22 # Directories in this file are referenced from the root of the project not this folder # This file is intended to be called from the root like so: # docker build -t grafana/promtail -f cmd/promtail/Dockerfile . -FROM golang:1.22.5-alpine as goenv +FROM golang:${GO_VERSION} as goenv RUN go env GOARCH > /goarch && \ go env GOARM > /goarm -FROM --platform=linux/amd64 $BUILD_IMAGE as build +FROM $BUILD_IMAGE as build COPY --from=goenv /goarch /goarm / COPY . /src/loki WORKDIR /src/loki RUN make clean && GOARCH=$(cat /goarch) GOARM=$(cat /goarm) make BUILD_IN_CONTAINER=false loki-canary -FROM alpine:3.18.5 -RUN apk add --update --no-cache ca-certificates +FROM gcr.io/distroless/base-nossl:debug COPY --from=build /src/loki/cmd/loki-canary/loki-canary /usr/bin/loki-canary +SHELL [ "/busybox/sh", "-c" ] +RUN ln -s /busybox/sh /bin/sh ENTRYPOINT [ "/usr/bin/loki-canary" ] diff --git a/cmd/loki/Dockerfile b/cmd/loki/Dockerfile index 6052f41ca54f..521c59159573 100644 --- a/cmd/loki/Dockerfile +++ b/cmd/loki/Dockerfile @@ -1,22 +1,23 @@ -FROM golang:1.22.5 as build +ARG GO_VERSION=1.22 +FROM golang:${GO_VERSION} as build COPY . /src/loki WORKDIR /src/loki RUN make clean && make BUILD_IN_CONTAINER=false loki -FROM alpine:3.18.5 - -RUN apk add --no-cache ca-certificates libcap +FROM gcr.io/distroless/base-nossl:debug COPY --from=build /src/loki/cmd/loki/loki /usr/bin/loki COPY cmd/loki/loki-docker-config.yaml /etc/loki/local-config.yaml +SHELL [ "/busybox/sh", "-c" ] + RUN addgroup -g 10001 -S loki && \ adduser -u 10001 -S loki -G loki RUN mkdir -p /loki/rules && \ mkdir -p /loki/rules-temp && \ - chown -R loki:loki /etc/loki /loki - + chown -R loki:loki /etc/loki /loki && \ + ln -s /busybox/sh /bin/sh USER 10001 EXPOSE 3100 ENTRYPOINT [ "/usr/bin/loki" ] diff --git a/cmd/loki/Dockerfile.cross b/cmd/loki/Dockerfile.cross index 52b18a302bdb..da9d358d28c5 100644 --- a/cmd/loki/Dockerfile.cross +++ b/cmd/loki/Dockerfile.cross @@ -1,28 +1,27 @@ -ARG BUILD_IMAGE=grafana/loki-build-image:0.33.4 +ARG GO_VERSION=1.22 # Directories in this file are referenced from the root of the project not this folder # This file is intended to be called from the root like so: # docker build -t grafana/loki -f cmd/loki/Dockerfile . -FROM golang:1.22.5-alpine as goenv +FROM golang:${GO_VERSION} as goenv RUN go env GOARCH > /goarch && \ go env GOARM > /goarm -FROM --platform=linux/amd64 $BUILD_IMAGE as build -COPY --from=goenv /goarch /goarm / COPY . /src/loki WORKDIR /src/loki RUN make clean && GOARCH=$(cat /goarch) GOARM=$(cat /goarm) make BUILD_IN_CONTAINER=false loki -FROM alpine:3.18.5 +FROM gcr.io/distroless/base-nossl:debug -RUN apk add --no-cache ca-certificates - -COPY --from=build /src/loki/cmd/loki/loki /usr/bin/loki +COPY --from=goenv /src/loki/cmd/loki/loki /usr/bin/loki COPY cmd/loki/loki-local-config.yaml /etc/loki/local-config.yaml +SHELL [ "/busybox/sh", "-c" ] + RUN addgroup -g 10001 -S loki && \ adduser -u 10001 -S loki -G loki RUN mkdir -p /loki && \ - chown -R loki:loki /etc/loki /loki + chown -R loki:loki /etc/loki /loki && \ + ln -s /busybox/sh /bin/sh USER 10001 EXPOSE 3100 diff --git a/cmd/loki/Dockerfile.debug b/cmd/loki/Dockerfile.debug index a943cb57f0d1..bb23778d9af3 100644 --- a/cmd/loki/Dockerfile.debug +++ b/cmd/loki/Dockerfile.debug @@ -1,22 +1,22 @@ ARG BUILD_IMAGE=grafana/loki-build-image:0.33.4 +ARG GO_VERSION=1.22 # Directories in this file are referenced from the root of the project not this folder # This file is intended to be called from the root like so: # docker build -t grafana/loki -f cmd/loki/Dockerfile.debug . -FROM golang:1.22.5-alpine as goenv +FROM golang:${GO_VERSION} as goenv RUN go env GOARCH > /goarch && \ go env GOARM > /goarm && \ go install github.com/go-delve/delve/cmd/dlv@latest -FROM --platform=linux/amd64 $BUILD_IMAGE as build +FROM $BUILD_IMAGE as build COPY --from=goenv /goarch /goarm / COPY . /src/loki WORKDIR /src/loki RUN make clean && \ GOARCH=$(cat /goarch) GOARM=$(cat /goarm) make BUILD_IN_CONTAINER=false loki-debug -FROM alpine:3.18.5 -RUN apk add --update --no-cache ca-certificates +FROM gcr.io/distroless/base-nossl:debug COPY --from=build /src/loki/cmd/loki/loki-debug /usr/bin/loki-debug COPY --from=goenv /go/bin/dlv /usr/bin/dlv COPY cmd/loki/loki-docker-config.yaml /etc/loki/local-config.yaml @@ -25,8 +25,9 @@ EXPOSE 3100 # Expose 40000 for delve EXPOSE 40000 -# Allow delve to run on Alpine based containers. -RUN apk add --no-cache libc6-compat +SHELL [ "/busybox/sh", "-c" ] +RUN ln -s /busybox/sh /bin/sh + # Run delve, ending with -- because we pass params via kubernetes, per the docs: # Pass flags to the program you are debugging using --, for example:` diff --git a/cmd/migrate/Dockerfile b/cmd/migrate/Dockerfile index e50554d50d6b..a24697a719f2 100644 --- a/cmd/migrate/Dockerfile +++ b/cmd/migrate/Dockerfile @@ -1,10 +1,12 @@ -FROM golang:1.22.5 as build +ARG GO_VERSION=1.22 +FROM golang:${GO_VERSION} as build COPY . /src/loki WORKDIR /src/loki RUN make clean && make BUILD_IN_CONTAINER=false migrate -FROM alpine:3.18.5 -RUN apk add --update --no-cache ca-certificates +FROM gcr.io/distroless/base-nossl:debug + COPY --from=build /src/loki/cmd/migrate/migrate /usr/bin/migrate -#ENTRYPOINT [ "/usr/bin/migrate" ] -CMD tail -f /dev/null +SHELL [ "/busybox/sh", "-c" ] +RUN ln -s /busybox/sh /bin/sh +ENTRYPOINT [ "/busybox/tail", "-f", "/dev/null" ] diff --git a/cmd/querytee/Dockerfile b/cmd/querytee/Dockerfile index f2bfcefa0266..ea86fe0249ee 100644 --- a/cmd/querytee/Dockerfile +++ b/cmd/querytee/Dockerfile @@ -1,10 +1,14 @@ -FROM golang:1.22.5 as build +ARG GO_VERSION=1.22 +FROM golang:${GO_VERSION} as build COPY . /src/loki WORKDIR /src/loki RUN make clean && make BUILD_IN_CONTAINER=false loki-querytee -FROM alpine:3.18.5 -RUN apk add --update --no-cache ca-certificates +FROM gcr.io/distroless/base-nossl:debug COPY --from=build /src/loki/cmd/querytee/querytee /usr/bin/querytee + +SHELL [ "/busybox/sh", "-c" ] +RUN ln -s /busybox/sh /bin/sh + ENTRYPOINT [ "/usr/bin/querytee" ] diff --git a/cmd/querytee/Dockerfile.cross b/cmd/querytee/Dockerfile.cross index 005ebabbc217..e0cec2a4cbdd 100644 --- a/cmd/querytee/Dockerfile.cross +++ b/cmd/querytee/Dockerfile.cross @@ -2,17 +2,19 @@ ARG BUILD_IMAGE=grafana/loki-build-image:0.33.4 # Directories in this file are referenced from the root of the project not this folder # This file is intended to be called from the root like so: # docker build -t grafana/promtail -f cmd/promtail/Dockerfile . -FROM golang:1.22.5-alpine as goenv +ARG GO_VERSION=1.22 +FROM golang:${GO_VERSION} as goenv RUN go env GOARCH > /goarch && \ go env GOARM > /goarm -FROM --platform=linux/amd64 $BUILD_IMAGE as build +FROM $BUILD_IMAGE as build COPY --from=goenv /goarch /goarm / COPY . /src/loki WORKDIR /src/loki RUN make clean && GOARCH=$(cat /goarch) GOARM=$(cat /goarm) make BUILD_IN_CONTAINER=false loki-querytee -FROM alpine:3.18.5 -RUN apk add --update --no-cache ca-certificates +FROM gcr.io/distroless/base-nossl:debug COPY --from=build /src/loki/cmd/querytee/querytee /usr/bin/querytee +SHELL [ "/busybox/sh", "-c" ] +RUN ln -s /busybox/sh /bin/sh ENTRYPOINT [ "/usr/bin/querytee" ] diff --git a/loki-build-image/Dockerfile b/loki-build-image/Dockerfile index 25c95578e453..6b49181606c4 100644 --- a/loki-build-image/Dockerfile +++ b/loki-build-image/Dockerfile @@ -4,9 +4,9 @@ # tag of the Docker image in `../.drone/drone.jsonnet` and run `make drone`. # See ../docs/sources/community/maintaining/release-loki-build-image.md for instructions # on how to publish a new build image. - +ARG GO_VERSION=1.22 # Install helm (https://helm.sh/) and helm-docs (https://github.com/norwoodj/helm-docs) for generating Helm Chart reference. -FROM golang:1.22.5-bookworm as helm +FROM golang:${GO_VERSION}-bookworm as helm ARG TARGETARCH ARG HELM_VER="v3.2.3" RUN curl -L "https://get.helm.sh/helm-${HELM_VER}-linux-$TARGETARCH.tar.gz" | tar zx && \ @@ -38,7 +38,7 @@ RUN apk add --no-cache curl && \ FROM alpine:3.18.6 as docker RUN apk add --no-cache docker-cli docker-cli-buildx -FROM golang:1.22.5-bookworm as drone +FROM golang:${GO_VERSION}-bookworm as drone ARG TARGETARCH RUN curl -L "https://github.com/drone/drone-cli/releases/download/v1.7.0/drone_linux_$TARGETARCH".tar.gz | tar zx && \ install -t /usr/local/bin drone @@ -48,35 +48,35 @@ RUN curl -L "https://github.com/drone/drone-cli/releases/download/v1.7.0/drone_l # Error: # github.com/fatih/faillint@v1.5.0 requires golang.org/x/tools@v0.0.0-20200207224406-61798d64f025 # (not golang.org/x/tools@v0.0.0-20190918214920-58d531046acd from golang.org/x/tools/cmd/goyacc@58d531046acdc757f177387bc1725bfa79895d69) -FROM golang:1.22.5-bookworm as faillint +FROM golang:${GO_VERSION}-bookworm as faillint RUN GO111MODULE=on go install github.com/fatih/faillint@v1.12.0 RUN GO111MODULE=on go install golang.org/x/tools/cmd/goimports@v0.7.0 -FROM golang:1.22.5-bookworm as delve +FROM golang:${GO_VERSION}-bookworm as delve RUN GO111MODULE=on go install github.com/go-delve/delve/cmd/dlv@latest # Install ghr used to push binaries and template the release # This collides with the version of go tools used in the base image, thus we install it in its own image and copy it over. -FROM golang:1.22.5-bookworm as ghr +FROM golang:${GO_VERSION}-bookworm as ghr RUN GO111MODULE=on go install github.com/tcnksm/ghr@9349474 # Install nfpm (https://nfpm.goreleaser.com) for creating .deb and .rpm packages. -FROM golang:1.22.5-bookworm as nfpm +FROM golang:${GO_VERSION}-bookworm as nfpm RUN GO111MODULE=on go install github.com/goreleaser/nfpm/v2/cmd/nfpm@v2.11.3 # Install gotestsum -FROM golang:1.22.5-bookworm as gotestsum +FROM golang:${GO_VERSION}-bookworm as gotestsum RUN GO111MODULE=on go install gotest.tools/gotestsum@v1.8.2 # Install tools used to compile jsonnet. -FROM golang:1.22.5-bookworm as jsonnet +FROM golang:${GO_VERSION}-bookworm as jsonnet RUN GO111MODULE=on go install github.com/jsonnet-bundler/jsonnet-bundler/cmd/jb@v0.5.1 RUN GO111MODULE=on go install github.com/monitoring-mixins/mixtool/cmd/mixtool@16dc166166d91e93475b86b9355a4faed2400c18 RUN GO111MODULE=on go install github.com/google/go-jsonnet/cmd/jsonnet@v0.20.0 FROM aquasec/trivy as trivy -FROM golang:1.22.5-bookworm +FROM golang:${GO_VERSION}-bookworm RUN apt-get update && \ apt-get install -qy \ musl gnupg ragel \ diff --git a/production/helm/loki/src/helm-test/Dockerfile b/production/helm/loki/src/helm-test/Dockerfile index 48ff7e8a6a27..bb71f28b98ed 100644 --- a/production/helm/loki/src/helm-test/Dockerfile +++ b/production/helm/loki/src/helm-test/Dockerfile @@ -1,4 +1,5 @@ -FROM golang:1.22.2 as build +ARG GO_VERSION=1.22 +FROM golang:${GO_VERSION} as build # build via Makefile target helm-test-image in root # Makefile. Building from this directory will not be @@ -7,7 +8,6 @@ COPY . /src/loki WORKDIR /src/loki RUN make clean && make BUILD_IN_CONTAINER=false helm-test -FROM alpine:3.18.5 -RUN apk add --update --no-cache ca-certificates=20230506-r0 +FROM gcr.io/distroless/base-nossl:debug COPY --from=build /src/loki/production/helm/loki/src/helm-test/helm-test /usr/bin/helm-test ENTRYPOINT [ "/usr/bin/helm-test" ] From 515e13cc6c92b08968bc87e220b8bca64683fd05 Mon Sep 17 00:00:00 2001 From: Quentin Bisson Date: Wed, 17 Jul 2024 22:20:08 +0200 Subject: [PATCH 23/23] fix: incorrect pod matcher for compactor in mixin when using ssd mode (#12846) --- .../dashboards/loki-deletion.json | 10 +++++----- .../loki-mixin-compiled/dashboards/loki-deletion.json | 10 +++++----- .../loki-mixin/dashboards/loki-deletion.libsonnet | 6 ++---- 3 files changed, 12 insertions(+), 14 deletions(-) diff --git a/production/loki-mixin-compiled-ssd/dashboards/loki-deletion.json b/production/loki-mixin-compiled-ssd/dashboards/loki-deletion.json index 0718507d941f..e630e9b9f5c0 100644 --- a/production/loki-mixin-compiled-ssd/dashboards/loki-deletion.json +++ b/production/loki-mixin-compiled-ssd/dashboards/loki-deletion.json @@ -379,7 +379,7 @@ "span": 4, "targets": [ { - "expr": "node_namespace_pod_container:container_cpu_usage_seconds_total:sum_irate{cluster=~\"$cluster\", namespace=~\"$namespace\", container=\"loki\", pod=~\"(loki.*|enterprise-logs)-backend.*\"}", + "expr": "node_namespace_pod_container:container_cpu_usage_seconds_total:sum_irate{cluster=~\"$cluster\", namespace=~\"$namespace\", pod=~\"(compactor|(loki.*|enterprise-logs)-backend.*|loki-single-binary)\"}", "format": "time_series", "legendFormat": "{{pod}}", "legendLink": null @@ -426,7 +426,7 @@ "span": 4, "targets": [ { - "expr": "go_memstats_heap_inuse_bytes{cluster=~\"$cluster\", namespace=~\"$namespace\", container=\"compactor\"} / 1024 / 1024 ", + "expr": "go_memstats_heap_inuse_bytes{cluster=~\"$cluster\", namespace=~\"$namespace\", pod=~\"(compactor|(loki.*|enterprise-logs)-backend.*|loki-single-binary)\"} / 1024 / 1024 ", "format": "time_series", "legendFormat": " {{pod}} ", "legendLink": null @@ -579,7 +579,7 @@ "span": 6, "targets": [ { - "expr": "sum(rate(loki_compactor_deleted_lines{cluster=~\"$cluster\", namespace=~\"$namespace\", container=\"loki\", pod=~\"(loki.*|enterprise-logs)-backend.*\"}[$__rate_interval])) by (user)", + "expr": "sum(rate(loki_compactor_deleted_lines{cluster=~\"$cluster\", namespace=~\"$namespace\", pod=~\"(compactor|(loki.*|enterprise-logs)-backend.*|loki-single-binary)\"}[$__rate_interval])) by (user)", "format": "time_series", "legendFormat": "{{user}}", "legendLink": null @@ -606,7 +606,7 @@ "span": 6, "targets": [ { - "expr": "{cluster=~\"$cluster\", namespace=~\"$namespace\", container=\"loki\", pod=~\"(loki.*|enterprise-logs)-backend.*\"} |~ \"Started processing delete request|delete request for user marked as processed\" | logfmt | line_format \"{{.ts}} user={{.user}} delete_request_id={{.delete_request_id}} msg={{.msg}}\" ", + "expr": "{cluster=~\"$cluster\", namespace=~\"$namespace\", pod=~\"(compactor|(loki.*|enterprise-logs)-backend.*|loki-single-binary)\"} |~ \"Started processing delete request|delete request for user marked as processed\" | logfmt | line_format \"{{.ts}} user={{.user}} delete_request_id={{.delete_request_id}} msg={{.msg}}\" ", "refId": "A" } ], @@ -619,7 +619,7 @@ "span": 6, "targets": [ { - "expr": "{cluster=~\"$cluster\", namespace=~\"$namespace\", container=\"loki\", pod=~\"(loki.*|enterprise-logs)-backend.*\"} |~ \"delete request for user added\" | logfmt | line_format \"{{.ts}} user={{.user}} query='{{.query}}'\"", + "expr": "{cluster=~\"$cluster\", namespace=~\"$namespace\", pod=~\"(compactor|(loki.*|enterprise-logs)-backend.*|loki-single-binary)\"} |~ \"delete request for user added\" | logfmt | line_format \"{{.ts}} user={{.user}} query='{{.query}}'\"", "refId": "A" } ], diff --git a/production/loki-mixin-compiled/dashboards/loki-deletion.json b/production/loki-mixin-compiled/dashboards/loki-deletion.json index 7b048f729e2b..e630e9b9f5c0 100644 --- a/production/loki-mixin-compiled/dashboards/loki-deletion.json +++ b/production/loki-mixin-compiled/dashboards/loki-deletion.json @@ -379,7 +379,7 @@ "span": 4, "targets": [ { - "expr": "node_namespace_pod_container:container_cpu_usage_seconds_total:sum_irate{cluster=~\"$cluster\", namespace=~\"$namespace\", container=\"compactor\"}", + "expr": "node_namespace_pod_container:container_cpu_usage_seconds_total:sum_irate{cluster=~\"$cluster\", namespace=~\"$namespace\", pod=~\"(compactor|(loki.*|enterprise-logs)-backend.*|loki-single-binary)\"}", "format": "time_series", "legendFormat": "{{pod}}", "legendLink": null @@ -426,7 +426,7 @@ "span": 4, "targets": [ { - "expr": "go_memstats_heap_inuse_bytes{cluster=~\"$cluster\", namespace=~\"$namespace\", container=\"compactor\"} / 1024 / 1024 ", + "expr": "go_memstats_heap_inuse_bytes{cluster=~\"$cluster\", namespace=~\"$namespace\", pod=~\"(compactor|(loki.*|enterprise-logs)-backend.*|loki-single-binary)\"} / 1024 / 1024 ", "format": "time_series", "legendFormat": " {{pod}} ", "legendLink": null @@ -579,7 +579,7 @@ "span": 6, "targets": [ { - "expr": "sum(rate(loki_compactor_deleted_lines{cluster=~\"$cluster\", namespace=~\"$namespace\", container=\"compactor\"}[$__rate_interval])) by (user)", + "expr": "sum(rate(loki_compactor_deleted_lines{cluster=~\"$cluster\", namespace=~\"$namespace\", pod=~\"(compactor|(loki.*|enterprise-logs)-backend.*|loki-single-binary)\"}[$__rate_interval])) by (user)", "format": "time_series", "legendFormat": "{{user}}", "legendLink": null @@ -606,7 +606,7 @@ "span": 6, "targets": [ { - "expr": "{cluster=~\"$cluster\", namespace=~\"$namespace\", container=\"compactor\"} |~ \"Started processing delete request|delete request for user marked as processed\" | logfmt | line_format \"{{.ts}} user={{.user}} delete_request_id={{.delete_request_id}} msg={{.msg}}\" ", + "expr": "{cluster=~\"$cluster\", namespace=~\"$namespace\", pod=~\"(compactor|(loki.*|enterprise-logs)-backend.*|loki-single-binary)\"} |~ \"Started processing delete request|delete request for user marked as processed\" | logfmt | line_format \"{{.ts}} user={{.user}} delete_request_id={{.delete_request_id}} msg={{.msg}}\" ", "refId": "A" } ], @@ -619,7 +619,7 @@ "span": 6, "targets": [ { - "expr": "{cluster=~\"$cluster\", namespace=~\"$namespace\", container=\"compactor\"} |~ \"delete request for user added\" | logfmt | line_format \"{{.ts}} user={{.user}} query='{{.query}}'\"", + "expr": "{cluster=~\"$cluster\", namespace=~\"$namespace\", pod=~\"(compactor|(loki.*|enterprise-logs)-backend.*|loki-single-binary)\"} |~ \"delete request for user added\" | logfmt | line_format \"{{.ts}} user={{.user}} query='{{.query}}'\"", "refId": "A" } ], diff --git a/production/loki-mixin/dashboards/loki-deletion.libsonnet b/production/loki-mixin/dashboards/loki-deletion.libsonnet index a1c50ecfa691..a4c4ae779e70 100644 --- a/production/loki-mixin/dashboards/loki-deletion.libsonnet +++ b/production/loki-mixin/dashboards/loki-deletion.libsonnet @@ -2,9 +2,7 @@ local g = import 'grafana-builder/grafana.libsonnet'; local utils = import 'mixin-utils/utils.libsonnet'; (import 'dashboard-utils.libsonnet') { - local compactor_matcher = if $._config.meta_monitoring.enabled - then 'pod=~"(compactor|%s-backend.*|loki-single-binary)"' % $._config.ssd.pod_prefix_matcher - else if $._config.ssd.enabled then 'container="loki", pod=~"%s-backend.*"' % $._config.ssd.pod_prefix_matcher else 'container="compactor"', + local compactor_matcher = 'pod=~"(compactor|%s-backend.*|loki-single-binary)"' % $._config.ssd.pod_prefix_matcher, grafanaDashboards+:: { 'loki-deletion.json': @@ -50,7 +48,7 @@ local utils = import 'mixin-utils/utils.libsonnet'; ) .addPanel( $.newQueryPanel('Compactor memory usage (MiB)') + - g.queryPanel('go_memstats_heap_inuse_bytes{%s, container="compactor"} / 1024 / 1024 ' % $.namespaceMatcher(), ' {{pod}} '), + g.queryPanel('go_memstats_heap_inuse_bytes{%s, %s} / 1024 / 1024 ' % [$.namespaceMatcher(), compactor_matcher], ' {{pod}} '), ) .addPanel( $.newQueryPanel('Compaction run duration (seconds)') +